]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: modify ScrubJob to hold two SchedTarget-s
authorRonen Friedman <rfriedma@redhat.com>
Fri, 12 Jul 2024 13:18:30 +0000 (08:18 -0500)
committerRonen Friedman <rfriedma@redhat.com>
Sun, 25 Aug 2024 13:01:00 +0000 (08:01 -0500)
ScrubJob will now hold two SchedTarget-s - two sets of scheduling
information (times, levels, etc.) for the next shallow and deep scrubs.

This is in preparation for the upcoming changes to the scheduling queue.
The change cannot stand on its own, as the partial implementation
creates some inconsistencies in the scheduling logic.

Specifically, here is what changes here, and how it differs from the
desired implementation:
- The OSD still maintains a queue of scrub jobs - one object only per
  PG.
  But now - each queue element holds two SchedTarget-s.
- When a scrub is initiated, the Scrubber is handed a ScrubJob object.
  Only in the next commit will it also receive the ID of the selected
  level. That causes some issues when re-determining the level of the
  initiated scrub. A failure to match the queue "intent" results in
  failures.
- the 'planned scrub' flags are still here, instead of directly
  encoding the characteristics of the next scrub in the relevant
  sched-entry.
- the 'urgency' levels do not cover the full required range of
  behaviors and priorities.

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
src/osd/scrubber/osd_scrub_sched.cc
src/osd/scrubber/pg_scrubber.cc
src/osd/scrubber/pg_scrubber.h
src/osd/scrubber/scrub_job.cc
src/osd/scrubber/scrub_job.h
src/osd/scrubber/scrub_machine_lstnr.h
src/osd/scrubber_common.h

