]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: modify after-repair-scrub triggering
authorRonen Friedman <rfriedma@redhat.com>
Sun, 28 Jul 2024 12:37:07 +0000 (07:37 -0500)
committerRonen Friedman <rfriedma@redhat.com>
Sun, 25 Aug 2024 13:01:00 +0000 (08:01 -0500)
... to manipulate the relevant scrub target directly, instead
of using the 'planned scrub' flags.

The relevant condition flag was moved from the PG and into the scrubber.

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
12 files changed:
src/osd/OSD.cc
src/osd/OSD.h
src/osd/PG.cc
src/osd/PG.h
src/osd/PrimaryLogPG.cc
src/osd/scheduler/OpSchedulerItem.cc
src/osd/scheduler/OpSchedulerItem.h
src/osd/scrubber/pg_scrubber.cc
src/osd/scrubber/pg_scrubber.h
src/osd/scrubber/scrub_machine.cc
src/osd/scrubber/scrub_machine.h
src/osd/scrubber_common.h

index 0d392d1b6a38d45232ef8c5acd26cfd97691459c..fb3a415a54204a6ca46cda56e46d1e804f4750a3 100644 (file)
@@ -1819,11 +1819,6 @@ void OSDService::queue_for_scrub(PG* pg, Scrub::scrub_prio_t with_priority)
   queue_scrub_event_msg_default_cost<PGScrub>(pg, with_priority);
 }
 
-void OSDService::queue_scrub_after_repair(PG* pg, Scrub::scrub_prio_t with_priority)
-{
-  queue_scrub_event_msg_default_cost<PGScrubAfterRepair>(pg, with_priority);
-}
-
 void OSDService::queue_for_rep_scrub(PG* pg,
                                     Scrub::scrub_prio_t with_priority,
                                     unsigned int qu_priority,
index 787827b8ff45b2518c5e6e9821f0c68c910780e0..7c9aed7c6ba764f761430989ff2e3dc430f13352 100644 (file)
@@ -514,8 +514,6 @@ public:
   void queue_for_snap_trim(PG *pg, uint64_t cost);
   void queue_for_scrub(PG* pg, Scrub::scrub_prio_t with_priority);
 
-  void queue_scrub_after_repair(PG* pg, Scrub::scrub_prio_t with_priority);
-
   /// Signals either (a) the end of a sleep period, or (b) a recheck of the availability
   /// of the primary map being created by the backend.
   void queue_for_scrub_resched(PG* pg, Scrub::scrub_prio_t with_priority);
index cda893397bdc658a61c1a898863af7a37656fce9..f7a5033574f7672404561e8dc99ab3f3d12a868d 100644 (file)
@@ -205,7 +205,6 @@ PG::PG(OSDService *o, OSDMapRef curmap,
   recovery_ops_active(0),
   backfill_reserving(false),
   finish_sync_event(NULL),
-  scrub_after_recovery(false),
   active_pushes(0),
   recovery_state(
     o->cct,
@@ -361,7 +360,6 @@ void PG::clear_primary_state()
   if (m_scrubber) {
     m_scrubber->on_new_interval();
   }
-  scrub_after_recovery = false;
 
   agent_clear();
 }
@@ -427,27 +425,6 @@ void PG::queue_recovery()
   }
 }
 
