]> git.apps.os.sepia.ceph.com Git - ceph-ci.git/commitdiff
osd/scrub: remove I/O (store) direct access from the Scrubber Backend
authorRonen Friedman <rfriedma@redhat.com>
Mon, 31 Jan 2022 11:29:51 +0000 (11:29 +0000)
committerRonen Friedman <rfriedma@redhat.com>
Thu, 24 Feb 2022 14:26:37 +0000 (14:26 +0000)
Target: improving Scrubber BE testability by reducing the number and
complexity of the external APIs it uses.

(note - first step of many)

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
src/osd/scrubber/ScrubStore.cc
src/osd/scrubber/ScrubStore.h
src/osd/scrubber/pg_scrubber.cc
src/osd/scrubber/pg_scrubber.h
src/osd/scrubber/scrub_backend.cc
src/osd/scrubber/scrub_backend.h

index 1787b3d8875ae86aa886b407c26466a7474e029c..a00ab2caecee6bfc96900daf162ac2a9b932b91d 100644 (file)
@@ -124,6 +124,11 @@ Store::~Store()
   ceph_assert(results.empty());
 }
 
+void Store::add_error(int64_t pool, const inconsistent_obj_wrapper& e)
+{
+  add_object_error(pool, e);
+}
+
 void Store::add_object_error(int64_t pool, const inconsistent_obj_wrapper& e)
 {
   bufferlist bl;
@@ -131,6 +136,11 @@ void Store::add_object_error(int64_t pool, const inconsistent_obj_wrapper& e)
   results[to_object_key(pool, e.object)] = bl;
 }
 
+void Store::add_error(int64_t pool, const inconsistent_snapset_wrapper& e)
+{
+  add_snap_error(pool, e);
+}
+
 void Store::add_snap_error(int64_t pool, const inconsistent_snapset_wrapper& e)
 {
   bufferlist bl;
index 57cd0e852d5cb4a095ab18bd2eff2867220f3a30..f3b5b5d98ddbc98e9bfbf4591eee8bfe3ae09332 100644 (file)
@@ -25,6 +25,11 @@ public:
                       const coll_t& coll);
   void add_object_error(int64_t pool, const inconsistent_obj_wrapper& e);
   void add_snap_error(int64_t pool, const inconsistent_snapset_wrapper& e);
+
+  // and a variant-friendly interface:
+  void add_error(int64_t pool, const inconsistent_obj_wrapper& e);
+  void add_error(int64_t pool, const inconsistent_snapset_wrapper& e);
+
   bool empty() const;
   void flush(ObjectStore::Transaction *);
   void cleanup(ObjectStore::Transaction *);
index af1876badfadd5628ce88a39d18cb2c65f81eb4b..59db35e0910ad4ecc6bf316882aef065c56ae8dd 100644 (file)
@@ -25,7 +25,6 @@
 
 using std::list;
 using std::pair;
-using std::set;
 using std::stringstream;
 using std::vector;
 using namespace Scrub;
@@ -1039,13 +1038,16 @@ int PgScrubber::build_replica_map_chunk()
     case 0: {
       // finished!
 
-      m_be->replica_clean_meta(replica_scrubmap, m_end.is_max(), m_start);
+      auto required_fixes = m_be->replica_clean_meta(
+        replica_scrubmap, m_end.is_max(), m_start, *this);
+      // actuate snap-mapper changes:
+      apply_snap_mapper_fixes(required_fixes);
 
       // the local map has been created. Send it to the primary.
       // Note: once the message reaches the Primary, it may ask us for another
-      // chunk - and we better be done with the current scrub. Thus - the preparation of
-      // the reply message is separate, and we clear the scrub state before actually
-      // sending it.
+      // chunk - and we better be done with the current scrub. Thus - the
+      // preparation of the reply message is separate, and we clear the scrub
+      // state before actually sending it.
 
       auto reply = prep_replica_map_msg(PreemptionNoted::no_preemption);
       replica_handling_done();
@@ -1129,10 +1131,107 @@ void PgScrubber::run_callbacks()
   }
 }
 
