]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd/scrub: OSD's scrub queue now holds SchedEntry-s
authorRonen Friedman <rfriedma@redhat.com>
Wed, 24 Jul 2024 07:02:46 +0000 (02:02 -0500)
committerRonen Friedman <rfriedma@redhat.com>
Sun, 25 Aug 2024 13:01:00 +0000 (08:01 -0500)
The OSD's scrub queue now holds SchedEntry-s, instead of ScrubJob-s.
The queue itself is implemented using the 'not_before_queue_t' class.

Note: this is not a stable state of the scrubber code. In the next
commits:
- modifying the way sched targets are modified and updated, to match the
  new queue implementation.
- removing the 'planned scrub' flags.

Important note: the interaction of initiate_scrub() and pop_ready_pg()
is not changed by this commit. Namely:

Currently - pop..() loops over all eligible jobs, until it finds one
that matches the environment restrictions (which most of the time, as the
concurrency limit is usually reached, would be 'high-priority-only').

The other option is to maintain Sam's 'not_before_q' clean interface: we
always pop the top, and if that top fails the preconds tests - we delay and
re-push. This has the following troubling implications:

- it would take a long time to find a viable scrub job, if the problem
  is related to, for example, 'no scrub'.
- local resources failure (inc_scrubs() failure) must be handles
  separately, as we do not want to reshuffle the queue for this
  very very common case.
- but the real problem: unneeded shuffling of the queue, even as the
  problem is not with the scrub job itself, but with the environment
  (esp. no-scrub etc.).
  This is a common case, and it would be wrong to reshuffle the queue
  for that.
- and - remember that any change to a sched-entry must be done under PG
  lock.

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
src/osd/PG.cc
src/osd/PG.h
src/osd/scrubber/osd_scrub.cc
src/osd/scrubber/osd_scrub.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
src/osd/scrubber_common.h

index 96fd6435f7217040d0226a972c3617d590a54dd8..cda893397bdc658a61c1a898863af7a37656fce9 100644 (file)
@@ -1325,12 +1325,12 @@ unsigned int PG::scrub_requeue_priority(Scrub::scrub_prio_t with_priority, unsig
 
 
 Scrub::schedule_result_t PG::start_scrubbing(
-    std::unique_ptr<Scrub::ScrubJob> candidate,
+    const Scrub::SchedEntry& candidate,
     Scrub::OSDRestrictions osd_restrictions)
 {
   dout(10) << fmt::format(
-                 "{}: {}+{} (env restrictions:{})", __func__,
-                 (is_active() ? "<active>" : "<not-active>"),
+                 "{}: scrubbing {}. {}+{} (env restrictions:{})", __func__,
+                 candidate, (is_active() ? "<active>" : "<not-active>"),
                  (is_clean() ? "<clean>" : "<not-clean>"), osd_restrictions)
           << dendl;
   ceph_assert(ceph_mutex_is_locked(_lock));
@@ -1349,7 +1349,7 @@ Scrub::schedule_result_t PG::start_scrubbing(
        get_pgbackend()->auto_repair_supported());
 
   return m_scrubber->start_scrub_session(
-      std::move(candidate), osd_restrictions, pg_cond, m_planned_scrub);
+      candidate.level, osd_restrictions, pg_cond, m_planned_scrub);
 }
 
 double PG::next_deepscrub_interval() const
