]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: complete ScrubJob transition from within ScrubQueue
authorRonen Friedman <rfriedma@redhat.com>
Mon, 11 Sep 2023 06:56:33 +0000 (01:56 -0500)
committerRonen Friedman <rfriedma@redhat.com>
Wed, 20 Sep 2023 06:39:10 +0000 (01:39 -0500)
ScrubJob is now in the Scrub namespace.

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
src/osd/CMakeLists.txt
src/osd/scrubber/osd_scrub.cc
src/osd/scrubber/osd_scrub_sched.cc
src/osd/scrubber/osd_scrub_sched.h
src/osd/scrubber/pg_scrubber.cc
src/osd/scrubber/pg_scrubber.h
src/osd/scrubber/scrub_job.cc [new file with mode: 0644]
src/osd/scrubber/scrub_job.h
src/test/osd/test_scrub_sched.cc

index 75f91fb53742acdf558ce5f938f8f07295daa812..7d19424b404a64ef8a7e78fe5f7fdd5f0d86c74a 100644 (file)
@@ -25,6 +25,7 @@ set(osd_srcs
   scrubber/osd_scrub.cc
   scrubber/osd_scrub_sched.cc
   scrubber/PrimaryLogScrub.cc
+  scrubber/scrub_job.cc
   scrubber/scrub_machine.cc
   scrubber/scrub_resources.cc
   scrubber/ScrubStore.cc
index 97f266d3dee47f6ecd4d8fc799a78d17183eee7b..2cf6df8ea93150b27709ef196bea4e038a9edab5 100644 (file)
@@ -157,13 +157,13 @@ void ScrubQueue::dump_scrubs(ceph::Formatter* f) const
 
   f->open_array_section("scrubs");
 
-  std::for_each(to_scrub.cbegin(), to_scrub.cend(), [&f](const ScrubJobRef& j) {
-    j->dump(f);
-  });
+  std::for_each(
+      to_scrub.cbegin(), to_scrub.cend(),
+      [&f](const Scrub::ScrubJobRef& j) { j->dump(f); });
 
-  std::for_each(penalized.cbegin(),
-               penalized.cend(),
-               [&f](const ScrubJobRef& j) { j->dump(f); });
+  std::for_each(
+      penalized.cbegin(), penalized.cend(),
+      [&f](const Scrub::ScrubJobRef& j) { j->dump(f); });
 
   f->close_section();
 }
@@ -238,7 +238,7 @@ Scrub::schedule_result_t ScrubQueue::select_pg_and_scrub(
 
 // not holding jobs_lock. 'group' is a copy of the actual list.
 Scrub::schedule_result_t ScrubQueue::select_from_group(
-    ScrubQContainer& group,
+    Scrub::ScrubQContainer& group,
     const Scrub::OSDRestrictions& preconds,
     utime_t now_is)
 {
index baec441f2623e70c866fc59a53490107047d5c86..4fcca8e940affa4ec718b1e1eca77a03beee180f 100644 (file)
@@ -9,67 +9,13 @@
 using namespace ::std::chrono;
 using namespace ::std::chrono_literals;
 using namespace ::std::literals;
+using qu_state_t = Scrub::qu_state_t;
+using must_scrub_t = Scrub::must_scrub_t;
+using ScrubQContainer = Scrub::ScrubQContainer;
+using sched_params_t = Scrub::sched_params_t;
+using OSDRestrictions = Scrub::OSDRestrictions;
+using ScrubJob = Scrub::ScrubJob;
 
-// ////////////////////////////////////////////////////////////////////////// //
-// ScrubJob
-
-#define dout_context (cct)
-#define dout_subsys ceph_subsys_osd
-#undef dout_prefix
-#define dout_prefix *_dout << "osd." << whoami << " "
-
-ScrubQueue::ScrubJob::ScrubJob(CephContext* cct, const spg_t& pg, int node_id)
-    : RefCountedObject{cct}
-    , pgid{pg}
-    , whoami{node_id}
-    , cct{cct}
-{}
-
-// debug usage only
-ostream& operator<<(ostream& out, const ScrubQueue::ScrubJob& sjob)
-{
-  out << sjob.pgid << ",  " << sjob.schedule.scheduled_at
-      << " dead: " << sjob.schedule.deadline << " - "
-      << sjob.registration_state() << " / failure: " << sjob.resources_failure
-      << " / pen. t.o.: " << sjob.penalty_timeout
-      << " / queue state: " << ScrubQueue::qu_state_text(sjob.state);
-
-  return out;
-}
-
-void ScrubQueue::ScrubJob::update_schedule(
-  const ScrubQueue::scrub_schedule_t& adjusted)
-{
-  schedule = adjusted;
-  penalty_timeout = utime_t(0, 0);  // helps with debugging
-
-  // 'updated' is changed here while not holding jobs_lock. That's OK, as
-  // the (atomic) flag will only be cleared by select_pg_and_scrub() after
-  // scan_penalized() is called and the job was moved to the to_scrub queue.
-  updated = true;
-  dout(10) << fmt::format("{}: pg[{}] adjusted: {:s} ({})", __func__, pgid,
-                          schedule.scheduled_at, registration_state()) << dendl;
-}
-
-std::string ScrubQueue::ScrubJob::scheduling_state(utime_t now_is,
-                                                  bool is_deep_expected) const
-{
-  // if not in the OSD scheduling queues, not a candidate for scrubbing
-  if (state != qu_state_t::registered) {
-    return "no scrub is scheduled";
-  }
-
-  // if the time has passed, we are surely in the queue
-  // (note that for now we do not tell client if 'penalized')
-  if (now_is > schedule.scheduled_at) {
-    // we are never sure that the next scrub will indeed be shallow:
-    return fmt::format("queued for {}scrub", (is_deep_expected ? "deep " : ""));
-  }
-
-  return fmt::format("{}scrub scheduled @ {:s}",
-                    (is_deep_expected ? "deep " : ""),
-                    schedule.scheduled_at);
-}
 
 
 // ////////////////////////////////////////////////////////////////////////// //
@@ -96,6 +42,7 @@ ScrubQueue::ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds)
   }
 }
 
