]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: relocate determine_scrub_time()
authorRonen Friedman <rfriedma@redhat.com>
Thu, 12 May 2022 15:35:12 +0000 (15:35 +0000)
committerRonen Friedman <rfriedma@redhat.com>
Mon, 16 May 2022 09:32:45 +0000 (09:32 +0000)
This aux func calculates the next scrub time based on
configuration data, scrub history and the requested scrub
flags. As part of creating unit-tests for ScrubQueue,
determine_scrub_time() is moved to that component.

A related change codifies the OSD services used by
the scheduler.

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
src/osd/OSD.h
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

index 2da5de10aa69bb426dff0287109d404b75c3fcde..8aa5b8cfbd711127da8acf240e44d0d8b6b4a3fd 100644 (file)
@@ -94,7 +94,7 @@ class MMonGetPurgedSnapsReply;
 
 class OSD;
 
-class OSDService {
+class OSDService : public Scrub::ScrubSchedListener {
   using OpSchedulerItem = ceph::osd::scheduler::OpSchedulerItem;
 public:
   OSD *osd;
@@ -147,7 +147,7 @@ public:
     superblock = block;
   }
 
-  int get_nodeid() const { return whoami; }
+  int get_nodeid() const final { return whoami; }
 
   std::atomic<epoch_t> max_oldest_map;
 private:
@@ -290,7 +290,9 @@ public:
    * @param allow_requested_repair_only
    * @return a Scrub::attempt_t detailing either a success, or the failure reason.
    */
-  Scrub::schedule_result_t initiate_a_scrub(spg_t pgid, bool allow_requested_repair_only);
+  Scrub::schedule_result_t initiate_a_scrub(
+    spg_t pgid,
+    bool allow_requested_repair_only) final;
 
 
  private:
index 225f6011e7a142817dfcf4d3b532850a519974af..ddc84c44482c6250540305e6073cf84d6b36995b 100644 (file)
@@ -2,9 +2,7 @@
 // vim: ts=8 sw=2 smarttab
 #include "./osd_scrub_sched.h"
 
-#include "include/utime_fmt.h"
 #include "osd/OSD.h"
-#include "osd/osd_types_fmt.h"
 
 #include "pg_scrubber.h"
 
@@ -79,11 +77,12 @@ std::string ScrubQueue::ScrubJob::scheduling_state(utime_t now_is,
 #undef dout_context
 #define dout_context (cct)
 #undef dout_prefix
-#define dout_prefix \
-  *_dout << "osd." << osd_service.whoami << " scrub-queue::" << __func__ << " "
+#define dout_prefix                                                            \
+  *_dout << "osd." << osd_service.get_nodeid() << " scrub-queue::" << __func__ \
+        << " "
 
 
-ScrubQueue::ScrubQueue(CephContext* cct, OSDService& osds)
+ScrubQueue::ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds)
     : cct{cct}
     , osd_service{osds}
 {
@@ -98,7 +97,7 @@ ScrubQueue::ScrubQueue(CephContext* cct, OSDService& osds)
 
 std::optional<double> ScrubQueue::update_load_average()
 {
-  int hb_interval = cct->_conf->osd_heartbeat_interval;
+  int hb_interval = conf()->osd_heartbeat_interval;
   int n_samples = 60 * 24 * 24;
   if (hb_interval > 1) {
     n_samples /= hb_interval;
@@ -220,6 +219,48 @@ void ScrubQueue::update_job(ScrubJobRef scrub_job,
   scrub_job->update_schedule(adjusted);
 }
 
+ScrubQueue::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;
+
+  //dout(15) << fmt::format(": requested_scrub_t: {}", request_flags) << dendl; // RRR
+  dout(15) << ": requested_scrub_t: {}" <<  request_flags << dendl; 
+
+
+  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;
+    // 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;
+
+  } else {
+    res.proposed_time = pg_info.history.last_scrub_stamp;
+    res.min_interval = pool_conf.value_or(pool_opts_t::SCRUB_MIN_INTERVAL, 0.0);
+    res.max_interval = pool_conf.value_or(pool_opts_t::SCRUB_MAX_INTERVAL, 0.0);
+  }
+
+  dout(15) << fmt::format(
+               ": suggested: {} hist: {} v: {}/{} must: {} pool-min: {}",
+               res.proposed_time,
+               pg_info.history.last_scrub_stamp,
+               (bool)pg_info.stats.stats_invalid,
+               conf()->osd_scrub_invalid_stats,
+               (res.is_must == must_scrub_t::mandatory ? "y" : "n"),
+               res.min_interval)
+          << dendl;
+  return res;
+}
+
+
 // used under jobs_lock
 void ScrubQueue::move_failed_pgs(utime_t now_is)
 {
@@ -237,7 +278,7 @@ void ScrubQueue::move_failed_pgs(utime_t now_is)
 
       // determine the penalty time, after which the job should be reinstated
       utime_t after = now_is;
-      after += cct->_conf->osd_scrub_sleep * 2 + utime_t{300'000ms};
+      after += conf()->osd_scrub_sleep * 2 + utime_t{300'000ms};
 
       // note: currently - not taking 'deadline' into account when determining
       // 'penalty_timeout'.
@@ -309,7 +350,7 @@ Scrub::schedule_result_t ScrubQueue::select_pg_and_scrub(
   dout(10) << " reg./pen. sizes: " << to_scrub.size() << " / "
           << penalized.size() << dendl;
 
-  utime_t now_is = ceph_clock_now();
+  utime_t now_is = time_now();
 
   preconds.time_permit = scrub_time_permit(now_is);
   preconds.load_is_low = scrub_load_below_threshold();
@@ -489,12 +530,12 @@ ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
 
   if (g_conf()->subsys.should_gather<ceph_subsys_osd, 20>()) {
     dout(20) << "min t: " << times.min_interval
-            << " osd: " << cct->_conf->osd_scrub_min_interval
+            << " osd: " << conf()->osd_scrub_min_interval
             << " max t: " << times.max_interval
-            << " osd: " << cct->_conf->osd_scrub_max_interval << dendl;
+            << " osd: " << conf()->osd_scrub_max_interval << dendl;
 
     dout(20) << "at " << sched_n_dead.scheduled_at << " ratio "
-            << cct->_conf->osd_scrub_interval_randomize_ratio << dendl;
+            << conf()->osd_scrub_interval_randomize_ratio << dendl;
   }
 
   if (times.is_must == ScrubQueue::must_scrub_t::not_mandatory) {
@@ -502,15 +543,15 @@ ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
     // unless explicitly requested, postpone the scrub with a random delay
     double scrub_min_interval = times.min_interval > 0
                                  ? times.min_interval
-                                 : cct->_conf->osd_scrub_min_interval;
+                                 : conf()->osd_scrub_min_interval;
     double scrub_max_interval = times.max_interval > 0
                                  ? times.max_interval
-                                 : cct->_conf->osd_scrub_max_interval;
+                                 : conf()->osd_scrub_max_interval;
 
     sched_n_dead.scheduled_at += scrub_min_interval;
     double r = rand() / (double)RAND_MAX;
     sched_n_dead.scheduled_at +=
-      scrub_min_interval * cct->_conf->osd_scrub_interval_randomize_ratio * r;
+      scrub_min_interval * conf()->osd_scrub_interval_randomize_ratio * r;
 
     if (scrub_max_interval <= 0) {
       sched_n_dead.deadline = utime_t{};
@@ -526,15 +567,15 @@ ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
 
 double ScrubQueue::scrub_sleep_time(bool must_scrub) const
 {
-  double regular_sleep_period = cct->_conf->osd_scrub_sleep;
+  double regular_sleep_period = conf()->osd_scrub_sleep;
 
-  if (must_scrub || scrub_time_permit(ceph_clock_now())) {
+  if (must_scrub || scrub_time_permit(time_now())) {
     return regular_sleep_period;
   }
 
   // relevant if scrubbing started during allowed time, but continued into
   // forbidden hours
-  double extended_sleep = cct->_conf->osd_scrub_extended_sleep;
+  double extended_sleep = conf()->osd_scrub_extended_sleep;
   dout(20) << "w/ extended sleep (" << extended_sleep << ")" << dendl;
   return std::max(extended_sleep, regular_sleep_period);
 }
@@ -550,9 +591,9 @@ bool ScrubQueue::scrub_load_below_threshold() const
   // allow scrub if below configured threshold
   long cpus = sysconf(_SC_NPROCESSORS_ONLN);
   double loadavg_per_cpu = cpus > 0 ? loadavgs[0] / cpus : loadavgs[0];
-  if (loadavg_per_cpu < cct->_conf->osd_scrub_load_threshold) {
+  if (loadavg_per_cpu < conf()->osd_scrub_load_threshold) {
     dout(20) << "loadavg per cpu " << loadavg_per_cpu << " < max "
-            << cct->_conf->osd_scrub_load_threshold << " = yes" << dendl;
+            << conf()->osd_scrub_load_threshold << " = yes" << dendl;
     return true;
   }
 
@@ -565,7 +606,7 @@ bool ScrubQueue::scrub_load_below_threshold() const
   }
 
   dout(20) << "loadavg " << loadavgs[0] << " >= max "
-          << cct->_conf->osd_scrub_load_threshold << " and ( >= daily_loadavg "
+          << conf()->osd_scrub_load_threshold << " and ( >= daily_loadavg "
           << daily_loadavg << " or >= 15m avg " << loadavgs[2] << ") = no"
           << dendl;
   return false;
@@ -616,22 +657,22 @@ bool ScrubQueue::scrub_time_permit(utime_t now) const
   time_t tt = now.sec();
   localtime_r(&tt, &bdt);
 
-  bool day_permit = isbetween_modulo(cct->_conf->osd_scrub_begin_week_day,
-                                    cct->_conf->osd_scrub_end_week_day,
+  bool day_permit = isbetween_modulo(conf()->osd_scrub_begin_week_day,
+                                    conf()->osd_scrub_end_week_day,
                                     bdt.tm_wday);
   if (!day_permit) {
     dout(20) << "should run between week day "
-            << cct->_conf->osd_scrub_begin_week_day << " - "
-            << cct->_conf->osd_scrub_end_week_day << " now " << bdt.tm_wday
+            << conf()->osd_scrub_begin_week_day << " - "
+            << conf()->osd_scrub_end_week_day << " now " << bdt.tm_wday
             << " - no" << dendl;
     return false;
   }
 
-  bool time_permit = isbetween_modulo(cct->_conf->osd_scrub_begin_hour,
-                                     cct->_conf->osd_scrub_end_hour,
+  bool time_permit = isbetween_modulo(conf()->osd_scrub_begin_hour,
+                                     conf()->osd_scrub_end_hour,
                                      bdt.tm_hour);
-  dout(20) << "should run between " << cct->_conf->osd_scrub_begin_hour << " - "
-          << cct->_conf->osd_scrub_end_hour << " now (" << bdt.tm_hour
+  dout(20) << "should run between " << conf()->osd_scrub_begin_hour << " - "
+          << conf()->osd_scrub_end_hour << " now (" << bdt.tm_hour
           << ") = " << (time_permit ? "yes" : "no") << dendl;
   return time_permit;
 }
@@ -694,12 +735,12 @@ bool ScrubQueue::can_inc_scrubs() const
   // inc_scrubs_local() failures
   std::lock_guard lck{resource_lock};
 
-  if (scrubs_local + scrubs_remote < cct->_conf->osd_max_scrubs) {
+  if (scrubs_local + scrubs_remote < conf()->osd_max_scrubs) {
     return true;
   }
 
   dout(20) << " == false. " << scrubs_local << " local + " << scrubs_remote
-          << " remote >= max " << cct->_conf->osd_max_scrubs << dendl;
+          << " remote >= max " << conf()->osd_max_scrubs << dendl;
   return false;
 }
 
@@ -707,13 +748,13 @@ bool ScrubQueue::inc_scrubs_local()
 {
   std::lock_guard lck{resource_lock};
 
-  if (scrubs_local + scrubs_remote < cct->_conf->osd_max_scrubs) {
+  if (scrubs_local + scrubs_remote < conf()->osd_max_scrubs) {
     ++scrubs_local;
     return true;
   }
 
   dout(20) << ": " << scrubs_local << " local + " << scrubs_remote
-          << " remote >= max " << cct->_conf->osd_max_scrubs << dendl;
+          << " remote >= max " << conf()->osd_max_scrubs << dendl;
   return false;
 }
 
@@ -721,7 +762,7 @@ void ScrubQueue::dec_scrubs_local()
 {
   std::lock_guard lck{resource_lock};
   dout(20) << ": " << scrubs_local << " -> " << (scrubs_local - 1) << " (max "
-          << cct->_conf->osd_max_scrubs << ", remote " << scrubs_remote << ")"
+          << conf()->osd_max_scrubs << ", remote " << scrubs_remote << ")"
           << dendl;
 
   --scrubs_local;
@@ -732,16 +773,16 @@ bool ScrubQueue::inc_scrubs_remote()
 {
   std::lock_guard lck{resource_lock};
 
-  if (scrubs_local + scrubs_remote < cct->_conf->osd_max_scrubs) {
+  if (scrubs_local + scrubs_remote < conf()->osd_max_scrubs) {
     dout(20) << ": " << scrubs_remote << " -> " << (scrubs_remote + 1)
-            << " (max " << cct->_conf->osd_max_scrubs << ", local "
+            << " (max " << conf()->osd_max_scrubs << ", local "
             << scrubs_local << ")" << dendl;
     ++scrubs_remote;
     return true;
   }
 
   dout(20) << ": " << scrubs_local << " local + " << scrubs_remote
-          << " remote >= max " << cct->_conf->osd_max_scrubs << dendl;
+          << " remote >= max " << conf()->osd_max_scrubs << dendl;
   return false;
 }
 
@@ -749,7 +790,7 @@ void ScrubQueue::dec_scrubs_remote()
 {
   std::lock_guard lck{resource_lock};
   dout(20) << ": " << scrubs_remote << " -> " << (scrubs_remote - 1) << " (max "
-          << cct->_conf->osd_max_scrubs << ", local " << scrubs_local << ")"
+          << conf()->osd_max_scrubs << ", local " << scrubs_local << ")"
           << dendl;
   --scrubs_remote;
   ceph_assert(scrubs_remote >= 0);
@@ -760,5 +801,5 @@ void ScrubQueue::dump_scrub_reservations(ceph::Formatter* f) const
   std::lock_guard lck{resource_lock};
   f->dump_int("scrubs_local", scrubs_local);
   f->dump_int("scrubs_remote", scrubs_remote);
-  f->dump_int("osd_max_scrubs", cct->_conf->osd_max_scrubs);
+  f->dump_int("osd_max_scrubs", conf()->osd_max_scrubs);
 }
index c15df6b55fe3cb7d7b64df57503fef7d101b4c02..90d3c7e2ec3413923eb94f71e2f6197114da1b6e 100644 (file)
@@ -117,7 +117,8 @@ SqrubQueue interfaces (main functions):
 #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"
 
 class PG;
@@ -137,6 +138,29 @@ enum class schedule_result_t {
   preconditions               // time, configuration, etc.
 };
 
+// the OSD services provided to the scrub scheduler
+class ScrubSchedListener {
+ public:
+  virtual int get_nodeid() const = 0;  // returns the OSD number ('whoami')
+
+  /**
+   * A callback used by the ScrubQueue object to initiate a scrub on a specific
+   * PG.
+   *
+   * The request might fail for multiple reasons, as ScrubQueue cannot by its
+   * own check some of the PG-specific preconditions and those are checked here.
+   * See attempt_t definition.
+   *
+   * @return a Scrub::attempt_t detailing either a success, or the failure
+   * reason.
+   */
+  virtual schedule_result_t initiate_a_scrub(
+    spg_t pgid,
+    bool allow_requested_repair_only) = 0;
+
+  virtual ~ScrubSchedListener() {}
+};
+
 }  // namespace Scrub
 
 /**
@@ -161,7 +185,8 @@ class ScrubQueue {
                     // under lock
   };
 
-  ScrubQueue(CephContext* cct, OSDService& osds);
+  ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds);
+  virtual ~ScrubQueue() = default;
 
   struct scrub_schedule_t {
     utime_t scheduled_at{};
@@ -328,6 +353,10 @@ class ScrubQueue {
    */
   void update_job(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,
+                                     const pool_opts_t pool_conf) const;
+
  public:
   void dump_scrubs(ceph::Formatter* f) const;
 
@@ -366,7 +395,13 @@ class ScrubQueue {
 
  private:
   CephContext* cct;
-  OSDService& osd_service;
+  Scrub::ScrubSchedListener& osd_service;
+
+#ifdef WITH_SEASTAR
+  auto& conf() const { return local_conf(); }
+#else
+  auto& conf() const { return cct->_conf; }
+#endif
 
   /**
    *  jobs_lock protects the job containers and the relevant scrub-jobs state
@@ -454,4 +489,30 @@ class ScrubQueue {
     ScrubQContainer& group,
     const Scrub::ScrubPreconds& preconds,
     utime_t now_is);
+
+protected: // used by the unit-tests
+  /**
+   * unit-tests will override this function to return a mock time
+   */
+  virtual utime_t time_now() const { return ceph_clock_now(); }
+};
+
+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(),
+      "{}, {} dead: {} - {} / failure: {} / pen. t.o.: {} / queue state: {}",
+      sjob.pgid,
+      sjob.schedule.scheduled_at,
+      sjob.schedule.deadline,
+      sjob.registration_state(),
+      sjob.resources_failure,
+      sjob.penalty_timeout,
+      ScrubQueue::qu_state_text(sjob.state));
+  }
 };
index 56f94c9e8e9712076c9a6dbd6aa0caacf1afee59..8379208094b6a9071d3e31a8c40f2e98fc07e69c 100644 (file)
@@ -503,7 +503,10 @@ void PgScrubber::on_primary_change(const requested_scrub_t& request_flags)
           << dendl;
 
   if (is_primary()) {
-    auto suggested = determine_scrub_time(request_flags);
+    auto suggested = m_osds->get_scrub_services().determine_scrub_time(
+      request_flags,
+      m_pg->info,
+      m_pg->get_pgpool().info.opts);
     m_osds->get_scrub_services().register_with_osd(m_scrub_job, suggested);
   } else {
     m_osds->get_scrub_services().remove_from_osd_queue(m_scrub_job);
@@ -534,51 +537,16 @@ void PgScrubber::update_scrub_job(const requested_scrub_t& request_flags)
   }
 
   if (is_primary() && m_scrub_job) {
-    auto suggested = determine_scrub_time(request_flags);
+    auto suggested = m_osds->get_scrub_services().determine_scrub_time(
+      request_flags,
+      m_pg->info,
+      m_pg->get_pgpool().info.opts);
     m_osds->get_scrub_services().update_job(m_scrub_job, suggested);
   }
 
   dout(15) << __func__ << " done " << registration_state() << dendl;
 }
 
-ScrubQueue::sched_params_t PgScrubber::determine_scrub_time(
-  const requested_scrub_t& request_flags) const
-{
-  ScrubQueue::sched_params_t res;
-
-  if (!is_primary()) {
-    return res;         // with ok_to_scrub set to 'false'
-  }
-
-  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;
-    // we do not need the interval data in this case
-
-  } else if (m_pg->info.stats.stats_invalid &&
-            m_pg->get_cct()->_conf->osd_scrub_invalid_stats) {
-    res.proposed_time = ceph_clock_now();
-    res.is_must = ScrubQueue::must_scrub_t::mandatory;
-
-  } else {
-    res.proposed_time = m_pg->info.history.last_scrub_stamp;
-    res.min_interval =
-      m_pg->get_pgpool().info.opts.value_or(pool_opts_t::SCRUB_MIN_INTERVAL, 0.0);
-    res.max_interval =
-      m_pg->get_pgpool().info.opts.value_or(pool_opts_t::SCRUB_MAX_INTERVAL, 0.0);
-  }
-
-  dout(15) << __func__ << " suggested: " << res.proposed_time
-          << " hist: " << m_pg->info.history.last_scrub_stamp
-          << " v:" << m_pg->info.stats.stats_invalid << " / "
-          << m_pg->cct->_conf->osd_scrub_invalid_stats << " must:"
-          << (res.is_must == ScrubQueue::must_scrub_t::mandatory ? "y" : "n")
-          << " pool min: " << res.min_interval << dendl;
-  return res;
-}
-
 void PgScrubber::scrub_requested(scrub_level_t scrub_level,
                                 scrub_type_t scrub_type,
                                 requested_scrub_t& req_flags)
index 295dd445737f2d04851be2bc027c12424eed7a65..0ed13f352d740d9e2eed44340078af0b8905bda9 100644 (file)
@@ -831,9 +831,6 @@ class PgScrubber : public ScrubPgIF,
    */
   void request_rescrubbing(requested_scrub_t& req_flags);
 
-  ScrubQueue::sched_params_t determine_scrub_time(
-    const requested_scrub_t& request_flags) const;
-
   void unregister_from_osd();
 
   /*