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>
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));
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
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(
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);
}
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.
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;
}
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;
// 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)
* 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.
* 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
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;
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);
}
* 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);
}
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
┌───────────────────────────▼────────────┐
│ │
│ │
- │ 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 ├───────────┘
+ │ └───────────────┤
└─────────────────────┘
#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"
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
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;
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
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;
*/
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
*/
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
"{}: 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;
}
}
/*
* 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.
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;
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)
{
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());
}
/*
* 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)
{
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);
// 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(
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
"{}: 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;
}
// (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;
}
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;
}
<< ": 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;
}
// 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;
}
// 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;
[[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;
/**
* 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:
*/
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{};
class ReplicaReservations;
struct ReplicaActive;
class ScrubJob;
+ struct SchedEntry;
}
/// reservation-related data sent by the primary to the replicas,
/**
* 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,
* 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;