]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: minor fixes split from main "scrub backend" commit
authorRonen Friedman <rfriedma@redhat.com>
Wed, 1 Dec 2021 11:37:04 +0000 (11:37 +0000)
committerRonen Friedman <rfriedma@redhat.com>
Sat, 22 Jan 2022 08:59:59 +0000 (08:59 +0000)
Reformatting, minor compiler warnings, etc.
Split from "osd/scrub: extract scrub backend code into separate classes"

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
16 files changed:
src/common/snap_types.h
src/osd/ECBackend.cc
src/osd/OSD.cc
src/osd/PG.h
src/osd/PrimaryLogPG.cc
src/osd/osd_types_fmt.h
src/osd/scheduler/OpSchedulerItem.h
src/osd/scrubber/PrimaryLogScrub.cc
src/osd/scrubber/PrimaryLogScrub.h
src/osd/scrubber/osd_scrub_sched.cc
src/osd/scrubber/pg_scrubber.cc
src/osd/scrubber/pg_scrubber.h
src/osd/scrubber/scrub_backend.cc
src/osd/scrubber/scrub_backend.h
src/osd/scrubber/scrub_machine.cc
src/osd/scrubber/scrub_machine.h

index 958aea339a183ec92cb95d14985d433ed908f012..1705f8834cd2c469acf56f979d41d2415a7dd38a 100644 (file)
@@ -25,7 +25,7 @@ struct SnapRealmInfo {
     h.seq = seq_;
     h.parent_since = current_parent_since_;
   }
-  
+
   inodeno_t ino() const { return inodeno_t(h.ino); }
   inodeno_t parent() const { return inodeno_t(h.parent); }
   snapid_t seq() const { return snapid_t(h.seq); }
@@ -74,6 +74,4 @@ inline std::ostream& operator<<(std::ostream& out, const SnapContext& snapc) {
   return out << snapc.seq << "=" << snapc.snaps;
 }
 
-//}
-
 #endif
