]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd: remove more (most? all?) of classic scrub
authorSage Weil <sage@redhat.com>
Wed, 10 Dec 2014 00:02:49 +0000 (16:02 -0800)
committerSage Weil <sage@redhat.com>
Sat, 20 Dec 2014 15:30:00 +0000 (07:30 -0800)
Hopefully this is everything

Signed-off-by: Sage Weil <sage@redhat.com>
src/messages/MOSDRepScrub.h
src/osd/OSD.cc
src/osd/OSD.h
src/osd/PG.cc
src/osd/PG.h
src/osd/ReplicatedPG.cc

index 5d0a6041c2fc38bb3ba968894f4958746a52128e..fb4080e704678a5b695c48144783af21208686e3 100644 (file)
@@ -43,17 +43,6 @@ struct MOSDRepScrub : public Message {
       deep(false),
       seed(0) { }
 
-  MOSDRepScrub(spg_t pgid, eversion_t scrub_from, eversion_t scrub_to,
-              epoch_t map_epoch)
-    : Message(MSG_OSD_REP_SCRUB, HEAD_VERSION, COMPAT_VERSION),
-      pgid(pgid),
-      scrub_from(scrub_from),
-      scrub_to(scrub_to),
-      map_epoch(map_epoch),
-      chunky(false),
-      deep(false),
-      seed(0) { }
-
   MOSDRepScrub(spg_t pgid, eversion_t scrub_to, epoch_t map_epoch,
                hobject_t start, hobject_t end, bool deep, uint32_t seed)
     : Message(MSG_OSD_REP_SCRUB, HEAD_VERSION, COMPAT_VERSION),
index bda3622e8c56099162256fb7ca9ba92d079060f5..c61cff545ea4dc52b48f1a67e993f88430c888f1 100644 (file)
@@ -194,7 +194,6 @@ OSDService::OSDService(OSD *osd) :
   recovery_wq(osd->recovery_wq),
   snap_trim_wq(osd->snap_trim_wq),
   scrub_wq(osd->scrub_wq),
-  scrub_finalize_wq(osd->scrub_finalize_wq),
   rep_scrub_wq(osd->rep_scrub_wq),
   recovery_gen_wq("recovery_gen_wq", cct->_conf->osd_recovery_thread_timeout,
                  &osd->recovery_tp),
@@ -1558,7 +1557,6 @@ OSD::OSD(CephContext *cct_, ObjectStore *store_,
   replay_queue_lock("OSD::replay_queue_lock"),
   snap_trim_wq(this, cct->_conf->osd_snap_trim_thread_timeout, &disk_tp),
   scrub_wq(this, cct->_conf->osd_scrub_thread_timeout, &disk_tp),
-  scrub_finalize_wq(cct->_conf->osd_scrub_finalize_thread_timeout, &osd_tp),
   rep_scrub_wq(this, cct->_conf->osd_scrub_thread_timeout, &disk_tp),
   remove_wq(store, cct->_conf->osd_remove_thread_timeout, &disk_tp),
   service(this)
@@ -2284,7 +2282,6 @@ int OSD::shutdown()
 
   osd_tp.drain();
   peering_wq.clear();
-  scrub_finalize_wq.clear();
   osd_tp.stop();
   dout(10) << "osd tp stopped" << dendl;
 
index 8dd1912426094e4ec7e0019b161ef12090256830..aeb857b140bd791f7ae1be0a140e50ca3e848897 100644 (file)
@@ -327,7 +327,6 @@ public:
   ThreadPool::WorkQueue<PG> &recovery_wq;
   ThreadPool::WorkQueue<PG> &snap_trim_wq;
   ThreadPool::WorkQueue<PG> &scrub_wq;
-  ThreadPool::WorkQueue<PG> &scrub_finalize_wq;
   ThreadPool::WorkQueue<MOSDRepScrub> &rep_scrub_wq;
   GenContextWQ recovery_gen_wq;
   GenContextWQ op_gen_wq;
@@ -2134,50 +2133,6 @@ protected:
     }
   } scrub_wq;
 