+
 /*
  * Modify the scrub job state:
  * - if 'registered' (as expected): mark as 'unregistering'. The job will be
@@ -106,7 +53,7 @@ ScrubQueue::ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds)
  *
  * Note: not holding the jobs lock
  */
-void ScrubQueue::remove_from_osd_queue(ScrubJobRef scrub_job)
+void ScrubQueue::remove_from_osd_queue(Scrub::ScrubJobRef scrub_job)
 {
   dout(15) << "removing pg[" << scrub_job->pgid << "] from OSD scrub queue"
           << dendl;
@@ -119,21 +66,21 @@ void ScrubQueue::remove_from_osd_queue(ScrubJobRef scrub_job)
   if (ret) {
 
     dout(10) << "pg[" << scrub_job->pgid << "] sched-state changed from "
-            << qu_state_text(expected_state) << " to "
-            << qu_state_text(scrub_job->state) << dendl;
+            << ScrubJob::qu_state_text(expected_state) << " to "
+            << ScrubJob::qu_state_text(scrub_job->state) << dendl;
 
   } else {
 
     // job wasn't in state 'registered' coming in
     dout(5) << "removing pg[" << scrub_job->pgid
-           << "] failed. State was: " << qu_state_text(expected_state)
+           << "] failed. State was: " << ScrubJob::qu_state_text(expected_state)
            << dendl;
   }
 }
 
 void ScrubQueue::register_with_osd(
-  ScrubJobRef scrub_job,
-  const ScrubQueue::sched_params_t& suggested)
+  Scrub::ScrubJobRef scrub_job,
+  const sched_params_t& suggested)
 {
   qu_state_t state_at_entry = scrub_job->state.load();
   dout(20) << fmt::format(
@@ -193,31 +140,31 @@ void ScrubQueue::register_with_osd(
 }
 
 // look mommy - no locks!
-void ScrubQueue::update_job(ScrubJobRef scrub_job,
-                           const ScrubQueue::sched_params_t& suggested)
+void ScrubQueue::update_job(Scrub::ScrubJobRef scrub_job,
+                           const sched_params_t& suggested)
 {
   // adjust the suggested scrub time according to OSD-wide status
   auto adjusted = adjust_target_time(suggested);
   scrub_job->update_schedule(adjusted);
 }
 
-ScrubQueue::sched_params_t ScrubQueue::determine_scrub_time(
+sched_params_t ScrubQueue::determine_scrub_time(
   const requested_scrub_t& request_flags,
   const pg_info_t& pg_info,
   const pool_opts_t& pool_conf) const
 {
-  ScrubQueue::sched_params_t res;
+  sched_params_t res;
 
   if (request_flags.must_scrub || request_flags.need_auto) {
 
     // Set the smallest time that isn't utime_t()
     res.proposed_time = PgScrubber::scrub_must_stamp();
-    res.is_must = ScrubQueue::must_scrub_t::mandatory;
+    res.is_must = Scrub::must_scrub_t::mandatory;
     // we do not need the interval data in this case
 
   } else if (pg_info.stats.stats_invalid && conf()->osd_scrub_invalid_stats) {
     res.proposed_time = time_now();
-    res.is_must = ScrubQueue::must_scrub_t::mandatory;
+    res.is_must = Scrub::must_scrub_t::mandatory;
 
   } else {
     res.proposed_time = pg_info.history.last_scrub_stamp;
@@ -250,7 +197,7 @@ void ScrubQueue::move_failed_pgs(utime_t now_is)
       // remote resources. Move it to the secondary scrub queue.
 
       dout(15) << "moving " << sjob->pgid
-              << " state: " << ScrubQueue::qu_state_text(sjob->state) << dendl;
+              << " state: " << ScrubJob::qu_state_text(sjob->state) << dendl;
 
       // determine the penalty time, after which the job should be reinstated
       utime_t after = now_is;
@@ -298,7 +245,7 @@ std::string_view ScrubQueue::attempt_res_text(Scrub::schedule_result_t v)
   return "(unknown)"sv;
 }
 
-std::string_view ScrubQueue::qu_state_text(qu_state_t st)
+std::string_view ScrubJob::qu_state_text(qu_state_t st)
 {
   switch (st) {
     case qu_state_t::not_registered: return "not registered w/ OSD"sv;
@@ -329,8 +276,8 @@ void ScrubQueue::rm_unregistered_jobs(ScrubQContainer& group)
 
 namespace {
 struct cmp_sched_time_t {
-  bool operator()(const ScrubQueue::ScrubJobRef& lhs,
-                 const ScrubQueue::ScrubJobRef& rhs) const
+  bool operator()(const Scrub::ScrubJobRef& lhs,
+                 const Scrub::ScrubJobRef& rhs) const
   {
     return lhs->schedule.scheduled_at < rhs->schedule.scheduled_at;
   }
@@ -338,14 +285,14 @@ struct cmp_sched_time_t {
 }  // namespace
 
 // called under lock
-ScrubQueue::ScrubQContainer ScrubQueue::collect_ripe_jobs(
+ScrubQContainer ScrubQueue::collect_ripe_jobs(
   ScrubQContainer& group,
   utime_t time_now)
 {
   rm_unregistered_jobs(group);
 
   // copy ripe jobs
-  ScrubQueue::ScrubQContainer ripes;
+  ScrubQContainer ripes;
   ripes.reserve(group.size());
 
   std::copy_if(group.begin(),
@@ -369,13 +316,13 @@ ScrubQueue::ScrubQContainer ScrubQueue::collect_ripe_jobs(
 }
 
 
-ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
+Scrub::scrub_schedule_t ScrubQueue::adjust_target_time(
   const sched_params_t& times) const
 {
-  ScrubQueue::scrub_schedule_t sched_n_dead{
+  Scrub::scrub_schedule_t sched_n_dead{
     times.proposed_time, times.proposed_time};
 
-  if (times.is_must == ScrubQueue::must_scrub_t::not_mandatory) {
+  if (times.is_must == Scrub::must_scrub_t::not_mandatory) {
     // unless explicitly requested, postpone the scrub with a random delay
     double scrub_min_interval = times.min_interval > 0
                                  ? times.min_interval
@@ -410,6 +357,7 @@ ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
   return sched_n_dead;
 }
 
+
 // note: called with jobs_lock held
 void ScrubQueue::scan_penalized(bool forgive_all, utime_t time_now)
 {
@@ -440,7 +388,7 @@ void ScrubQueue::scan_penalized(bool forgive_all, utime_t time_now)
   }
 }
 
-void ScrubQueue::ScrubJob::dump(ceph::Formatter* f) const
+void ScrubJob::dump(ceph::Formatter* f) const
 {
   f->open_object_section("scrub");
   f->dump_stream("pgid") << pgid;
@@ -451,9 +399,9 @@ void ScrubQueue::ScrubJob::dump(ceph::Formatter* f) const
   f->close_section();
 }
 
-ScrubQueue::ScrubQContainer ScrubQueue::list_registered_jobs() const
+ScrubQContainer ScrubQueue::list_registered_jobs() const
 {
-  ScrubQueue::ScrubQContainer all_jobs;
+  ScrubQContainer all_jobs;
   all_jobs.reserve(to_scrub.size() + penalized.size());
   dout(20) << " size: " << all_jobs.capacity() << dendl;
 
index 4727f3c7d5b7dcc64379dbb06737219f75d2cf8d..90b5c9f400894797b2f53d378b9fee5b6ba31483 100644 (file)
@@ -107,20 +107,9 @@ ScrubQueue interfaces (main functions):
  */
 // clang-format on
 
-#include <atomic>
-#include <chrono>
-#include <memory>
 #include <optional>
-#include <vector>
-
-#include "common/RefCountedObj.h"
-#include "common/ceph_atomic.h"
-#include "osd/osd_types.h"
-#include "osd/scrubber_common.h"
-#include "include/utime_fmt.h"
-#include "osd/osd_types_fmt.h"
 #include "utime.h"
-
+#include "scrub_job.h"
 class PG;
 
 namespace Scrub {
@@ -163,6 +152,7 @@ class ScrubSchedListener {
 
 }  // namespace Scrub
 
+
 /**
  * the queue of PGs waiting to be scrubbed.
  * Main operations are scheduling/unscheduling a PG to be scrubbed at a certain
@@ -175,28 +165,12 @@ class ScrubSchedListener {
  */
 class ScrubQueue {
  public:
-  enum class must_scrub_t { not_mandatory, mandatory };
-
-  enum class qu_state_t {
-    not_registered,  // not a primary, thus not considered for scrubbing by this
-                    // OSD (also the temporary state when just created)
-    registered,             // in either of the two queues ('to_scrub' or 'penalized')
-    unregistering    // in the process of being unregistered. Will be finalized
-                    // under lock
-  };
-
   ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds);
   virtual ~ScrubQueue() = default;
 
-#include "osd/scrubber/scrub_job.h"
-
   friend class TestOSDScrub;
   friend class ScrubSchedTestWrapper; ///< unit-tests structure
-
-  using ScrubJobRef = ceph::ref_t<ScrubJob>;
-  using ScrubQContainer = std::vector<ScrubJobRef>;
-
-  static std::string_view qu_state_text(qu_state_t st);
+  using sched_params_t = Scrub::sched_params_t;
 
   /**
    * called periodically by the OSD to select the first scrub-eligible PG
@@ -225,13 +199,13 @@ class ScrubQueue {
    * remove the pg from set of PGs to be scanned for scrubbing.
    * To be used if we are no longer the PG's primary, or if the PG is removed.
    */
-  void remove_from_osd_queue(ScrubJobRef sjob);
+  void remove_from_osd_queue(Scrub::ScrubJobRef sjob);
 
   /**
    * @return the list (not std::set!) of all scrub jobs registered
    *   (apart from PGs in the process of being removed)
    */
-  ScrubQContainer list_registered_jobs() const;
+  Scrub::ScrubQContainer list_registered_jobs() const;
 
   /**
    * Add the scrub job to the list of jobs (i.e. list of PGs) to be periodically
@@ -243,7 +217,7 @@ class ScrubQueue {
    *
    * locking: might lock jobs_lock
    */
-  void register_with_osd(ScrubJobRef sjob, const sched_params_t& suggested);
+  void register_with_osd(Scrub::ScrubJobRef sjob, const sched_params_t& suggested);
 
   /**
    * modify a scrub-job's scheduled time and deadline
@@ -265,7 +239,7 @@ class ScrubQueue {
    *
    *  locking: not using the jobs_lock
    */
-  void update_job(ScrubJobRef sjob, const sched_params_t& suggested);
+  void update_job(Scrub::ScrubJobRef sjob, const sched_params_t& suggested);
 
   sched_params_t determine_scrub_time(const requested_scrub_t& request_flags,
                                      const pg_info_t& pg_info,
@@ -346,18 +320,18 @@ class ScrubQueue {
    */
   mutable ceph::mutex jobs_lock = ceph::make_mutex("ScrubQueue::jobs_lock");
 
-  ScrubQContainer to_scrub;   ///< scrub jobs (i.e. PGs) to scrub
-  ScrubQContainer penalized;  ///< those that failed to reserve remote resources
+  Scrub::ScrubQContainer to_scrub;   ///< scrub jobs (i.e. PGs) to scrub
+  Scrub::ScrubQContainer penalized;  ///< those that failed to reserve remote resources
   bool restore_penalized{false};
 
   double daily_loadavg{0.0};
 
   static inline constexpr auto registered_job = [](const auto& jobref) -> bool {
-    return jobref->state == qu_state_t::registered;
+    return jobref->state == Scrub::qu_state_t::registered;
   };
 
   static inline constexpr auto invalid_state = [](const auto& jobref) -> bool {
-    return jobref->state == qu_state_t::not_registered;
+    return jobref->state == Scrub::qu_state_t::not_registered;
   };
 
   /**
@@ -369,7 +343,7 @@ class ScrubQueue {
    * clear dead entries (unregistered, or belonging to removed PGs) from a
    * queue. Job state is changed to match new status.
    */
-  void rm_unregistered_jobs(ScrubQContainer& group);
+  void rm_unregistered_jobs(Scrub::ScrubQContainer& group);
 
   /**
    * the set of all scrub jobs in 'group' which are ready to be scrubbed
@@ -379,8 +353,9 @@ class ScrubQueue {
    *
    * Note that the returned container holds independent refs to the
    * scrub jobs.
+   * Note also that OSDRestrictions is 1L size, thus copied.
    */
-  ScrubQContainer collect_ripe_jobs(ScrubQContainer& group, utime_t time_now);
+  Scrub::ScrubQContainer collect_ripe_jobs(Scrub::ScrubQContainer& group, utime_t time_now);
 
 
   /// scrub resources management lock (guarding scrubs_local & scrubs_remote)
@@ -416,8 +391,8 @@ class ScrubQueue {
    *
    * @return a pair of values: the determined scrub time, and the deadline
    */
-  scrub_schedule_t adjust_target_time(
-    const sched_params_t& recomputed_params) const;
+  Scrub::scrub_schedule_t adjust_target_time(
+    const Scrub::sched_params_t& recomputed_params) const;
 
   /**
    * Look for scrub jobs that have their 'resources_failure' set. These jobs
@@ -430,7 +405,7 @@ class ScrubQueue {
   void move_failed_pgs(utime_t now_is);
 
   Scrub::schedule_result_t select_from_group(
-    ScrubQContainer& group,
+    Scrub::ScrubQContainer& group,
     const Scrub::OSDRestrictions& preconds,
     utime_t now_is);
 
@@ -440,34 +415,3 @@ protected: // used by the unit-tests
    */
   virtual utime_t time_now() const { return ceph_clock_now(); }
 };
-
-template <>
-struct fmt::formatter<ScrubQueue::qu_state_t>
-    : fmt::formatter<std::string_view> {
-  template <typename FormatContext>
-  auto format(const ScrubQueue::qu_state_t& s, FormatContext& ctx)
-  {
-    auto out = ctx.out();
-    out = fmt::formatter<string_view>::format(
-      std::string{ScrubQueue::qu_state_text(s)}, ctx);
-    return out;
-  }
-};
-
-template <>
-struct fmt::formatter<ScrubQueue::ScrubJob> {
-  constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
-
-  template <typename FormatContext>
-  auto format(const ScrubQueue::ScrubJob& sjob, FormatContext& ctx)
-  {
-    return fmt::format_to(
-      ctx.out(),
-      "pg[{}] @ {:s} (dl:{:s}) - <{}> / failure: {} / pen. t.o.: {:s} / queue "
-      "state: {:.7}",
-      sjob.pgid, sjob.schedule.scheduled_at, sjob.schedule.deadline,
-      sjob.registration_state(), sjob.resources_failure, sjob.penalty_timeout,
-      sjob.state.load(std::memory_order_relaxed));
-  }
-};
-
index 339fe1e7155ac9ae3ec9032a2cd303c78b70bc73..6a40f2bb1179c926c25c47e4020cf6f631425dc8 100644 (file)
@@ -2085,7 +2085,7 @@ pg_scrubbing_status_t PgScrubber::get_schedule() const
        false /* is periodic? unknown, actually */};
     }
   }
-  if (m_scrub_job->state != ScrubQueue::qu_state_t::registered) {
+  if (m_scrub_job->state != Scrub::qu_state_t::registered) {
     return pg_scrubbing_status_t{utime_t{},
                                 0,
                                 pg_scrub_sched_status_t::not_queued,
@@ -2168,9 +2168,8 @@ PgScrubber::PgScrubber(PG* pg)
   m_fsm = std::make_unique<ScrubMachine>(m_pg, this);
   m_fsm->initiate();
 
-  m_scrub_job = ceph::make_ref<ScrubQueue::ScrubJob>(m_osds->cct,
-                                                    m_pg->pg_id,
-                                                    m_osds->get_nodeid());
+  m_scrub_job = ceph::make_ref<Scrub::ScrubJob>(
+      m_osds->cct, m_pg->pg_id, m_osds->get_nodeid());
 }
 
 void PgScrubber::set_scrub_begin_time()
@@ -2469,7 +2468,7 @@ void ReplicaReservations::release_replica(pg_shard_t peer, epoch_t epoch)
 ReplicaReservations::ReplicaReservations(
   PG* pg,
   pg_shard_t whoami,
-  ScrubQueue::ScrubJobRef scrubjob,
+  Scrub::ScrubJobRef scrubjob,
   const ConfigProxy& conf)
     : m_pg{pg}
     , m_acting_set{pg->get_actingset()}
index 5810f1fd986fa72f53758a3ec6c57996b5ff061e..52428599514e7d0b01cea8ca0369b3b6d97a7bfd 100644 (file)
@@ -131,7 +131,7 @@ class ReplicaReservations {
   bool m_had_rejections{false};
   int m_pending{-1};
   const pg_info_t& m_pg_info;
-  ScrubQueue::ScrubJobRef m_scrub_job; ///< a ref to this PG's scrub job
+  Scrub::ScrubJobRef m_scrub_job;      ///< a ref to this PG's scrub job
   const ConfigProxy& m_conf;
 
   // detecting slow peers (see 'slow-secondary' above)
@@ -161,7 +161,7 @@ class ReplicaReservations {
 
   ReplicaReservations(PG* pg,
                       pg_shard_t whoami,
-                      ScrubQueue::ScrubJobRef scrubjob,
+                      Scrub::ScrubJobRef scrubjob,
                       const ConfigProxy& conf); 
 
   ~ReplicaReservations();
@@ -595,7 +595,7 @@ class PgScrubber : public ScrubPgIF,
   virtual void _scrub_clear_state() {}
 
   utime_t m_scrub_reg_stamp;           ///< stamp we registered for
-  ScrubQueue::ScrubJobRef m_scrub_job; ///< the scrub-job used by the OSD to
+  Scrub::ScrubJobRef m_scrub_job;      ///< the scrub-job used by the OSD to
                                        ///< schedule us
 
   ostream& show(ostream& out) const override;
diff --git a/src/osd/scrubber/scrub_job.cc b/src/osd/scrubber/scrub_job.cc
new file mode 100644 (file)
index 0000000..bf9649c
--- /dev/null
@@ -0,0 +1,83 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "./scrub_job.h"
+
+using qu_state_t = Scrub::qu_state_t;
+using must_scrub_t = Scrub::must_scrub_t;
+using ScrubQContainer = Scrub::ScrubQContainer;
+using sched_params_t = Scrub::sched_params_t;
+using OSDRestrictions = Scrub::OSDRestrictions;
+using ScrubJob = Scrub::ScrubJob;
+
+
+// ////////////////////////////////////////////////////////////////////////// //
+// ScrubJob
+
+#define dout_subsys ceph_subsys_osd
+#undef dout_context
+#define dout_context (cct)
+#undef dout_prefix
+#define dout_prefix _prefix_fn(_dout, this, __func__)
+
+template <class T>
+static std::ostream& _prefix_fn(std::ostream* _dout, T* t, std::string fn = "")
+{
+  return t->gen_prefix(*_dout, fn);
+}
+
+ScrubJob::ScrubJob(CephContext* cct, const spg_t& pg, int node_id)
+    : RefCountedObject{cct}
+    , pgid{pg}
+    , whoami{node_id}
+    , cct{cct}
+    , log_msg_prefix{fmt::format("osd.{}: scrub-job:pg[{}]:", node_id, pgid)}
+{}
+
+// debug usage only
+namespace std {
+ostream& operator<<(ostream& out, const ScrubJob& sjob)
+{
+  return out << fmt::format("{}", sjob);
+}
+}  // namespace std
+
+void ScrubJob::update_schedule(const Scrub::scrub_schedule_t& adjusted)
+{
+  schedule = adjusted;
+  penalty_timeout = utime_t(0, 0);  // helps with debugging
+
+  // 'updated' is changed here while not holding jobs_lock. That's OK, as
+  // the (atomic) flag will only be cleared by select_pg_and_scrub() after
+  // scan_penalized() is called and the job was moved to the to_scrub queue.
+  updated = true;
+  dout(10) << fmt::format(
+                 "adjusted: {:s} ({})", schedule.scheduled_at,
+                 registration_state())
+          << dendl;
+}
+
+std::string ScrubJob::scheduling_state(utime_t now_is, bool is_deep_expected)
+    const
+{
+  // if not in the OSD scheduling queues, not a candidate for scrubbing
+  if (state != qu_state_t::registered) {
+    return "no scrub is scheduled";
+  }
+
+  // if the time has passed, we are surely in the queue
+  // (note that for now we do not tell client if 'penalized')
+  if (now_is > schedule.scheduled_at) {
+    // we are never sure that the next scrub will indeed be shallow:
+    return fmt::format("queued for {}scrub", (is_deep_expected ? "deep " : ""));
+  }
+
+  return fmt::format(
+      "{}scrub scheduled @ {:s}", (is_deep_expected ? "deep " : ""),
+      schedule.scheduled_at);
+}
+
+std::ostream& ScrubJob::gen_prefix(std::ostream& out, std::string_view fn) const
+{
+  return out << log_msg_prefix << fn << ": ";
+}
index 5bddb50200e92db2e780f521ce524c2bf90e78c6..6f8191cdb42d4b87a951205a7e815414b3002aa3 100644 (file)
@@ -1,21 +1,47 @@
 // -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
+#pragma once
+
+#include <atomic>
+#include <chrono>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+#include "common/RefCountedObj.h"
+#include "common/ceph_atomic.h"
+#include "include/utime_fmt.h"
+#include "osd/osd_types.h"
+#include "osd/osd_types_fmt.h"
+#include "osd/scrubber_common.h"
+
+
+namespace Scrub {
+
+enum class must_scrub_t { not_mandatory, mandatory };
+
+enum class qu_state_t {
+  not_registered,  // not a primary, thus not considered for scrubbing by this
+                  // OSD (also the temporary state when just created)
+  registered,     // in either of the two queues ('to_scrub' or 'penalized')
+  unregistering           // in the process of being unregistered. Will be finalized
+                  // under lock
+};
 
 struct scrub_schedule_t {
   utime_t scheduled_at{};
   utime_t deadline{0, 0};
 };
 
-
 struct sched_params_t {
   utime_t proposed_time{};
   double min_interval{0.0};
   double max_interval{0.0};
-    must_scrub_t is_must{ScrubQueue::must_scrub_t::not_mandatory};
+  must_scrub_t is_must{must_scrub_t::not_mandatory};
 };
 
-  struct ScrubJob final : public RefCountedObject {
-
+class ScrubJob final : public RefCountedObject {
+ public:
   /**
    * a time scheduled for scrub, and a deadline: The scrub could be delayed
    * if system load is too high (but not if after the deadline),or if trying
@@ -64,6 +90,7 @@ struct sched_params_t {
 
   utime_t get_sched_time() const { return schedule.scheduled_at; }
 
+  static std::string_view qu_state_text(qu_state_t st);
 
   /**
    * relatively low-cost(*) access to the scrub job's state, to be used in
@@ -72,10 +99,10 @@ struct sched_params_t {
    */
   std::string_view state_desc() const
   {
-      return ScrubQueue::qu_state_text(state.load(std::memory_order_relaxed));
+    return qu_state_text(state.load(std::memory_order_relaxed));
   }
 
-    void update_schedule(const ScrubQueue::scrub_schedule_t& adjusted);
+  void update_schedule(const scrub_schedule_t& adjusted);
 
   void dump(ceph::Formatter* f) const;
 
@@ -102,5 +129,46 @@ struct sched_params_t {
    */
   std::string scheduling_state(utime_t now_is, bool is_deep_expected) const;
 
-  friend std::ostream& operator<<(std::ostream& out, const ScrubJob& pg);
+  std::ostream& gen_prefix(std::ostream& out, std::string_view fn) const;
+  const std::string log_msg_prefix;
+};
+
+using ScrubJobRef = ceph::ref_t<ScrubJob>;
+using ScrubQContainer = std::vector<ScrubJobRef>;
+}  // namespace Scrub
+
+namespace std {
+std::ostream& operator<<(std::ostream& out, const Scrub::ScrubJob& pg);
+}  // namespace std
+
+namespace fmt {
+template <>
+struct formatter<Scrub::qu_state_t> : formatter<std::string_view> {
+  template <typename FormatContext>
+  auto format(const Scrub::qu_state_t& s, FormatContext& ctx)
+  {
+    auto out = ctx.out();
+    out = fmt::formatter<string_view>::format(
+       std::string{Scrub::ScrubJob::qu_state_text(s)}, ctx);
+    return out;
+  }
+};
+
+template <>
+struct formatter<Scrub::ScrubJob> {
+  constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+  template <typename FormatContext>
+  auto format(const Scrub::ScrubJob& sjob, FormatContext& ctx)
+  {
+    return fmt::format_to(
+       ctx.out(),
+       "pg[{}] @ {:s} (dl:{:s}) - <{}> / failure: {} / pen. t.o.: {:s} / "
+       "queue "
+       "state: {:.7}",
+       sjob.pgid, sjob.schedule.scheduled_at, sjob.schedule.deadline,
+       sjob.registration_state(), sjob.resources_failure, sjob.penalty_timeout,
+       sjob.state.load(std::memory_order_relaxed));
+  }
 };
+}  // namespace fmt
index efd7ba3240dad87bf6f02e50f4b1215f95fdca7a..a3f1a348584aa2432006ca481290bebf1cd47adf 100644 (file)
@@ -42,8 +42,10 @@ int main(int argc, char** argv)
 }
 
 using schedule_result_t = Scrub::schedule_result_t;
-using ScrubJobRef = ScrubQueue::ScrubJobRef;
-using qu_state_t = ScrubQueue::qu_state_t;
+using ScrubJobRef = Scrub::ScrubJobRef;
+using qu_state_t = Scrub::qu_state_t;
+using scrub_schedule_t = Scrub::scrub_schedule_t;
+using ScrubQContainer = Scrub::ScrubQContainer;
 
 /// enabling access into ScrubQueue internals
 class ScrubSchedTestWrapper : public ScrubQueue {
@@ -128,7 +130,7 @@ struct sjob_config_t {
   std::optional<double> pool_conf_max;
   bool is_must;
   bool is_need_auto;
-  ScrubQueue::scrub_schedule_t initial_schedule;
+  scrub_schedule_t initial_schedule;
 };
 
 
@@ -141,7 +143,7 @@ struct sjob_dynamic_data_t {
   pg_info_t mocked_pg_info;
   pool_opts_t mocked_pool_opts;
   requested_scrub_t request_flags;
-  ScrubQueue::ScrubJobRef job;
+  ScrubJobRef job;
 };
 
 class TestScrubSched : public ::testing::Test {
@@ -197,7 +199,7 @@ class TestScrubSched : public ::testing::Test {
     dyn_data.request_flags.need_auto = sjob_data.is_need_auto;
 
     // create the scrub job
-    dyn_data.job = ceph::make_ref<ScrubQueue::ScrubJob>(g_ceph_context,
+    dyn_data.job = ceph::make_ref<Scrub::ScrubJob>(g_ceph_context,
                                                        sjob_data.spg,
                                                        m_osd_num);
     m_scrub_jobs.push_back(dyn_data);
@@ -252,7 +254,7 @@ class TestScrubSched : public ::testing::Test {
   }
 
   void debug_print_jobs(std::string hdr,
-                       const ScrubQueue::ScrubQContainer& jobs)
+                       const ScrubQContainer& jobs)
   {
     std::cout << fmt::format("{}: time now {}", hdr, m_sched->time_now())
              << std::endl;
@@ -287,7 +289,7 @@ std::vector<sjob_config_t> sjob_configs = {
     std::nullopt,                  // max scrub delay in pool config
     false,                         // must-scrub
     false,                         // need-auto
-    ScrubQueue::scrub_schedule_t{}  // initial schedule
+    scrub_schedule_t{}  // initial schedule
   },
 
   {spg_t{pg_t{4, 1}},
@@ -297,7 +299,7 @@ std::vector<sjob_config_t> sjob_configs = {
    std::nullopt,
    true,
    false,
-   ScrubQueue::scrub_schedule_t{}},
+   scrub_schedule_t{}},
 
   {spg_t{pg_t{7, 1}},
    true,
@@ -306,7 +308,7 @@ std::vector<sjob_config_t> sjob_configs = {
    std::nullopt,
    false,
    false,
-   ScrubQueue::scrub_schedule_t{}},
+   scrub_schedule_t{}},
 
   {spg_t{pg_t{5, 1}},
    true,
@@ -315,7 +317,7 @@ std::vector<sjob_config_t> sjob_configs = {
    std::nullopt,
    false,
    false,
-   ScrubQueue::scrub_schedule_t{}}};
+   scrub_schedule_t{}}};
 
 }  // anonymous namespace