-void PG::queue_scrub_after_repair()
-{
-  dout(10) << __func__ << dendl;
-  ceph_assert(ceph_mutex_is_locked(_lock));
-
-  m_planned_scrub.must_deep_scrub = true;
-  m_planned_scrub.check_repair = true;
-  m_planned_scrub.must_scrub = true;
-  m_planned_scrub.calculated_to_deep = true;
-
-  if (is_scrub_queued_or_active()) {
-    dout(10) << __func__ << ": scrubbing already ("
-             << (is_scrubbing() ? "active)" : "queued)") << dendl;
-    return;
-  }
-
-  m_scrubber->set_op_parameters(m_planned_scrub);
-  dout(15) << __func__ << ": queueing" << dendl;
-
-  osd->queue_scrub_after_repair(this, Scrub::scrub_prio_t::high_priority);
-}
 
 unsigned PG::get_scrub_priority()
 {
@@ -484,17 +461,15 @@ void PG::_finish_recovery(Context* c)
   // When recovery is initiated by a repair, that flag is left on
   state_clear(PG_STATE_REPAIR);
   if (c == finish_sync_event) {
-    dout(15) << __func__ << " scrub_after_recovery? " << scrub_after_recovery << dendl;
+    dout(15) << fmt::format("{}: scrub_after_recovery: {}", __func__,
+      m_scrubber->is_after_repair_required()) << dendl;
     finish_sync_event = 0;
     recovery_state.purge_strays();
 
     publish_stats_to_osd();
 
-    if (scrub_after_recovery) {
-      dout(10) << "_finish_recovery requeueing for scrub" << dendl;
-      scrub_after_recovery = false;
-      queue_scrub_after_repair();
-    }
+    // notify the scrubber that recovery is done. This may trigger a scrub.
+    m_scrubber->recovery_completed();
   } else {
     dout(10) << "_finish_recovery -- stale" << dendl;
   }
index 5f3cd32e0230577008d9f0d02ca98bac48efcb7b..6bcb77ee5a9d18ea13cb69a2813a6c56f81f8e77 100644 (file)
@@ -425,19 +425,6 @@ public:
     forward_scrub_event(&ScrubPgIF::initiate_regular_scrub, queued, "StartScrub");
   }
 
-  /**
-   *  a special version of PG::scrub(), which:
-   *  - is initiated after repair, and
-   * (not true anymore:)
-   *  - is not required to allocate local/remote OSD scrub resources
-   */
-  void recovery_scrub(epoch_t queued, ThreadPool::TPHandle& handle)
-  {
-    // a new scrub
-    forward_scrub_event(&ScrubPgIF::initiate_scrub_after_repair, queued,
-                       "AfterRepairScrub");
-  }
-
   void replica_scrub(epoch_t queued,
                     Scrub::act_token_t act_token,
                     ThreadPool::TPHandle& handle);
@@ -1219,8 +1206,6 @@ public:
 
   // -- scrub --
 protected:
-  bool scrub_after_recovery;
-
   int active_pushes;
 
   [[nodiscard]] bool ops_blocked_by_scrub() const;
@@ -1354,7 +1339,6 @@ protected:
   virtual void snap_trimmer_scrub_complete() = 0;
 
   void queue_recovery();
-  void queue_scrub_after_repair();
   unsigned int get_scrub_priority();
 
   bool try_flush_or_schedule_async() override;