-  struct ScrubFinalizeWQ : public ThreadPool::WorkQueue<PG> {
-  private:
-    xlist<PG*> scrub_finalize_queue;
-
-  public:
-    ScrubFinalizeWQ(time_t ti, ThreadPool *tp)
-      : ThreadPool::WorkQueue<PG>("OSD::ScrubFinalizeWQ", ti, ti*10, tp) {}
-
-    bool _empty() {
-      return scrub_finalize_queue.empty();
-    }
-    bool _enqueue(PG *pg) {
-      if (pg->scrub_finalize_item.is_on_list()) {
-       return false;
-      }
-      pg->get("ScrubFinalizeWQ");
-      scrub_finalize_queue.push_back(&pg->scrub_finalize_item);
-      return true;
-    }
-    void _dequeue(PG *pg) {
-      if (pg->scrub_finalize_item.remove_myself()) {
-       pg->put("ScrubFinalizeWQ");
-      }
-    }
-    PG *_dequeue() {
-      if (scrub_finalize_queue.empty())
-       return NULL;
-      PG *pg = scrub_finalize_queue.front();
-      scrub_finalize_queue.pop_front();
-      return pg;
-    }
-    void _process(PG *pg) {
-      pg->scrub_finalize();
-      pg->put("ScrubFinalizeWQ");
-    }
-    void _clear() {
-      while (!scrub_finalize_queue.empty()) {
-       PG *pg = scrub_finalize_queue.front();
-       scrub_finalize_queue.pop_front();
-       pg->put("ScrubFinalizeWQ");
-      }
-    }
-  } scrub_finalize_wq;
-
   struct RepScrubWQ : public ThreadPool::WorkQueue<MOSDRepScrub> {
   private: 
     OSD *osd;
index 71be9c8a3ed0643a7335dec6f68b4663d537d242..5e3539f78f93c3916c78e2cd0d51de5f957723c6 100644 (file)
@@ -191,7 +191,7 @@ PG::PG(OSDService *o, OSDMapRef curmap,
   coll(p), pg_log(cct),
   pgmeta_oid(p.make_pgmeta_oid()),
   missing_loc(this),
-  recovery_item(this), scrub_item(this), scrub_finalize_item(this), snap_trim_item(this), stat_queue_item(this),
+  recovery_item(this), scrub_item(this), snap_trim_item(this), stat_queue_item(this),
   recovery_ops_active(0),
   role(0),
   state(0),
@@ -3270,20 +3270,6 @@ void PG::sub_op_scrub_map(OpRequestRef op)
   }
 }
 
