]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: mark PG as being scrubbed, from scrub initiation to Inactive state 46767/head
authorRonen Friedman <rfriedma@redhat.com>
Mon, 9 Aug 2021 18:20:37 +0000 (18:20 +0000)
committerRonen Friedman <rfriedma@redhat.com>
Sun, 26 Jun 2022 09:33:30 +0000 (09:33 +0000)
The scrubber's state-machine changes states only following a message dispatched
via the OSD queue. That creates some vulnerability periods, from when the
decision to change the state is made, till when the message carrying the event
is dequeued and processed by the state-machine.

One of the problems thus created is a second scrub being started on a PG, before
the previous scrub is fully terminated and cleaned up.

Here we add a 'being-scrubbed' flag, that is asserted when the first scrub
initiation message is queued and is only cleared when the state machine reaches
Inactive state after the scrub is done.

To note: scrub_finish() is now part of the FSM transition from WaitDigest to Inactive,
closing this specific vulnerability period;

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
(cherry picked from commit 2809a58411f7f5dcecf2caf98e9549842268566f)

Conflicts:
src/osd/OSD.cc
src/osd/PG.cc
src/osd/PG.h
src/osd/pg_scrubber.cc
src/osd/scrub_machine.cc

Conflicts resolved by:
- removing 'scrub-duration' code that was pulled in;
- replacing log lines that used fmtlib;
- completing pieces of scrub_finish() handling that were only partially
  included.

src/osd/OSD.cc
src/osd/PG.cc
src/osd/PG.h
src/osd/PrimaryLogPG.cc
src/osd/pg_scrubber.cc
src/osd/pg_scrubber.h
src/osd/scrub_machine.cc
src/osd/scrub_machine.h
src/osd/scrub_machine_lstnr.h
src/osd/scrubber_common.h

index bb7f23954e8f74ed5487e6d0a587c8a4bb932638..45470ad8c703482aa23fc4835a34b7e8d8ad2fee 100644 (file)
@@ -7700,7 +7700,7 @@ void OSD::sched_scrub()
       }
 
       // This has already started, so go on to the next scrub job