index b061e0f69ca14160bff2947ffb7bad2740b17e1c..06c75049f226a81486633e31350ee60fa97657cf 100644 (file)
@@ -381,7 +381,7 @@ void ECBackend::handle_recovery_push(
     if ((get_parent()->pgb_is_primary())) {
       ceph_assert(recovery_ops.count(op.soid));
       ceph_assert(recovery_ops[op.soid].obc);
-      if (get_parent()->pg_is_repair())
+      if (get_parent()->pg_is_repair() || is_repair)
         get_parent()->inc_osd_stat_repaired();
       get_parent()->on_local_recover(
        op.soid,
index 8b3205a2160ace38febaa0c4a65376e81f0e72fb..39447c07cda5dd9b2a523d8d3e8805f96166d2f3 100644 (file)
@@ -7473,7 +7473,8 @@ bool OSD::scrub_random_backoff()
   bool coin_flip = (rand() / (double)RAND_MAX >=
                    cct->_conf->osd_scrub_backoff_ratio);
   if (!coin_flip) {
-    dout(20) << "scrub_random_backoff lost coin flip, randomly backing off" << dendl;
+    dout(20) << "scrub_random_backoff lost coin flip, randomly backing off (ratio: "
+            << cct->_conf->osd_scrub_backoff_ratio << ")" << dendl;
     return true;
   }
   return false;
index a859e41b97e12fb6e1e4547d7986a77624e13df1..2bd4a5af067b48f47cf3d36200795e8f01ebd389 100644 (file)
@@ -54,8 +54,8 @@
 
 class OSD;
 class OSDService;
-class OSDShard;
-class OSDShardPGSlot;
+struct OSDShard;
+struct OSDShardPGSlot;
 
 class PG;
 struct OpRequest;
index b4b8f00ba8b1cb08c1c3bf34b295c62782e83703..e91213bdee92168004feae70291f62581397b240 100644 (file)
@@ -14,8 +14,6 @@
  * Foundation.  See file COPYING.
  *
  */
-#include "PrimaryLogPG.h"
-
 #include <errno.h>
 
 #include <charconv>
@@ -6443,12 +6441,13 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
       {
        uint64_t ver = op.assert_ver.ver;
        tracepoint(osd, do_osd_op_pre_assert_ver, soid.oid.name.c_str(), soid.snap.val, ver);
-       if (!ver)
+       if (!ver) {
          result = -EINVAL;
-        else if (ver < oi.user_version)
+        } else if (ver < oi.user_version) {
          result = -ERANGE;
-       else if (ver > oi.user_version)
+        } else if (ver > oi.user_version) {
          result = -EOVERFLOW;
+        }
       }
       break;
 
@@ -12278,6 +12277,8 @@ int PrimaryLogPG::recover_missing(
   int priority,
   PGBackend::RecoveryHandle *h)
 {
+  dout(10) << __func__ << " sar: " << scrub_after_recovery << dendl;
+
   if (recovery_state.get_missing_loc().is_unfound(soid)) {
     dout(7) << __func__ << " " << soid
            << " v " << v
index 3bca80f40f58918d36921976d3a9adae5944954d..f210425c6d15d259162f762d2d69f144b28974f0 100644 (file)
@@ -105,3 +105,30 @@ struct fmt::formatter<object_info_t> {
     return fmt::format_to(ctx.out(), ")");
   }
 };
+
+template <>
+struct fmt::formatter<pg_t> {
+  constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+  template <typename FormatContext>
+  auto format(const pg_t& pg, FormatContext& ctx)
+  {
+    return fmt::format_to(ctx.out(), "{}.{}", pg.pool(), pg.m_seed);
+  }
+};
+
+
+template <>
+struct fmt::formatter<spg_t> {
+  constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+  template <typename FormatContext>
+  auto format(const spg_t& spg, FormatContext& ctx)
+  {
+    if (shard_id_t::NO_SHARD == spg.shard.id) {
+      return fmt::format_to(ctx.out(), "{}", spg.pgid);
+    } else {
+      return fmt::format_to(ctx.out(), "{}s{}>", spg.pgid, spg.shard.id);
+    }
+  }
+};
index 68ee0d734c228416102e6a77305a6be8f6c26510..2a4c00d0dcd8f04d278eefad78cddd95e16fc527 100644 (file)
@@ -25,7 +25,7 @@
 
 
 class OSD;
-class OSDShard;
+struct OSDShard;
 
 namespace ceph::osd::scheduler {
 
index 4ba9221b010ddf8a6f7bd402dae4e14b6d6bba8d..53247cae4c5a2332e821c2dd1116ee4cfd7f3897 100644 (file)
@@ -1,7 +1,9 @@
 // -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 
-#include "PrimaryLogScrub.h"
+#include "./PrimaryLogScrub.h"
+
+#include <sstream>
 
 #include "common/scrub_types.h"
 #include "osd/osd_types_fmt.h"
@@ -15,8 +17,6 @@
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, this)
 
-using std::vector;
-
 template <class T>
 static ostream& _prefix(std::ostream* _dout, T* t)
 {
@@ -122,8 +122,9 @@ void PrimaryLogScrub::_scrub_finish()
       return false;
     });
 
-    if (m_pl_pg->agent_state)
+    if (m_pl_pg->agent_state) {
       m_pl_pg->agent_choose_mode();
+    }
   }
 
   dout(10) << m_mode_desc << " got " << m_scrub_cstat.sum.num_objects << "/"
@@ -164,6 +165,7 @@ void PrimaryLogScrub::_scrub_finish()
        !info.stats.manifest_stats_invalid) ||
       m_scrub_cstat.sum.num_whiteouts != info.stats.stats.sum.num_whiteouts ||
       m_scrub_cstat.sum.num_bytes != info.stats.stats.sum.num_bytes) {
+
     m_osds->clog->error() << info.pgid << " " << m_mode_desc << " : stat mismatch, got "
                          << m_scrub_cstat.sum.num_objects << "/"
                          << info.stats.stats.sum.num_objects << " objects, "
index b112fc6b9ac1e7789f1209364d0d2e51e7cf02ab..90d1a49adcc069515d75e76b7c6d16ec789699a1 100644 (file)
@@ -5,10 +5,6 @@
 // the './' includes are marked this way to affect clang-format
 #include "./pg_scrubber.h"
 
-#include <iostream>
-#include <sstream>
-#include <vector>
-
 #include "debug.h"
 
 #include "common/errno.h"
index 99f73b13164886e60e7daa3e909d9985ecf3d915..e34cdc9eff0302a4cdb78f8391cdc94c20baf371 100644 (file)
@@ -485,7 +485,7 @@ ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
 
   if (times.is_must == ScrubQueue::must_scrub_t::not_mandatory) {
 
-    // if not explicitly requested, postpone the scrub with a random delay
+    // unless explicitly requested, postpone the scrub with a random delay
     double scrub_min_interval = times.min_interval > 0
                                  ? times.min_interval
                                  : cct->_conf->osd_scrub_min_interval;
index 432c0c90d0d144d07f61a5f21aa60664e0cb6687..34d0763e7314a0293104116b2fd8b0be2cd80dcc 100644 (file)
@@ -23,7 +23,6 @@
 #include "scrub_machine.h"
 
 using std::list;
-using std::map;
 using std::pair;
 using std::set;
 using std::stringstream;
@@ -422,8 +421,8 @@ unsigned int PgScrubber::scrub_requeue_priority(Scrub::scrub_prio_t with_priorit
                                                unsigned int suggested_priority) const
 {
   if (with_priority == Scrub::scrub_prio_t::high_priority) {
-    suggested_priority = std::max(suggested_priority,
-                                 (unsigned int)m_pg->cct->_conf->osd_client_op_priority);
+    suggested_priority = std::max(
+      suggested_priority, (unsigned int)m_pg->get_cct()->_conf->osd_client_op_priority);
   }
   return suggested_priority;
 }
@@ -524,7 +523,7 @@ PgScrubber::determine_scrub_time(const requested_scrub_t& request_flags) const
     // we do not need the interval data in this case
 
   } else if (m_pg->info.stats.stats_invalid &&
-           m_pg->cct->_conf->osd_scrub_invalid_stats) {
+           m_pg->get_cct()->_conf->osd_scrub_invalid_stats) {
     res.proposed_time = ceph_clock_now();
     res.is_must = ScrubQueue::must_scrub_t::mandatory;
 
@@ -998,7 +997,8 @@ void PgScrubber::on_replica_init()
 int PgScrubber::build_primary_map_chunk()
 {
   epoch_t map_building_since = m_pg->get_osdmap_epoch();
-  dout(20) << __func__ << ": initiated at epoch " << map_building_since << dendl;
+  dout(20) << __func__ << ": initiated at epoch " << map_building_since
+           << dendl;
 
   auto ret = build_scrub_map_chunk(*m_primary_scrubmap,
                                    m_primary_scrubmap_pos,
@@ -1283,7 +1283,7 @@ void PgScrubber::send_preempted_replica()
                                  m_replica_min_epoch, m_pg_whoami);
 
   reply->preempted = true;
-  ::encode(replica_scrubmap, reply->get_data()); // must not skip this
+  ::encode(replica_scrubmap, reply->get_data());  // skipping this crashes the scrubber
   m_pg->send_cluster_message(m_pg->get_primary().osd, reply, m_replica_min_epoch, false);
 }
 
@@ -1576,6 +1576,9 @@ void PgScrubber::scrub_finish()
   // Since we don't know which errors were fixed, we can only clear them
   // when every one has been fixed.
   if (m_is_repair) {
+    dout(15) << fmt::format("{}: {} errors. {} errors fixed", __func__,
+                            m_shallow_errors + m_deep_errors, m_fixed_count)
+             << dendl;
     if (m_fixed_count == m_shallow_errors + m_deep_errors) {
 
       ceph_assert(m_is_deep);
@@ -1588,10 +1591,11 @@ void PgScrubber::scrub_finish()
       // Deep scrub in order to get corrected error counts
       m_pg->scrub_after_recovery = true;
       m_pg->m_planned_scrub.req_scrub =
-       m_pg->m_planned_scrub.req_scrub || m_flags.required;
+        m_pg->m_planned_scrub.req_scrub || m_flags.required;
 
       dout(20) << __func__ << " Current 'required': " << m_flags.required
-              << " Planned 'req_scrub': " << m_pg->m_planned_scrub.req_scrub << dendl;
+               << " Planned 'req_scrub': " << m_pg->m_planned_scrub.req_scrub
+               << dendl;
 
     } else if (m_shallow_errors || m_deep_errors) {
 
@@ -1599,7 +1603,7 @@ void PgScrubber::scrub_finish()
       // possible.
       state_set(PG_STATE_FAILED_REPAIR);
       dout(10) << __func__ << " " << (m_shallow_errors + m_deep_errors)
-              << " error(s) present with no repair possible" << dendl;
+               << " error(s) present with no repair possible" << dendl;
     }
   }
 
index 488e38b46c77e9460bbb66b29b02426f31e0be78..86d774fcd5a431649a5a7da086b0e208db5b643f 100644 (file)
@@ -155,7 +155,7 @@ class LocalReservation {
 };
 
 /**
- *  wraps the OSD resource we are using when reserved as a replica by a scrubbing master.
+ *  wraps the OSD resource we are using when reserved as a replica by a scrubbing primary.
  */
 class ReservedByRemotePrimary {
   const PgScrubber* m_scrubber; ///< we will be using its gen_prefix()
@@ -904,7 +904,7 @@ private:
 
    private:
     PG* m_pg;
-    mutable std::mutex m_preemption_lock;
+    mutable ceph::mutex m_preemption_lock;
     bool m_preemptable{false};
     bool m_preempted{false};
     int m_left;
index 3bdbf03b65dae81d7776f5c5fd7774eb3b7a7831..326dd5bab388e802e41349da89ad1757250577f9 100644 (file)
@@ -115,11 +115,11 @@ ScrubMap* ScrubBackend::new_chunk()
   return &this_chunk->received_maps[m_pg_whoami];
 }
 
-void ScrubBackend::merge_to_master_set()
+void ScrubBackend::merge_to_authoritative_set()
 {
   dout(15) << __func__ << dendl;
   ceph_assert(m_pg.is_primary());
-  ceph_assert(this_chunk->master_set.empty() &&
+  ceph_assert(this_chunk->authoritative_set.empty() &&
               "the scrubber-backend should be empty");
 
   if (g_conf()->subsys.should_gather<ceph_subsys_osd, 15>()) {
@@ -132,13 +132,13 @@ void ScrubBackend::merge_to_master_set()
     }
   }
 
-  // Construct the master set of objects
+  // Construct the authoritative set of objects
   for (const auto& map : this_chunk->received_maps) {
-    std::transform(
-      map.second.objects.begin(),
-      map.second.objects.end(),
-      std::inserter(this_chunk->master_set, this_chunk->master_set.end()),
-      [](const auto& i) { return i.first; });
+    std::transform(map.second.objects.begin(),
+                   map.second.objects.end(),
+                   std::inserter(this_chunk->authoritative_set,
+                                 this_chunk->authoritative_set.end()),
+                   [](const auto& i) { return i.first; });
   }
 }
 
@@ -187,7 +187,7 @@ void ScrubBackend::scrub_compare_maps(bool max_reached)
   // construct authoritative scrub map for type-specific scrubbing
 
   m_cleaned_meta_map.insert(my_map());
-  merge_to_master_set();
+  merge_to_authoritative_set();
 
   // collect some omap statistics into m_omap_stats
   omap_checks();
@@ -238,7 +238,7 @@ void ScrubBackend::omap_checks()
   stringstream wss;
 
   // Iterate through objects and update omap stats
-  for (const auto& ho : this_chunk->master_set) {
+  for (const auto& ho : this_chunk->authoritative_set) {
 
     for (const auto& [srd, smap] : this_chunk->received_maps) {
       if (srd != m_pg.get_primary()) {
@@ -776,8 +776,7 @@ shard_as_auth_t ScrubBackend::possible_auth_shard(const hobject_t& obj,
                       &shard_info_wrapper::set_obj_size_info_mismatch)) {
 
     errstream << sep(err) << "candidate size " << smap_obj.size << " info size "
-              << m_pgbe.be_get_ondisk_size(oi.size)
-              << " mismatch";
+              << m_pgbe.be_get_ondisk_size(oi.size) << " mismatch";
   }
 
   std::optional<uint32_t> digest;
@@ -797,13 +796,14 @@ shard_as_auth_t ScrubBackend::possible_auth_shard(const hobject_t& obj,
 // re-implementation of PGBackend::be_compare_scrubmaps()
 std::optional<std::string> ScrubBackend::compare_smaps()
 {
-  dout(10) << __func__ << ": master-set #: " << this_chunk->master_set.size()
+  dout(10) << __func__
+           << ": authoritative-set #: " << this_chunk->authoritative_set.size()
            << dendl;
 
   std::stringstream errstream;
   std::for_each(
-    this_chunk->master_set.begin(),
-    this_chunk->master_set.end(),
+    this_chunk->authoritative_set.begin(),
+    this_chunk->authoritative_set.end(),
     [this, &errstream](const auto& ho) { compare_obj_in_maps(ho, errstream); });
 
   if (errstream.str().empty()) {
@@ -892,12 +892,12 @@ void ScrubBackend::compare_obj_in_maps(const hobject_t& ho,
 }
 
 
-std::optional<ScrubBackend::auth_and_obj_errs_t> ScrubBackend::for_empty_auth_list(
-  std::list<pg_shard_t>&& auths,
-  std::set<pg_shard_t>&& obj_errors,
-  shard_to_scrubmap_t::iterator auth,
-  const hobject_t& ho,
-  stringstream& errstream)
+std::optional<ScrubBackend::auth_and_obj_errs_t>
+ScrubBackend::for_empty_auth_list(std::list<pg_shard_t>&& auths,
+                                  std::set<pg_shard_t>&& obj_errors,
+                                  shard_to_scrubmap_t::iterator auth,
+                                  const hobject_t& ho,
+                                  stringstream& errstream)
 {
   if (auths.empty()) {
     if (obj_errors.empty()) {
@@ -919,7 +919,7 @@ std::optional<ScrubBackend::auth_and_obj_errs_t> ScrubBackend::for_empty_auth_li
   }
 
   return ScrubBackend::auth_and_obj_errs_t{std::move(auths),
-                                        std::move(obj_errors)};
+                                           std::move(obj_errors)};
 }
 
 
@@ -1900,7 +1900,7 @@ void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
       ceph::condition_variable my_cond;
       ceph::mutex my_lock = ceph::make_mutex("PG::_scan_snaps my_lock");
       int e = 0;
-      bool done;
+      bool done;  // note: initialized to 'false' by C_SafeCond
 
       t.register_on_applied_sync(new C_SafeCond(my_lock, my_cond, &done, &e));
 
@@ -1911,7 +1911,9 @@ void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
       } 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;
     }
   }
 }
index cc11affd1c0a862c16f1a9cdc4698be7bd5dcbc0..30a685dfbcaffae6072b028a52ad9e2c05dfcb74 100644 (file)
@@ -167,7 +167,7 @@ struct scrub_chunk_t {
   std::map<pg_shard_t, ScrubMap> received_maps;
 
   /// a collection of all objs mentioned in the maps
-  std::set<hobject_t> master_set;
+  std::set<hobject_t> authoritative_set;
 
   utime_t started{ceph_clock_now()};
 
@@ -269,7 +269,15 @@ class ScrubBackend {
   ConfigProxy& m_conf;
   LogChannelRef clog;
 
+  int num_digest_updates_pending{0};
+
+ public:  
+  // as used by PgScrubber::final_cstat_update(). consider relocating.
+  // actually - only filled in by the PG backend, and used by the scrubber.
+  // We are not handling it. So consider getting it from the Scrubber, or
+  // creating it by the PG-BE
+  omap_stat_t m_omap_stats = (const struct omap_stat_t){0};
+
  private:
   using auth_and_obj_errs_t =
     std::tuple<std::list<pg_shard_t>,  ///< the auth-list
@@ -291,11 +299,11 @@ class ScrubBackend {
   ScrubMap& my_map();
 
   /**
-   *  merge_to_master_set() updates
+   *  merge_to_authoritative_set() updates
    *   - this_chunk->maps[from] with the replicas' scrub-maps;
-   *   - this_chunk->master_set as a union of all the maps' objects;
+   *   - this_chunk->authoritative_set as a union of all the maps' objects;
    */
-  void merge_to_master_set();
+  void merge_to_authoritative_set();
 
   // note: used by both Primary & replicas
   static ScrubMap clean_meta_map(ScrubMap& cleaned, bool max_reached);
index 2610bf351a15029e0d2f79375197178d06072701..ba2512735d625d511f23da11b24cd7c7926337eb 100644 (file)
@@ -19,7 +19,6 @@
 
 using namespace std::chrono;
 using namespace std::chrono_literals;
-namespace sc = boost::statechart;
 
 #define DECLARE_LOCALS                                           \
   ScrubMachineListener* scrbr = context<ScrubMachine>().m_scrbr; \
@@ -435,6 +434,12 @@ sc::result WaitReplicas::react(const GotReplicas&)
   }
 }
 
+sc::result WaitReplicas::react(const DigestUpdate&)
+{
+  dout(10) << "WaitReplicas::react(const DigestUpdate&) - too early" << dendl;
+  return discard_event();
+}
+
 // ----------------------- WaitDigestUpdate -----------------------------------
 
 WaitDigestUpdate::WaitDigestUpdate(my_context ctx) : my_base(ctx)
index d9132eedca8581ada060f917cc491160c6667ba1..41669138a9cc2fb6dedc0a343439014537092629 100644 (file)
@@ -299,11 +299,11 @@ struct WaitReplicas : sc::state<WaitReplicas, ActiveScrubbing> {
   using reactions =
     mpl::list<sc::custom_reaction<GotReplicas>,         // all replicas are accounted for
              sc::transition<MapsCompared, WaitDigestUpdate>,
-             sc::deferral<DigestUpdate>  // might arrive before we've reached WDU
+             sc::custom_reaction<DigestUpdate>
              >;
 
   sc::result react(const GotReplicas&);
-
+  sc::result react(const DigestUpdate&);
   bool all_maps_already_called{false}; // see comment in react code
 };