-// send scrub v2-compatible messages (classic scrub)
-void PG::_request_scrub_map_classic(pg_shard_t replica, eversion_t version)
-{
-  assert(replica != pg_whoami);
-  dout(10) << "scrub  requesting scrubmap from osd." << replica << dendl;
-  MOSDRepScrub *repscrubop =
-    new MOSDRepScrub(
-      spg_t(info.pgid.pgid, replica.shard), version,
-      last_update_applied,
-      get_osdmap()->get_epoch());
-  osd->send_message_osd_cluster(
-    replica.osd, repscrubop, get_osdmap()->get_epoch());
-}
-
 // send scrub v3 messages (chunky scrub)
 void PG::_request_scrub_map(
   pg_shard_t replica, eversion_t version,
@@ -3593,76 +3579,6 @@ int PG::build_scrub_map_chunk(
   return 0;
 }
 
-/*
- * build a (sorted) summary of pg content for purposes of scrubbing
- * called while holding pg lock
- */ 
-void PG::build_scrub_map(ScrubMap &map, ThreadPool::TPHandle &handle)
-{
-  dout(10) << "build_scrub_map" << dendl;
-
-  map.valid_through = info.last_update;
-  epoch_t epoch = get_osdmap()->get_epoch();
-
-  unlock();
-
-  // wait for any writes on our pg to flush to disk first.  this avoids races
-  // with scrub starting immediately after trim or recovery completion.
-  osr->flush();
-
-  // objects
-  vector<hobject_t> ls;
-  osd->store->collection_list(coll, ls);
-
-  get_pgbackend()->be_scan_list(map, ls, false, 0, handle);
-  lock();
-  _scan_snaps(map);
-
-  if (pg_has_reset_since(epoch)) {
-    dout(10) << "scrub  pg changed, aborting" << dendl;
-    return;
-  }
-
-
-  dout(10) << "PG relocked, finalizing" << dendl;
-
-  dout(10) << __func__ << " done." << dendl;
-}
-
-
-/* 
- * build a summary of pg content changed starting after v
- * called while holding pg lock
- */
-void PG::build_inc_scrub_map(
-  ScrubMap &map, eversion_t v,
-  ThreadPool::TPHandle &handle)
-{
-  map.valid_through = last_update_applied;
-  map.incr_since = v;
-  vector<hobject_t> ls;
-  list<pg_log_entry_t>::const_iterator p;
-  if (v == pg_log.get_tail()) {
-    p = pg_log.get_log().log.begin();
-  } else if (v > pg_log.get_tail()) {
-    p = pg_log.get_log().find_entry(v);
-    ++p;
-  } else {
-    assert(0);
-  }
-  
-  for (; p != pg_log.get_log().log.end(); ++p) {
-    if (p->is_update()) {
-      ls.push_back(p->soid);
-      map.objects[p->soid].negative = false;
-    } else if (p->is_delete()) {
-      map.objects[p->soid].negative = true;
-    }
-  }
-
-  get_pgbackend()->be_scan_list(map, ls, false, 0, handle);
-}
-
 void PG::repair_object(
   const hobject_t& soid, ScrubMap::object *po,
   pg_shard_t bad_peer, pg_shard_t ok_peer)
@@ -3752,7 +3668,7 @@ void PG::replica_scrub(
  * PG_STATE_SCRUBBING is set when the scrub is queued
  * 
  * scrub will be chunky if all OSDs in PG support chunky scrub
- * scrub will fall back to classic in any other case
+ * scrub will fail if OSDs are too old.
  */
 void PG::scrub(ThreadPool::TPHandle &handle)
 {
@@ -4137,30 +4053,6 @@ void PG::scrub_clear_state()
   _scrub_clear_state();
 }
 
-bool PG::scrub_gather_replica_maps()
-{
-  assert(scrubber.waiting_on == 0);
-  assert(_lock.is_locked());
-
-  for (map<pg_shard_t, ScrubMap>::iterator p = scrubber.received_maps.begin();
-       p != scrubber.received_maps.end();
-       ++p) {
-    
-    if (scrubber.received_maps[p->first].valid_through != pg_log.get_head()) {
-      scrubber.waiting_on++;
-      scrubber.waiting_on_whom.insert(p->first);
-      // Need to request another incremental map
-      _request_scrub_map_classic(p->first, p->second.valid_through);
-    }
-  }
-  
-  if (scrubber.waiting_on > 0) {
-    return false;
-  } else {
-    return true;
-  }
-}
-
 void PG::scrub_compare_maps() 
 {
   dout(10) << __func__ << " has maps, analyzing" << dendl;
@@ -4277,38 +4169,6 @@ void PG::scrub_process_inconsistent()
   }
 }
 
-void PG::scrub_finalize()
-{
-  lock();
-  if (deleting) {
-    unlock();
-    return;
-  }
-
-  assert(last_update_applied == info.last_update);
-
-  if (scrubber.epoch_start != info.history.same_interval_since) {
-    dout(10) << "scrub  pg changed, aborting" << dendl;
-    scrub_clear_state();
-    scrub_unreserve_replicas();
-    unlock();
-    return;
-  }
-
-  if (!scrub_gather_replica_maps()) {
-    dout(10) << "maps not yet up to date, sent out new requests" << dendl;
-    unlock();
-    return;
-  }
-
-  scrub_compare_maps();
-
-  scrub_finish();
-
-  dout(10) << "scrub done" << dendl;
-  unlock();
-}
-
 // the part that actually finalizes a scrub
 void PG::scrub_finish() 
 {
index 25ee5cd6c977011ca18460c88ddae096b8014784..9dfa2b9b018f01000774f544bee650ed00ca19d2 100644 (file)
@@ -431,7 +431,7 @@ public:
 
   /* You should not use these items without taking their respective queue locks
    * (if they have one) */
-  xlist<PG*>::item recovery_item, scrub_item, scrub_finalize_item, snap_trim_item, stat_queue_item;
+  xlist<PG*>::item recovery_item, scrub_item, snap_trim_item, stat_queue_item;
   int recovery_ops_active;
   set<pg_shard_t> waiting_on_backfill;
 #ifdef DEBUG_RECOVERY_OIDS
@@ -1172,19 +1172,15 @@ public:
     pg_shard_t ok_peer);
 
   void scrub(ThreadPool::TPHandle &handle);
-  void classic_scrub(ThreadPool::TPHandle &handle);
   void chunky_scrub(ThreadPool::TPHandle &handle);
   void scrub_compare_maps();
   void scrub_process_inconsistent();
-  void scrub_finalize();
   void scrub_finish();
   void scrub_clear_state();
-  bool scrub_gather_replica_maps();
   void _scan_snaps(ScrubMap &map);
   void _scan_rollback_obs(
     const vector<ghobject_t> &rollback_obs,
     ThreadPool::TPHandle &handle);
-  void _request_scrub_map_classic(pg_shard_t replica, eversion_t version);
   void _request_scrub_map(pg_shard_t replica, eversion_t version,
                           hobject_t start, hobject_t end, bool deep,
                          uint32_t seed);
@@ -1192,9 +1188,6 @@ public:
     ScrubMap &map,
     hobject_t start, hobject_t end, bool deep, uint32_t seed,
     ThreadPool::TPHandle &handle);
-  void build_scrub_map(ScrubMap &map, ThreadPool::TPHandle &handle);
-  void build_inc_scrub_map(
-    ScrubMap &map, eversion_t v, ThreadPool::TPHandle &handle);
   /**
    * returns true if [begin, end) is good to scrub at this time
    * a false return value obliges the implementer to requeue scrub when the
index 5c98422a8addfadf5aa719f0fa6862debbf1b8af..815720bad06a0f1044bb80c6ccc1c4b3fcdf2e2c 100644 (file)
@@ -9807,7 +9807,6 @@ void ReplicatedPG::on_shutdown()
   // remove from queues
   osd->recovery_wq.dequeue(this);
   osd->scrub_wq.dequeue(this);
-  osd->scrub_finalize_wq.dequeue(this);
   osd->snap_trim_wq.dequeue(this);
   osd->pg_stat_queue_dequeue(this);
   osd->dequeue_pg(this, 0);