index 5f6ebc0c2f71993fb969cf8186069120e4c4c372..06d6947b53516c691b277cdedf13b9779d5e5007 100644 (file)
@@ -76,16 +76,22 @@ std::unique_ptr<ScrubJob> ScrubQueue::pop_ready_pg(
     utime_t time_now)
 {
   std::unique_lock lck{jobs_lock};
-
   const auto eligible_filtr = [time_now, rst = restrictions](
                                  const std::unique_ptr<ScrubJob>& jb) -> bool {
-    // look for jobs that have their n.b. in the past, and are not
-    // blocked by restrictions
-    return jb->get_sched_time() <= time_now &&
-          (jb->high_priority ||
-           (!rst.high_priority_only &&
-            (!rst.only_deadlined || (!jb->schedule.deadline.is_zero() &&
-                                     jb->schedule.deadline <= time_now))));
+    // look for jobs that at least one of their pair of targets has a ripe n.b.
+    // and is not blocked by restrictions
+    const auto eligible_target = [time_now,
+                                 rst](const Scrub::SchedTarget& t) -> bool {
+      return t.is_ripe(time_now) &&
+            (t.is_high_priority() ||
+             (!rst.high_priority_only &&
+              (!rst.only_deadlined ||
+               (!t.sched_info_ref().schedule.deadline.is_zero() &&
+                t.sched_info_ref().schedule.deadline <= time_now))));
+    };
+
+    return eligible_target(jb->shallow_target) ||
+          eligible_target(jb->deep_target);
   };
 
   auto not_ripes =
@@ -110,19 +116,6 @@ std::unique_ptr<ScrubJob> ScrubQueue::pop_ready_pg(
 }
 
 
-namespace {
-struct cmp_time_n_priority_t {
-  bool operator()(const Scrub::ScrubJob& lhs, const Scrub::ScrubJob& rhs)
-      const
-  {
-    return lhs.is_high_priority() > rhs.is_high_priority() ||
-          (lhs.is_high_priority() == rhs.is_high_priority() &&
-           lhs.schedule.scheduled_at < rhs.schedule.scheduled_at);
-  }
-};
-}  // namespace
-
-
 /**
  * the set of all PGs named by the entries in the queue (but only those
  * entries that satisfy the predicate)
index 659050201505ed3d35e2f400feb0893a60f9730a..7d912daaa4ee1b555b58415ac480fb7a46a711f3 100644 (file)
@@ -502,35 +502,81 @@ void PgScrubber::rm_from_osd_scrubbing()
   }
 }
 
-sched_params_t PgScrubber::determine_initial_schedule(
+
+bool PgScrubber::flags_to_deep_priority(
     const Scrub::sched_conf_t& app_conf,
-    utime_t scrub_clock_now) const
+    utime_t scrub_clock_now)
 {
-  sched_params_t res;
+  auto& targ = m_scrub_job->deep_target;
+  auto& entry = m_scrub_job->deep_target.sched_info_ref();
 
-  if (m_planned_scrub.must_scrub || m_planned_scrub.need_auto) {
+  // note: as we depend on the returned value to distinguish between existing h.p.
+  // and an instance in which that is set here, there is the added "not already
+  // high-priority" condition.
+  if (targ.is_high_priority()) {
+    return false;
+  }
 
+  if (m_planned_scrub.need_auto || m_planned_scrub.must_deep_scrub) {
     // Set the smallest time that isn't utime_t()
-    res.proposed_time = PgScrubber::scrub_must_stamp();
-    res.is_must = Scrub::must_scrub_t::mandatory;
+    entry.schedule.scheduled_at = PgScrubber::scrub_must_stamp();
+    entry.urgency = urgency_t::operator_requested;
+    return true;
+  }
 
-  } else if (m_pg->info.stats.stats_invalid && app_conf.mandatory_on_invalid) {
-    res.proposed_time = scrub_clock_now;
-    res.is_must = Scrub::must_scrub_t::mandatory;
+  return false; // not set to high-priority *by this function*
+}
 
-  } else {
-    res.proposed_time = m_pg->info.history.last_scrub_stamp;
+
+void PgScrubber::flags_to_shallow_priority(
+    const Scrub::sched_conf_t& app_conf,
+    utime_t scrub_clock_now)
+{
+  auto& entry = m_scrub_job->shallow_target.sched_info_ref();
+
+  if (m_planned_scrub.must_scrub) {
+
+    // Set the smallest time that isn't utime_t()
+    entry.schedule.scheduled_at = PgScrubber::scrub_must_stamp();
+    ///\todo missing a distinct urgency level for 'must' scrubs
+    entry.urgency = urgency_t::operator_requested;
+
+  } else if (m_pg->info.stats.stats_invalid && app_conf.mandatory_on_invalid) {
+    entry.schedule.scheduled_at = scrub_clock_now;
+    entry.urgency = urgency_t::operator_requested;
   }
+}
 
-  dout(15) << fmt::format(
-                 "{}: suggested:{:s}(must:{:c}) hist:{:s} valid:{}/{} flags:{}",
-                 __func__, res.proposed_time,
-                 (res.is_must == must_scrub_t::mandatory ? 'y' : 'n'),
-                 m_pg->info.history.last_scrub_stamp,
-                 !(bool)m_pg->info.stats.stats_invalid,
-                 app_conf.mandatory_on_invalid, m_planned_scrub)
+
+void PgScrubber::update_targets(
+    const requested_scrub_t& planned,
+    utime_t scrub_clock_now)
+{
+  const auto applicable_conf = populate_config_params();
+
+  dout(10) << fmt::format(
+                 "{}: config:{} flags:<{}> job on entry:{}{}", __func__,
+                 applicable_conf, planned, *m_scrub_job,
+                 m_pg->info.stats.stats_invalid ? " invalid-stats" : "")
           << dendl;
-  return res;
+
+  // first, use the planned-scrub flags to possibly set one of the
+  // targets as high-priority.
+  // Note - this step is to be removed in the followup commits.
+  auto deep_hp_set = flags_to_deep_priority(applicable_conf, scrub_clock_now);
+  if (!deep_hp_set) {
+    flags_to_shallow_priority(populate_config_params(), scrub_clock_now);
+  }
+
+  // the next periodic scrubs:
+  m_scrub_job->adjust_shallow_schedule(
+      m_pg->info.history.last_scrub_stamp, applicable_conf, scrub_clock_now,
+      delay_ready_t::delay_ready);
+  m_scrub_job->adjust_deep_schedule(
+      m_pg->info.history.last_deep_scrub_stamp, applicable_conf,
+      scrub_clock_now, delay_ready_t::delay_ready);
+
+  dout(10) << fmt::format("{}: adjusted:{}", __func__, *m_scrub_job) << dendl;
 }
 
 
@@ -576,18 +622,19 @@ void PgScrubber::update_scrub_job(Scrub::delay_ready_t delay_ready)
     return;
   }
 
+  dout(15) << fmt::format(
+                 "{}: flags:<{}> job on entry:{}", __func__, m_planned_scrub,
+                 *m_scrub_job)
+          << dendl;
+
   // if we were marked as 'not registered' - do not try to push into
-  // the queue. And if we are already in the queue - do not push again.
-  if (!m_scrub_job->registered) {
+  // the queue. And if we are already in the queue - dequeue.
+  if (!m_scrub_job->is_registered()) {
     dout(10) << fmt::format("{}: PG[{}] not registered", __func__, m_pg_id)
             << dendl;
     return;
   }
-
-  dout(15) << fmt::format(
-                 "{}: flags:<{}> job on entry:{}", __func__, m_planned_scrub,
-                 *m_scrub_job)
-          << dendl;
+  ceph_assert(m_pg->is_locked());
   if (m_scrub_job->target_queued) {
     m_osds->get_scrub_services().remove_from_osd_queue(m_pg_id);
     m_scrub_job->target_queued = false;
@@ -596,15 +643,9 @@ void PgScrubber::update_scrub_job(Scrub::delay_ready_t delay_ready)
             << dendl;
   }
 
-  ceph_assert(m_pg->is_locked());
-  const auto applicable_conf = populate_config_params();
   const auto scrub_clock_now = ceph_clock_now();
-  const auto suggested =
-      determine_initial_schedule(applicable_conf, scrub_clock_now);
+  update_targets(m_planned_scrub, scrub_clock_now);
 
-  ceph_assert(m_scrub_job->is_registered());
-  m_scrub_job->adjust_schedule(
-      suggested, applicable_conf, scrub_clock_now, delay_ready);
   m_osds->get_scrub_services().enqueue_target(*m_scrub_job);
   m_scrub_job->target_queued = true;
   m_pg->publish_stats_to_osd();
@@ -615,6 +656,7 @@ void PgScrubber::update_scrub_job(Scrub::delay_ready_t delay_ready)
           << dendl;
 }
 
+
 scrub_level_t PgScrubber::scrub_requested(
     scrub_level_t scrub_level,
     scrub_type_t scrub_type,
@@ -653,16 +695,28 @@ void PgScrubber::request_rescrubbing(requested_scrub_t& request_flags)
   update_scrub_job(delay_ready_t::no_delay);
 }
 
+
+/*
+ * Implementation:
+ * try to create the reservation object (which translates into asking the
+ * OSD for a local scrub resource). The object returned is a
+ * a wrapper around the actual reservation, and that object releases
+ * the local resource automatically when reset.
+ */
 bool PgScrubber::reserve_local()
 {
-  // try to create the reservation object (which translates into asking the
-  // OSD for a local scrub resource). The object returned is a
-  // a wrapper around the actual reservation, and that object releases
-  // the local resource automatically when reset.
-  m_local_osd_resource = m_osds->get_scrub_services().inc_scrubs_local(
-      m_scrub_job->is_high_priority());
+  // Implementation note re the 'is high priority' parameter:
+  // In this step in the scrub scheduling rework, at the point of the call to
+  // this function, set_op_params() was not yet called, and we cannot rely
+  // on m_is_deep to determine the scrub level. So for now - we check both
+  // targets here.
+  const bool is_hp =
+      m_scrub_job->is_job_high_priority(scrub_level_t::shallow) ||
+      m_scrub_job->is_job_high_priority(scrub_level_t::deep);
+
+  m_local_osd_resource = m_osds->get_scrub_services().inc_scrubs_local(is_hp);
   if (m_local_osd_resource) {
-    dout(15) << __func__ << ": local resources reserved" << dendl;
+    dout(10) << __func__ << ": local resources reserved" << dendl;
     return true;
   }
 
@@ -670,6 +724,8 @@ bool PgScrubber::reserve_local()
   return false;
 }
 
+
+
 Scrub::sched_conf_t PgScrubber::populate_config_params() const
 {
   const pool_opts_t& pool_conf = m_pg->get_pgpool().info.opts;
@@ -2081,24 +2137,18 @@ void PgScrubber::on_digest_updates()
  * And - it does not (at least initially, i.e. immediately after
  * set_op_parameters()), have high priority.
  *
- * Alas, the scrub session that was initiated was aborted. We must now
- * merge the two sets of parameters, using the highest priority and the
- * nearest target time for the next scrub.
- *
- * Note: only half-functioning in this commit. As the scrub-job copy
- * (the one that was in the scheduling queue, and was passed to the scrubber)
- * does not have the 'urgency' parameter, we are missing some information
- * that is still encoded in the 'planned scrub' flags. This will be fixed in
- * the next step.
+ * Note: only half-functioning in this commit: The m_active_target
+ * points (for now) to a pair of targets. We must use 'm_is_deep'
+ * to know what was the actual aborted target.
  */
 void PgScrubber::on_mid_scrub_abort(Scrub::delay_cause_t issue)
 {
   if (!m_scrub_job->is_registered()) {
     dout(10) << fmt::format(
-                    "{}: PG not registered for scrubbing on this OSD. Won't "
-                    "requeue!",
-                    __func__)
-             << dendl;
+                   "{}: PG not registered for scrubbing on this OSD. Won't "
+                   "requeue!",
+                   __func__)
+            << dendl;
     return;
   }
 
@@ -2107,7 +2157,7 @@ void PgScrubber::on_mid_scrub_abort(Scrub::delay_cause_t issue)
 
   // note again: this is not how merging should work in the final version:
   // e.g. - the 'aborted_schedule' data should be passed thru the scrubber.
-  // In this current patchworik, for example, we are only guessing at
+  // In this current patchwork, for example, we are only guessing at
   // the original value of 'must_deep_scrub'.
   m_planned_scrub.must_deep_scrub =
       m_planned_scrub.must_deep_scrub || (m_flags.required && m_is_deep);
@@ -2120,9 +2170,27 @@ void PgScrubber::on_mid_scrub_abort(Scrub::delay_cause_t issue)
   m_planned_scrub.check_repair =
       m_planned_scrub.check_repair || m_flags.check_repair;
 
-  m_scrub_job->merge_and_delay(
-      m_active_target->schedule, issue, m_planned_scrub, ceph_clock_now());
+  // copy the aborted target
+  const auto aborted_target = m_is_deep ? m_active_target->deep_target
+                                       : m_active_target->shallow_target;
+
+  const auto scrub_clock_now = ceph_clock_now();
+  update_targets(m_planned_scrub, scrub_clock_now);
+
+  // we may have updated both targets. For sure - we took notice of any change
+  // that made any of the targets into a high-priority one. All that's left:
+  // delay the specific target that was aborted.
+
+  m_scrub_job->delay_on_failure(
+      m_is_deep ? scrub_level_t::deep : scrub_level_t::shallow, 5s, issue,
+      scrub_clock_now);
+
+  /// \todo complete the merging of the deadline & target for non-hp targets
+  if (!aborted_target.is_high_priority()) {
+    std::ignore = aborted_target;
+  }
   ceph_assert(!m_scrub_job->target_queued);
+
   m_osds->get_scrub_services().enqueue_target(*m_scrub_job);
   m_scrub_job->target_queued = true;
 }
@@ -2139,7 +2207,9 @@ void PgScrubber::requeue_penalized(Scrub::delay_cause_t cause)
     return;
   }
   /// \todo fix the 5s' to use a cause-specific delay parameter