-      if (pg->is_scrub_active()) {
+      if (pg->is_scrub_queued_or_active()) {
        pg->unlock();
        dout(20) << __func__ << ": already in progress pgid " << scrub_job.pgid << dendl;
        continue;
index 6121ca1cfc38de7b220e6291984eedec9809da23..5b10f146616f3c4745a99044a75f01adb8670905 100644 (file)
@@ -203,7 +203,6 @@ PG::PG(OSDService *o, OSDMapRef curmap,
   info_struct_v(0),
   pgmeta_oid(p.make_pgmeta_oid()),
   stat_queue_item(this),
-  scrub_queued(false),
   recovery_queued(false),
   recovery_ops_active(0),
   backfill_reserving(false),
@@ -436,19 +435,16 @@ void PG::queue_scrub_after_repair()
   m_planned_scrub.check_repair = true;
   m_planned_scrub.must_scrub = true;
 
-  if (is_scrubbing()) {
-    dout(10) << __func__ << ": scrubbing already" << dendl;
-    return;
-  }
-  if (scrub_queued) {
-    dout(10) << __func__ << ": already queued" << dendl;
+  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;
 
-  scrub_queued = true;
+  m_scrubber->set_queued_or_active();
   osd->queue_scrub_after_repair(this, Scrub::scrub_prio_t::high_priority);
 }
 
@@ -1337,18 +1333,11 @@ bool PG::sched_scrub()
          << (is_clean() ? " <clean>" : " <not-clean>") << dendl;
   ceph_assert(ceph_mutex_is_locked(_lock));
 
-  if (m_scrubber && m_scrubber->is_scrub_active()) {
-    return false;
-  }
-  
   if (!is_primary() || !is_active() || !is_clean()) {
     return false;
   }
 
-  if (scrub_queued) {
-    // only applicable to the very first time a scrub event is queued
-    // (until handled and posted to the scrub FSM)
-    dout(10) << __func__ << ": already queued" << dendl;
+  if (is_scrub_queued_or_active()) {
     return false;
   }
 
@@ -1382,8 +1371,7 @@ bool PG::sched_scrub()
   m_scrubber->set_op_parameters(m_planned_scrub);
 
   dout(10) << __func__ << ": queueing" << dendl;
-
-  scrub_queued = true;
+  m_scrubber->set_queued_or_active();
   osd->queue_for_scrub(this, Scrub::scrub_prio_t::low_priority);
   return true;
 }
@@ -1576,9 +1564,10 @@ void PG::on_role_change() {
   plpg_on_role_change();
 }
 
-void PG::on_new_interval() {
-  dout(20) << __func__ << " scrub_queued was " << scrub_queued << " flags: " << m_planned_scrub << dendl;
-  scrub_queued = false;
+void PG::on_new_interval()
+{
+  dout(20) << __func__ << ": scrub flags on new interval: " << m_planned_scrub
+          << dendl;
   projected_last_update = eversion_t();
   cancel_recovery();
 }
@@ -2076,7 +2065,8 @@ void PG::repair_object(
 void PG::forward_scrub_event(ScrubAPI fn, epoch_t epoch_queued, std::string_view desc)
 {
   dout(20) << __func__ << ": " << desc << " queued at: " << epoch_queued << dendl;
-  if (is_active() && m_scrubber) {
+  ceph_assert(m_scrubber);
+  if (is_active()) {
     ((*m_scrubber).*fn)(epoch_queued);
   } else {
     // pg might be in the process of being deleted
@@ -2092,7 +2082,8 @@ void PG::forward_scrub_event(ScrubSafeAPI fn,
 {
   dout(20) << __func__ << ": " << desc << " queued: " << epoch_queued
           << " token: " << act_token << dendl;
-  if (is_active() && m_scrubber) {
+  ceph_assert(m_scrubber);
+  if (is_active()) {
     ((*m_scrubber).*fn)(epoch_queued, act_token);
   } else {
     // pg might be in the process of being deleted
@@ -2105,8 +2096,8 @@ void PG::forward_scrub_event(ScrubSafeAPI fn,
 void PG::replica_scrub(OpRequestRef op, ThreadPool::TPHandle& handle)
 {
   dout(10) << __func__ << " (op)" << dendl;
-  if (m_scrubber)
-    m_scrubber->replica_scrub_op(op);
+  ceph_assert(m_scrubber);
+  m_scrubber->replica_scrub_op(op);
 }
 
 void PG::replica_scrub(epoch_t epoch_queued,
@@ -2115,7 +2106,6 @@ void PG::replica_scrub(epoch_t epoch_queued,
 {
   dout(10) << __func__ << " queued at: " << epoch_queued
           << (is_primary() ? " (primary)" : " (replica)") << dendl;
-  scrub_queued = false;
   forward_scrub_event(&ScrubPgIF::send_start_replica, epoch_queued, act_token,
                      "StartReplica/nw");
 }
index 1fa9b808e13a50fd917dcc4ce3a63a9cf66d130d..61adae1205eb2ffc3c3e8984b15dabb922ff1713 100644 (file)
@@ -184,6 +184,10 @@ public:
   /// scrubbing state for both Primary & replicas
   bool is_scrub_active() const { return m_scrubber->is_scrub_active(); }
 
+  /// set when the scrub request is queued, and reset after scrubbing fully
+  /// cleaned up.
+  bool is_scrub_queued_or_active() const { return m_scrubber->is_queued_or_active(); }
+
 public:
   // -- members --
   const coll_t coll;
@@ -381,7 +385,6 @@ public:
   void scrub(epoch_t queued, ThreadPool::TPHandle& handle)
   {
     // a new scrub
-    scrub_queued = false;
     forward_scrub_event(&ScrubPgIF::initiate_regular_scrub, queued, "StartScrub"sv);
   }
 
@@ -394,7 +397,6 @@ public:
   void recovery_scrub(epoch_t queued, ThreadPool::TPHandle& handle)
   {
     // a new scrub
-    scrub_queued = false;
     forward_scrub_event(&ScrubPgIF::initiate_scrub_after_repair, queued,
                        "AfterRepairScrub"sv);
   }
@@ -407,7 +409,6 @@ public:
                             Scrub::act_token_t act_token,
                             ThreadPool::TPHandle& handle)
   {
-    scrub_queued = false;
     forward_scrub_event(&ScrubPgIF::send_sched_replica, queued, act_token,
                        "SchedReplica");
   }
@@ -425,8 +426,7 @@ public:
 
   void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle& handle)
   {
-    scrub_queued = false;
-    forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub"sv);
+    forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub");
   }
 
   void scrub_send_pushes_update(epoch_t queued, ThreadPool::TPHandle& handle)
@@ -805,7 +805,6 @@ protected:
   /* You should not use these items without taking their respective queue locks
    * (if they have one) */
   xlist<PG*>::item stat_queue_item;
-  bool scrub_queued;
   bool recovery_queued;
 
   int recovery_ops_active;
index 1342f9d439a265d49715fe0a6b9943b1edf8bc22..22c12b151f50828de5592b46b3f86a802cf2be2c 100644 (file)
@@ -15178,7 +15178,7 @@ void PrimaryLogPG::SnapTrimmer::log_exit(const char *state_name, utime_t enter_t
 bool PrimaryLogPG::SnapTrimmer::permit_trim() {
   return
     pg->is_clean() &&
-    !pg->m_scrubber->is_scrub_active() &&
+    !pg->is_scrub_queued_or_active() &&
     !pg->snap_trimq.empty();
 }
 
@@ -15214,7 +15214,7 @@ boost::statechart::result PrimaryLogPG::NotTrimming::react(const KickTrim&)
     ldout(pg->cct, 10) << "NotTrimming not clean or nothing to trim" << dendl;
     return discard_event();
   }
-  if (pg->m_scrubber->is_scrub_active()) {
+  if (pg->is_scrub_queued_or_active()) {
     ldout(pg->cct, 10) << " scrubbing, will requeue snap_trimmer after" << dendl;
     return transit< WaitScrub >();
   } else {
index 59372e552fc2875d5ad1e51651c92db9fa752c12..ecf42204ed2345bb8ea6e13884f75762352d5741 100644 (file)
@@ -181,6 +181,7 @@ void PgScrubber::initiate_regular_scrub(epoch_t epoch_queued)
   } else {
     // and just in case snap trimming was blocked by the aborted scrub
     m_pg->snap_trimmer_scrub_complete();
+    clear_queued_or_active();
   }
 }
 
@@ -195,6 +196,7 @@ void PgScrubber::initiate_scrub_after_repair(epoch_t epoch_queued)
     dout(10) << "scrubber event --<< AfterRepairScrub" << dendl;
   } else {
     m_pg->snap_trimmer_scrub_complete();
+    clear_queued_or_active();
   }
 }
 void PgScrubber::send_scrub_unblock(epoch_t epoch_queued)
@@ -1200,6 +1202,26 @@ void PgScrubber::replica_scrub_op(OpRequestRef op)
     return;
   }
 
+  if (is_queued_or_active()) {
+    // this is bug!
+    // Somehow, we have received a new scrub request from our Primary, before
+    // having finished with the previous one. Did we go through an interval
+    // change without reseting the FSM? Possible responses:
+    // - crashing (the original assert_not_active() implemented that one), or
+    // - trying to recover:
+    //  - (logging enough information to debug this scenario)
+    //  - reset the FSM.
+    m_osds->clog->warn()
+      << __func__
+      << ": error: a second scrub-op received while handling the previous one";
+
+    scrub_clear_state();
+    m_osds->clog->warn() << __func__
+                        << ": after a reset. Now handling the new OP";
+  }
+  // make sure the FSM is at NotActive
+  m_fsm->assert_not_active();
+
   replica_scrubmap = ScrubMap{};
   replica_scrubmap_pos = ScrubMapBuilder{};
 
@@ -1218,11 +1240,9 @@ void PgScrubber::replica_scrub_op(OpRequestRef op)
 
   replica_scrubmap_pos.reset();
 
-  // make sure the FSM is at NotActive
-  m_fsm->assert_not_active();
-
-  m_osds->queue_for_rep_scrub(m_pg, m_replica_request_priority, m_flags.priority,
-                             m_current_token);
+  set_queued_or_active();
+  m_osds->queue_for_rep_scrub(m_pg, m_replica_request_priority,
+                              m_flags.priority, m_current_token);
 }
 
 void PgScrubber::set_op_parameters(requested_scrub_t& request)
@@ -1591,6 +1611,21 @@ void PgScrubber::unreserve_replicas()
   m_reservations.reset();
 }
 
+void PgScrubber::set_queued_or_active()
+{
+  m_queued_or_active = true;
+}
+
+void PgScrubber::clear_queued_or_active()
+{
+  m_queued_or_active = false;
+}
+
+bool PgScrubber::is_queued_or_active() const
+{
+  return m_queued_or_active;
+}
+
 [[nodiscard]] bool PgScrubber::scrub_process_inconsistent()
 {
   dout(10) << __func__ << ": checking authoritative (mode="
@@ -1642,6 +1677,7 @@ void PgScrubber::scrub_finish()
           << ". deep_scrub_on_error: " << m_flags.deep_scrub_on_error << dendl;
 
   ceph_assert(m_pg->is_locked());
+  ceph_assert(is_queued_or_active());
 
   m_pg->m_planned_scrub = requested_scrub_t{};
 
@@ -1795,9 +1831,10 @@ void PgScrubber::scrub_finish()
 
 void PgScrubber::on_digest_updates()
 {
-  dout(10) << __func__ << " #pending: " << num_digest_updates_pending << " pending? "
-          << num_digest_updates_pending
-          << (m_end.is_max() ? " <last chunk> " : " <mid chunk> ") << dendl;
+  dout(10) << __func__ << " #pending: " << num_digest_updates_pending
+          << (m_end.is_max() ? " <last chunk>" : " <mid chunk>")
+           << (is_queued_or_active() ? "" : " ** not marked as scrubbing **")
+           << dendl;
 
   if (num_digest_updates_pending > 0) {
     // do nothing for now. We will be called again when new updates arrive
@@ -1806,10 +1843,7 @@ void PgScrubber::on_digest_updates()
 
   // got all updates, and finished with this chunk. Any more?
   if (m_end.is_max()) {
-
-    scrub_finish();
     m_osds->queue_scrub_is_finished(m_pg);
-
   } else {
     // go get a new chunk (via "requeue")
     preemption_data.reset();
@@ -2007,6 +2041,7 @@ void PgScrubber::reset_internal_state()
   m_sleep_started_at = utime_t{};
 
   m_active = false;
+  clear_queued_or_active();
 }
 
 // note that only applicable to the Replica:
index 7721d368249c3678b3e47bf2d2b8de7ddce032aa..0406544fbed4ba2f41050934ecc0ea0ced1c6299 100644 (file)
@@ -313,6 +313,8 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
 
   void scrub_clear_state() final;
 
+  bool is_queued_or_active() const final;
+
   /**
    *  add to scrub statistics, but only if the soid is below the scrub start
    */
@@ -373,6 +375,8 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
 
   void on_digest_updates() final;
 
+  void scrub_finish() final;
+
   ScrubMachineListener::MsgAndEpoch
   prep_replica_map_msg(Scrub::PreemptionNoted was_preempted) final;
 
@@ -402,6 +406,9 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
 
   [[nodiscard]] bool was_epoch_changed() const final;
 
+  void set_queued_or_active() final;
+  void clear_queued_or_active() final;
+
   void mark_local_map_ready() final;
 
   [[nodiscard]] bool are_all_maps_available() const final;
@@ -528,9 +535,6 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
   void cleanup_on_finish();  // scrub_clear_state() as called for a Primary when
                             // Active->NotActive
 
-  /// the part that actually finalizes a scrub
-  void scrub_finish();
-
  protected:
   PG* const m_pg;
 
@@ -578,6 +582,23 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
 
   bool m_active{false};
 
+  /**
+   * a flag designed to prevent the initiation of a second scrub on a PG for which scrubbing
+   * has been initiated.
+   *
+   * set once scrubbing was initiated (i.e. - even before the FSM event that
+   * will trigger a state-change out of Inactive was handled), and only reset
+   * once the FSM is back in Inactive.
+   * In other words - its ON period encompasses:
+   *   - the time period covered today by 'queued', and
+   *   - the time when m_active is set, and
+   *   - all the time from scrub_finish() calling update_stats() till the
+   *     FSM handles the 'finished' event
+   *
+   * Compared with 'm_active', this flag is asserted earlier  and remains ON for longer.
+   */
+  bool m_queued_or_active{false};
+
   eversion_t m_subset_last_update{};
 
   std::unique_ptr<Scrub::Store> m_store;
index e130702784263910688873ec8ef50998b6fef348..1d08af5812796b32dda9f744c7af823887a19823 100644 (file)
@@ -93,6 +93,8 @@ std::ostream& ScrubMachine::gen_prefix(std::ostream& out) const
 NotActive::NotActive(my_context ctx) : my_base(ctx)
 {
   dout(10) << "-- state -->> NotActive" << dendl;
+  DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
+  scrbr->clear_queued_or_active();
 }
 
 // ----------------------- ReservingReplicas ---------------------------------
@@ -140,6 +142,7 @@ ActiveScrubbing::~ActiveScrubbing()
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
   dout(15) << __func__ << dendl;
   scrbr->unreserve_replicas();
+  scrbr->clear_queued_or_active();
 }
 
 /*
@@ -408,7 +411,9 @@ sc::result WaitReplicas::react(const GotReplicas&)
 
 WaitDigestUpdate::WaitDigestUpdate(my_context ctx) : my_base(ctx)
 {
+  DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
   dout(10) << "-- state -->> Act/WaitDigestUpdate" << dendl;
+
   // perform an initial check: maybe we already
   // have all the updates we need:
   // (note that DigestUpdate is usually an external event)
@@ -430,6 +435,14 @@ sc::result WaitDigestUpdate::react(const DigestUpdate&)
   return discard_event();
 }
 
+sc::result WaitDigestUpdate::react(const ScrubFinished&)
+{
+  DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
+  dout(10) << "WaitDigestUpdate::react(const ScrubFinished&)" << dendl;
+  scrbr->scrub_finish();
+  return transit<NotActive>();
+}
+
 ScrubMachine::ScrubMachine(PG* pg, ScrubMachineListener* pg_scrub)
     : m_pg_id{pg->pg_id}, m_scrbr{pg_scrub}
 {
index bd7ae37d2e3b96b5adfa730c67b4abfc25793631..7f88a675a2b66217605ea5c3d615f6206e855ee4 100644 (file)
@@ -306,9 +306,10 @@ struct WaitDigestUpdate : sc::state<WaitDigestUpdate, ActiveScrubbing> {
   explicit WaitDigestUpdate(my_context ctx);
 
   using reactions = mpl::list<sc::custom_reaction<DigestUpdate>,
-                             sc::transition<NextChunk, PendingTimer>,
-                             sc::transition<ScrubFinished, NotActive>>;
+                             sc::custom_reaction<ScrubFinished>,
+                             sc::transition<NextChunk, PendingTimer>>;
   sc::result react(const DigestUpdate&);
+  sc::result react(const ScrubFinished&);
 };
 
 // ----------------------------- the "replica active" states -----------------------
index 68953e4573ff812ab2603623c279402fdfb609a4..564fa77a1f062bd47e80f18fd5a9dc211e210783 100644 (file)
@@ -89,6 +89,9 @@ struct ScrubMachineListener {
 
   virtual void on_digest_updates() = 0;
 
+  /// the part that actually finalizes a scrub
+  virtual void scrub_finish() = 0;
+
   /**
    * Prepare a MOSDRepScrubMap message carrying the requested scrub map
    * @param was_preempted - were we preempted?
@@ -134,6 +137,12 @@ struct ScrubMachineListener {
 
   virtual void unreserve_replicas() = 0;
 
+  /**
+   * Manipulate the 'I am being scrubbed now' Scrubber's flag
+   */
+  virtual void set_queued_or_active() = 0;
+  virtual void clear_queued_or_active() = 0;
+
   /**
    * the FSM interface into the "are we waiting for maps, either our own or from
    * replicas" state.
index 34c63f5c5eee02e495e3edb8f2da233a6135a3e8..65014b594d6cbf4c46815095e2c22f60230c87d2 100644 (file)
@@ -170,6 +170,23 @@ struct ScrubPgIF {
    */
   [[nodiscard]] virtual bool is_scrub_active() const = 0;
 
+  /**
+   * 'true' until after the FSM processes the 'scrub-finished' event,
+   * and scrubbing is completely cleaned-up.
+   *
+   * In other words - holds longer than is_scrub_active(), thus preventing
+   * a rescrubbing of the same PG while the previous scrub has not fully
+   * terminated.
+   */
+  [[nodiscard]] virtual bool is_queued_or_active() const = 0;
+
+  /**
+   * Manipulate the 'scrubbing request has been queued, or - we are
+   * actually scrubbing' Scrubber's flag
+   */
+  virtual void set_queued_or_active() = 0;
+  virtual void clear_queued_or_active() = 0;
+
   /// are we waiting for resource reservation grants form our replicas?
   [[nodiscard]] virtual bool is_reserving() const = 0;