index 68c3a203bf934deda02203d64ca2cdbfdb568488..1df284bcfc3169ed5a27c3efc67f0ce06870bf09 100644 (file)
@@ -12384,7 +12384,10 @@ int PrimaryLogPG::recover_missing(
   int priority,
   PGBackend::RecoveryHandle *h)
 {
-  dout(10) << __func__ << " sar: " << scrub_after_recovery << dendl;
+  dout(10) << fmt::format(
+                 "{} sar: {}", __func__,
+                 m_scrubber->is_after_repair_required())
+          << dendl;
 
   if (recovery_state.get_missing_loc().is_unfound(soid)) {
     dout(7) << __func__ << " " << soid
@@ -12415,7 +12418,7 @@ int PrimaryLogPG::recover_missing(
         if (!object_missing) {
           object_stat_sum_t stat_diff;
           stat_diff.num_objects_recovered = 1;
-          if (scrub_after_recovery)
+          if (m_scrubber->is_after_repair_required())
             stat_diff.num_objects_repaired = 1;
           on_global_recover(soid, stat_diff, true);
         } else {
index ecad656bab9a4b1a30a51337f127c0c92d81de35..1a6434618884559584eb130c40434c58dccff32e 100644 (file)
@@ -59,15 +59,6 @@ void PGScrub::run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& ha
   pg->unlock();
 }
 
-void PGScrubAfterRepair::run(OSD* osd,
-                         OSDShard* sdata,
-                         PGRef& pg,
-                         ThreadPool::TPHandle& handle)
-{
-  pg->recovery_scrub(epoch_queued, handle);
-  pg->unlock();
-}
-
 void PGScrubResched::run(OSD* osd,
                         OSDShard* sdata,
                         PGRef& pg,
index 0b28e64158e4ef0e2bbf31637bebf8ad00a21684..d0281cf84e7f3ca6cae6f069edddf78381137d58 100644 (file)
@@ -373,18 +373,6 @@ class PGScrubResched : public PGScrubItem {
   void run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& handle) final;
 };
 
-/**
- *  called when a repair process completes, to initiate scrubbing. No local/remote
- *  resources are allocated.
- */
-class PGScrubAfterRepair : public PGScrubItem {
- public:
-  PGScrubAfterRepair(spg_t pg, epoch_t epoch_queued)
-      : PGScrubItem{pg, epoch_queued, "PGScrubAfterRepair"}
-  {}
-  void run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& handle) final;
-};
-
 class PGScrubPushesUpdate : public PGScrubItem {
  public:
   PGScrubPushesUpdate(spg_t pg, epoch_t epoch_queued)
index 4b439bf8ec5cf289109fe3be0b45ae11f486d103..6c2dff17bd7be108606bec94354433381ef5a032 100644 (file)
@@ -67,8 +67,6 @@ ostream& operator<<(ostream& out, const requested_scrub_t& sf)
     out << " MUST_REPAIR";
   if (sf.auto_repair)
     out << " planned AUTO_REPAIR";
-  if (sf.check_repair)
-    out << " planned CHECK_REPAIR";
   if (sf.deep_scrub_on_error)
     out << " planned DEEP_SCRUB_ON_ERROR";
   if (sf.must_deep_scrub)
@@ -189,7 +187,7 @@ bool PgScrubber::should_abort() const
 /*
  * a note re the checks performed before sending scrub-initiating messages:
  *
- * For those ('StartScrub', 'AfterRepairScrub') scrub-initiation messages that
+ * For those scrub-initiation messages ('StartScrub') that
  * possibly were in the queue while the PG changed state and became unavailable
  * for scrubbing:
  *
@@ -228,22 +226,6 @@ void PgScrubber::advance_token()
   m_current_token++;
 }
 
-void PgScrubber::initiate_scrub_after_repair(epoch_t epoch_queued)
-{
-  dout(10) << fmt::format(
-                 "{}: epoch:{} is PrimaryIdle:{}", __func__, epoch_queued,
-                 m_fsm->is_primary_idle())
-          << dendl;
-  // we may have lost our Primary status while the message languished in the
-  // queue
-  if (check_interval(epoch_queued)) {
-    dout(10) << "scrubber event -->> AfterRepairScrub epoch: " << epoch_queued
-            << dendl;
-    m_fsm->process_event(AfterRepairScrub{});
-    dout(10) << "scrubber event --<< AfterRepairScrub" << dendl;
-  }
-}
-
 void PgScrubber::send_scrub_unblock(epoch_t epoch_queued)
 {
   dout(10) << "scrubber event -->> " << __func__ << " epoch: " << epoch_queued
@@ -470,6 +452,7 @@ void PgScrubber::on_new_interval()
                  (is_primary() ? "Primary" : "Replica/other"),
                  is_scrub_active(), is_queued_or_active())
           << dendl;
+  m_after_repair_scrub_required = false;
   m_fsm->process_event(IntervalChanged{});
   // the following asserts were added due to a past bug, where PG flags were
   // left set in some scenarios.
@@ -707,6 +690,30 @@ scrub_level_t PgScrubber::scrub_requested(
 }
 
 
+void PgScrubber::recovery_completed()
+{
+  dout(15) << fmt::format(
+                 "{}: is scrub required? {}", __func__,
+                 m_after_repair_scrub_required)
+          << dendl;
+  if (m_after_repair_scrub_required) {
+    m_after_repair_scrub_required = false;
+    m_osds->get_scrub_services().dequeue_target(m_pg_id, scrub_level_t::deep);
+    auto& trgt = m_scrub_job->get_target(scrub_level_t::deep);
+    trgt.up_urgency_to(urgency_t::after_repair);
+    trgt.sched_info.schedule.scheduled_at = {0, 0};
+    trgt.sched_info.schedule.not_before = ceph_clock_now();
+    m_osds->get_scrub_services().enqueue_target(trgt);
+  }
+}
+
+
+bool PgScrubber::is_after_repair_required() const
+{
+  return m_after_repair_scrub_required;
+}
+
+
 void PgScrubber::request_rescrubbing(requested_scrub_t& request_flags)
 {
   dout(10) << __func__ << " flags: " << request_flags << dendl;
@@ -1703,7 +1710,7 @@ void PgScrubber::set_op_parameters(const requested_scrub_t& request)
   // to discard stale messages from previous aborted scrubs.
   m_epoch_start = m_pg->get_osdmap_epoch();
 
-  m_flags.check_repair = request.check_repair;
+  m_flags.check_repair = m_active_target->urgency() == urgency_t::after_repair;
   m_flags.auto_repair = request.auto_repair || request.need_auto;
   m_flags.required = request.req_scrub || request.must_scrub;
 
@@ -2016,7 +2023,7 @@ void PgScrubber::scrub_finish()
     } else if (has_error) {
 
       // Deep scrub in order to get corrected error counts
-      m_pg->scrub_after_recovery = true;
+      m_after_repair_scrub_required = true;
       m_planned_scrub.req_scrub = m_planned_scrub.req_scrub || m_flags.required;
 
       dout(20) << __func__ << " Current 'required': " << m_flags.required
@@ -2185,8 +2192,6 @@ void PgScrubber::on_mid_scrub_abort(Scrub::delay_cause_t issue)
   m_planned_scrub.need_auto = m_planned_scrub.need_auto || m_flags.auto_repair;
   m_planned_scrub.deep_scrub_on_error =
       m_planned_scrub.deep_scrub_on_error || m_flags.deep_scrub_on_error;
-  m_planned_scrub.check_repair =
-      m_planned_scrub.check_repair || m_flags.check_repair;
 
   // copy the aborted target
   const auto aborted_target = *m_active_target;
index 0b114e5458846f7b0cd17769cdfd3860c174c7b8..f657dcc690cf913b918442b6e2139573464c8536 100644 (file)
@@ -150,7 +150,8 @@ struct scrub_flags_t {
   bool auto_repair{false};
 
   /// this flag indicates that we are scrubbing post repair to verify everything
-  /// is fixed
+  /// is fixed (otherwise - PG_STATE_FAILED_REPAIR will be asserted.)
+  /// Update (July 2024): now reflects an 'after-repair' urgency.
   bool check_repair{false};
 
   /// checked at the end of the scrub, to possibly initiate a deep-scrub
@@ -166,6 +167,35 @@ struct scrub_flags_t {
 
 ostream& operator<<(ostream& out, const scrub_flags_t& sf);
 
+namespace fmt {
+template <>
+struct formatter<scrub_flags_t> {
+  constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+  template <typename FormatContext>
+  auto format(scrub_flags_t& sf, FormatContext& ctx) const
+  {
+    std::string txt;
+    bool sep{false};
+    if (sf.auto_repair) {
+      txt = "auto-repair";
+      sep = true;
+    }
+    if (sf.check_repair) {
+      txt += sep ? ",check-repair" : "check-repair";
+      sep = true;
+    }
+    if (sf.deep_scrub_on_error) {
+      txt += sep ? ",deep-scrub-on-error" : "deep-scrub-on-error";
+      sep = true;
+    }
+    if (sf.required) {
+      txt += sep ? ",required" : "required";
+    }
+    return fmt::format_to(ctx.out(), "{}", txt);
+  }
+};
+}  // namespace fmt
+
 
 /**
  * The part of PG-scrubbing code that isn't state-machine wiring.
@@ -195,8 +225,6 @@ class PgScrubber : public ScrubPgIF,
 
   void initiate_regular_scrub(epoch_t epoch_queued) final;
 
-  void initiate_scrub_after_repair(epoch_t epoch_queued) final;
-
   void send_scrub_resched(epoch_t epoch_queued) final;
 
   void active_pushes_notification(epoch_t epoch_queued) final;
@@ -267,6 +295,14 @@ class PgScrubber : public ScrubPgIF,
       scrub_type_t scrub_type,
       requested_scrub_t& req_flags) final;
 
+  /**
+   * let the scrubber know that a recovery operation has completed.
+   * This might trigger an 'after repair' scrub.
+   */
+  void recovery_completed() final;
+
+  bool is_after_repair_required() const final;
+
   /**
    * Reserve local scrub resources (managed by the OSD)
    *
@@ -506,6 +542,11 @@ class PgScrubber : public ScrubPgIF,
   /// control the order of construction/destruction.
   std::optional<Scrub::ScrubJob> m_scrub_job;
 
+
+  /// the scrubber has initiated a recovery, and is waiting for the recovery
+  /// to complete (in order to perform an 'after-repair' scrub)
+  bool m_after_repair_scrub_required{false};
+
   ostream& show(ostream& out) const override;
 
  public:
index 547a046d5bd6d15c0ac05f31f61e5f0e231d7674..81e9bd7b6d8b1b249b1baaf6437e5d882b9c2f85 100644 (file)
@@ -159,14 +159,6 @@ sc::result PrimaryIdle::react(const StartScrub&)
   return transit<ReservingReplicas>();
 }
 
-sc::result PrimaryIdle::react(const AfterRepairScrub&)
-{
-  dout(10) << "PrimaryIdle::react(const AfterRepairScrub&)" << dendl;
-  DECLARE_LOCALS;
-  scrbr->reset_epoch();
-  return transit<ReservingReplicas>();
-}
-
 void PrimaryIdle::clear_state(const FullReset&) {
   dout(10) << "PrimaryIdle::react(const FullReset&): clearing state flags"
            << dendl;
index cf8d28c765b0480e22fff3ab6d98e43f7f5b2ee9..d1edfd37c9859270a2343f71b86e93bfb77f54c6 100644 (file)
@@ -163,9 +163,6 @@ MEV(RemotesReserved)
 /// initiate a new scrubbing session (relevant if we are a Primary)
 MEV(StartScrub)
 
-/// initiate a new scrubbing session. Only triggered at Recovery completion
-MEV(AfterRepairScrub)
-
 /// triggered when the PG unblocked an object that was marked for scrubbing.
 /// Via the PGScrubUnblocked op
 MEV(Unblocked)
@@ -472,11 +469,8 @@ struct PrimaryIdle;
  *  The basic state for an active Primary. Ready to accept a new scrub request.
  *  State managed here: being in the OSD's scrub queue (unless when scrubbing).
  *
- *  Scrubbing is triggered by one of the following events:
- *  - (standard scenario for a Primary): 'StartScrub'. Initiates the OSDs
- *    resources reservation process. Will be issued by PG::scrub(), following a
- *    queued "PGScrub" op.
- *  - a special end-of-recovery Primary scrub event ('AfterRepairScrub').
+ *  Scrubbing is triggered by a 'StartScrub' event, which is issued by
+ *  PG::scrub(), following a queued "PGScrub" op.
  */
 struct PrimaryActive : sc::state<PrimaryActive, ScrubMachine, PrimaryIdle>,
                         NamedSimply {
@@ -517,13 +511,10 @@ struct PrimaryIdle : sc::state<PrimaryIdle, PrimaryActive>, NamedSimply {
 
   using reactions = mpl::list<
       sc::custom_reaction<StartScrub>,
-      // a scrubbing that was initiated at recovery completion:
-      sc::custom_reaction<AfterRepairScrub>,
       // undoing set_op_params(), if aborted before starting the scrub:
       sc::in_state_reaction<FullReset, PrimaryIdle, &PrimaryIdle::clear_state>>;
 
   sc::result react(const StartScrub&);
-  sc::result react(const AfterRepairScrub&);
 };
 
 /**
index 60cb19a9af2d92922e305234d32ac17a2940caee..8e2e13c97a604787d48e5fc07a3dbeaf44ca9cef 100644 (file)
@@ -354,12 +354,6 @@ struct requested_scrub_t {
    */
   bool auto_repair{false};
 
-  /**
-   * indicating that we are scrubbing post repair to verify everything is fixed.
-   * Otherwise - PG_STATE_FAILED_REPAIR will be asserted.
-   */
-  bool check_repair{false};
-
   /**
    * Used to indicate, both in client-facing listings and internally, that
    * the planned scrub will be a deep one.
@@ -377,10 +371,9 @@ struct fmt::formatter<requested_scrub_t> {
   auto format(const requested_scrub_t& rs, FormatContext& ctx)
   {
     return fmt::format_to(ctx.out(),
-                          "(plnd:{}{}{}{}{}{}{}{}{}{})",
+                          "(plnd:{}{}{}{}{}{}{}{}{})",
                           rs.must_repair ? " must_repair" : "",
                           rs.auto_repair ? " auto_repair" : "",
-                          rs.check_repair ? " check_repair" : "",
                           rs.deep_scrub_on_error ? " deep_scrub_on_error" : "",
                           rs.must_deep_scrub ? " must_deep_scrub" : "",
                           rs.must_scrub ? " must_scrub" : "",
@@ -409,8 +402,6 @@ struct ScrubPgIF {
 
   virtual void initiate_regular_scrub(epoch_t epoch_queued) = 0;
 
-  virtual void initiate_scrub_after_repair(epoch_t epoch_queued) = 0;
-
   virtual void send_scrub_resched(epoch_t epoch_queued) = 0;
 
   virtual void active_pushes_notification(epoch_t epoch_queued) = 0;
@@ -601,6 +592,23 @@ struct ScrubPgIF {
    */
   virtual void update_scrub_job(Scrub::delay_ready_t delay_ready) = 0;
 
+  virtual scrub_level_t scrub_requested(
+      scrub_level_t scrub_level,
+      scrub_type_t scrub_type,
+      requested_scrub_t& req_flags) = 0;
+
+  /**
+   * let the scrubber know that a recovery operation has completed.
+   * This might trigger an 'after repair' scrub.
+   */
+  virtual void recovery_completed() = 0;
+
+  /**
+   * m_after_repair_scrub_required is set, and recovery_complete() is
+   * expected to trigger a deep scrub
+   */
+  virtual bool is_after_repair_required() const = 0;
+
 
   // --------------- reservations -----------------------------------
 
@@ -612,11 +620,6 @@ struct ScrubPgIF {
    */
   virtual void handle_scrub_reserve_msgs(OpRequestRef op) = 0;
 
-  virtual scrub_level_t scrub_requested(
-      scrub_level_t scrub_level,
-      scrub_type_t scrub_type,
-      requested_scrub_t& req_flags) = 0;
-
   // --------------- debugging via the asok ------------------------------
 
   virtual int asok_debug(std::string_view cmd,