]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: move ReplicaReservations into the Scrubber FSM
authorRonen Friedman <rfriedma@redhat.com>
Fri, 13 Oct 2023 17:14:31 +0000 (12:14 -0500)
committerRonen Friedman <rfriedma@redhat.com>
Sat, 14 Oct 2023 18:49:01 +0000 (21:49 +0300)
Handle grant/deny messages within the FSM.
One exception at this point: the handling of "granted by everyone"
(due to the technical inconvenience of having to handle the
"0 replicas" case in the FSM state constructor).

Note: after this commit, ScrubMachineListener - an API which is
a subset of the Scrubber API to be used by the Scrubber FSM - does
no longer make sense. The FSM should now have full access to the
scrubber, and that interface will be removed in a subsequent PR.

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
13 files changed:
src/osd/OSD.cc
src/osd/OSD.h
src/osd/PG.h
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/scrub_machine_lstnr.h
src/osd/scrubber/scrub_reservations.cc
src/osd/scrubber/scrub_reservations.h
src/osd/scrubber_common.h

index d6948cf05a76f0b8cb23bc2a3c09bac72c7c57a1..88a0bc037564bc2b0ef07d581fd1d26aa1c17de3 100644 (file)
@@ -1805,12 +1805,6 @@ void OSDService::queue_for_scrub_granted(PG* pg, Scrub::scrub_prio_t with_priori
   queue_scrub_event_msg<PGScrubResourcesOK>(pg, with_priority);
 }
 
-void OSDService::queue_for_scrub_denied(PG* pg, Scrub::scrub_prio_t with_priority)
-{
-  // Resulting scrub event: 'ReservationFailure'
-  queue_scrub_event_msg<PGScrubDenied>(pg, with_priority);
-}
-
 void OSDService::queue_for_scrub_resched(PG* pg, Scrub::scrub_prio_t with_priority)
 {
   // Resulting scrub event: 'InternalSchedScrub'
index 4ba3d7681194835e2ced856c39f6c7c392628e06..d86443351cf28084ff27e2a6b2088a0e114e65f8 100644 (file)
@@ -507,9 +507,6 @@ public:
   /// queue the message (-> event) that all replicas have reserved scrub resources for us
   void queue_for_scrub_granted(PG* pg, Scrub::scrub_prio_t with_priority);
 
-  /// queue the message (-> event) that some replicas denied our scrub resources request
-  void queue_for_scrub_denied(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 56ac9a65bf1a04ce6e3c3382bfbb2b42d7bce71a..fe335b85e0003e5073dbf2003ecb2e8b00c567b9 100644 (file)
@@ -454,12 +454,6 @@ public:
     forward_scrub_event(&ScrubPgIF::send_remotes_reserved, queued, "RemotesReserved");
   }
 
-  void scrub_send_resources_denied(epoch_t queued, ThreadPool::TPHandle& handle)
-  {
-    forward_scrub_event(&ScrubPgIF::send_reservation_failure, queued,
-                       "ReservationFailure");
-  }
-
   void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle& handle)
   {
     forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub");
index d1abc264a8f8c5a1f491e8cc646c488a0a59264c..0641aafdc1c94310caa2c627c92cffa185a8ea08 100644 (file)
@@ -86,15 +86,6 @@ void PGScrubResourcesOK::run(OSD* osd,
   pg->unlock();
 }
 
-void PGScrubDenied::run(OSD* osd,
-                       OSDShard* sdata,
-                       PGRef& pg,
-                       ThreadPool::TPHandle& handle)
-{
-  pg->scrub_send_resources_denied(epoch_queued, handle);
-  pg->unlock();
-}
-
 void PGScrubPushesUpdate::run(OSD* osd,
                              OSDShard* sdata,
                              PGRef& pg,
index 3222901dc70644526ffa1dbb74f3dc1df63d50d0..2803169a9bf973eccc75ee3c3c6a8b2c7fd408d2 100644 (file)
@@ -384,17 +384,6 @@ class PGScrubResourcesOK : public PGScrubItem {
   void run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& handle) final;
 };
 
-/**
- *  scrub resources requests denied by replica(s)
- */
-class PGScrubDenied : public PGScrubItem {
- public:
-  PGScrubDenied(spg_t pg, epoch_t epoch_queued)
-      : PGScrubItem{pg, epoch_queued, "PGScrubDenied"}
-  {}
-  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.
index 609eb0a72e6a83efbafe98687ab8f60db7d23ebd..98290a06ac4591e1683c9c81f2f1a7784f07aae2 100644 (file)
@@ -370,16 +370,6 @@ void PgScrubber::send_remotes_reserved(epoch_t epoch_queued)
   dout(10) << "scrubber event --<< " << __func__ << dendl;
 }
 
-void PgScrubber::send_reservation_failure(epoch_t epoch_queued)
-{
-  dout(10) << "scrubber event -->> " << __func__ << " epoch: " << epoch_queued
-          << dendl;
-  if (check_interval(epoch_queued)) {  // do not check for 'active'!
-    m_fsm->process_event(ReservationFailure{});
-  }
-  dout(10) << "scrubber event --<< " << __func__ << dendl;
-}
-
 void PgScrubber::send_chunk_free(epoch_t epoch_queued)
 {
   dout(10) << "scrubber event -->> " << __func__ << " epoch: " << epoch_queued
@@ -1623,25 +1613,6 @@ void PgScrubber::handle_scrub_reserve_request(OpRequestRef op)
   m_osds->send_message_osd_cluster(reply, op->get_req()->get_connection());
 }
 
-
-/// temporary
-void PgScrubber::grant_from_replica(OpRequestRef op, pg_shard_t from)
-{
-  dout(10) << fmt::format("{}: {}", __func__, *op->get_req()) << dendl;
-  ceph_assert(m_reservations.has_value()); // the FSM should know
-  m_reservations->handle_reserve_grant(op, from);
-}
-
-
-/// temporary
-void PgScrubber::reject_from_replica(OpRequestRef op, pg_shard_t from)
-{
-  dout(10) << fmt::format("{}: {}", __func__, *op->get_req()) << dendl;
-  ceph_assert(m_reservations.has_value()); // the FSM should know
-  m_reservations->handle_reserve_reject(op, from);
-}
-
-
 void PgScrubber::handle_scrub_reserve_release(OpRequestRef op)
 {
   dout(10) << __func__ << " " << *op->get_req() << dendl;
@@ -1656,21 +1627,6 @@ void PgScrubber::handle_scrub_reserve_release(OpRequestRef op)
   m_fsm->process_event(FullReset{});
 }
 
-void PgScrubber::discard_replica_reservations()
-{
-  dout(10) << __func__ << dendl;
-  if (m_reservations.has_value()) {
-    m_reservations->discard_remote_reservations();
-  }
-}
-
-void PgScrubber::clear_scrub_reservations()
-{
-  dout(10) << __func__ << dendl;
-  m_reservations.reset();        // the remote reservations
-  m_local_osd_resource.reset();          // the local reservation
-}
-
 bool PgScrubber::set_reserving_now() {
   return m_osds->get_scrub_services().set_reserving_now(m_pg_id,
                                                         ceph_clock_now());
@@ -2181,12 +2137,6 @@ void PgScrubber::set_scrub_duration()
   });
 }
 
