]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/osd/scrubber/osd_scrub_sched.cc
update ceph source to reef 18.1.2
[ceph.git] / ceph / src / osd / scrubber / osd_scrub_sched.cc
index 99f73b13164886e60e7daa3e909d9985ecf3d915..82b7c689d4cd45568567a5193503f70f4dabdd3f 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"
 
@@ -16,18 +14,21 @@ using namespace ::std::literals;
 #define dout_context (cct)
 #define dout_subsys ceph_subsys_osd
 #undef dout_prefix
-#define dout_prefix *_dout << "osd." << whoami << "  "
+#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}
+    : 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
+      << " 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);
 
@@ -44,9 +45,8 @@ void ScrubQueue::ScrubJob::update_schedule(
   // 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) << " pg[" << pgid << "] adjusted: " << schedule.scheduled_at << "  "
-          << registration_state() << dendl;
+  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,
@@ -64,7 +64,8 @@ std::string ScrubQueue::ScrubJob::scheduling_state(utime_t now_is,
     return fmt::format("queued for {}scrub", (is_deep_expected ? "deep " : ""));
   }
 
-  return fmt::format("{}scrub scheduled @ {}", (is_deep_expected ? "deep " : ""),
+  return fmt::format("{}scrub scheduled @ {:s}",
+                    (is_deep_expected ? "deep " : ""),
                     schedule.scheduled_at);
 }
 
@@ -75,12 +76,14 @@ 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)
-    : cct{cct}, osd_service{osds}
+ScrubQueue::ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds)
+    : cct{cct}
+    , osd_service{osds}
 {
   // initialize the daily loadavg with current 15min loadavg
   if (double loadavgs[3]; getloadavg(loadavgs, 3) == 3) {
@@ -93,7 +96,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;
@@ -128,8 +131,9 @@ void ScrubQueue::remove_from_osd_queue(ScrubJobRef scrub_job)
           << dendl;
 
   qu_state_t expected_state{qu_state_t::registered};
-  auto ret = scrub_job->state.compare_exchange_strong(expected_state,
-                                                     qu_state_t::unregistering);
+  auto ret =
+    scrub_job->state.compare_exchange_strong(expected_state,
+                                            qu_state_t::unregistering);
 
   if (ret) {
 
@@ -141,17 +145,20 @@ void ScrubQueue::remove_from_osd_queue(ScrubJobRef scrub_job)
 
     // job wasn't in state 'registered' coming in
     dout(5) << "removing pg[" << scrub_job->pgid
-           << "] failed. State was: " << qu_state_text(expected_state) << dendl;
+           << "] failed. State was: " << qu_state_text(expected_state)
+           << dendl;
   }
 }
 
-void ScrubQueue::register_with_osd(ScrubJobRef scrub_job,
-                                  const ScrubQueue::sched_params_t& suggested)
+void ScrubQueue::register_with_osd(
+  ScrubJobRef scrub_job,
+  const ScrubQueue::sched_params_t& suggested)
 {
   qu_state_t state_at_entry = scrub_job->state.load();
-
-  dout(15) << "pg[" << scrub_job->pgid << "] was "
-          << qu_state_text(state_at_entry) << dendl;
+  dout(20) << fmt::format(
+               "pg[{}] state at entry: <{:.14}>", scrub_job->pgid,
+               state_at_entry)
+          << dendl;
 
   switch (state_at_entry) {
     case qu_state_t::registered:
@@ -166,7 +173,7 @@ void ScrubQueue::register_with_osd(ScrubJobRef scrub_job,
 
        if (state_at_entry != scrub_job->state) {
          lck.unlock();
-         dout(5) << " scrub job state changed" << dendl;
+         dout(5) << " scrub job state changed. Retrying." << dendl;
          // retry
          register_with_osd(scrub_job, suggested);
          break;
@@ -177,7 +184,6 @@ void ScrubQueue::register_with_osd(ScrubJobRef scrub_job,
        scrub_job->in_queues = true;
        scrub_job->state = qu_state_t::registered;
       }
-
       break;
 
     case qu_state_t::unregistering:
@@ -198,10 +204,11 @@ void ScrubQueue::register_with_osd(ScrubJobRef scrub_job,
       break;
   }
 
-  dout(10) << "pg(" << scrub_job->pgid << ") sched-state changed from "
-          << qu_state_text(state_at_entry) << " to "
-          << qu_state_text(scrub_job->state)
-          << " at: " << scrub_job->schedule.scheduled_at << dendl;
+  dout(10) << fmt::format(
+               "pg[{}] sched-state changed from <{:.14}> to <{:.14}> (@{:s})",
+               scrub_job->pgid, state_at_entry, scrub_job->state.load(),
+               scrub_job->schedule.scheduled_at)
+          << dendl;
 }
 
 // look mommy - no locks!
@@ -213,6 +220,42 @@ 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;
+
+  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: {:s} hist: {:s} 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, request_flags)
+          << dendl;
+  return res;
+}
+
+
 // used under jobs_lock
 void ScrubQueue::move_failed_pgs(utime_t now_is)
 {
@@ -230,7 +273,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'.
@@ -299,10 +342,10 @@ std::string_view ScrubQueue::qu_state_text(qu_state_t st)
 Scrub::schedule_result_t ScrubQueue::select_pg_and_scrub(
   Scrub::ScrubPreconds& preconds)
 {
-  dout(10) << " reg./pen. sizes: " << to_scrub.size() << " / " << penalized.size()
-          << dendl;
+  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();
@@ -322,7 +365,8 @@ Scrub::schedule_result_t ScrubQueue::select_pg_and_scrub(
   restore_penalized = false;
 
   // remove the 'updated' flag from all entries
-  std::for_each(to_scrub.begin(), to_scrub.end(),
+  std::for_each(to_scrub.begin(),
+               to_scrub.end(),
                [](const auto& jobref) -> void { jobref->updated = false; });
 
   // add failed scrub attempts to the penalized list
@@ -343,8 +387,8 @@ Scrub::schedule_result_t ScrubQueue::select_pg_and_scrub(
   // - we will try the penalized
   if (res == Scrub::schedule_result_t::none_ready && !penalized_copy.empty()) {
     res = select_from_group(penalized_copy, preconds, now_is);
-    dout(10) << "tried the penalized. Res: " << ScrubQueue::attempt_res_text(res)
-            << dendl;
+    dout(10) << "tried the penalized. Res: "
+            << ScrubQueue::attempt_res_text(res) << dendl;
     restore_penalized = true;
   }
 
@@ -379,8 +423,9 @@ struct cmp_sched_time_t {
 }  // namespace
 
 // called under lock
-ScrubQueue::ScrubQContainer ScrubQueue::collect_ripe_jobs(ScrubQContainer& group,
-                                                         utime_t time_now)
+ScrubQueue::ScrubQContainer ScrubQueue::collect_ripe_jobs(
+  ScrubQContainer& group,
+  utime_t time_now)
 {
   rm_unregistered_jobs(group);
 
@@ -388,7 +433,9 @@ ScrubQueue::ScrubQContainer ScrubQueue::collect_ripe_jobs(ScrubQContainer& group
   ScrubQueue::ScrubQContainer ripes;
   ripes.reserve(group.size());
 
-  std::copy_if(group.begin(), group.end(), std::back_inserter(ripes),
+  std::copy_if(group.begin(),
+              group.end(),
+              std::back_inserter(ripes),
               [time_now](const auto& jobref) -> bool {
                 return jobref->schedule.scheduled_at <= time_now;
               });
@@ -408,7 +455,9 @@ ScrubQueue::ScrubQContainer ScrubQueue::collect_ripe_jobs(ScrubQContainer& group
 
 // not holding jobs_lock. 'group' is a copy of the actual list.
 Scrub::schedule_result_t ScrubQueue::select_from_group(
-  ScrubQContainer& group, const Scrub::ScrubPreconds& preconds, utime_t now_is)
+  ScrubQContainer& group,
+  const Scrub::ScrubPreconds& preconds,
+  utime_t now_is)
 {
   dout(15) << "jobs #: " << group.size() << dendl;
 
@@ -429,8 +478,9 @@ Scrub::schedule_result_t ScrubQueue::select_from_group(
     // we have a candidate to scrub. We turn to the OSD to verify that the PG
     // configuration allows the specified type of scrub, and to initiate the
     // scrub.
-    switch (osd_service.initiate_a_scrub(candidate->pgid,
-                                        preconds.allow_requested_repair_only)) {
+    switch (
+      osd_service.initiate_a_scrub(candidate->pgid,
+                                  preconds.allow_requested_repair_only)) {
 
       case Scrub::schedule_result_t::scrub_initiated:
        // the happy path. We are done
@@ -463,64 +513,62 @@ Scrub::schedule_result_t ScrubQueue::select_from_group(
     }
   }
 
-  dout(20) << " returning 'none ready' " << dendl;
+  dout(20) << " returning 'none ready'" << dendl;
   return Scrub::schedule_result_t::none_ready;
 }
 
 ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
   const sched_params_t& times) const
 {
-  ScrubQueue::scrub_schedule_t sched_n_dead{times.proposed_time,
-                                           times.proposed_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
-            << " max t: " << times.max_interval
-            << " osd: " << cct->_conf->osd_scrub_max_interval << dendl;
-
-    dout(20) << "at " << sched_n_dead.scheduled_at << " ratio "
-            << cct->_conf->osd_scrub_interval_randomize_ratio << dendl;
-  }
+  ScrubQueue::scrub_schedule_t sched_n_dead{
+    times.proposed_time, times.proposed_time};
 
   if (times.is_must == ScrubQueue::must_scrub_t::not_mandatory) {
-
-    // if not explicitly requested, postpone the scrub with a random delay
+    // 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{};
     } else {
       sched_n_dead.deadline += scrub_max_interval;
     }
+    // note: no specific job can be named in the log message
+    dout(20) << fmt::format(
+                 "not-must. Was:{:s} {{min:{}/{} max:{}/{} ratio:{}}} "
+                 "Adjusted:{:s} ({:s})",
+                 times.proposed_time, fmt::group_digits(times.min_interval),
+                 fmt::group_digits(conf()->osd_scrub_min_interval),
+                 fmt::group_digits(times.max_interval),
+                 fmt::group_digits(conf()->osd_scrub_max_interval),
+                 conf()->osd_scrub_interval_randomize_ratio,
+                 sched_n_dead.scheduled_at, sched_n_dead.deadline)
+            << dendl;
   }
-
-  dout(17) << "at (final) " << sched_n_dead.scheduled_at << " - "
-          << sched_n_dead.deadline << dendl;
+  // else - no log needed. All relevant data will be logged by the caller
   return sched_n_dead;
 }
 
 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);
 }
@@ -536,21 +584,22 @@ 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;
   }
 
   // allow scrub if below daily avg and currently decreasing
   if (loadavgs[0] < daily_loadavg && loadavgs[0] < loadavgs[2]) {
-    dout(20) << "loadavg " << loadavgs[0] << " < daily_loadavg " << daily_loadavg
-            << " and < 15m avg " << loadavgs[2] << " = yes" << dendl;
+    dout(20) << "loadavg " << loadavgs[0] << " < daily_loadavg "
+            << daily_loadavg << " and < 15m avg " << loadavgs[2] << " = yes"
+            << dendl;
     return true;
   }
 
   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;
@@ -575,7 +624,9 @@ void ScrubQueue::scan_penalized(bool forgive_all, utime_t time_now)
   } else {
 
     auto forgiven_last = std::partition(
-      penalized.begin(), penalized.end(), [time_now](const auto& e) {
+      penalized.begin(),
+      penalized.end(),
+      [time_now](const auto& e) {
        return (*e).updated || ((*e).penalty_timeout <= time_now);
       });
 
@@ -599,22 +650,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, bdt.tm_wday);
+  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, bdt.tm_hour);
-  dout(20) << "should run between " << cct->_conf->osd_scrub_begin_hour << " - "
-          << cct->_conf->osd_scrub_end_hour << " now (" << bdt.tm_hour
+  bool time_permit = isbetween_modulo(conf()->osd_scrub_begin_hour,
+                                     conf()->osd_scrub_end_hour,
+                                     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;
 }
@@ -625,7 +676,8 @@ void ScrubQueue::ScrubJob::dump(ceph::Formatter* f) const
   f->dump_stream("pgid") << pgid;
   f->dump_stream("sched_time") << schedule.scheduled_at;
   f->dump_stream("deadline") << schedule.deadline;
-  f->dump_bool("forced", schedule.scheduled_at == PgScrubber::scrub_must_stamp());
+  f->dump_bool("forced",
+              schedule.scheduled_at == PgScrubber::scrub_must_stamp());
   f->close_section();
 }
 
@@ -636,10 +688,12 @@ 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 ScrubJobRef& j) {
+    j->dump(f);
+  });
 
-  std::for_each(penalized.cbegin(), penalized.cend(),
+  std::for_each(penalized.cbegin(),
+               penalized.cend(),
                [&f](const ScrubJobRef& j) { j->dump(f); });
 
   f->close_section();
@@ -653,16 +707,20 @@ ScrubQueue::ScrubQContainer ScrubQueue::list_registered_jobs() const
 
   std::lock_guard lck{jobs_lock};
 
-  std::copy_if(to_scrub.begin(), to_scrub.end(), std::back_inserter(all_jobs),
+  std::copy_if(to_scrub.begin(),
+              to_scrub.end(),
+              std::back_inserter(all_jobs),
               registered_job);
-  std::copy_if(penalized.begin(), penalized.end(), std::back_inserter(all_jobs),
+  std::copy_if(penalized.begin(),
+              penalized.end(),
+              std::back_inserter(all_jobs),
               registered_job);
 
   return all_jobs;
 }
 
 // ////////////////////////////////////////////////////////////////////////// //
-// ScrubJob - scrub resource management
+// ScrubQueue - scrub resource management
 
 bool ScrubQueue::can_inc_scrubs() const
 {
@@ -670,12 +728,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;
 }
 
@@ -683,13 +741,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;
 }
 
@@ -697,7 +755,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;
@@ -708,16 +766,16 @@ bool ScrubQueue::inc_scrubs_remote()
 {
   std::lock_guard lck{resource_lock};
 
-  if (scrubs_local + scrubs_remote < cct->_conf->osd_max_scrubs) {
-    dout(20) << ": " << scrubs_remote << " -> " << (scrubs_remote + 1) << " (max "
-            << cct->_conf->osd_max_scrubs << ", local " << scrubs_local << ")"
-            << dendl;
+  if (scrubs_local + scrubs_remote < conf()->osd_max_scrubs) {
+    dout(20) << ": " << scrubs_remote << " -> " << (scrubs_remote + 1)
+            << " (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;
 }
 
@@ -725,7 +783,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);
@@ -736,5 +794,24 @@ 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);
+}
+
+void ScrubQueue::clear_pg_scrub_blocked(spg_t blocked_pg)
+{
+  dout(5) << fmt::format(": pg {} is unblocked", blocked_pg) << dendl;
+  --blocked_scrubs_cnt;
+  ceph_assert(blocked_scrubs_cnt >= 0);
+}
+
+void ScrubQueue::mark_pg_scrub_blocked(spg_t blocked_pg)
+{
+  dout(5) << fmt::format(": pg {} is blocked on an object", blocked_pg)
+         << dendl;
+  ++blocked_scrubs_cnt;
+}
+
+int ScrubQueue::get_blocked_pgs_count() const
+{
+  return blocked_scrubs_cnt;
 }