-  m_scrub_job->delay_on_failure(5s, cause, ceph_clock_now());
+  m_scrub_job->delay_on_failure(
+      m_is_deep ? scrub_level_t::deep : scrub_level_t::shallow, 5s, cause,
+      ceph_clock_now());
   ceph_assert(!m_scrub_job->target_queued);
   m_osds->get_scrub_services().enqueue_target(*m_scrub_job);
   m_scrub_job->target_queued = true;
@@ -2228,6 +2298,12 @@ Scrub::schedule_result_t PgScrubber::start_scrub_session(
 
   set_op_parameters(m_planned_scrub);
 
+  // clear all special handling urgency/flags from the target that is
+  // executing now.
+  auto& selected_target =
+      m_is_deep ? m_scrub_job->deep_target : m_scrub_job->shallow_target;
+  selected_target.reset();
+
   // using the OSD queue, as to not execute the scrub code as part of the tick.
   dout(10) << __func__ << ": queueing" << dendl;
   m_osds->queue_for_scrub(m_pg, Scrub::scrub_prio_t::low_priority);
@@ -2317,10 +2393,9 @@ pg_scrubbing_status_t PgScrubber::get_schedule() const
     return pg_scrubbing_status_t{};
   }
 
-  dout(25) << fmt::format("{}: active:{} blocked:{}",
-                         __func__,
-                         m_active,
-                         m_scrub_job->blocked)
+  dout(25) << fmt::format(
+                 "{}: active:{} blocked:{}", __func__, m_active,
+                 m_scrub_job->blocked)
           << dendl;
 
   auto now_is = ceph_clock_now();
@@ -2331,14 +2406,14 @@ pg_scrubbing_status_t PgScrubber::get_schedule() const
     if (m_scrub_job->blocked) {
       // a bug. An object is held locked.
       int32_t blocked_for =
-       (utime_t{now_is} - m_scrub_job->blocked_since).sec();
+         (utime_t{now_is} - m_scrub_job->blocked_since).sec();
       return pg_scrubbing_status_t{
-       utime_t{},
-       blocked_for,
-       pg_scrub_sched_status_t::blocked,
-       true,  // active
-       (m_is_deep ? scrub_level_t::deep : scrub_level_t::shallow),
-       false};
+         utime_t{},
+         blocked_for,
+         pg_scrub_sched_status_t::blocked,
+         true,  // active
+         (m_is_deep ? scrub_level_t::deep : scrub_level_t::shallow),
+         false};
 
     } else {
       int32_t dur_seconds =
@@ -2353,43 +2428,40 @@ pg_scrubbing_status_t PgScrubber::get_schedule() const
     }
   }
   if (!m_scrub_job->is_registered()) {
-    return pg_scrubbing_status_t{utime_t{},
-                                0,
-                                pg_scrub_sched_status_t::not_queued,
-                                false,
-                                scrub_level_t::shallow,
-                                false};
-  }
-
-  // Will next scrub surely be a deep one? note that deep-scrub might be
-  // selected even if we report a regular scrub here.
-  bool deep_expected = (now_is >= m_pg->next_deepscrub_interval()) ||
-                      m_planned_scrub.must_deep_scrub ||
-                      m_planned_scrub.need_auto;
-  scrub_level_t expected_level =
-    deep_expected ? scrub_level_t::deep : scrub_level_t::shallow;
-  bool periodic = !m_planned_scrub.must_scrub && !m_planned_scrub.need_auto &&
-                 !m_planned_scrub.must_deep_scrub;
-
-  // are we ripe for scrubbing?
-  if (now_is > m_scrub_job->schedule.not_before) {
-    // we are waiting for our turn at the OSD.
-    return pg_scrubbing_status_t{m_scrub_job->schedule.scheduled_at,
-                                0,
-                                pg_scrub_sched_status_t::queued,
-                                false,
-                                expected_level,
-                                periodic};
-  }
-
-  return pg_scrubbing_status_t{m_scrub_job->schedule.not_before,
-                              0,
-                              pg_scrub_sched_status_t::scheduled,
-                              false,
-                              expected_level,
-                              periodic};
+    return pg_scrubbing_status_t{
+       utime_t{},
+       0,
+       pg_scrub_sched_status_t::not_queued,
+       false,
+       scrub_level_t::shallow,
+       false};
+  }
+
+  // not taking 'no-*scrub' flags into account here.
+  const auto first_ready = m_scrub_job->earliest_eligible(now_is);
+  if (first_ready) {
+    const auto& targ = first_ready->get();
+    return pg_scrubbing_status_t{
+       targ.get_sched_time(),
+       0,
+       pg_scrub_sched_status_t::queued,
+       false,
+       (targ.is_deep() ? scrub_level_t::deep : scrub_level_t::shallow),
+       !targ.is_high_priority()};
+  }
+
+  // both targets are not ready yet
+  const auto targ = m_scrub_job->earliest_target();
+  return pg_scrubbing_status_t{
+      targ.get_sched_time(),
+      0,
+      pg_scrub_sched_status_t::scheduled,
+      false,
+      (targ.is_deep() ? scrub_level_t::deep : scrub_level_t::shallow),
+      !targ.is_high_priority()};
 }
 
