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 =
}
-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)
}
}
-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;
}
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;
<< 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();
<< dendl;
}
+
scrub_level_t PgScrubber::scrub_requested(
scrub_level_t scrub_level,
scrub_type_t scrub_type,
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;
}
return false;
}
+
+
Scrub::sched_conf_t PgScrubber::populate_config_params() const
{
const pool_opts_t& pool_conf = m_pg->get_pgpool().info.opts;
* 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;
}
// 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);
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;
}
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;
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);
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();
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 =
}
}
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;
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:{})",
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(
// 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;
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
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)}
{}
} // 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) ||
// 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
{
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
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;
+}
// 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 };
};
-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,
* 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
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:
// 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 {
}
};
+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(); }
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());
}
};
/// 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;
/**
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;