]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd: introduce acting_recovery_backfill shards
authorNeha Ojha <nojha@redhat.com>
Wed, 29 Nov 2017 09:57:43 +0000 (15:27 +0530)
committerNeha Ojha <nojha@redhat.com>
Wed, 14 Mar 2018 22:56:04 +0000 (15:56 -0700)
Signed-off-by: Neha Ojha <nojha@redhat.com>
src/osd/ECBackend.cc
src/osd/PG.cc
src/osd/PG.h
src/osd/PGBackend.cc
src/osd/PGBackend.h
src/osd/PrimaryLogPG.cc
src/osd/PrimaryLogPG.h
src/osd/ReplicatedBackend.cc

index 321ddb6cd8f5357187dc3ffc9bb9c98cf16fbef0..9ae86046bdc8051359272b502027577373bbce32 100644 (file)
@@ -728,8 +728,8 @@ int ECBackend::recover_object(
   }
   h->ops.back().recovery_progress.omap_complete = true;
   for (set<pg_shard_t>::const_iterator i =
-        get_parent()->get_actingbackfill_shards().begin();
-       i != get_parent()->get_actingbackfill_shards().end();
+        get_parent()->get_acting_recovery_backfill_shards().begin();
+       i != get_parent()->get_acting_recovery_backfill_shards().end();
        ++i) {
     dout(10) << "checking " << *i << dendl;
     if (get_parent()->get_shard_missing(*i).is_missing(hoid)) {
@@ -1848,8 +1848,8 @@ bool ECBackend::try_reads_to_commit()
 
   map<shard_id_t, ObjectStore::Transaction> trans;
   for (set<pg_shard_t>::const_iterator i =
-        get_parent()->get_actingbackfill_shards().begin();
-       i != get_parent()->get_actingbackfill_shards().end();
+        get_parent()->get_acting_recovery_backfill_shards().begin();
+       i != get_parent()->get_acting_recovery_backfill_shards().end();
        ++i) {
     trans[i->shard];
   }
@@ -1906,8 +1906,8 @@ bool ECBackend::try_reads_to_commit()
   bool should_write_local = false;
   ECSubWrite local_write_op;
   for (set<pg_shard_t>::const_iterator i =
-        get_parent()->get_actingbackfill_shards().begin();
-       i != get_parent()->get_actingbackfill_shards().end();
+        get_parent()->get_acting_recovery_backfill_shards().begin();
+       i != get_parent()->get_acting_recovery_backfill_shards().end();
        ++i) {
     op->pending_apply.insert(*i);
     op->pending_commit.insert(*i);
index 4bab4b8e60970c49c5e3215945837d06addbe216..3aadda0640953d585959b8acabd83cf4fd093ae8 100644 (file)
@@ -850,9 +850,9 @@ bool PG::needs_recovery() const
     return true;
   }
 
-  assert(!actingbackfill.empty());
-  set<pg_shard_t>::const_iterator end = actingbackfill.end();
-  set<pg_shard_t>::const_iterator a = actingbackfill.begin();
+  assert(!acting_recovery_backfill.empty());
+  set<pg_shard_t>::const_iterator end = acting_recovery_backfill.end();
+  set<pg_shard_t>::const_iterator a = acting_recovery_backfill.begin();
   for (; a != end; ++a) {
     if (*a == get_primary()) continue;
     pg_shard_t peer = *a;
@@ -1700,8 +1700,8 @@ bool PG::choose_acting(pg_shard_t &auth_log_shard_id,
     return false;
   }
   want_acting.clear();
-  actingbackfill = want_acting_backfill;
-  dout(10) << "actingbackfill is " << actingbackfill << dendl;
+  acting_recovery_backfill = want_acting_backfill;
+  dout(10) << "acting_recovery_backfill is " << acting_recovery_backfill << dendl;
   assert(backfill_targets.empty() || backfill_targets == want_backfill);
   if (backfill_targets.empty()) {
     // Caller is GetInfo
@@ -1858,9 +1858,9 @@ void PG::activate(ObjectStore::Transaction& t,
     assert(ctx);
     // start up replicas
 
-    assert(!actingbackfill.empty());
-    for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-        i != actingbackfill.end();
+    assert(!acting_recovery_backfill.empty());
+    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+        i != acting_recovery_backfill.end();
         ++i) {
       if (*i == pg_whoami) continue;
       pg_shard_t peer = *i;
@@ -1999,8 +1999,8 @@ void PG::activate(ObjectStore::Transaction& t,
 
     // Set up missing_loc
     set<pg_shard_t> complete_shards;
-    for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-        i != actingbackfill.end();
+    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+        i != acting_recovery_backfill.end();
         ++i) {
       dout(20) << __func__ << " setting up missing_loc from shard " << *i << " " << dendl;
       if (*i == get_primary()) {
@@ -2026,13 +2026,13 @@ void PG::activate(ObjectStore::Transaction& t,
       // source, this is considered safe since the PGLogs have been merged locally,
       // and covers vast majority of the use cases, like one OSD/host is down for
       // a while for hardware repairing
-      if (complete_shards.size() + 1 == actingbackfill.size()) {
+      if (complete_shards.size() + 1 == acting_recovery_backfill.size()) {
         missing_loc.add_batch_sources_info(complete_shards, ctx->handle);
       } else {
         missing_loc.add_source_info(pg_whoami, info, pg_log.get_missing(),
                                    ctx->handle);
-        for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-            i != actingbackfill.end();
+        for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+            i != acting_recovery_backfill.end();
             ++i) {
          if (*i == pg_whoami) continue;
          dout(10) << __func__ << ": adding " << *i << " as a source" << dendl;
@@ -2048,7 +2048,7 @@ void PG::activate(ObjectStore::Transaction& t,
       for (map<pg_shard_t, pg_missing_t>::iterator i = peer_missing.begin();
           i != peer_missing.end();
           ++i) {
-       if (is_actingbackfill(i->first))
+       if (is_acting_recovery_backfill(i->first))
          continue;
        assert(peer_info.count(i->first));
        search_for_missing(
@@ -2136,8 +2136,8 @@ void PG::_activate_committed(epoch_t epoch, epoch_t activation_epoch)
             << " last_interval_started " << info.history.last_interval_started
             << " last_epoch_started " << info.history.last_epoch_started
             << " same_interval_since " << info.history.same_interval_since << dendl;
-    assert(!actingbackfill.empty());
-    if (peer_activated.size() == actingbackfill.size())
+    assert(!acting_recovery_backfill.empty());
+    if (peer_activated.size() == acting_recovery_backfill.size())
       all_activated_and_committed();
   } else {
     dout(10) << "_activate_committed " << epoch << " telling primary" << dendl;
@@ -2185,8 +2185,8 @@ void PG::all_activated_and_committed()
 {
   dout(10) << "all_activated_and_committed" << dendl;
   assert(is_primary());
-  assert(peer_activated.size() == actingbackfill.size());
-  assert(!actingbackfill.empty());
+  assert(peer_activated.size() == acting_recovery_backfill.size());
+  assert(!acting_recovery_backfill.empty());
   assert(blocked_by.empty());
 
   // Degraded?
@@ -2755,7 +2755,7 @@ void PG::purge_strays()
   for (set<pg_shard_t>::iterator p = stray_set.begin();
        p != stray_set.end();
        ++p) {
-    assert(!is_actingbackfill(*p));
+    assert(!is_acting_recovery_backfill(*p));
     if (get_osdmap()->is_up(p->osd)) {
       dout(10) << "sending PGRemove to osd." << *p << dendl;
       vector<spg_t> to_remove;
@@ -2901,10 +2901,10 @@ void PG::_update_calc_stats()
 
   if ((is_remapped() || is_undersized() || !is_clean()) && (is_peered() || is_activating())) {
     dout(20) << __func__ << " actingset " << actingset << " upset "
-             << upset << " actingbackfill " << actingbackfill << dendl;
+             << upset << " acting_recovery_backfill " << acting_recovery_backfill << dendl;
     dout(20) << __func__ << " acting " << acting << " up " << up << dendl;
 
-    assert(!actingbackfill.empty());
+    assert(!acting_recovery_backfill.empty());
 
     bool estimate = false;
 
@@ -2924,7 +2924,7 @@ void PG::_update_calc_stats()
 
       // Primary first
       missing = pg_log.get_missing().num_missing();
-      assert(actingbackfill.count(pg_whoami));
+      assert(acting_recovery_backfill.count(pg_whoami));
       if (upset.count(pg_whoami)) {
         missing_target_objects.insert(make_pair(missing, pg_whoami));
       } else {
@@ -2935,7 +2935,6 @@ void PG::_update_calc_stats()
                << " primary objects " << num_objects
                << " missing " << missing
                << dendl;
-
     }
 
     // All other peers
@@ -3548,9 +3547,9 @@ void PG::trim_log()
   dout(10) << __func__ << " to " << pg_trim_to << dendl;
   if (pg_trim_to != eversion_t()) {
     // inform peers to trim log
-    assert(!actingbackfill.empty());
-    for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-        i != actingbackfill.end();
+    assert(!acting_recovery_backfill.empty());
+    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+        i != acting_recovery_backfill.end();
         ++i) {
       if (*i == pg_whoami) continue;
       osd->send_message_osd_cluster(
@@ -4325,8 +4324,8 @@ void PG::clear_scrub_reserved()
 void PG::scrub_reserve_replicas()
 {
   assert(backfill_targets.empty());
-  for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-       i != actingbackfill.end();
+  for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+       i != acting_recovery_backfill.end();
        ++i) {
     if (*i == pg_whoami) continue;
     dout(10) << "scrub requesting reserve from osd." << *i << dendl;
@@ -4342,8 +4341,8 @@ void PG::scrub_reserve_replicas()
 void PG::scrub_unreserve_replicas()
 {
   assert(backfill_targets.empty());
-  for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-       i != actingbackfill.end();
+  for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+       i != acting_recovery_backfill.end();
        ++i) {
     if (*i == pg_whoami) continue;
     dout(10) << "scrub requesting unreserve from osd." << *i << dendl;
@@ -4996,8 +4995,8 @@ void PG::chunky_scrub(ThreadPool::TPHandle &handle)
         scrubber.waiting_on_whom.insert(pg_whoami);
 
         // request maps from replicas
-       for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-            i != actingbackfill.end();
+       for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+            i != acting_recovery_backfill.end();
             ++i) {
          if (*i == pg_whoami) continue;
           _request_scrub_map(*i, scrubber.subset_last_update,
@@ -5222,7 +5221,7 @@ void PG::scrub_compare_maps()
   map<pg_shard_t, ScrubMap *> maps;
   maps[pg_whoami] = &scrubber.primary_scrubmap;
 
-  for (const auto& i : actingbackfill) {
+  for (const auto& i : acting_recovery_backfill) {
     if (i == pg_whoami) continue;
     dout(2) << __func__ << " replica " << i << " has "
             << scrubber.received_maps[i].objects.size()
@@ -5498,9 +5497,9 @@ void PG::share_pg_info()
   dout(10) << "share_pg_info" << dendl;
 
   // share new pg_info_t with replicas
-  assert(!actingbackfill.empty());
-  for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-       i != actingbackfill.end();
+  assert(!acting_recovery_backfill.empty());
+  for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+       i != acting_recovery_backfill.end();
        ++i) {
     if (*i == pg_whoami) continue;
     auto pg_shard = *i;
@@ -5576,8 +5575,8 @@ void PG::merge_new_log_entries(
   assert(is_primary());
 
   bool rebuild_missing = append_log_entries_update_missing(entries, t, trim_to, roll_forward_to);
-  for (set<pg_shard_t>::const_iterator i = actingbackfill.begin();
-       i != actingbackfill.end();
+  for (set<pg_shard_t>::const_iterator i = acting_recovery_backfill.begin();
+       i != acting_recovery_backfill.end();
        ++i) {
     pg_shard_t peer(*i);
     if (peer == pg_whoami) continue;
@@ -5608,7 +5607,7 @@ void PG::merge_new_log_entries(
     missing_loc.rebuild(
       i.soid,
       pg_whoami,
-      actingbackfill,
+      acting_recovery_backfill,
       info,
       pg_log.get_missing(),
       peer_missing,
@@ -5933,7 +5932,7 @@ void PG::start_peering_interval(
   state_clear(PG_STATE_RECOVERING);
 
   peer_purged.clear();
-  actingbackfill.clear();
+  acting_recovery_backfill.clear();
   scrub_queued = false;
 
   // reset primary state?
@@ -7517,7 +7516,7 @@ PG::RecoveryState::WaitLocalRecoveryReserved::WaitLocalRecoveryReserved(my_conte
 
   // Make sure all nodes that part of the recovery aren't full
   if (!pg->cct->_conf->osd_debug_skip_full_check_in_recovery &&
-      pg->osd->check_osdmap_full(pg->actingbackfill)) {
+      pg->osd->check_osdmap_full(pg->acting_recovery_backfill)) {
     post_event(RecoveryTooFull());
     return;
   }
@@ -7702,9 +7701,9 @@ PG::RecoveryState::Recovered::Recovered(my_context ctx)
 
   // if we finished backfill, all acting are active; recheck if
   // DEGRADED | UNDERSIZED is appropriate.
-  assert(!pg->actingbackfill.empty());
+  assert(!pg->acting_recovery_backfill.empty());
   if (pg->get_osdmap()->get_pg_size(pg->info.pgid.pgid) <=
-      pg->actingbackfill.size()) {
+      pg->acting_recovery_backfill.size()) {
     pg->state_clear(PG_STATE_FORCED_BACKFILL | PG_STATE_FORCED_RECOVERY);
     pg->publish_stats_to_osd();
   }
@@ -7784,7 +7783,7 @@ PG::RecoveryState::Active::Active(my_context ctx)
     remote_shards_to_reserve_recovery(
       unique_osd_shard_set(
        context< RecoveryMachine >().pg->pg_whoami,
-       context< RecoveryMachine >().pg->actingbackfill)),
+       context< RecoveryMachine >().pg->acting_recovery_backfill)),
     remote_shards_to_reserve_backfill(
       unique_osd_shard_set(
        context< RecoveryMachine >().pg->pg_whoami,
@@ -7812,8 +7811,8 @@ PG::RecoveryState::Active::Active(my_context ctx)
 
   // everyone has to commit/ack before we are truly active
   pg->blocked_by.clear();
-  for (set<pg_shard_t>::iterator p = pg->actingbackfill.begin();
-       p != pg->actingbackfill.end();
+  for (set<pg_shard_t>::iterator p = pg->acting_recovery_backfill.begin();
+       p != pg->acting_recovery_backfill.end();
        ++p) {
     if (p->shard != pg->pg_whoami.shard) {
       pg->blocked_by.insert(p->shard);
@@ -8031,18 +8030,18 @@ boost::statechart::result PG::RecoveryState::Active::react(const MInfoRec& infoe
   PG *pg = context< RecoveryMachine >().pg;
   assert(pg->is_primary());
 
-  assert(!pg->actingbackfill.empty());
+  assert(!pg->acting_recovery_backfill.empty());
   // don't update history (yet) if we are active and primary; the replica
   // may be telling us they have activated (and committed) but we can't
   // share that until _everyone_ does the same.
-  if (pg->is_actingbackfill(infoevt.from) &&
+  if (pg->is_acting_recovery_backfill(infoevt.from) &&
       pg->peer_activated.count(infoevt.from) == 0) {
     ldout(pg->cct, 10) << " peer osd." << infoevt.from
                       << " activated and committed" << dendl;
     pg->peer_activated.insert(infoevt.from);
     pg->blocked_by.erase(infoevt.from.shard);
     pg->publish_stats_to_osd();
-    if (pg->peer_activated.size() == pg->actingbackfill.size()) {
+    if (pg->peer_activated.size() == pg->acting_recovery_backfill.size()) {
       pg->all_activated_and_committed();
     }
   }
@@ -8669,9 +8668,9 @@ PG::RecoveryState::GetLog::GetLog(my_context ctx)
 
   // how much log to request?
   eversion_t request_log_from = pg->info.last_update;
-  assert(!pg->actingbackfill.empty());
-  for (set<pg_shard_t>::iterator p = pg->actingbackfill.begin();
-       p != pg->actingbackfill.end();
+  assert(!pg->acting_recovery_backfill.empty());
+  for (set<pg_shard_t>::iterator p = pg->acting_recovery_backfill.begin();
+       p != pg->acting_recovery_backfill.end();
        ++p) {
     if (*p == pg->pg_whoami) continue;
     pg_info_t& ri = pg->peer_info[*p];
@@ -8941,10 +8940,10 @@ PG::RecoveryState::GetMissing::GetMissing(my_context ctx)
   context< RecoveryMachine >().log_enter(state_name);
 
   PG *pg = context< RecoveryMachine >().pg;
-  assert(!pg->actingbackfill.empty());
+  assert(!pg->acting_recovery_backfill.empty());
   eversion_t since;
-  for (set<pg_shard_t>::iterator i = pg->actingbackfill.begin();
-       i != pg->actingbackfill.end();
+  for (set<pg_shard_t>::iterator i = pg->acting_recovery_backfill.begin();
+       i != pg->acting_recovery_backfill.end();
        ++i) {
     if (*i == pg->get_primary()) continue;
     const pg_info_t& pi = pg->peer_info[*i];
index 589206d3cb67c64b03befcaddf12f7add0ff41f9..ed4907daaa139826b4d6244220e8359b22ccae23 100644 (file)
@@ -936,7 +936,9 @@ protected:
   pg_shard_t pg_whoami;
   pg_shard_t up_primary;
   vector<int> up, acting, want_acting;
-  set<pg_shard_t> actingbackfill, actingset, upset;
+  // acting_recovery_backfill contains shards that are acting,
+  // async recovery targets, or backfill targets.
+  set<pg_shard_t> acting_recovery_backfill, actingset, upset;
   map<pg_shard_t,eversion_t> peer_last_complete_ondisk;
   eversion_t  min_last_complete_ondisk;  // up: min over last_complete_ondisk, peer_last_complete_ondisk
   eversion_t  pg_trim_to;
@@ -1278,8 +1280,8 @@ protected:
 
   void clear_primary_state();
 
-  bool is_actingbackfill(pg_shard_t osd) const {
-    return actingbackfill.count(osd);
+  bool is_acting_recovery_backfill(pg_shard_t osd) const {
+    return acting_recovery_backfill.count(osd);
   }
   bool is_acting(pg_shard_t osd) const {
     return has_shard(pool.info.is_erasure(), acting, osd);
@@ -1334,9 +1336,9 @@ protected:
 
   bool calc_min_last_complete_ondisk() {
     eversion_t min = last_complete_ondisk;
-    assert(!actingbackfill.empty());
-    for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-        i != actingbackfill.end();
+    assert(!acting_recovery_backfill.empty());
+    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+        i != acting_recovery_backfill.end();
         ++i) {
       if (*i == get_primary()) continue;
       if (peer_last_complete_ondisk.count(*i) == 0)
@@ -2930,7 +2932,7 @@ protected:
 
   /**
    * Merge entries updating missing as necessary on all
-   * actingbackfill logs and missings (also missing_loc)
+   * acting_recovery_backfill logs and missings (also missing_loc)
    */
   void merge_new_log_entries(
     const mempool::osd_pglog::list<pg_log_entry_t> &entries,
index 8b610a4d84ebf8e96c3c3077bdc260734051b634..da87d2fd66b39016db59a6d1264c47c1f9b908a9 100644 (file)
@@ -42,8 +42,8 @@ static ostream& _prefix(std::ostream *_dout, PGBackend *pgb) {
 void PGBackend::recover_delete_object(const hobject_t &oid, eversion_t v,
                                      RecoveryHandle *h)
 {
-  assert(get_parent()->get_actingbackfill_shards().size() > 0);
-  for (const auto& shard : get_parent()->get_actingbackfill_shards()) {
+  assert(get_parent()->get_acting_recovery_backfill_shards().size() > 0);
+  for (const auto& shard : get_parent()->get_acting_recovery_backfill_shards()) {
     if (shard == get_parent()->whoami_shard())
       continue;
     if (get_parent()->get_shard_missing(shard).is_missing(oid)) {
@@ -159,7 +159,7 @@ void PGBackend::handle_recovery_delete_reply(OpRequestRef op)
     recovery_info.version = p.second;
     get_parent()->on_peer_recover(m->from, oid, recovery_info);
     bool peers_recovered = true;
-    for (const auto& shard : get_parent()->get_actingbackfill_shards()) {
+    for (const auto& shard : get_parent()->get_acting_recovery_backfill_shards()) {
       if (shard == get_parent()->whoami_shard())
        continue;
       if (get_parent()->get_shard_missing(shard).is_missing(oid)) {
index 68da2e89b6fd0c0389a0e1b92bd46b16e4e0805e..26fda688e8ee3ef1c17adeeb6014399f50398279 100644 (file)
@@ -156,7 +156,7 @@ typedef ceph::shared_ptr<const OSDMap> OSDMapRef;
      virtual epoch_t get_interval_start_epoch() const = 0;
      virtual epoch_t get_last_peering_reset_epoch() const = 0;
 
-     virtual const set<pg_shard_t> &get_actingbackfill_shards() const = 0;
+     virtual const set<pg_shard_t> &get_acting_recovery_backfill_shards() const = 0;
      virtual const set<pg_shard_t> &get_acting_shards() const = 0;
      virtual const set<pg_shard_t> &get_backfill_shards() const = 0;
 
index 021f0d9756d662a58422117f3b1d8fa50277726d..c9811373be4aae047deb5777fb688c811b0d9d5a 100644 (file)
@@ -590,9 +590,9 @@ bool PrimaryLogPG::is_degraded_or_backfilling_object(const hobject_t& soid)
     return true;
   if (pg_log.get_missing().get_items().count(soid))
     return true;
-  assert(!actingbackfill.empty());
-  for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-       i != actingbackfill.end();
+  assert(!acting_recovery_backfill.empty());
+  for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+       i != acting_recovery_backfill.end();
        ++i) {
     if (*i == get_primary()) continue;
     pg_shard_t peer = *i;
@@ -711,9 +711,9 @@ void PrimaryLogPG::maybe_force_recovery()
     min_version = pg_log.get_missing().get_rmissing().begin()->first;
     soid = pg_log.get_missing().get_rmissing().begin()->second;
   }
-  assert(!actingbackfill.empty());
-  for (set<pg_shard_t>::iterator it = actingbackfill.begin();
-       it != actingbackfill.end();
+  assert(!acting_recovery_backfill.empty());
+  for (set<pg_shard_t>::iterator it = acting_recovery_backfill.begin();
+       it != acting_recovery_backfill.end();
        ++it) {
     if (*it == get_primary()) continue;
     pg_shard_t peer = *it;
@@ -940,10 +940,10 @@ int PrimaryLogPG::do_command(
         f->dump_stream("shard") << *p;
       f->close_section();
     }
-    if (!actingbackfill.empty()) {
-      f->open_array_section("actingbackfill");
-      for (set<pg_shard_t>::iterator p = actingbackfill.begin();
-          p != actingbackfill.end();
+    if (!acting_recovery_backfill.empty()) {
+      f->open_array_section("acting_recovery_backfill");
+      for (set<pg_shard_t>::iterator p = acting_recovery_backfill.begin();
+          p != acting_recovery_backfill.end();
           ++p)
         f->dump_stream("shard") << *p;
       f->close_section();
@@ -9995,8 +9995,8 @@ void PrimaryLogPG::issue_repop(RepGather *repop, OpContext *ctx)
 
   repop->v = ctx->at_version;
   if (ctx->at_version > eversion_t()) {
-    for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-        i != actingbackfill.end();
+    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+        i != acting_recovery_backfill.end();
         ++i) {
       if (*i == get_primary()) continue;
       pg_info_t &pinfo = peer_info[*i];
@@ -10163,8 +10163,8 @@ void PrimaryLogPG::submit_log_entries(
 
 
       set<pg_shard_t> waiting_on;
-      for (set<pg_shard_t>::const_iterator i = actingbackfill.begin();
-          i != actingbackfill.end();
+      for (set<pg_shard_t>::const_iterator i = acting_recovery_backfill.begin();
+          i != acting_recovery_backfill.end();
           ++i) {
        pg_shard_t peer(*i);
        if (peer == pg_whoami) continue;
@@ -10907,7 +10907,7 @@ int PrimaryLogPG::recover_missing(
        lock();
        if (!pg_has_reset_since(cur_epoch)) {
         bool object_missing = false;
-        for (const auto& shard : actingbackfill) {
+        for (const auto& shard : acting_recovery_backfill) {
           if (shard == pg_whoami)
             continue;
           if (peer_missing[shard].is_missing(soid)) {
@@ -11144,9 +11144,9 @@ eversion_t PrimaryLogPG::pick_newest_available(const hobject_t& oid)
   v = pmi.have;
   dout(10) << "pick_newest_available " << oid << " " << v << " on osd." << osd->whoami << " (local)" << dendl;
 
-  assert(!actingbackfill.empty());
-  for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-       i != actingbackfill.end();
+  assert(!acting_recovery_backfill.empty());
+  for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+       i != acting_recovery_backfill.end();
        ++i) {
     if (*i == get_primary()) continue;
     pg_shard_t peer = *i;
@@ -12172,9 +12172,9 @@ bool PrimaryLogPG::primary_error(
   pg_log.set_last_requested(0);
   missing_loc.remove_location(soid, pg_whoami);
   bool uhoh = true;
-  assert(!actingbackfill.empty());
-  for (set<pg_shard_t>::iterator i = actingbackfill.begin();
-       i != actingbackfill.end();
+  assert(!acting_recovery_backfill.empty());
+  for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+       i != acting_recovery_backfill.end();
        ++i) {
     if (*i == get_primary()) continue;
     pg_shard_t peer = *i;
@@ -12263,12 +12263,12 @@ uint64_t PrimaryLogPG::recover_replicas(uint64_t max, ThreadPool::TPHandle &hand
   PGBackend::RecoveryHandle *h = pgbackend->open_recovery_op();
 
   // this is FAR from an optimal recovery order.  pretty lame, really.
-  assert(!actingbackfill.empty());
+  assert(!acting_recovery_backfill.empty());
   // choose replicas to recover, replica has the shortest missing list first
   // so we can bring it back to normal ASAP
   std::vector<std::pair<unsigned int, pg_shard_t>> replicas_by_num_missing;
-  replicas_by_num_missing.reserve(actingbackfill.size() - 1);
-  for (auto &p: actingbackfill) {
+  replicas_by_num_missing.reserve(acting_recovery_backfill.size() - 1);
+  for (auto &p: acting_recovery_backfill) {
     if (p == get_primary()) {
       continue;
     }
@@ -14587,7 +14587,7 @@ int PrimaryLogPG::rep_repair_primary_object(const hobject_t& soid, OpRequestRef
   assert(is_primary());
 
   dout(10) << __func__ << " " << soid
-          << " peers osd.{" << actingbackfill << "}" << dendl;
+          << " peers osd.{" << acting_recovery_backfill << "}" << dendl;
 
   if (!is_clean()) {
     block_for_clean(soid, op);
index 90713db5365b45b5c494750d52291e791dd08b54..3443185c8659a09fe75d0ea9f4014310d9430642 100644 (file)
@@ -325,8 +325,8 @@ public:
   epoch_t get_last_peering_reset_epoch() const override {
     return get_last_peering_reset();
   }
-  const set<pg_shard_t> &get_actingbackfill_shards() const override {
-    return actingbackfill;
+  const set<pg_shard_t> &get_acting_recovery_backfill_shards() const override {
+    return acting_recovery_backfill;
   }
   const set<pg_shard_t> &get_acting_shards() const override {
     return actingset;
@@ -878,7 +878,7 @@ protected:
   void simple_opc_submit(OpContextUPtr ctx);
 
   /**
-   * Merge entries atomically into all actingbackfill osds
+   * Merge entries atomically into all acting_recovery_backfill osds
    * adjusting missing and recovery state as necessary.
    *
    * Also used to store error log entries for dup detection.
index e786c5003313c7c79e4b18b9b6b7e5c21a873036..6ed7bbc61ad6b7cb56a38782576128b5a0bceaa1 100644 (file)
@@ -462,8 +462,8 @@ void ReplicatedBackend::submit_transaction(
   InProgressOp &op = insert_res.first->second;
 
   op.waiting_for_commit.insert(
-    parent->get_actingbackfill_shards().begin(),
-    parent->get_actingbackfill_shards().end());
+    parent->get_acting_recovery_backfill_shards().begin(),
+    parent->get_acting_recovery_backfill_shards().end());
 
   issue_op(
     soid,
@@ -949,11 +949,11 @@ void ReplicatedBackend::issue_op(
   InProgressOp *op,
   ObjectStore::Transaction &op_t)
 {
-  if (parent->get_actingbackfill_shards().size() > 1) {
+  if (parent->get_acting_recovery_backfill_shards().size() > 1) {
     if (op->op) {
       op->op->pg_trace.event("issue replication ops");
       ostringstream ss;
-      set<pg_shard_t> replicas = parent->get_actingbackfill_shards();
+      set<pg_shard_t> replicas = parent->get_acting_recovery_backfill_shards();
       replicas.erase(parent->whoami_shard());
       ss << "waiting for subops from " << replicas;
       op->op->mark_sub_op_sent(ss.str());
@@ -963,7 +963,7 @@ void ReplicatedBackend::issue_op(
     bufferlist logs;
     encode(log_entries, logs);
 
-    for (const auto& shard : get_parent()->get_actingbackfill_shards()) {
+    for (const auto& shard : get_parent()->get_acting_recovery_backfill_shards()) {
       if (shard == parent->whoami_shard()) continue;
       const pg_info_t &pinfo = parent->get_shard_info().find(shard)->second;
 
@@ -2153,10 +2153,10 @@ int ReplicatedBackend::start_pushes(
 
   dout(20) << __func__ << " soid " << soid << dendl;
   // who needs it?
-  assert(get_parent()->get_actingbackfill_shards().size() > 0);
+  assert(get_parent()->get_acting_recovery_backfill_shards().size() > 0);
   for (set<pg_shard_t>::iterator i =
-        get_parent()->get_actingbackfill_shards().begin();
-       i != get_parent()->get_actingbackfill_shards().end();
+        get_parent()->get_acting_recovery_backfill_shards().begin();
+       i != get_parent()->get_acting_recovery_backfill_shards().end();
        ++i) {
     if (*i == get_parent()->whoami_shard()) continue;
     pg_shard_t peer = *i;