]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: mark PG as being scrubbed, from scrub initiation to Inactive state
authorRonen Friedman <rfriedma@redhat.com>
Mon, 9 Aug 2021 18:20:37 +0000 (18:20 +0000)
committerRonen Friedman <rfriedma@redhat.com>
Tue, 16 Nov 2021 16:05:02 +0000 (16:05 +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>
src/osd/OSD.cc
src/osd/PG.cc
src/osd/PG.h
src/osd/PrimaryLogPG.cc
src/osd/scrubber/pg_scrubber.cc
src/osd/scrubber/pg_scrubber.h
src/osd/scrubber/scrub_machine.cc
src/osd/scrubber/scrub_machine_lstnr.h
src/osd/scrubber_common.h

index 5b73d77d00639c997092ab37774024e7039dcf5b..83d1df5975498768e2ef232925598a90f49fcccc 100644 (file)
@@ -7542,7 +7542,7 @@ Scrub::schedule_result_t OSDService::initiate_a_scrub(spg_t pgid,
   }
 
   // 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 " << pgid << dendl;
     return Scrub::schedule_result_t::already_started;
index 455eb80a7178c80027a1c82d455dbd51aee989de..25278eb7585dee434ebdbab1fcdc6b3ea869e711 100644 (file)
@@ -201,7 +201,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),
@@ -433,19 +432,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);
 }
 
@@ -1338,15 +1334,12 @@ Scrub::schedule_result_t PG::sched_scrub()
          << (is_clean() ? " <clean>" : " <not-clean>") << dendl;
   ceph_assert(ceph_mutex_is_locked(_lock));
 
-  if (!is_primary() || !is_active() || !is_clean()) {
-    return Scrub::schedule_result_t::bad_pg_state;
+  if (is_scrub_queued_or_active()) {
+    return Scrub::schedule_result_t::already_started;
   }
 
-  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;
-    return Scrub::schedule_result_t::already_started;
+  if (!is_primary() || !is_active() || !is_clean()) {
+    return Scrub::schedule_result_t::bad_pg_state;
   }
 
   // analyse the combination of the requested scrub flags, the osd/pool configuration
@@ -1379,8 +1372,7 @@ Scrub::schedule_result_t 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 Scrub::schedule_result_t::scrub_initiated;
 }
@@ -1598,14 +1590,17 @@ 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()
+{
   projected_last_update = eversion_t();
   cancel_recovery();
-  if (m_scrubber) {
-    m_scrubber->on_maybe_registration_change(m_planned_scrub);
-  }
+
+  assert(m_scrubber);
+  // log some scrub data before we react to the interval
+  dout(20) << __func__ << (is_scrub_queued_or_active() ? " scrubbing " : " ")
+           << "flags: " << m_planned_scrub << dendl;
+
+  m_scrubber->on_maybe_registration_change(m_planned_scrub);
 }
 
 epoch_t PG::oldest_stored_osdmap() {
@@ -2101,7 +2096,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
@@ -2117,7 +2113,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
@@ -2130,8 +2127,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,
@@ -2140,7 +2137,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 de6a0a32bed7524b6874fdf2e3c2108b849c3b35..b68f1bfda60445cf4226ef2f7f8c22035cfb62bf 100644 (file)
@@ -188,6 +188,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;
@@ -384,7 +388,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");
   }
 
@@ -397,7 +400,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");
   }
@@ -410,7 +412,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");
   }
@@ -428,7 +429,6 @@ public:
 
   void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle& handle)
   {
-    scrub_queued = false;
     forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub");
   }
 
@@ -808,7 +808,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 88f074123b6914f10d4dc15439b91efd3658ba3d..fae6281ca1d9a75cfe3f852cf1e67e689614a3ca 100644 (file)
@@ -15501,7 +15501,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();
 }
 
@@ -15537,7 +15537,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 297b52a51311e6f08ef0add85e9ada4c6c1eefb8..db18749b91509be3c95fce6ce3b90f0a2f88bfa4 100644 (file)
@@ -17,6 +17,7 @@
 #include "messages/MOSDScrubReserve.h"
 
 #include "osd/OSD.h"
+#include "osd/osd_types_fmt.h"
 #include "ScrubStore.h"
 #include "scrub_machine.h"
 
@@ -186,6 +187,8 @@ void PgScrubber::initiate_regular_scrub(epoch_t epoch_queued)
     reset_epoch(epoch_queued);
     m_fsm->process_event(StartScrub{});
     dout(10) << "scrubber event --<< StartScrub" << dendl;
+  } else {
+    clear_queued_or_active();
   }
 }
 
@@ -198,6 +201,8 @@ void PgScrubber::initiate_scrub_after_repair(epoch_t epoch_queued)
     reset_epoch(epoch_queued);
     m_fsm->process_event(AfterRepairScrub{});
     dout(10) << "scrubber event --<< AfterRepairScrub" << dendl;
+  } else {
+    clear_queued_or_active();
   }
 }
 
@@ -1273,6 +1278,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() << fmt::format(
+      "{}: error: a second scrub-op received while handling the previous one",
+      __func__);
+
+    scrub_clear_state();
+    m_osds->clog->warn() << fmt::format(
+      "{}: after a reset. Now handling the new OP", __func__);
+  }
+  // make sure the FSM is at NotActive
+  m_fsm->assert_not_active();
+
   replica_scrubmap = ScrubMap{};
   replica_scrubmap_pos = ScrubMapBuilder{};
 
@@ -1291,11 +1316,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)
@@ -1674,6 +1697,20 @@ void PgScrubber::clear_reserving_now()
   m_osds->get_scrub_services().clear_reserving_now();
 }
 
+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()
 {
@@ -1726,6 +1763,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{};
 
@@ -1881,9 +1919,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
@@ -1892,10 +1931,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();
@@ -2200,6 +2236,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 4bcdb5e1557813aa6e4d8d32bc55ffb04d601c0b..d64eb994c53ecfb513c3b766903a5b688213fdaf 100644 (file)
@@ -324,6 +324,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
    */
@@ -392,6 +394,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;
 
@@ -424,6 +428,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;
@@ -562,9 +569,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;
 
@@ -612,6 +616,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 5190c8d8510e67f5502760bf6207ccfe6d9d14d5..8f88a77b52e06e7204effddc3a28855a2b0cfe11 100644 (file)
@@ -92,6 +92,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();
 }
 
 sc::result NotActive::react(const StartScrub&)
@@ -157,6 +159,7 @@ ActiveScrubbing::~ActiveScrubbing()
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
   dout(15) << __func__ << dendl;
   scrbr->unreserve_replicas();
+  scrbr->clear_queued_or_active();
 }
 
 /*
@@ -436,7 +439,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)
@@ -463,6 +468,7 @@ sc::result WaitDigestUpdate::react(const ScrubFinished&)
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
   dout(10) << "WaitDigestUpdate::react(const ScrubFinished&)" << dendl;
   scrbr->set_scrub_duration();
+  scrbr->scrub_finish();
   return transit<NotActive>();
 }
 
index 91c20e34946d16fe4ffd9ba6e64cddc43da22963..2fed0a399f160519abcd14d7288cd5ec822009d2 100644 (file)
@@ -103,6 +103,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?
@@ -161,6 +164,12 @@ struct ScrubMachineListener {
   virtual void set_reserving_now() = 0;
   virtual void clear_reserving_now() = 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 12302dad2abb2173c23a8e5676a56d5132bdb116..2b4d370be11ddb589eb150f6dc98a3aafdd78e2c 100644 (file)
@@ -178,6 +178,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;