+
 void PgScrubber::handle_query_state(ceph::Formatter* f)
 {
   dout(15) << __func__ << dendl;
@@ -2916,7 +2988,8 @@ std::optional<requested_scrub_t> PgScrubber::validate_scrub_mode(
   const bool time_for_deep = is_time_for_deep(pg_cond, m_planned_scrub);
   std::optional<requested_scrub_t> upd_flags;
 
-  if (m_scrub_job->is_high_priority()) {
+  if (m_scrub_job->is_job_high_priority(
+         time_for_deep ? scrub_level_t::deep : scrub_level_t::shallow)) {
     // 'initiated' scrubs
     dout(10) << fmt::format(
                    "{}: initiated (\"must\") scrub (target:{} pg:{})",
index e970bd7219db52554ff388157a98306543d2d566..19f20b48e4ad81a2f978fd850e2af5f42a2559b7 100644 (file)
@@ -809,15 +809,37 @@ class PgScrubber : public ScrubPgIF,
   Scrub::sched_conf_t populate_config_params() const;
 
   /**
-   * determine the time when the next scrub should be scheduled
+   * use the 'planned scrub' flags to determine the urgency attribute
+   * of the 'deep target' part of the ScrubJob object.
    *
-   * based on the planned scrub's flags, time of last scrub, and
-   * the pool's scrub configuration. This is only an initial "proposal",
-   * and will be further adjusted based on the scheduling parameters.
+   * Returns 'true' if a high-priority 'urgency' level was set by this
+   * call (note: not if it was already set).
+   *
+   * Note: in the previous implementation, if the shallow scrub had
+   * high priority, and it was time for the periodic deep scrub, a
+   * high priority deep scrub was initiated. This behavior is not
+   * replicated here.
+   */
+  bool flags_to_deep_priority(
+      const Scrub::sched_conf_t& app_conf,
+      utime_t scrub_clock_now);
+
+  /**
+   * use the 'planned scrub' flags to determine the urgency attribute
+   * of the 'shallow target' part of the ScrubJob object.
    */
-  Scrub::sched_params_t determine_initial_schedule(
+  void flags_to_shallow_priority(
       const Scrub::sched_conf_t& app_conf,
-      utime_t scrub_clock_now) const;
+      utime_t scrub_clock_now);
+
+  /**
+   * recompute the two ScrubJob targets, taking into account not
+   * only the up-to-date 'last' stamps, but also the 'planned scrub'
+   * flags.
+   */
+  void update_targets(
+      const requested_scrub_t& planned,
+      utime_t scrub_clock_now);
 
   /// should we perform deep scrub?
   bool is_time_for_deep(
index 911ecc66ddb6f3966fb44842eea4f400957dc874..a983f4b70476d009d80636d9f4e8ec045118c4f2 100644 (file)
@@ -2,10 +2,10 @@
 // vim: ts=8 sw=2 smarttab
 
 #include "./scrub_job.h"
+
 #include "pg_scrubber.h"
 
 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 sched_conf_t = Scrub::sched_conf_t;
@@ -13,6 +13,36 @@ using scrub_schedule_t = Scrub::scrub_schedule_t;
 using ScrubJob = Scrub::ScrubJob;
 using delay_ready_t = Scrub::delay_ready_t;
 
+namespace {
+utime_t add_double(utime_t t, double d)
+{
+  double int_part;
+  double frac_as_ns = 1'000'000'000 * std::modf(d, &int_part);
+  return utime_t{
+      t.sec() + static_cast<int>(int_part),
+      static_cast<int>(t.nsec() + frac_as_ns)};
+}
+}  // namespace
+
+using SchedEntry = Scrub::SchedEntry;
+
+// ////////////////////////////////////////////////////////////////////////// //
+// SchedTarget
+
+using SchedTarget = Scrub::SchedTarget;
+
+void SchedTarget::reset()
+{
+  // a bit convoluted, but the standard way to guarantee we keep the
+  // same set of member defaults as the constructor
+  *this = SchedTarget{sched_info.pgid, sched_info.level};
+}
+
+void SchedTarget::up_urgency_to(urgency_t u)
+{
+  sched_info.urgency = std::max(sched_info.urgency, u);
+}
+
 
 // ////////////////////////////////////////////////////////////////////////// //
 // ScrubJob
@@ -32,6 +62,8 @@ static std::ostream& _prefix_fn(std::ostream* _dout, T* t, std::string fn = "")
 ScrubJob::ScrubJob(CephContext* cct, const spg_t& pg, int node_id)
     : pgid{pg}
     , whoami{node_id}
+    , shallow_target{pg, scrub_level_t::shallow}
+    , deep_target{pg, scrub_level_t::deep}
     , cct{cct}
     , log_msg_prefix{fmt::format("osd.{} scrub-job:pg[{}]:", node_id, pgid)}
 {}
@@ -45,27 +77,59 @@ ostream& operator<<(ostream& out, const ScrubJob& sjob)
 }  // namespace std
 
 
-void ScrubJob::adjust_schedule(
-    const Scrub::sched_params_t& suggested,
+SchedTarget& ScrubJob::get_target(scrub_level_t s_or_d)
+{
+  return (s_or_d == scrub_level_t::deep) ? deep_target : shallow_target;
+}
+
+
+bool ScrubJob::is_queued() const
+{
+  return shallow_target.queued || deep_target.queued;
+}
+
+
+void ScrubJob::clear_both_targets_queued()
+{
+  shallow_target.queued = false;
+  deep_target.queued = false;
+}
+
+
+void ScrubJob::set_both_targets_queued()
+{
+  shallow_target.queued = true;
+  deep_target.queued = true;
+}
+
+
+void ScrubJob::adjust_shallow_schedule(
+    utime_t last_scrub,
     const Scrub::sched_conf_t& app_conf,
     utime_t scrub_clock_now,
     delay_ready_t modify_ready_targets)
 {
   dout(10) << fmt::format(
-                 "{} current h.p.:{:c} conf:{} also-ready?{:c} "
-                 "sjob@entry:{}",
-                 suggested, high_priority ? 'y' : 'n', app_conf,
+                 "at entry: shallow target:{}, conf:{}, last-stamp:{:s} "
+                 "also-ready?{:c}",
+                 shallow_target, app_conf, last_scrub,
                  (modify_ready_targets == delay_ready_t::delay_ready) ? 'y'
-                                                                      : 'n',
-                 *this)
+                                                                      : 'n')
           << dendl;
 
-  high_priority = (suggested.is_must == must_scrub_t::mandatory);
-  utime_t adj_not_before = suggested.proposed_time;
-  utime_t adj_target = suggested.proposed_time;
-  schedule.deadline = adj_target;
+  auto& sh_times = shallow_target.sched_info.schedule; // shorthand
+
+  if (!ScrubJob::requires_randomization(shallow_target.urgency())) {
+    // the target time is already set. Make sure to reset the n.b. and
+    // the (irrelevant) deadline
+    sh_times.not_before = sh_times.scheduled_at;
+    sh_times.deadline = sh_times.scheduled_at;
+
+  } else {
+    utime_t adj_not_before = last_scrub;
+    utime_t adj_target = last_scrub;
+    sh_times.deadline = adj_target;
 
-  if (!high_priority) {
     // add a random delay to the proposed scheduled time - but only for periodic
     // scrubs that are not already eligible for scrubbing.
     if ((modify_ready_targets == delay_ready_t::delay_ready) ||
@@ -78,53 +142,151 @@ void ScrubJob::adjust_schedule(
 
     // the deadline can be updated directly into the scrub-job
     if (app_conf.max_shallow) {
-      schedule.deadline += *app_conf.max_shallow;
+      sh_times.deadline += *app_conf.max_shallow;
     } else {
-      schedule.deadline = utime_t{};
+      sh_times.deadline = utime_t{};
     }
-
     if (adj_not_before < adj_target) {
       adj_not_before = adj_target;
     }
+    sh_times.scheduled_at = adj_target;
+    sh_times.not_before = adj_not_before;
   }
 
-  schedule.scheduled_at = adj_target;
-  schedule.not_before = adj_not_before;
   dout(10) << fmt::format(
                  "adjusted: nb:{:s} target:{:s} deadline:{:s} ({})",
-                 schedule.not_before, schedule.scheduled_at, schedule.deadline,
+                 sh_times.not_before, sh_times.scheduled_at, sh_times.deadline,
                  state_desc())
           << dendl;
 }
 
 
-void ScrubJob::merge_and_delay(
-    const scrub_schedule_t& aborted_schedule,
-    delay_cause_t issue,
-    requested_scrub_t updated_flags,
+std::optional<std::reference_wrapper<SchedTarget>> ScrubJob::earliest_eligible(
     utime_t scrub_clock_now)
 {
-  // merge the schedule targets:
-  schedule.scheduled_at =
-      std::min(aborted_schedule.scheduled_at, schedule.scheduled_at);
-  high_priority = high_priority || updated_flags.must_scrub;
-  delay_on_failure(5s, issue, scrub_clock_now);
+  std::weak_ordering compr = cmp_entries(
+      scrub_clock_now, shallow_target.queued_element(),
+      deep_target.queued_element());
+
+  auto poss_ret = (compr == std::weak_ordering::less)
+                     ? std::ref<SchedTarget>(shallow_target)
+                     : std::ref<SchedTarget>(deep_target);
+  if (poss_ret.get().sched_info.schedule.not_before <= scrub_clock_now) {
+    return poss_ret;
+  }
+  return std::nullopt;
+}
+
+std::optional<std::reference_wrapper<const SchedTarget>>
+ScrubJob::earliest_eligible(utime_t scrub_clock_now) const
+{
+  std::weak_ordering compr = cmp_entries(
+      scrub_clock_now, shallow_target.queued_element(),
+      deep_target.queued_element());
+
+  auto poss_ret = (compr == std::weak_ordering::less)
+                     ? std::cref<SchedTarget>(shallow_target)
+                     : std::cref<SchedTarget>(deep_target);
+  if (poss_ret.get().sched_info.schedule.not_before <= scrub_clock_now) {
+    return poss_ret;
+  }
+  return std::nullopt;
+}
+
+
+SchedTarget& ScrubJob::earliest_target()
+{
+  std::weak_ordering compr = cmp_future_entries(
+      shallow_target.queued_element(), deep_target.queued_element());
+  return (compr == std::weak_ordering::less) ? shallow_target : deep_target;
+}
 
-  // the new deadline is the minimum of the two
-  schedule.deadline = std::min(aborted_schedule.deadline, schedule.deadline);
+const SchedTarget& ScrubJob::earliest_target() const
+{
+  std::weak_ordering compr = cmp_future_entries(
+      shallow_target.queued_element(), deep_target.queued_element());
+  return (compr == std::weak_ordering::less) ? shallow_target : deep_target;
 }
 
+utime_t ScrubJob::get_sched_time() const
+{
+  return earliest_target().sched_info.schedule.not_before;
+}
 
-void ScrubJob::delay_on_failure(
+void ScrubJob::adjust_deep_schedule(
+    utime_t last_deep,
+    const Scrub::sched_conf_t& app_conf,
+    utime_t scrub_clock_now,
+    delay_ready_t modify_ready_targets)
+{
+  dout(10) << fmt::format(
+                 "at entry: deep target:{}, conf:{}, last-stamp:{:s} "
+                 "also-ready?{:c}",
+                 deep_target, app_conf, last_deep,
+                 (modify_ready_targets == delay_ready_t::delay_ready) ? 'y'
+                                                                      : 'n')
+          << dendl;
+
+  auto& dp_times = deep_target.sched_info.schedule;  // shorthand
+
+  if (!ScrubJob::requires_randomization(deep_target.urgency())) {
+    // the target time is already set. Make sure to reset the n.b. and
+    // the (irrelevant) deadline
+    dp_times.not_before = dp_times.scheduled_at;
+    dp_times.deadline = dp_times.scheduled_at;
+
+  } else {
+    utime_t adj_not_before = last_deep;
+    utime_t adj_target = last_deep;
+    dp_times.deadline = adj_target;
+
+    // add a random delay to the proposed scheduled time - but only for periodic
+    // scrubs that are not already eligible for scrubbing.
+    if ((modify_ready_targets == delay_ready_t::delay_ready) ||
+       adj_not_before > scrub_clock_now) {
+      adj_target += app_conf.deep_interval;
+      double r = rand() / (double)RAND_MAX;
+      adj_target += app_conf.deep_interval * app_conf.interval_randomize_ratio *
+                   r;  // RRR fix
+    }
+
+    // the deadline can be updated directly into the scrub-job
+    if (app_conf.max_shallow) {
+      dp_times.deadline += *app_conf.max_shallow;  // RRR fix
+    } else {
+      dp_times.deadline = utime_t{};
+    }
+    if (adj_not_before < adj_target) {
+      adj_not_before = adj_target;
+    }
+    dp_times.scheduled_at = adj_target;
+    dp_times.not_before = adj_not_before;
+  }
+
+  dout(10) << fmt::format(
+                 "adjusted: nb:{:s} target:{:s} deadline:{:s} ({})",
+                 dp_times.not_before, dp_times.scheduled_at, dp_times.deadline,
+                 state_desc())
+          << dendl;
+}
+
+
+SchedTarget& ScrubJob::delay_on_failure(
+    scrub_level_t level,
     std::chrono::seconds delay,
     Scrub::delay_cause_t delay_cause,
     utime_t scrub_clock_now)
 {
-  schedule.not_before =
-      std::max(scrub_clock_now, schedule.not_before) + utime_t{delay};
-  last_issue = delay_cause;
+  auto& delayed_target =
+      (level == scrub_level_t::deep) ? deep_target : shallow_target;
+  delayed_target.sched_info.schedule.not_before =
+      std::max(scrub_clock_now, delayed_target.sched_info.schedule.not_before) +
+      utime_t{delay};
+  delayed_target.sched_info.last_issue = delay_cause;
+  return delayed_target;
 }
 
+
 std::string ScrubJob::scheduling_state(utime_t now_is, bool is_deep_expected)
     const
 {
@@ -132,20 +294,27 @@ std::string ScrubJob::scheduling_state(utime_t now_is, bool is_deep_expected)
   if (!registered) {
     return "not registered for scrubbing";
   }
-  if (!target_queued) {
+  if (!is_queued()) {
     // if not currently queued - we are being scrubbed
     return "scrubbing";
   }
 
-  // if the time has passed, we are surely in the queue
-  if (now_is > schedule.not_before) {
-    // we are never sure that the next scrub will indeed be shallow:
-    return fmt::format("queued for {}scrub", (is_deep_expected ? "deep " : ""));
+  const auto first_ready = earliest_eligible(now_is);
+  if (first_ready) {
+    // the target is ready to be scrubbed
+    return fmt::format(
+       "queued for {}scrub at {:s} (debug RRR: {})",
+       (first_ready->get().is_deep() ? "deep " : ""),
+       first_ready->get().sched_info.schedule.scheduled_at,
+       (is_deep_expected ? "deep " : ""));
+  } else {
+    // both targets are in the future
+    const auto& nearest = earliest_target();
+    return fmt::format(
+       "{}scrub scheduled @ {:s} ({:s})", (nearest.is_deep() ? "deep " : ""),
+       nearest.sched_info.schedule.not_before,
+       nearest.sched_info.schedule.scheduled_at);
   }
-
-  return fmt::format(
-      "{}scrub scheduled @ {:s} ({:s})", (is_deep_expected ? "deep " : ""),
-      schedule.not_before, schedule.scheduled_at);
 }
 
 std::ostream& ScrubJob::gen_prefix(std::ostream& out, std::string_view fn) const
@@ -155,12 +324,46 @@ std::ostream& ScrubJob::gen_prefix(std::ostream& out, std::string_view fn) const
 
 void ScrubJob::dump(ceph::Formatter* f) const
 {
+  const auto& entry = earliest_target().sched_info;
+  const auto& sch = entry.schedule;
   f->open_object_section("scrub");
   f->dump_stream("pgid") << pgid;
-  f->dump_stream("sched_time") << schedule.not_before;
-  f->dump_stream("orig_sched_time") << schedule.scheduled_at;
-  f->dump_stream("deadline") << schedule.deadline;
-  f->dump_bool("forced",
-              schedule.scheduled_at == PgScrubber::scrub_must_stamp());
+  f->dump_stream("sched_time") << get_sched_time();
+  f->dump_stream("orig_sched_time") << sch.scheduled_at;
+  f->dump_stream("deadline") << sch.deadline;
+  f->dump_bool("forced", entry.urgency >= urgency_t::operator_requested);
   f->close_section();
 }
+
+// a set of static functions to determine, given a scheduling target's urgency,
+// what restrictions apply to that target (and what exemptions it has).
+
+bool ScrubJob::observes_noscrub_flags(urgency_t urgency)
+{
+  return urgency < urgency_t::after_repair;
+}
+
+bool ScrubJob::observes_allowed_hours(urgency_t urgency)
+{
+  return urgency < urgency_t::operator_requested;
+}
+
+bool ScrubJob::observes_load_limit(urgency_t urgency)
+{
+  return urgency < urgency_t::after_repair;
+}
+
+bool ScrubJob::requires_reservation(urgency_t urgency)
+{
+  return urgency < urgency_t::after_repair;
+}
+
+bool ScrubJob::requires_randomization(urgency_t urgency)
+{
+  return urgency == urgency_t::periodic_regular;
+}
+
+bool ScrubJob::observes_max_concurrency(urgency_t urgency)
+{
+  return urgency < urgency_t::operator_requested;
+}
index 3b0fb8cb8b251673857c35f4f66743f439d1a527..cba61ec24c8e7e95edefe8a0a798961743ac9705 100644 (file)
@@ -2,7 +2,6 @@
 // vim: ts=8 sw=2 smarttab
 #pragma once
 
-#include <atomic>
 #include <chrono>
 #include <compare>
 #include <iostream>
 #include <vector>
 
 #include "common/ceph_atomic.h"
+#include "common/fmt_common.h"
 #include "include/utime_fmt.h"
 #include "osd/osd_types.h"
 #include "osd/osd_types_fmt.h"
 #include "osd/scrubber_common.h"
 #include "scrub_queue_entry.h"
 
-/**
- * The ID used to name a candidate to scrub:
- * - in this version: a PG is identified by its spg_t
- * - in the (near) future: a PG + a scrub type (shallow/deep)
- */
-using ScrubTargetId = spg_t;
-
-
 namespace Scrub {
 
 enum class must_scrub_t { not_mandatory, mandatory };
@@ -86,58 +78,120 @@ struct sched_conf_t {
 };
 
 
-class ScrubJob {
- public:
+/**
+ * a wrapper around a Scrub::SchedEntry, adding some state flags
+ * to be used only by the Scrubber. Note that the SchedEntry itself is known to
+ * multiple objects (and must be kept small in size).
+*/
+struct SchedTarget {
+  constexpr explicit SchedTarget(spg_t pg_id, scrub_level_t scrub_level)
+      : sched_info{pg_id, scrub_level}
+  {}
+
+  /// our ID and scheduling parameters
+  SchedEntry sched_info;
+
   /**
-   * 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
-   * to scrub out of scrub hours.
+   * is this target (meaning - a copy of this specific combination of
+   * PG and scrub type) currently in the queue?
    */
-  scrub_schedule_t schedule;
+  bool queued{false};
+
+  // some helper functions
+
+  /// resets to the after-construction state
+  void reset();
+
+  /// set the urgency to the max of the current and the provided urgency
+  void up_urgency_to(urgency_t u);
+
+  /// access that part of the SchedTarget that is queued in the scrub queue
+  const SchedEntry& queued_element() const { return sched_info; }
+
+  bool is_deep() const { return sched_info.level == scrub_level_t::deep; }
+
+  bool is_shallow() const { return sched_info.level == scrub_level_t::shallow; }
+
+  scrub_level_t level() const { return sched_info.level; }
+
+  urgency_t urgency() const { return sched_info.urgency; }
 
+  bool was_delayed() const { return sched_info.last_issue != delay_cause_t::none; }
+
+  /// provides r/w access to the scheduling sub-object
+  SchedEntry& sched_info_ref() { return sched_info; }
+};
+
+
+
+class ScrubJob {
+ public:
   /// pg to be scrubbed
   spg_t pgid;
 
   /// the OSD id (for the log)
   int whoami;
 
+  /*
+   * the schedule for the next scrub at the specific level. Also - the
+   * urgency and characteristics of the scrub (e.g. - high priority,
+   * must-repair, ...)
+   */
+  SchedTarget shallow_target;
+  SchedTarget deep_target;
+
   /**
    * Set whenever the PG scrubs are managed by the OSD (i.e. - from becoming
    * an active Primary till the end of the interval).
    */
   bool registered{false};
 
-  /**
-   * there is a scrub target for this PG in the queue.
-   * \attn: temporary. Will be replaced with a pair of flags in the
-   * two level-specific scheduling targets.
-   */
-  bool target_queued{false};
-
   /// how the last attempt to scrub this PG ended
   delay_cause_t last_issue{delay_cause_t::none};
 
   /**
-    * the scrubber is waiting for locked objects to be unlocked.
-    * Set after a grace period has passed.
-    */
+   * the scrubber is waiting for locked objects to be unlocked.
+   * Set after a grace period has passed.
+   */
   bool blocked{false};
   utime_t blocked_since{};
 
   CephContext* cct;
 
-  bool high_priority{false};
-
   ScrubJob(CephContext* cct, const spg_t& pg, int node_id);
 
-  utime_t get_sched_time() const { return schedule.not_before; }
+  /**
+   * returns a possible reference to the earliest target that is eligible. If
+   * both the shallow and the deep targets have their n.b. in the future,
+   * nullopt is returned.
+   */
+  std::optional<std::reference_wrapper<SchedTarget>> earliest_eligible(
+      utime_t scrub_clock_now);
+  std::optional<std::reference_wrapper<const SchedTarget>> earliest_eligible(
+      utime_t scrub_clock_now) const;
+
+  /**
+   * the target with the earliest 'not-before' time (i.e. - assuming
+   * both targets are in the future).
+   * \attn: might return the wrong answer if both targets are eligible.
+   * If a need arises, a version that accepts the current time as a parameter
+   * should be added. Then - a correct determination can be made for
+   * all cases.
+   */
+  const SchedTarget& earliest_target() const;
+  SchedTarget& earliest_target();
+
+  /// the not-before of our earliest target (either shallow or deep)
+  utime_t get_sched_time() const;
 
   std::string_view state_desc() const
   {
-    return registered ? (target_queued ? "queued" : "registered")
+    return registered ? (is_queued() ? "queued" : "registered")
                      : "not-registered";
   }
 
+  SchedTarget& get_target(scrub_level_t s_or_d);
+
   /**
    * Given a proposed time for the next scrub, and the relevant
    * configuration, adjust_schedule() determines the actual target time,
@@ -151,36 +205,31 @@ class ScrubJob {
    *   on the configuration; the deadline is set further out (if configured)
    *   and the n.b. is reset to the target.
    */
-  void adjust_schedule(
-    const Scrub::sched_params_t& suggested,
-    const Scrub::sched_conf_t& aconf,
+  void adjust_shallow_schedule(
+    utime_t last_scrub,
+    const Scrub::sched_conf_t& app_conf,
+    utime_t scrub_clock_now,
+    delay_ready_t modify_ready_targets);
+
+  void adjust_deep_schedule(
+    utime_t last_deep,
+    const Scrub::sched_conf_t& app_conf,
     utime_t scrub_clock_now,
-    Scrub::delay_ready_t modify_ready_targets);
+    delay_ready_t modify_ready_targets);
 
   /**
-   * push the 'not_before' time out by 'delay' seconds, so that this scrub target
+   * For the level specified, set the 'not-before' time to 'now+delay',
+   * so that this scrub target
    * would not be retried before 'delay' seconds have passed.
+   * The 'last_issue' is updated to the cause of the delay.
+   * \returns a reference to the target that was modified.
    */
-  void delay_on_failure(
+  [[maybe_unused]] SchedTarget& delay_on_failure(
+      scrub_level_t level,
       std::chrono::seconds delay,
       delay_cause_t delay_cause,
       utime_t scrub_clock_now);
 
-  /**
-   *  Recalculating any possible updates to the scrub schedule, following an
-   *  aborted scrub attempt.
-   *  Usually - we can use the same schedule that triggered the aborted scrub.
-   *  But we must take into account scenarios where "something" caused the
-   *  parameters prepared for the *next* scrub to show higher urgency or
-   *  priority. "Something" - as in an operator command requiring immediate
-   *  scrubbing, or a change in the pool/cluster configuration.
-   */
-  void merge_and_delay(
-      const scrub_schedule_t& aborted_schedule,
-      Scrub::delay_cause_t issue,
-      requested_scrub_t updated_flags,
-      utime_t scrub_clock_now);
-
  /**
    * recalculate the scheduling parameters for the periodic scrub targets.
    * Used whenever the "external state" of the PG changes, e.g. when made
@@ -199,11 +248,12 @@ class ScrubJob {
 
   bool is_registered() const { return registered; }
 
-  /**
-   * is this a high priority scrub job?
-   * High priority - (usually) a scrub that was initiated by the operator
-   */
-  bool is_high_priority() const { return high_priority; }
+  /// are any of our two SchedTargets queued in the scrub queue?
+  bool is_queued() const;
+
+  /// mark both targets as queued / not queued
+  void clear_both_targets_queued();
+  void set_both_targets_queued();
 
   /**
    * a text description of the "scheduling intentions" of this PG:
@@ -220,12 +270,71 @@ class ScrubJob {
   // SchedTarget(s).
   std::partial_ordering operator<=>(const ScrubJob& rhs) const
   {
-    return schedule <=> rhs.schedule;
+    return cmp_entries(
+      ceph_clock_now(), shallow_target.queued_element(),
+      deep_target.queued_element());
   };
-};
 
-using ScrubQContainer = std::vector<std::unique_ptr<ScrubJob>>;
 
+ /*
+ * Restrictions and limitations that apply to each urgency level:
+ * -------------------------------------------------------------
+ * Some types of scrubs are exempt from some or all of the preconditions and
+ * limitations that apply to regular scrubs. The following table
+ * details the specific set of exemptions per 'urgency' level:
+ * (note: regular scrubs that are overdue are also allowed a specific
+ * set of exemptions. Those will be covered elsewhere).
+ *
+ * The relevant limitations are:
+ * - reservation: the scrub must reserve replicas;
+ * - dow/time: the scrub must adhere to the allowed days-of-week/hours;
+ * - ext-sleep: if initiated during allowed hours, the scrub is penalized
+ *   if continued into the forbidden times, by having a longer sleep time;
+ *   (note that this is only applicable to the wq scheduler).
+ * - load: the scrub must not be initiated if the OSD is under heavy CPU load;
+ * - noscrub: the scrub is aborted if the 'noscrub' flag (or the
+ *  'nodeep-scrub' flag for deep scrubs) is set;
+ * - randomization: the scrub's target time is extended by a random
+ *   duration. This only applies to periodic scrubs.
+ * - configuration changes: the target time may be modified following
+ *   a change in the configuration. This only applies to periodic scrubs.
+ * - max-scrubs: the scrub must not be initiated if the OSD is already
+ *   scrubbing too many PGs (the 'osd_max_scrubs' limit).
+ * - backoff: the scrub must not be initiated this tick if a dice roll
+ *   failed.
+ * - recovery: the scrub must not be initiated if the OSD is currently
+ *   recovering PGs.
+ *
+ * The following table summarizes the limitations in effect per urgency level:
+ *
+ *  +------------+------------+--------------+----------+-------------+
+ *  | limitation | must-scrub | after-repair | operator | must-repair |
+ *  +------------+------------+--------------+----------+-------------+
+ *  | reservation|    yes!    |      no      |     no   |      no     |
+ *  | dow/time   |    yes     |     yes      |     no   |      no     |
+ *  | ext-sleep  |    no?     |      no      |     no   |      no     |
+ *  | load       |    yes     |      no      |     no   |      no     |
+ *  | noscrub    |    yes     |      no?     |     no   |      no     |
+ *  | max-scrubs |    yes     |      yes?    |     no   |      no     |
+ *  | backoff    |    yes     |      no      |     no   |      no     |
+ *  +------------+------------+--------------+----------+-------------+
+ */
+
+  // a set of helper functions for determining, for each urgency level, what
+  // restrictions and limitations apply to that level.
+
+  static bool observes_noscrub_flags(urgency_t urgency);
+
+  static bool observes_allowed_hours(urgency_t urgency);
+
+  static bool observes_load_limit(urgency_t urgency);
+
+  static bool requires_reservation(urgency_t urgency);
+
+  static bool requires_randomization(urgency_t urgency);
+
+  static bool observes_max_concurrency(urgency_t urgency);
+};
 }  // namespace Scrub
 
 namespace std {
@@ -246,6 +355,18 @@ struct formatter<Scrub::sched_params_t> {
   }
 };
 
+template <>
+struct formatter<Scrub::SchedTarget> {
+  constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+  template <typename FormatContext>
+  auto format(const Scrub::SchedTarget& st, FormatContext& ctx) const
+  {
+     return fmt::format_to(
+       ctx.out(), "{},q:{:c},issue:{}", st.sched_info,
+       st.queued ? '+' : '-', st.sched_info.last_issue);
+  }
+};
+
 template <>
 struct formatter<Scrub::ScrubJob> {
   constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
@@ -254,9 +375,8 @@ struct formatter<Scrub::ScrubJob> {
   auto format(const Scrub::ScrubJob& sjob, FormatContext& ctx) const
   {
     return fmt::format_to(
-       ctx.out(), "pg[{}]:nb:{:s} / trg:{:s} / dl:{:s} <{}>",
-       sjob.pgid, sjob.schedule.not_before, sjob.schedule.scheduled_at,
-       sjob.schedule.deadline, sjob.state_desc());
+       ctx.out(), "pg[{}]:sh:{}/dp:{}<{}>",
+       sjob.pgid, sjob.shallow_target, sjob.deep_target, sjob.state_desc());
   }
 };
 
index 2177c7e85ab427ea0c0466a60afc49e934533526..39b853bb6f9742f6f50bd0df78e0980e8e012ea4 100644 (file)
@@ -162,7 +162,11 @@ struct ScrubMachineListener {
   /// the part that actually finalizes a scrub
   virtual void scrub_finish() = 0;
 
-  /// notify the scrubber about a scrub failure
+  /**
+   * The scrub session was aborted. We must restore the scheduling object
+   * that triggered the scrub back to the queue - but we may have to update
+   * it with changes requested (e.g. by an operator command).
+   */
   virtual void on_mid_scrub_abort(Scrub::delay_cause_t cause) = 0;
 
   /**
index d4648c68a037afb5eace49bfd6f32e9742366520..bc493b8437f9660a51c2f9c9af041b4f1a37e14c 100644 (file)
@@ -605,10 +605,9 @@ struct ScrubPgIF {
   virtual bool reserve_local() = 0;
 
   /**
-   * Recalculate the required scrub time.
+   * Recalculate scrub (both deep & shallow) schedules
    *
-   * This function assumes that the queue registration status is up-to-date,
-   * i.e. the OSD "knows our name" if-f we are the Primary.
+   * Dequeues the scrub job, and re-queues it with the new schedule.
    */
   virtual void update_scrub_job(Scrub::delay_ready_t delay_ready) = 0;