-void PgScrubber::reserve_replicas()
-{
-  dout(10) << __func__ << dendl;
-  m_reservations.emplace(*this);
-}
-
 void PgScrubber::cleanup_on_finish()
 {
   dout(10) << __func__ << dendl;
@@ -2195,7 +2145,7 @@ void PgScrubber::cleanup_on_finish()
   state_clear(PG_STATE_SCRUBBING);
   state_clear(PG_STATE_DEEP_SCRUB);
 
-  clear_scrub_reservations();
+  m_local_osd_resource.reset();
   requeue_waiting();
 
   reset_internal_state();
@@ -2229,7 +2179,7 @@ void PgScrubber::clear_pgscrub_state()
 
   state_clear(PG_STATE_REPAIR);
 
-  clear_scrub_reservations();
+  m_local_osd_resource.reset();
   requeue_waiting();
 
   reset_internal_state();
index fc7b735d219ef17bfa93ce5c2a7059f5be38e6a0..0c8fa8c34fb790cafbfd93ea9689448ac7530d67 100644 (file)
@@ -265,8 +265,6 @@ class PgScrubber : public ScrubPgIF,
   void handle_scrub_reserve_request(OpRequestRef op);
 
   void handle_scrub_reserve_release(OpRequestRef op);
-  void discard_replica_reservations() final;
-  void clear_scrub_reservations() final;  // PG::clear... fwds to here
 
   // managing scrub op registration
 
@@ -448,7 +446,6 @@ class PgScrubber : public ScrubPgIF,
   void send_preempted_replica() final;
 
   void send_remotes_reserved(epoch_t epoch_queued) final;
-  void send_reservation_failure(epoch_t epoch_queued) final;
 
   /**
    *  does the PG have newer updates than what we (the scrubber) know?
@@ -465,8 +462,6 @@ class PgScrubber : public ScrubPgIF,
 
   int build_replica_map_chunk() final;
 
-  void reserve_replicas() final;
-
   bool set_reserving_now() final;
   void clear_reserving_now() final;
 
@@ -501,10 +496,6 @@ class PgScrubber : public ScrubPgIF,
 
   void log_cluster_warning(const std::string& warning) const final;
 
-  // temporary interface to handle forwarded reservation messages:
-  void grant_from_replica(OpRequestRef op, pg_shard_t from) final;
-  void reject_from_replica(OpRequestRef op, pg_shard_t from) final;
-
  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); }
@@ -633,9 +624,8 @@ class PgScrubber : public ScrubPgIF,
 
   epoch_t m_last_aborted{};  // last time we've noticed a request to abort
 
-  // 'optional', as 'ReplicaReservations' & 'LocalReservation' are
+  // 'optional', as 'LocalReservation' is
   // 'RAII-designed' to guarantee un-reserving when deleted.
-  std::optional<Scrub::ReplicaReservations> m_reservations;
   std::optional<Scrub::LocalReservation> m_local_osd_resource;
 
   void cleanup_on_finish();  // scrub_clear_state() as called for a Primary when
index f6eb471530a201cb866d30c1fbed3fd79456893a..cc257a47f0e314e254a026493c92b5ae7859d4da 100644 (file)
@@ -131,6 +131,7 @@ Session::Session(my_context ctx)
 Session::~Session()
 {
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
+  m_reservations.reset();
 
   // note the interaction between clearing the 'queued' flag and two
   // other states: the snap-mapper and the scrubber internal state.
@@ -145,8 +146,7 @@ sc::result Session::react(const IntervalChanged&)
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
   dout(10) << "Session::react(const IntervalChanged&)" << dendl;
 
-  /// \todo (future commit): the reservations will be local to this state
-  scrbr->discard_replica_reservations();
+  m_reservations->discard_remote_reservations();
   return transit<NotActive>();
 }
 
@@ -160,10 +160,11 @@ ReservingReplicas::ReservingReplicas(my_context ctx)
   dout(10) << "-- state -->> ReservingReplicas" << dendl;
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
 
-  scrbr->reserve_replicas();
+  // initiate the reservation process
+  context<Session>().m_reservations.emplace(*scrbr);
 
-  auto timeout = scrbr->get_pg_cct()->_conf.get_val<
-    std::chrono::milliseconds>("osd_scrub_reservation_timeout");
+  auto timeout = scrbr->get_pg_cct()->_conf.get_val<milliseconds>(
+      "osd_scrub_reservation_timeout");
   if (timeout.count() > 0) {
     // Start a timer to handle case where the replicas take a long time to
     // ack the reservation.  See ReservationTimeout handler below.
@@ -182,24 +183,29 @@ ReservingReplicas::~ReservingReplicas()
 
 sc::result ReservingReplicas::react(const ReplicaGrant& ev)
 {
-  // for now - route the message back to the scrubber, as it holds the
-  // ReplicaReservations object
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
   dout(10) << "ReservingReplicas::react(const ReplicaGrant&)" << dendl;
 
-  scrbr->grant_from_replica(ev.m_op, ev.m_from);
+  context<Session>().m_reservations->handle_reserve_grant(ev.m_op, ev.m_from);
   return discard_event();
 }
 
 sc::result ReservingReplicas::react(const ReplicaReject& ev)
 {
-  // for now - route the message back to the scrubber, as it holds the
-  // ReplicaReservations object
   DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
   dout(10) << "ReservingReplicas::react(const ReplicaReject&)" << dendl;
 
-  scrbr->reject_from_replica(ev.m_op, ev.m_from);
-  return discard_event();
+  // manipulate the 'next to reserve' iterator to exclude
+  // the rejecting replica from the set of replicas requiring release
+  context<Session>().m_reservations->verify_rejections_source(
+      ev.m_op, ev.m_from);
+
+  // set 'reservation failure' as the scrub termination cause (affecting
+  // the rescheduling of this PG)
+  scrbr->flag_reservations_failure();
+
+  // 'Session' state dtor stops the scrubber
+  return transit<NotActive>();
 }
 
 sc::result ReservingReplicas::react(const ReservationTimeout&)
@@ -208,10 +214,10 @@ sc::result ReservingReplicas::react(const ReservationTimeout&)
   dout(10) << "ReservingReplicas::react(const ReservationTimeout&)" << dendl;
 
   const auto msg = fmt::format(
-      "PgScrubber: {} timeout on reserving replicas (since {})",
-      scrbr->get_spgid(), entered_at);
-  dout(5) << msg << dendl;
-  scrbr->get_clog()->warn() << "osd." << scrbr->get_whoami() << " " << msg;
+      "osd.{} PgScrubber: {} timeout on reserving replicas (since {})",
+      scrbr->get_whoami(), scrbr->get_spgid(), entered_at);
+  dout(1) << msg << dendl;
+  scrbr->get_clog()->warn() << msg;
 
   // cause the scrubber to stop the scrub session, marking 'reservation
   // failure' as the cause (affecting future scheduling)
@@ -219,13 +225,6 @@ sc::result ReservingReplicas::react(const ReservationTimeout&)
   return transit<NotActive>();
 }
 
-sc::result ReservingReplicas::react(const ReservationFailure&)
-{
-  DECLARE_LOCALS;  // 'scrbr' & 'pg_id' aliases
-  dout(10) << "ReservingReplicas::react(const ReservationFailure&)" << dendl;
-  return transit<NotActive>();
-}
-
 // ----------------------- ActiveScrubbing -----------------------------------
 
 ActiveScrubbing::ActiveScrubbing(my_context ctx)
index 806376e3032d3b980696cf7a1ff5502eca7b542f..2f73cbbefb5d61306aa4693cd57cfd507c9e10b9 100644 (file)
@@ -24,6 +24,7 @@
 #include "osd/scrubber_common.h"
 
 #include "scrub_machine_lstnr.h"
+#include "scrub_reservations.h"
 
 /// a wrapper that sets the FSM state description used by the
 /// PgScrubber
@@ -101,9 +102,6 @@ struct ReplicaReject : sc::event<ReplicaReject> {
 /// all replicas have granted our reserve request
 MEV(RemotesReserved)
 
-/// a reservation request has failed
-MEV(ReservationFailure)
-
 /// reservations have timed out
 MEV(ReservationTimeout)
 
@@ -391,19 +389,20 @@ struct Session : sc::state<Session, ScrubMachine, ReservingReplicas>,
                               sc::custom_reaction<IntervalChanged>>;
 
   sc::result react(const IntervalChanged&);
+
+  /// managing the scrub session's reservations (optional, as
+  /// it's an RAII wrapper around the state of 'holding reservations')
+  std::optional<ReplicaReservations> m_reservations{std::nullopt};
 };
 
 struct ReservingReplicas : sc::state<ReservingReplicas, Session>,
                           NamedSimply {
   explicit ReservingReplicas(my_context ctx);
   ~ReservingReplicas();
-  using reactions = mpl::list<
-                             // all replicas granted our resources request
-                             sc::custom_reaction<ReplicaGrant>,
+  using reactions = mpl::list<sc::custom_reaction<ReplicaGrant>,
                              sc::custom_reaction<ReplicaReject>,
                              sc::transition<RemotesReserved, ActiveScrubbing>,
-                             sc::custom_reaction<ReservationTimeout>,
-                             sc::custom_reaction<ReservationFailure>>;
+                             sc::custom_reaction<ReservationTimeout>>;
 
   ceph::coarse_real_clock::time_point entered_at =
     ceph::coarse_real_clock::now();
@@ -416,9 +415,6 @@ struct ReservingReplicas : sc::state<ReservingReplicas, Session>,
   sc::result react(const ReplicaReject&);
 
   sc::result react(const ReservationTimeout&);
-
-  /// at least one replica denied us the scrub resources we've requested
-  sc::result react(const ReservationFailure&);
 };
 
 
index e760ddf0055d9f2662e5ed259f57b6099f1e2f8a..4206c789f91a02078cd9f39f2987ec1f69ccadf5 100644 (file)
@@ -5,11 +5,13 @@
 /**
  * \file the PgScrubber interface used by the scrub FSM
  */
+#include "common/LogClient.h"
 #include "common/version.h"
 #include "include/Context.h"
 #include "osd/osd_types.h"
+#include "osd/scrubber_common.h"
 
-struct ScrubMachineListener;
+class PG;
 
 namespace Scrub {
 
@@ -172,15 +174,6 @@ struct ScrubMachineListener {
    */
   virtual void maps_compare_n_cleanup() = 0;
 
-  /**
-   * order the PgScrubber to initiate the process of reserving replicas' scrub
-   * resources.
-   */
-  virtual void reserve_replicas() = 0;
-
-  /// discard replica reservations without sending a message (on interval)
-  virtual void discard_replica_reservations() = 0;
-
   virtual void set_scrub_begin_time() = 0;
 
   virtual void set_scrub_duration() = 0;
@@ -239,11 +232,6 @@ struct ScrubMachineListener {
   /// sending cluster-log warnings
   virtual void log_cluster_warning(const std::string& msg) const = 0;
 
-  // temporary interface (to be discarded in a follow-up commit)
-  // to handle replica reservation messages routed thru the FSM
-  virtual void grant_from_replica(OpRequestRef op, pg_shard_t from) = 0;
-  virtual void reject_from_replica(OpRequestRef op, pg_shard_t from) = 0;
-
   // temporary interface (to be discarded in a follow-up PR)
   /// set the 'resources_failure' flag in the scrub-job object
   virtual void flag_reservations_failure() = 0;
index bd5f61c9b9ec5284539ca4f286dd0785272aaf36..011ace2a28363b4d9d3015f5b19000995b6f6cd2 100644 (file)
@@ -142,8 +142,6 @@ void ReplicaReservations::send_next_reservation_or_complete()
   }
 }
 
-// temporary comment: the part of handle_reserve_reject() that will not
-// be delegated to the FSM in the following commits
 void ReplicaReservations::verify_rejections_source(
     OpRequestRef op,
     pg_shard_t from)
@@ -171,17 +169,6 @@ void ReplicaReservations::verify_rejections_source(
   }
 }
 
-// to be delegated to the FSM in the following commits
-void ReplicaReservations::handle_reserve_reject(
-    OpRequestRef op,
-    pg_shard_t from)
-{
-  verify_rejections_source(op, from);
-  release_all();
-  m_scrubber.flag_reservations_failure();
-  m_osds->queue_for_scrub_denied(m_pg, scrub_prio_t::low_priority);
-}
-
 std::optional<pg_shard_t> ReplicaReservations::get_last_sent() const
 {
   if (m_next_to_request == m_sorted_secondaries.cbegin()) {
index a52f385a4fdfecee6d5283c2f4b96d1650e1d739..634e7e580027fed687b2df853663b7c961445f17 100644 (file)
@@ -93,8 +93,6 @@ class ReplicaReservations {
    */
   void verify_rejections_source(OpRequestRef op, pg_shard_t from);
 
-  void handle_reserve_reject(OpRequestRef op, pg_shard_t from);
-
   /**
    * Notifies implementation that it is no longer responsible for releasing
    * tracked remote reservations.
index 24ef8206980a05e5a3c386b132ff414ef984c1c3..61be0bd5a62c4121693edb841f59a81c6a6891ac 100644 (file)
@@ -20,7 +20,7 @@ namespace Scrub {
   class ReplicaReservations;
 }
 
-/// Facilitating scrub-realated object access to private PG data
+/// Facilitating scrub-related object access to private PG data
 class ScrubberPasskey {
 private:
   friend class Scrub::ReplicaReservations;
@@ -362,12 +362,6 @@ struct ScrubPgIF {
    */
   virtual void send_remotes_reserved(epoch_t epoch_queued) = 0;
 
-  /**
-   * triggers the 'ReservationFailure' (at least one replica denied us the
-   * requested resources) state-machine event
-   */
-  virtual void send_reservation_failure(epoch_t epoch_queued) = 0;
-
   virtual void cleanup_store(ObjectStore::Transaction* t) = 0;
 
   virtual bool get_store_errors(const scrub_ls_arg_t& arg,
@@ -382,20 +376,6 @@ struct ScrubPgIF {
 
   // --------------- reservations -----------------------------------
 
-  /**
-   *  "forget" all replica reservations. No messages are sent to the
-   *  previously-reserved.
-   *
-   *  Used upon interval change. The replicas' state is guaranteed to
-   *  be reset separately by the interval-change event.
-   */
-  virtual void discard_replica_reservations() = 0;
-
-  /**
-   * clear both local and OSD-managed resource reservation flags
-   */
-  virtual void clear_scrub_reservations() = 0;
-
   /**
    * Reserve local scrub resources (managed by the OSD)
    *