+void PgScrubber::persist_scrub_results(inconsistent_objs_t&& all_errors)
+{
+  dout(10) << __func__ << " " << all_errors.size() << " errors" << dendl;
+
+  for (auto& e : all_errors) {
+    std::visit([this](auto& e) { m_store->add_error(m_pg->pool.id, e); }, e);
+  }
+
+  ObjectStore::Transaction t;
+  m_store->flush(&t);
+  m_osds->store->queue_transaction(m_pg->ch, std::move(t), nullptr);
+}
+
+void PgScrubber::apply_snap_mapper_fixes(
+  const std::vector<snap_mapper_fix_t>& fix_list)
+{
+  dout(15) << __func__ << " " << fix_list.size() << " fixes" << dendl;
+
+  if (fix_list.empty()) {
+    return;
+  }
+
+  ObjectStore::Transaction t;
+  OSDriver::OSTransaction t_drv(m_pg->osdriver.get_transaction(&t));
+
+  for (auto& [fix_op, hoid, snaps, bogus_snaps] : fix_list) {
+
+    if (fix_op == snap_mapper_op_t::update) {
+
+      // must remove the existing snap-set before inserting the correct one
+      if (auto r = m_pg->snap_mapper.remove_oid(hoid, &t_drv); r < 0) {
+
+        derr << __func__ << ": remove_oid returned " << cpp_strerror(r)
+             << dendl;
+        ceph_abort();
+      }
+
+      m_osds->clog->error() << fmt::format(
+        "osd.{} found snap mapper error on pg {} oid {} snaps in mapper: {}, "
+        "oi: "
+        "{} ...repaired",
+        m_pg_whoami, m_pg_id, hoid, bogus_snaps, snaps);
+
+    } else {
+
+      m_osds->clog->error() << fmt::format(
+        "osd.{} found snap mapper error on pg {} oid {} snaps missing in "
+        "mapper, should be: {} ...repaired",
+        m_pg_whoami, m_pg_id, hoid, snaps);
+    }
+
+    // now - insert the correct snap-set
+
+    m_pg->snap_mapper.add_oid(hoid, snaps, &t_drv);
+  }
+
+  // wait for repair to apply to avoid confusing other bits of the system.
+  {
+    dout(15) << __func__ << " wait on repair!" << dendl;
+
+    ceph::condition_variable my_cond;
+    ceph::mutex my_lock = ceph::make_mutex("PG::_scan_snaps my_lock");
+    int e = 0;
+    bool done{false};
+
+    t.register_on_applied_sync(new C_SafeCond(my_lock, my_cond, &done, &e));
+
+    if (e = m_pg->osd->store->queue_transaction(m_pg->ch, std::move(t));
+        e != 0) {
+      derr << __func__ << ": queue_transaction got " << cpp_strerror(e)
+           << dendl;
+    } else {
+      std::unique_lock l{my_lock};
+      my_cond.wait(l, [&done] { return done; });
+      ceph_assert(m_pg->osd->store);  // RRR why?
+    }
+    dout(15) << __func__ << " wait on repair - done" << dendl;
+  }
+}
+
 void PgScrubber::maps_compare_n_cleanup()
 {
   m_pg->add_objects_scrubbed_count(m_be->get_primary_scrubmap().objects.size());
-  m_be->scrub_compare_maps(m_end.is_max());
+
+  auto required_fixes = m_be->scrub_compare_maps(m_end.is_max(), *this);
+  if (!required_fixes.inconsistent_objs.empty()) {
+    if (state_test(PG_STATE_REPAIR)) {
+      dout(10) << __func__ << ": discarding scrub results (repairing)" << dendl;
+    } else {
+      // perform the ordered scrub-store I/O:
+      persist_scrub_results(std::move(required_fixes.inconsistent_objs));
+    }
+  }
+
+  // actuate snap-mapper changes:
+  apply_snap_mapper_fixes(required_fixes.snap_fix_list);
+
+  auto chunk_err_counts = m_be->get_error_counts();
+  m_shallow_errors += chunk_err_counts.shallow_errors;
+  m_deep_errors += chunk_err_counts.deep_errors;
+
   m_start = m_end;
   run_callbacks();
   requeue_waiting();
index 4263e24603eb2f3c34e41f8f8303093bd6b39924..e9df04d6d0ffdce3f19e458e128efaf5297d17dc 100644 (file)
@@ -255,8 +255,9 @@ ostream& operator<<(ostream& out, const scrub_flags_t& sf);
  * am forced to strongly decouple the state-machine implementation details from
  * the actual scrubbing code.
  */
-class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
-
+class PgScrubber : public ScrubPgIF,
+                   public ScrubMachineListener,
+                   public SnapMapperAccessor {
  public:
   explicit PgScrubber(PG* pg);
 
@@ -509,6 +510,12 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
   utime_t scrub_begin_stamp;
   std::ostream& gen_prefix(std::ostream& out) const final;
 
+  //  fetching the snap-set for a given object (used by the scrub-backend)
+  int get_snaps(const hobject_t& hoid, std::set<snapid_t>* snaps_set) const final
+  {
+    return m_pg->snap_mapper.get_snaps(hoid, snaps_set);
+  }
+
  protected:
   bool state_test(uint64_t m) const { return m_pg->state_test(m); }
   void state_set(uint64_t m) { m_pg->state_set(m); }
@@ -815,9 +822,8 @@ private:
 
   Scrub::MapsCollectionStatus m_maps_status;
 
-
-  /// Maps from object with errors to good peers
-  std::map<hobject_t, std::list<std::pair<ScrubMap::object, pg_shard_t>>> m_authoritative;
+  void persist_scrub_results(inconsistent_objs_t&& all_errors);
+  void apply_snap_mapper_fixes(const std::vector<snap_mapper_fix_t>& fix_list);
 
   // ------------ members used if we are a replica
 
index 46bc57f6ba6e06b03b2c3b8f127b95ea02b49440..d19c39782cab9c0e633a0d0f4c3e487cf9373d62 100644 (file)
@@ -163,9 +163,11 @@ void ScrubBackend::decode_received_map(pg_shard_t from,
 }
 
 
-void ScrubBackend::replica_clean_meta(ScrubMap& repl_map,
-                                      bool max_reached,
-                                      const hobject_t& start)
+std::vector<snap_mapper_fix_t> ScrubBackend::replica_clean_meta(
+  ScrubMap& repl_map,
+  bool max_reached,
+  const hobject_t& start,
+  SnapMapperAccessor& snaps_getter)
 {
   dout(15) << __func__ << ": REPL META # " << m_cleaned_meta_map.objects.size()
            << " objects" << dendl;
@@ -173,7 +175,7 @@ void ScrubBackend::replica_clean_meta(ScrubMap& repl_map,
   m_cleaned_meta_map.clear_from(start);  // RRR how can this be required?
   m_cleaned_meta_map.insert(repl_map);
   auto for_meta_scrub = clean_meta_map(m_cleaned_meta_map, max_reached);
-  scan_snaps(for_meta_scrub);
+  return scan_snaps(for_meta_scrub, snaps_getter);
 }
 
 
@@ -183,7 +185,9 @@ void ScrubBackend::replica_clean_meta(ScrubMap& repl_map,
 //
 // /////////////////////////////////////////////////////////////////////////////
 
-void ScrubBackend::scrub_compare_maps(bool max_reached)
+objs_fix_list_t ScrubBackend::scrub_compare_maps(
+  bool max_reached,
+  SnapMapperAccessor& snaps_getter)
 {
   dout(10) << __func__ << " has maps, analyzing" << dendl;
   ceph_assert(m_pg.is_primary());
@@ -204,26 +208,8 @@ void ScrubBackend::scrub_compare_maps(bool max_reached)
   // (Validates consistency of the object info and snap sets)
   scrub_snapshot_metadata(for_meta_scrub);
 
-  // Called here on the primary. Can use an authoritative map if it isn't the
-  // primary
-  scan_snaps(for_meta_scrub);
-
-  if (!m_scrubber.m_store->empty()) {
-
-    if (m_scrubber.state_test(PG_STATE_REPAIR)) {
-      dout(10) << __func__ << ": discarding scrub results" << dendl;
-      m_scrubber.m_store->flush(nullptr);
-
-    } else {
-
-      dout(10) << __func__ << ": updating scrub object" << dendl;
-      ObjectStore::Transaction t;
-      m_scrubber.m_store->flush(&t);
-      m_scrubber.m_osds->store->queue_transaction(m_pg.ch,
-                                                  std::move(t),
-                                                  nullptr);
-    }
-  }
+  return objs_fix_list_t{std::move(this_chunk->m_inconsistent_objs),
+                         scan_snaps(for_meta_scrub, snaps_getter)};
 }
 
 void ScrubBackend::omap_checks()
@@ -390,18 +376,13 @@ int ScrubBackend::scrub_process_inconsistent()
                              m_missing.size(),
                              m_inconsistent.size());
 
-  dout(2) << err_msg << dendl;
+  dout(4) << err_msg << dendl;
   clog->error() << err_msg;
 
+  ceph_assert(m_repair);
   int fixed_cnt{0};
-  if (m_repair) {
-    m_scrubber.state_clear(PG_STATE_CLEAN);
-    // we know we have a problem, so it's OK to set the user-visible flag
-    // even if we only reached here via auto-repair
-    m_scrubber.state_set(PG_STATE_REPAIR);
-    m_scrubber.update_op_mode_text();
 
-    for (const auto& [hobj, shrd_list] : m_scrubber.m_authoritative) {
+  for (const auto& [hobj, shrd_list] : m_auth_peers) {
 
       auto missing_entry = m_missing.find(hobj);
 
@@ -415,13 +396,12 @@ int ScrubBackend::scrub_process_inconsistent()
         fixed_cnt += m_inconsistent[hobj].size();
       }
     }
-  }
   return fixed_cnt;
 }
 
 void ScrubBackend::repair_object(
   const hobject_t& soid,
-  const list<pair<ScrubMap::object, pg_shard_t>>& ok_peers,
+  const auth_peers_t& ok_peers,
   const set<pg_shard_t>& bad_peers)
 {
   if (g_conf()->subsys.should_gather<ceph_subsys_osd, 20>()) {
@@ -886,8 +866,8 @@ std::optional<std::string> ScrubBackend::compare_obj_in_maps(const hobject_t& ho
     match_in_shards(ho, auth_res, object_error, errstream);
 
   auto opt_ers =
-    for_empty_auth_list(std::forward<std::list<pg_shard_t>>(auths),
-                        std::forward<std::set<pg_shard_t>>(objerrs),
+    for_empty_auth_list(std::move(auths),
+                        std::move(objerrs),
                         auth,
                         ho,
                         errstream);
@@ -911,7 +891,7 @@ std::optional<std::string> ScrubBackend::compare_obj_in_maps(const hobject_t& ho
   }
 
   if (object_error.errors || object_error.union_shards.errors) {
-    m_scrubber.m_store->add_object_error(ho.pool, object_error);
+    this_chunk->m_inconsistent_objs.push_back(std::move(object_error));
   }
 
   if (errstream.str().empty()) {
@@ -961,8 +941,8 @@ void ScrubBackend::inconsistents(const hobject_t& ho,
                                  auth_and_obj_errs_t&& auth_n_errs,
                                  stringstream& errstream)
 {
-  auto& object_errors = std::get<1>(auth_n_errs);
-  auto& auth_list = std::get<0>(auth_n_errs);
+  auto& object_errors = auth_n_errs.object_errors;
+  auto& auth_list = auth_n_errs.auth_list;
 
   this_chunk->cur_inconsistent.insert(object_errors.begin(),
                                       object_errors.end());  // merge?
@@ -1734,7 +1714,7 @@ void ScrubBackend::scrub_snapshot_metadata(ScrubMap& map)
       // what's next?
       ++curclone;
       if (soid_error.errors) {
-        m_scrubber.m_store->add_snap_error(pool.id, soid_error);
+        this_chunk->m_inconsistent_objs.push_back(std::move(soid_error));
         ++soid_error_count;
       }
     }
@@ -1759,8 +1739,9 @@ void ScrubBackend::scrub_snapshot_metadata(ScrubMap& map)
   if (missing) {
     log_missing(missing, head, __func__, allow_incomplete_clones);
   }
-  if (head && (head_error.errors || soid_error_count))
-    m_scrubber.m_store->add_snap_error(pool.id, head_error);
+  if (head && (head_error.errors || soid_error_count)) {
+    this_chunk->m_inconsistent_objs.push_back(std::move(head_error));
+  }
 
   // fix data/omap digests
   m_scrubber.submit_digest_fixes(this_chunk->missing_digest);
@@ -1821,8 +1802,11 @@ void ScrubBackend::log_missing(int missing,
 
 // ////////////////////////////////////////////////////////////////////////////////
 
-void ScrubBackend::scan_snaps(ScrubMap& smap)
+std::vector<snap_mapper_fix_t> ScrubBackend::scan_snaps(
+  ScrubMap& smap,
+  SnapMapperAccessor& snaps_getter)
 {
+  std::vector<snap_mapper_fix_t> out_orders;
   hobject_t head;
   SnapSet snapset;
 
@@ -1866,14 +1850,22 @@ void ScrubBackend::scan_snaps(ScrubMap& smap)
         continue;
       }
 
-      scan_object_snaps(hoid, o, snapset);
+      auto maybe_fix_order = scan_object_snaps(hoid, snapset, snaps_getter);
+      if (maybe_fix_order) {
+        out_orders.push_back(std::move(*maybe_fix_order));
+      }
+
     }
   }
+
+  dout(15) << __func__ << " " << out_orders.size() << " fix orders" << dendl;
+  return out_orders;
 }
 
-void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
-                                     ScrubMap::object& scrmap_obj,
-                                     const SnapSet& snapset)
+std::optional<snap_mapper_fix_t> ScrubBackend::scan_object_snaps(
+  const hobject_t& hoid,
+  const SnapSet& snapset,
+  SnapMapperAccessor& snaps_getter)
 {
   // check and if necessary fix snap_mapper
 
@@ -1881,65 +1873,27 @@ void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
   if (p == snapset.clone_snaps.end()) {
     derr << __func__ << " no clone_snaps for " << hoid << " in " << snapset
          << dendl;
-    return;
+    return std::nullopt;
   }
   set<snapid_t> obj_snaps{p->second.begin(), p->second.end()};
 
   set<snapid_t> cur_snaps;
-  int r = m_pg.snap_mapper.get_snaps(hoid, &cur_snaps);
+  int r = snaps_getter.get_snaps(hoid, &cur_snaps);
   if (r != 0 && r != -ENOENT) {
     derr << __func__ << ": get_snaps returned " << cpp_strerror(r) << dendl;
     ceph_abort();
   }
   if (r == -ENOENT || cur_snaps != obj_snaps) {
-    ObjectStore::Transaction t;
-    OSDriver::OSTransaction _t(m_pg.osdriver.get_transaction(&t));
-    if (r == 0) {
-      r = m_pg.snap_mapper.remove_oid(hoid, &_t);
-      if (r != 0) {
-        derr << __func__ << ": remove_oid returned " << cpp_strerror(r)
-             << dendl;
-        ceph_abort();
-      }
-      clog->error() << "osd." << m_pg_whoami
-                    << " found snap mapper error on pg " << m_pg_id << " oid "
-                    << hoid << " snaps in mapper: " << cur_snaps
-                    << ", oi: " << obj_snaps << "...repaired";
-    } else {
-      clog->error() << "osd." << m_pg_whoami
-                    << " found snap mapper error on pg " << m_pg_id << " oid "
-                    << hoid << " snaps missing in mapper"
-                    << ", should be: " << obj_snaps << " was " << cur_snaps
-                    << " r " << r << "...repaired";
-    }
-    m_pg.snap_mapper.add_oid(hoid, obj_snaps, &_t);
-
-    // wait for repair to apply to avoid confusing other bits of the system.
-    {
-      dout(15) << __func__ << " wait on repair!" << dendl;
 
-      ceph::condition_variable my_cond;
-      ceph::mutex my_lock = ceph::make_mutex("PG::_scan_snaps my_lock");
-      int e = 0;
-      bool done;  // note: initialized to 'false' by C_SafeCond
-
-      t.register_on_applied_sync(new C_SafeCond(my_lock, my_cond, &done, &e));
-
-      e = m_pg.osd->store->queue_transaction(m_pg.ch, std::move(t));
-      if (e != 0) {
-        derr << __func__ << ": queue_transaction got " << cpp_strerror(e)
-             << dendl;
-      } else {
-        std::unique_lock l{my_lock};
-        my_cond.wait(l, [&done] { return done; });
-        ceph_assert(m_pg.osd->store);
-      }
-      dout(15) << __func__ << " wait on repair - done" << dendl;
-    }
+    // add this object to the list of snapsets that needs fixing. Note
+    // that we also collect the existing (bogus) list, as legacy log lines show those
+    snap_mapper_op_t fixing_op =
+      (r == -ENOENT ? snap_mapper_op_t::add : snap_mapper_op_t::update);
+    return snap_mapper_fix_t{fixing_op, hoid, obj_snaps, cur_snaps};
   }
+  return std::nullopt;
 }
 
-
 /*
  * Process:
  * Building a map of objects suitable for snapshot validation.
index faecd6e3e7a8715243fd2c4e99b2eedeeb55aac0..c67a1a43accb2b998106409ffad26ec6e17fb394 100644 (file)
@@ -64,6 +64,11 @@ using digests_fixes_t = std::vector<std::pair<hobject_t, data_omap_digests_t>>;
 using shard_info_map_t = std::map<pg_shard_t, shard_info_wrapper>;
 using shard_to_scrubmap_t = std::map<pg_shard_t, ScrubMap>;
 
+using auth_peers_t = std::vector<std::pair<ScrubMap::object, pg_shard_t>>;
+
+using wrapped_err_t =
+  std::variant<inconsistent_obj_wrapper, inconsistent_snapset_wrapper>;
+using inconsistent_objs_t = std::vector<wrapped_err_t>;
 
 /// omap-specific stats
 struct omap_stat_t {
@@ -73,6 +78,43 @@ struct omap_stat_t {
 };
 
 
+/*
+ * snaps-related aux structures:
+ * the scrub-backend scans the snaps associated with each scrubbed object, and
+ * fixes corrupted snap-sets.
+ * The actual access to the PG's snap_mapper, and the actual I/O transactions,
+ * are performed by the main PgScrubber object.
+ * the following aux structures are used to facilitate the required exchanges:
+ * - pre-fix snap-sets are accessed by the scrub-backend, and:
+ * - a list of fix-orders (either insert or replace operations) are returned
+ */
+
+struct SnapMapperAccessor {
+  virtual int get_snaps(const hobject_t& hoid,
+                        std::set<snapid_t>* snaps_set) const = 0;
+  virtual ~SnapMapperAccessor() = default;
+};
+
+enum class snap_mapper_op_t {
+  add,
+  update,
+};
+
+struct snap_mapper_fix_t {
+  snap_mapper_op_t op;
+  hobject_t hoid;
+  std::set<snapid_t> snaps;
+  std::set<snapid_t> wrong_snaps;  // only collected & returned for logging sake
+};
+
+// and - as the main scrub-backend entry point - scrub_compare_maps() - must
+// be able to return both a list of snap fixes and a list of inconsistent
+// objects:
+struct objs_fix_list_t {
+  inconsistent_objs_t inconsistent_objs;
+  std::vector<snap_mapper_fix_t> snap_fix_list;
+};
+
 /**
  * A structure used internally by select_auth_object()
  *
@@ -150,8 +192,9 @@ struct fmt::formatter<shard_as_auth_t> {
       // note: 'if' chain, as hard to consistently (on all compilers) avoid some
       // warnings for a switch plus multiple return paths
       if (as_auth.possible_auth == shard_as_auth_t::usable_t::not_usable) {
-        return format_to(
-          ctx.out(), "{{shard-not-usable:{}}}", as_auth.error_text);
+        return format_to(ctx.out(),
+                         "{{shard-not-usable:{}}}",
+                         as_auth.error_text);
       }
       if (as_auth.possible_auth == shard_as_auth_t::usable_t::not_found) {
         return format_to(ctx.out(), "{{shard-not-found}}");
@@ -231,6 +274,8 @@ struct scrub_chunk_t {
   /// Map from object with errors to good peers
   std::map<hobject_t, std::list<pg_shard_t>> authoritative;
 
+  inconsistent_objs_t m_inconsistent_objs;
+
 
   // these must be reset for each element:
 
@@ -284,9 +329,11 @@ class ScrubBackend {
    */
   void update_repair_status(bool should_repair);
 
-  void replica_clean_meta(ScrubMap& smap,
-                          bool max_reached,
-                          const hobject_t& start);
+  std::vector<snap_mapper_fix_t> replica_clean_meta(
+    ScrubMap& smap,
+    bool max_reached,
+    const hobject_t& start,
+    SnapMapperAccessor& snaps_getter);
 
   /**
    * decode the arriving MOSDRepScrubMap message, placing the replica's
@@ -296,7 +343,8 @@ class ScrubBackend {
    */
   void decode_received_map(pg_shard_t from, const MOSDRepScrubMap& msg);
 
-  void scrub_compare_maps(bool max_reached);
+  objs_fix_list_t scrub_compare_maps(bool max_reached,
+                                     SnapMapperAccessor& snaps_getter);
 
   int scrub_process_inconsistent();
 
@@ -322,15 +370,19 @@ class ScrubBackend {
  /// collecting some scrub-session-wide omap stats
   omap_stat_t m_omap_stats;
 
+  /// Mapping from object with errors to good peers
+  std::map<hobject_t, auth_peers_t> m_auth_peers;
+
   // shorthands:
   ConfigProxy& m_conf;
   LogChannelRef clog;
 
  private:
-  using auth_and_obj_errs_t =
-    std::tuple<std::list<pg_shard_t>,  ///< the auth-list
-               std::set<pg_shard_t>    ///< object_errors
-               >;
+
+  struct auth_and_obj_errs_t {
+    std::list<pg_shard_t> auth_list;
+    std::set<pg_shard_t> object_errors;
+  };
 
   std::optional<scrub_chunk_t> this_chunk;
 
@@ -385,11 +437,9 @@ class ScrubBackend {
                            std::stringstream& errorstream,
                            bool has_snapset);
 
-
-  void repair_object(
-    const hobject_t& soid,
-    const std::list<std::pair<ScrubMap::object, pg_shard_t>>& ok_peers,
-    const std::set<pg_shard_t>& bad_peers);
+  void repair_object(const hobject_t& soid,
+                     const auth_peers_t& ok_peers,
+                     const std::set<pg_shard_t>& bad_peers);
 
   /**
    * An auxiliary used by select_auth_object() to test a specific shard
@@ -452,9 +502,22 @@ class ScrubBackend {
                    const char* logged_func_name,
                    bool allow_incomplete_clones);
 
-  void scan_snaps(ScrubMap& smap);
+  /**
+   * returns a list of snaps "fix orders"
+   */
+  std::vector<snap_mapper_fix_t> scan_snaps(
+    ScrubMap& smap,
+    SnapMapperAccessor& snaps_getter);
+
+  /**
+   * an aux used by scan_snaps(), possibly returning a fix-order
+   * for a specific hobject.
+   */
+  std::optional<snap_mapper_fix_t> scan_object_snaps(
+    const hobject_t& hoid,
+    const SnapSet& snapset,
+    SnapMapperAccessor& snaps_getter);
 
-  void scan_object_snaps(const hobject_t& hoid,
-                         ScrubMap::object& scrmap_obj,
-                         const SnapSet& snapset);
+  // accessing the PG backend for this translation service
+  uint64_t logical_to_ondisk_size(uint64_t logical_size) const;
 };