index b18e3e163896298ed5bfe2da099c63c13ff6bdcc..5f3cd32e0230577008d9f0d02ca98bac48efcb7b 100644 (file)
@@ -701,7 +701,7 @@ public:
   bool get_must_scrub() const;
 
   Scrub::schedule_result_t start_scrubbing(
-    std::unique_ptr<Scrub::ScrubJob> candidate,
+    const Scrub::SchedEntry& candidate,
     Scrub::OSDRestrictions osd_restrictions);
 
   unsigned int scrub_requeue_priority(
index 76e590a0930daa4d48a08e607dff7385f60e485d..e2e1aa34661b87c5f5ccc124cf393a37c914d259 100644 (file)
@@ -89,7 +89,7 @@ bool OsdScrub::scrub_random_backoff() const
 
 void OsdScrub::debug_log_all_jobs() const
 {
-  m_queue.for_each_job([this](const Scrub::ScrubJob& sj) {
+  m_queue.for_each_job([this](const Scrub::SchedEntry& sj) {
     dout(20) << fmt::format("\tscrub-queue jobs: {}", sj) << dendl;
   }, 20);
 }
@@ -125,16 +125,13 @@ void OsdScrub::initiate_scrub(bool is_recovery_active)
     debug_log_all_jobs();
   }
 
-  auto candidate = m_queue.pop_ready_pg(env_restrictions, scrub_time);
+  auto candidate = m_queue.pop_ready_entry(env_restrictions, scrub_time);
   if (!candidate) {
     dout(20) << "no PGs are ready for scrubbing" << dendl;
     return;
   }
 
-  auto candidate_pg = candidate->pgid;
-  auto res = initiate_a_scrub(std::move(candidate), env_restrictions);
-
-  switch (res) {
+  switch (initiate_a_scrub(*candidate, env_restrictions)) {
     case schedule_result_t::target_specific_failure:
     case schedule_result_t::osd_wide_failure:
       // No scrub this tick.
@@ -142,7 +139,7 @@ void OsdScrub::initiate_scrub(bool is_recovery_active)
       break;
 
     case schedule_result_t::scrub_initiated:
-      dout(20) << fmt::format("scrub initiated for pg[{}]", candidate_pg)
+      dout(20) << fmt::format("scrub initiated for pg[{}]", candidate->pgid)
               << dendl;
       break;
   }
@@ -195,36 +192,34 @@ Scrub::OSDRestrictions OsdScrub::restrictions_on_scrubbing(
 
 
 Scrub::schedule_result_t OsdScrub::initiate_a_scrub(
-    std::unique_ptr<Scrub::ScrubJob> candidate,
+    const Scrub::SchedEntry& candidate,
     Scrub::OSDRestrictions restrictions)
 {
-  dout(20) << fmt::format("trying pg[{}]", candidate->pgid) << dendl;
+  dout(20) << fmt::format("trying pg[{}]", candidate.pgid) << dendl;
 
   // we have a candidate to scrub. We need some PG information to
   // know if scrubbing is allowed
 
-  auto locked_pg = m_osd_svc.get_locked_pg(candidate->pgid);
+  auto locked_pg = m_osd_svc.get_locked_pg(candidate.pgid);
   if (!locked_pg) {
     // the PG was dequeued in the short timespan between querying the
     // scrub queue - and now.
-    dout(5) << fmt::format("pg[{}] not found", candidate->pgid) << dendl;
+    dout(5) << fmt::format("pg[{}] not found", candidate.pgid) << dendl;
     return Scrub::schedule_result_t::target_specific_failure;
   }
 
-  // note: the 'candidate', which in this step is a copy of the scrub job,
+  // note: the 'candidate' (a SchedEntry, identifying PG & level)
   // was already dequeued. The "original" scrub job cannot be accessed from
   // here directly. Thus - we leave it to start_scrubbing() (via a call
-  // to PgScrubber::start_scrub_session() to mark it as dequeued.
-  return locked_pg->pg()->start_scrubbing(std::move(candidate), restrictions);
+  // to PgScrubber::start_scrub_session()) to mark it as dequeued.
+  return locked_pg->pg()->start_scrubbing(candidate, restrictions);
 }
 
 
 void OsdScrub::on_config_change()
 {
-  auto to_notify = m_queue.get_pgs([](const Scrub::ScrubJob& sj) -> bool {
-    ceph_assert(sj.registered);
-    return true;
-  });
+  auto to_notify = m_queue.get_pgs(
+      [](const Scrub::SchedEntry& sj, bool) -> bool { return true; });
 
   for (const auto& p : to_notify) {
     dout(30) << fmt::format("rescheduling pg[{}] scrubs", p) << dendl;
@@ -426,9 +421,14 @@ PerfCounters* OsdScrub::get_perf_counters(int pool_type, scrub_level_t level)
 // forwarders to the queue
 
 
-void OsdScrub::enqueue_target(const Scrub::ScrubJob& sjob)
+void OsdScrub::enqueue_scrub_job(const Scrub::ScrubJob& sjob)
+{
+  m_queue.enqueue_scrub_job(sjob);
+}
+
+void OsdScrub::enqueue_target(const Scrub::SchedTarget& trgt)
 {
-  m_queue.enqueue_target(sjob);
+  m_queue.enqueue_target(trgt);
 }
 
 void OsdScrub::remove_from_osd_queue(spg_t pgid)
index 535282a2580f56176c192157f28e59f5663b13aa..1f1c62bc8dab7ac9befc744d40eab43bededd4fc 100644 (file)
@@ -80,8 +80,13 @@ class OsdScrub {
    * Add the scrub job to the list of jobs (i.e. list of PGs) to be periodically
    * scrubbed by the OSD.
    */
-  void enqueue_target(const Scrub::ScrubJob& sjob);
+  void enqueue_scrub_job(const Scrub::ScrubJob& sjob);
 
+  /**
+   * copy the scheduling element (the SchedEntry sub-object) part of
+   * the SchedTarget to the queue.
+   */
+  void enqueue_target(const Scrub::SchedTarget& trgt);
 
   /**
    * remove the pg from set of PGs to be scanned for scrubbing.
@@ -153,7 +158,7 @@ class OsdScrub {
    *          initiated, and if not - why.
    */
   Scrub::schedule_result_t initiate_a_scrub(
-      std::unique_ptr<Scrub::ScrubJob> candidate,
+      const Scrub::SchedEntry& candidate,
       Scrub::OSDRestrictions restrictions);
 
   /// resource reservation management
index 06d6947b53516c691b277cdedf13b9779d5e5007..8784438ce633670cc4cff07d04f734daa5505c81 100644 (file)
@@ -12,10 +12,10 @@ using namespace ::std::chrono_literals;
 using namespace ::std::literals;
 
 using must_scrub_t = Scrub::must_scrub_t;
-using ScrubQContainer = Scrub::ScrubQContainer;
 using sched_params_t = Scrub::sched_params_t;
 using OSDRestrictions = Scrub::OSDRestrictions;
 using ScrubJob = Scrub::ScrubJob;
+using SchedEntry = ::Scrub::SchedEntry;
 
 
 
@@ -55,64 +55,41 @@ void ScrubQueue::remove_from_osd_queue(spg_t pgid)
   dout(10) << fmt::format(
                  "removing pg[{}] from OSD scrub queue", pgid)
           << dendl;
-
   std::unique_lock lck{jobs_lock};
-  std::erase_if(to_scrub, [pgid](const auto& job) {
-    return job->pgid == pgid;
-  });
+  to_scrub.remove_by_class<spg_t>(pgid);
 }
 
 
-void ScrubQueue::enqueue_target(const Scrub::ScrubJob& sjob)
+void ScrubQueue::enqueue_scrub_job(const Scrub::ScrubJob& sjob)
 {
   std::unique_lock lck{jobs_lock};
-  // the costly copying is only for this stage
-  to_scrub.push_back(std::make_unique<ScrubJob>(sjob));
+  to_scrub.enqueue(sjob.shallow_target.queued_element());
+  to_scrub.enqueue(sjob.deep_target.queued_element());
 }
 
+void ScrubQueue::enqueue_target(const Scrub::SchedTarget& trgt)
+{
+  std::unique_lock lck{jobs_lock};
+  to_scrub.enqueue(trgt.queued_element());
+}
 
-std::unique_ptr<ScrubJob> ScrubQueue::pop_ready_pg(
+std::optional<Scrub::SchedEntry> ScrubQueue::pop_ready_entry(
     OSDRestrictions restrictions,  // note: 4B in size! (thus - copy)
     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 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() ||
+  auto eligible_filtr = [time_now, rst = restrictions](
+                                 const SchedEntry& e) -> bool {
+      return e.is_ripe(time_now) &&
+            (e.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);
+               (!e.schedule.deadline.is_zero() &&
+                e.schedule.deadline <= time_now))));
   };
 
-  auto not_ripes =
-      std::partition(to_scrub.begin(), to_scrub.end(), eligible_filtr);
-  if (not_ripes == to_scrub.begin()) {
-    return nullptr;
-  }
-  auto top = std::min_element(
-      to_scrub.begin(), not_ripes,
-      [](const std::unique_ptr<ScrubJob>& lhs,
-        const std::unique_ptr<ScrubJob>& rhs) -> bool {
-       return lhs->get_sched_time() < rhs->get_sched_time();
-      });
-
-  if (top == not_ripes) {
-    return nullptr;
-  }
-
-  auto top_job = std::move(*top);
-  to_scrub.erase(top);
-  return top_job;
+  std::unique_lock lck{jobs_lock};
+  to_scrub.advance_time(time_now);
+  return to_scrub.dequeue_by_pred(eligible_filtr);
 }
 
 
@@ -120,30 +97,39 @@ std::unique_ptr<ScrubJob> ScrubQueue::pop_ready_pg(
  * the set of all PGs named by the entries in the queue (but only those
  * entries that satisfy the predicate)
  */
-std::set<spg_t> ScrubQueue::get_pgs(const ScrubQueue::EntryPred& cond) const
+std::set<spg_t> ScrubQueue::get_pgs(const ScrubQueue::EntryPred& pred) const
 {
-  std::lock_guard lck{jobs_lock};
-  std::set<spg_t> pgs_w_matching_entries;
-  for (const auto& job : to_scrub) {
-    if (cond(*job)) {
-      pgs_w_matching_entries.insert(job->pgid);
-    }
-  }
-  return pgs_w_matching_entries;
+  std::lock_guard lck(jobs_lock);
+
+  using acc_t = std::set<spg_t>;
+  auto extract_pg =
+      [pred](acc_t&& acc, const SchedEntry& se, bool is_eligible) {
+       if (pred(se, is_eligible)) {
+         acc.insert(se.pgid);
+       }
+       return std::move(acc);
+      };
+
+  return to_scrub.accumulate<acc_t, decltype(extract_pg)>(
+      std::move(extract_pg));
 }
 
 
 void ScrubQueue::for_each_job(
-    std::function<void(const Scrub::ScrubJob&)> fn,
+    std::function<void(const Scrub::SchedEntry&)> fn,
     int max_jobs) const
 {
+  auto fn_call = [fn](const SchedEntry& e, bool) -> void { fn(e); };
   std::lock_guard lck(jobs_lock);
-  // implementation note: not using 'for_each_n()', as it is UB
-  // if there aren't enough elements
-  std::for_each(
-      to_scrub.begin(),
-      to_scrub.begin() + std::min(max_jobs, static_cast<int>(to_scrub.size())),
-      [fn](const auto& job) { fn(*job); });
+  to_scrub.for_each_n<decltype(fn_call)>(std::move(fn_call), max_jobs);
+}
+
+
+bool ScrubQueue::remove_entry_unlocked(spg_t pgid, scrub_level_t s_or_d)
+{
+  auto same_lvl = [s_or_d](const SchedEntry& e) { return e.level == s_or_d; };
+  return to_scrub.remove_if_by_class<spg_t, decltype(same_lvl)>(
+      pgid, std::move(same_lvl), 1);
 }
 
 
@@ -152,12 +138,21 @@ void ScrubQueue::dump_scrubs(ceph::Formatter* f) const
   ceph_assert(f != nullptr);
   f->open_array_section("scrubs");
   for_each_job(
-      [&f](const Scrub::ScrubJob& j) { j.dump(f); },
+      [&f](const Scrub::SchedEntry& e) {
+       f->open_object_section("scrub");
+       f->dump_stream("pgid") << e.pgid;
+       f->dump_stream("sched_time") << e.schedule.not_before;
+       f->dump_stream("orig_sched_time") << e.schedule.scheduled_at;
+       f->dump_stream("deadline") << e.schedule.deadline;
+       f->dump_bool(
+           "forced",
+           e.schedule.scheduled_at == PgScrubber::scrub_must_stamp());
+       f->close_section();
+      },
       std::numeric_limits<int>::max());
   f->close_section();
 }
 
-
 // ////////////////////////////////////////////////////////////////////////// //
 // ScrubQueue - maintaining the 'blocked on a locked object' count
 
index 68a655450f3b2a7962a4297a25de4898daf79569..5f8aaec3a8ce78574e74fdb71d362dc66cf3cea8 100644 (file)
@@ -29,7 +29,7 @@
   ┌───────────────────────────▼────────────┐
   │                                        │
   │                                        │
-  │  ScrubQContainer    to_scrub <>────────┼────────┐
+  │  not_before_queue_t to_scrub <>────────┼────────┐
   │                                        │        │
   │                                        │        │
   │  OSD_wide resource counters            │        │
     │      ┌─────▼──────┐
     │      │Copy of     │
     │      │job's       ├┐
-    │      │sched params││
-    │      │(*)         │┼┐
+    │      │sched targts││
+    │      │            │┼┐
     │      │            │┼┘◄────────────────────────┐
     └──────┤            ││                          │
-           │            ││   (*) for now - a copy   │
-           │            ││       of the whole SJ    │
+           │            ││                          │
+           │            ││                          │
            │            ││                          │
            └┬───────────┼│                          │
             └─┼┼┼┼┼┼┼┼┼┼┼│                          │
   │               │ScrubJob             │           │
   │               │                     │           │
   │               │     ┌───────────────┤           │
-  │               │     │Sched params   ├───────────┘
+  │               │     │Sched target   ├───────────┘
   └───────────────┤     └───────────────┤
-                  │                     │
+                  │                     │           ^
+                  │     ┌───────────────┤           |
+                  │     │Sched target   ├───────────┘
+                  │     └───────────────┤
                   └─────────────────────┘
 
 
@@ -110,6 +113,7 @@ ScrubQueue interfaces (main functions):
 #include <optional>
 
 #include "common/AsyncReserver.h"
+#include "common/not_before_queue.h"
 #include "utime.h"
 #include "osd/scrubber/scrub_job.h"
 #include "osd/PG.h"
@@ -164,7 +168,8 @@ class ScrubQueue {
   void remove_from_osd_queue(spg_t pgid);
 
   /// A predicate over the entries in the queue
-  using EntryPred = std::function<bool(const Scrub::ScrubJob&)>;
+  using EntryPred =
+      std::function<bool(const ::Scrub::SchedEntry&, bool only_eligibles)>;
 
   /**
    * the set of all PGs named by the entries in the queue (but only those
@@ -173,10 +178,16 @@ class ScrubQueue {
   std::set<spg_t> get_pgs(const EntryPred&) const;
 
   /**
-   * Add the scrub job to the list of jobs (i.e. list of PGs) to be periodically
-   * scrubbed by the OSD.
+   * Add the scrub job (both SchedTargets) to the list of jobs (i.e. list of
+   * PGs) to be periodically scrubbed by the OSD.
+   */
+  void enqueue_scrub_job(const Scrub::ScrubJob& sjob);
+
+  /**
+   * copy the scheduling element (the SchedEntry sub-object) part of
+   * the SchedTarget to the queue.
    */
-  void enqueue_target(const Scrub::ScrubJob& sjob);
+  void enqueue_target(const Scrub::SchedTarget& trgt);
 
   std::ostream& gen_prefix(std::ostream& out, std::string_view fn) const;
 
@@ -184,7 +195,7 @@ class ScrubQueue {
   void dump_scrubs(ceph::Formatter* f) const;
 
   void for_each_job(
-      std::function<void(const Scrub::ScrubJob&)> fn,
+      std::function<void(const Scrub::SchedEntry&)> fn,
       int max_jobs) const;
 
   /// counting the number of PGs stuck while scrubbing, waiting for objects
@@ -193,14 +204,14 @@ class ScrubQueue {
   int get_blocked_pgs_count() const;
 
   /**
-   * find the nearest scrub-job (later on - scrub target) that is ready to
+   * find the nearest scheduling entry that is ready to
    * to be scrubbed (taking 'restrictions' into account).
    * The selected entry in the queue is dequeued and returned.
-   * A nullptr is returned if no eligible entry is found.
+   * nullopt is returned if no such entry exists.
    */
-  std::unique_ptr<Scrub::ScrubJob> pop_ready_pg(
-      Scrub::OSDRestrictions restrictions,  // note: 4B in size! (copy)
-      utime_t time_now);
+  std::optional<Scrub::SchedEntry> pop_ready_entry(
+    Scrub::OSDRestrictions restrictions,
+    utime_t time_now);
 
  private:
   CephContext* cct;
@@ -219,7 +230,7 @@ class ScrubQueue {
    */
   mutable ceph::mutex jobs_lock = ceph::make_mutex("ScrubQueue::jobs_lock");
 
-  Scrub::ScrubQContainer to_scrub;   ///< scrub jobs (i.e. PGs) to scrub
+  not_before_queue_t<Scrub::SchedEntry> to_scrub;
 
   /**
    * The scrubbing of PGs might be delayed if the scrubbed chunk of objects is
@@ -233,6 +244,12 @@ class ScrubQueue {
    */
   std::atomic_int_fast16_t blocked_scrubs_cnt{0};
 
+  /**
+   * remove the entry from the queue.
+   * returns: true if it was there, false otherwise.
+   */
+  bool remove_entry_unlocked(spg_t pgid, scrub_level_t s_or_d);
+
 protected: // used by the unit-tests
   /**
    * unit-tests will override this function to return a mock time
index 7d912daaa4ee1b555b58415ac480fb7a46a711f3..8546af02cd2cadc522af0079fa1f7e7a0105bfb8 100644 (file)
@@ -498,6 +498,7 @@ void PgScrubber::rm_from_osd_scrubbing()
                    "{}: prev. state: {}", __func__, registration_state())
             << dendl;
     m_osds->get_scrub_services().remove_from_osd_queue(m_pg_id);
+    m_scrub_job->clear_both_targets_queued();
     m_scrub_job->registered = false;
   }
 }
@@ -605,7 +606,8 @@ void PgScrubber::on_primary_active_clean()
 /*
  * A note re the call to publish_stats_to_osd() below:
  * - we are called from either request_rescrubbing() or scrub_requested().
- * - in both cases - the schedule was modified, and needs to be published;
+ * Update: also from scrub_finish() & schedule_scrub_with_osd().
+ * - in all cases - the schedule was modified, and needs to be published;
  * - we are a Primary.
  * - in the 1st case - the call is made as part of scrub_finish(), which
  *   guarantees that the PG is locked and the interval is still the same.
@@ -635,9 +637,10 @@ void PgScrubber::update_scrub_job(Scrub::delay_ready_t delay_ready)
     return;
   }
   ceph_assert(m_pg->is_locked());
-  if (m_scrub_job->target_queued) {
+  if (m_scrub_job->is_queued()) {
+    // one or both of the targets are in the queue. Remove them.
     m_osds->get_scrub_services().remove_from_osd_queue(m_pg_id);
-    m_scrub_job->target_queued = false;
+    m_scrub_job->clear_both_targets_queued();
     dout(20) << fmt::format(
                    "{}: PG[{}] dequeuing for an update", __func__, m_pg_id)
             << dendl;
@@ -645,11 +648,9 @@ void PgScrubber::update_scrub_job(Scrub::delay_ready_t delay_ready)
 
   const auto scrub_clock_now = ceph_clock_now();
   update_targets(m_planned_scrub, scrub_clock_now);
-
-  m_osds->get_scrub_services().enqueue_target(*m_scrub_job);
-  m_scrub_job->target_queued = true;
+  m_osds->get_scrub_services().enqueue_scrub_job(*m_scrub_job);
+  m_scrub_job->set_both_targets_queued();
   m_pg->publish_stats_to_osd();
-
   dout(10) << fmt::format(
                  "{}: flags:<{}> job on exit:{}", __func__, m_planned_scrub,
                  *m_scrub_job)
@@ -1888,7 +1889,9 @@ void PgScrubber::flag_reservations_failure()
 {
   dout(10) << __func__ << dendl;
   // delay the next invocation of the scrubber on this target
-  requeue_penalized(Scrub::delay_cause_t::replicas);
+  requeue_penalized(
+      m_active_target->level(), Scrub::delay_cause_t::replicas,
+      ceph_clock_now());
 }
 
 /*
@@ -2136,10 +2139,6 @@ void PgScrubber::on_digest_updates()
  * have had its priority, flags, or schedule modified in the meantime.
  * And - it does not (at least initially, i.e. immediately after
  * set_op_parameters()), have high priority.
- *
- * 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)
 {
@@ -2171,8 +2170,8 @@ void PgScrubber::on_mid_scrub_abort(Scrub::delay_cause_t issue)
       m_planned_scrub.check_repair || m_flags.check_repair;
 
   // copy the aborted target
-  const auto aborted_target = m_is_deep ? m_active_target->deep_target
-                                       : m_active_target->shallow_target;
+  const auto aborted_target = *m_active_target;
+  m_active_target.reset();
 
   const auto scrub_clock_now = ceph_clock_now();
   update_targets(m_planned_scrub, scrub_clock_now);
@@ -2181,22 +2180,24 @@ void PgScrubber::on_mid_scrub_abort(Scrub::delay_cause_t issue)
   // 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,
+  auto& trgt = m_scrub_job->delay_on_failure(aborted_target.level(), 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;
+  ceph_assert(!trgt.is_queued());
+  ceph_assert(!m_scrub_job->is_queued());
+  m_osds->get_scrub_services().enqueue_target(trgt);
+  trgt.queued = true;
 }
 
 
-void PgScrubber::requeue_penalized(Scrub::delay_cause_t cause)
+void PgScrubber::requeue_penalized(
+    scrub_level_t s_or_d,
+    Scrub::delay_cause_t cause,
+    utime_t scrub_clock_now)
 {
   if (!m_scrub_job->is_registered()) {
     dout(10) << fmt::format(
@@ -2207,31 +2208,39 @@ 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(
-      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;
+  auto& trgt = m_scrub_job->delay_on_failure(s_or_d, 5s, cause, scrub_clock_now);
+  ceph_assert(!trgt.is_queued());
+  m_osds->get_scrub_services().enqueue_target(trgt);
+  trgt.queued = true;
 }
 
 
 Scrub::schedule_result_t PgScrubber::start_scrub_session(
-    std::unique_ptr<Scrub::ScrubJob> candidate,
+    scrub_level_t s_or_d,
     Scrub::OSDRestrictions osd_restrictions,
     Scrub::ScrubPGPreconds pg_cond,
     const requested_scrub_t& requested_flags)
 {
-  m_scrub_job->target_queued = false;
+  auto& trgt = m_scrub_job->get_target(s_or_d);
+  dout(10) << fmt::format(
+                 "{}: pg[{}] {} {} target: {}", __func__, m_pg_id,
+                 (m_pg->is_active() ? "<active>" : "<not-active>"),
+                 (m_pg->is_clean() ? "<clean>" : "<not-clean>"), trgt)
+          << dendl;
+  // mark our target as not-in-queue. If any error is encountered - that
+  // target must be requeued!
+  trgt.clear_queued();
 
   if (is_queued_or_active()) {
-    // not a real option when the queue entry is the whole ScrubJob, but
-    // will be possible when using level-specific targets
     dout(10) << __func__ << ": scrub already in progress" << dendl;
+    // no need to requeue
     return schedule_result_t::target_specific_failure;
   }
 
-  m_active_target = std::move(candidate);
+  // a few checks. If failing - the 'not-before' is modified, and the target
+  // is requeued.
+  auto clock_now = ceph_clock_now();
+  m_active_target = trgt;
 
   if (!is_primary() || !m_pg->is_active()) {
     // the PG is not expected to be 'registered' in this state. And we should
@@ -2247,7 +2256,8 @@ Scrub::schedule_result_t PgScrubber::start_scrub_session(
                    "{}: cannot scrub (not clean). Registered?{:c}", __func__,
                    m_scrub_job->is_registered() ? 'Y' : 'n')
             << dendl;
-    requeue_penalized(Scrub::delay_cause_t::pg_state);
+    requeue_penalized(
+       m_active_target->level(), Scrub::delay_cause_t::pg_state, clock_now);
     return schedule_result_t::target_specific_failure;
   }
 
@@ -2256,7 +2266,8 @@ Scrub::schedule_result_t PgScrubber::start_scrub_session(
     // (on the transition from NotTrimming to Trimming/WaitReservation),
     // i.e. some time before setting 'snaptrim'.
     dout(10) << __func__ << ": cannot scrub while snap-trimming" << dendl;
-    requeue_penalized(Scrub::delay_cause_t::pg_state);
+    requeue_penalized(
+       m_active_target->level(), Scrub::delay_cause_t::pg_state, clock_now);
     return schedule_result_t::target_specific_failure;
   }
 
@@ -2266,7 +2277,9 @@ Scrub::schedule_result_t PgScrubber::start_scrub_session(
   auto updated_flags = validate_scrub_mode(osd_restrictions, pg_cond);
   if (!updated_flags) {
     dout(10) << __func__ << ": scrub not allowed" << dendl;
-    requeue_penalized(Scrub::delay_cause_t::scrub_params);
+    requeue_penalized(
+       m_active_target->level(), Scrub::delay_cause_t::scrub_params,
+       clock_now);
     return schedule_result_t::target_specific_failure;
   }
 
@@ -2278,7 +2291,9 @@ Scrub::schedule_result_t PgScrubber::start_scrub_session(
             << ": skipping this PG as repairing was not explicitly "
                "requested for it"
             << dendl;
-    requeue_penalized(Scrub::delay_cause_t::scrub_params);
+    requeue_penalized(
+       m_active_target->level(), Scrub::delay_cause_t::scrub_params,
+       clock_now);
     return schedule_result_t::target_specific_failure;
   }
 
@@ -2286,7 +2301,9 @@ Scrub::schedule_result_t PgScrubber::start_scrub_session(
   // be retried by the OSD later on.
   if (!reserve_local()) {
     dout(10) << __func__ << ": failed to reserve locally" << dendl;
-    requeue_penalized(Scrub::delay_cause_t::local_resources);
+    requeue_penalized(
+       m_active_target->level(), Scrub::delay_cause_t::local_resources,
+       clock_now);
     return schedule_result_t::osd_wide_failure;
   }
 
@@ -2300,9 +2317,7 @@ Scrub::schedule_result_t PgScrubber::start_scrub_session(
 
   // 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();
+  trgt.reset();
 
   // using the OSD queue, as to not execute the scrub code as part of the tick.
   dout(10) << __func__ << ": queueing" << dendl;
index 19f20b48e4ad81a2f978fd850e2af5f42a2559b7..f2ce4c3000ed55914a2d3f0ca23f49201e62d29b 100644 (file)
@@ -188,7 +188,7 @@ class PgScrubber : public ScrubPgIF,
   [[nodiscard]] bool is_reserving() const final;
 
   Scrub::schedule_result_t start_scrub_session(
-      std::unique_ptr<Scrub::ScrubJob> candidate,
+      scrub_level_t s_or_d,
       Scrub::OSDRestrictions osd_restrictions,
       Scrub::ScrubPGPreconds pg_cond,
       const requested_scrub_t& requested_flags) final;
@@ -570,9 +570,12 @@ class PgScrubber : public ScrubPgIF,
   /**
    * move the 'not before' to a later time (with a delay amount that is
    * based on the delay cause). Also saves the cause.
-   * Pushes the updated scrub-job into the OSD's queue.
+   * Pushes the updated scheduling entry into the OSD's queue.
    */
-  void requeue_penalized(Scrub::delay_cause_t cause);
+  void requeue_penalized(
+      scrub_level_t s_or_d,
+      Scrub::delay_cause_t cause,
+      utime_t scrub_clock_now);
 
   // -----     methods used to verify the relevance of incoming events:
 
@@ -732,14 +735,9 @@ class PgScrubber : public ScrubPgIF,
    */
   bool m_queued_or_active{false};
 
-  /**
-   * A copy of the specific scheduling target (either shallow_target or
-   * deep_target in the scrub_job) that was selected for this active scrub
-   * session.
-   * \ATTN: in this initial step - a copy of the whole scrub-job is passed
-   * around. Later on this would be just a part of a Scrub::SchedTarget
-   */
-  std::unique_ptr<Scrub::ScrubJob> m_active_target;
+  /// A copy of the specific scheduling target (either shallow_target or
+  /// deep_target in the scrub_job) that was selected for this active scrub
+  std::optional<Scrub::SchedTarget> m_active_target;
 
   eversion_t m_subset_last_update{};
 
index bc493b8437f9660a51c2f9c9af041b4f1a37e14c..7c6c9239730ebe0f9dc4d407112af43f0093e8fd 100644 (file)
@@ -24,6 +24,7 @@ namespace Scrub {
   class ReplicaReservations;
   struct ReplicaActive;
   class ScrubJob;
+  struct SchedEntry;
 }
 
 /// reservation-related data sent by the primary to the replicas,
@@ -485,11 +486,8 @@ struct ScrubPgIF {
 
   /**
    * attempt to initiate a scrub session.
-   * @param candidate the scrub job to start. Later on - this will be the
-   *   specific queue entry (that carries the information about the level,
-   *   priority, etc. of the scrub that should be initiated on this PG).
-   *   This parameter is saved by the scrubber for the whole duration of
-   *   the scrub session (to be used if the scrub is aborted).
+   * param s_or_d: the scrub level to start. This identifies the specific
+   *   target to be scrubbed.
    * @param osd_restrictions limitations on the types of scrubs that can
    *   be initiated on this OSD at this time.
    * @param preconds the PG state re scrubbing at the time of the request,
@@ -501,7 +499,7 @@ struct ScrubPgIF {
    *   external reasons.
    */
   virtual Scrub::schedule_result_t start_scrub_session(
-      std::unique_ptr<Scrub::ScrubJob> candidate,
+      scrub_level_t s_or_d,
       Scrub::OSDRestrictions osd_restrictions,
       Scrub::ScrubPGPreconds pg_cond,
       const requested_scrub_t& requested_flags) = 0;