osd_restrictions, pg_cond, m_planned_scrub);
}
-
double PG::next_deepscrub_interval() const
{
double deep_scrub_interval =
deep_scrub_interval = cct->_conf->osd_deep_scrub_interval;
return info.history.last_deep_scrub_stamp + deep_scrub_interval;
}
+
void PG::on_scrub_schedule_input_change()
{
if (is_active() && is_primary()) {
#include "osd/osd_types.h"
#include <fmt/chrono.h>
#include <fmt/ranges.h>
+#include <fmt/std.h>
#if FMT_VERSION >= 90000
#include <fmt/ostream.h>
#endif
// forwarders to the queue
-void OsdScrub::delay_on_failure(
- Scrub::ScrubJob& sjob,
- std::chrono::seconds delay,
- Scrub::delay_cause_t delay_cause,
- utime_t now_is)
-{
- m_queue.delay_on_failure(sjob, delay, delay_cause, now_is);
-}
-
void OsdScrub::enqueue_target(const Scrub::ScrubJob& sjob)
{
m_queue.enqueue_target(sjob);
utime_t t,
bool high_priority_scrub) const;
- /**
- * push the 'not_before' time out by 'delay' seconds, so that this scrub target
- * would not be retried before 'delay' seconds have passed.
- */
- void delay_on_failure(
- Scrub::ScrubJob& sjob,
- std::chrono::seconds delay,
- Scrub::delay_cause_t delay_cause,
- utime_t now_is);
-
/**
* \returns true if the current time is within the scrub time window
}
-void ScrubQueue::delay_on_failure(
- Scrub::ScrubJob& sjob,
- std::chrono::seconds delay,
- Scrub::delay_cause_t delay_cause,
- utime_t now_is)
-{
- dout(10) << fmt::format(
- "pg[{}] delay_on_failure: delay:{} now:{:s}",
- sjob.pgid, delay, now_is)
- << dendl;
- sjob.delay_on_failure(delay, delay_cause, now_is);
-}
-
-
std::unique_ptr<ScrubJob> ScrubQueue::pop_ready_pg(
OSDRestrictions restrictions, // note: 4B in size! (thus - copy)
utime_t time_now)
*/
void enqueue_target(const Scrub::ScrubJob& sjob);
- void delay_on_failure(
- Scrub::ScrubJob& sjob,
- std::chrono::seconds delay,
- Scrub::delay_cause_t delay_cause,
- utime_t now_is);
-
std::ostream& gen_prefix(std::ostream& out, std::string_view fn) const;
public:
}
}
-sched_params_t PgScrubber::determine_scrub_time(
- const pool_opts_t& pool_conf) const
+sched_params_t PgScrubber::determine_initial_schedule(
+ const Scrub::sched_conf_t& app_conf,
+ utime_t scrub_clock_now) const
{
sched_params_t res;
// Set the smallest time that isn't utime_t()
res.proposed_time = PgScrubber::scrub_must_stamp();
res.is_must = Scrub::must_scrub_t::mandatory;
- // we do not need the interval data in this case
- } else if (
- m_pg->info.stats.stats_invalid &&
- get_pg_cct()->_conf->osd_scrub_invalid_stats) {
- res.proposed_time = ceph_clock_now();
+ } 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;
} else {
res.proposed_time = m_pg->info.history.last_scrub_stamp;
- res.min_interval = pool_conf.value_or(pool_opts_t::SCRUB_MIN_INTERVAL, 0.0);
- res.max_interval = pool_conf.value_or(pool_opts_t::SCRUB_MAX_INTERVAL, 0.0);
- }
-
- dout(15)
- << fmt::format(
- "{}: suggested: {:s} hist: {:s} v:{}/{} must:{} pool-min:{} {}",
- __func__, res.proposed_time, m_pg->info.history.last_scrub_stamp,
- (bool)m_pg->info.stats.stats_invalid,
- get_pg_cct()->_conf->osd_scrub_invalid_stats,
- (res.is_must == must_scrub_t::mandatory ? "y" : "n"),
- res.min_interval, m_planned_scrub)
- << dendl;
+ }
+
+ 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)
+ << dendl;
return res;
}
auto pre_reg = registration_state();
m_scrub_job->registered = true;
- auto suggested = determine_scrub_time(m_pg->get_pgpool().info.opts);
- auto applicable_conf = populate_config_params();
+ const auto applicable_conf = populate_config_params();
+ const auto scrub_clock_now = ceph_clock_now();
+ auto suggested = determine_initial_schedule(applicable_conf, scrub_clock_now);
m_scrub_job->init_targets(
- suggested, m_pg->info, applicable_conf, ceph_clock_now());
+ suggested, m_pg->info, applicable_conf, scrub_clock_now);
m_osds->get_scrub_services().enqueue_target(*m_scrub_job);
ceph_assert(m_pg->is_locked());
- const auto suggested = determine_scrub_time(m_pg->get_pgpool().info.opts);
const auto applicable_conf = populate_config_params();
- m_scrub_job->on_periods_change(suggested, applicable_conf, ceph_clock_now());
+ const auto scrub_clock_now = ceph_clock_now();
+ const auto suggested = determine_initial_schedule(applicable_conf, scrub_clock_now);
+ m_scrub_job->on_periods_change(suggested, applicable_conf, scrub_clock_now);
m_osds->get_scrub_services().enqueue_target(*m_scrub_job);
m_scrub_job->target_queued = true;
m_pg->publish_stats_to_osd();
- dout(15) << __func__ << ": done " << registration_state() << dendl;
+ dout(15) << fmt::format(
+ "{}: flags:<{}> job on exit:{}", __func__, request_flags,
+ *m_scrub_job)
+ << dendl;
}
scrub_level_t PgScrubber::scrub_requested(
{
dout(10) << __func__ << dendl;
// delay the next invocation of the scrubber on this target
- penalize_next_scrub(Scrub::delay_cause_t::replicas);
+ requeue_penalized(Scrub::delay_cause_t::replicas);
}
/*
}
}
-/*
- * note: arbitrary delay used in this early version of the
- * scheduler refactoring.
- */
-void PgScrubber::penalize_next_scrub(Scrub::delay_cause_t cause)
-{
- m_osds->get_scrub_services().delay_on_failure(
- *m_scrub_job, 5s, cause, ceph_clock_now());
-}
void PgScrubber::on_digest_updates()
{
}
-// a placeholder. requeue_penalized() is fully implemented in the
-// following commits of this PR
void PgScrubber::requeue_penalized(Scrub::delay_cause_t cause)
{
- penalize_next_scrub(cause);
+ /// \todo fix the 5s' to use a cause-specific delay parameter
+ m_scrub_job->delay_on_failure(5s, cause, ceph_clock_now());
+ ceph_assert(m_scrub_job->is_registered());
+ ceph_assert(!m_scrub_job->target_queued);
+ m_osds->get_scrub_services().enqueue_target(*m_scrub_job);
+ m_scrub_job->target_queued = true;
}
// 'query' command data for an active scrub
void dump_active_scrubber(ceph::Formatter* f, bool is_deep) const;
- /// calls penalize_next_scrub() to push the 'not before' to a later time
- /// (for now. The fuller implementation will also push the scrub job back
- /// into the queue).
+ /**
+ * 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.
+ */
void requeue_penalized(Scrub::delay_cause_t cause);
// ----- methods used to verify the relevance of incoming events:
* determine the time when the next scrub should be scheduled
*
* based on the planned scrub's flags, time of last scrub, and
- * the pool's scrub configuration.
+ * the pool's scrub configuration. This is only an initial "proposal",
+ * and will be further adjusted based on the scheduling parameters.
*/
- Scrub::sched_params_t determine_scrub_time(
- const pool_opts_t& pool_conf) const;
+ Scrub::sched_params_t determine_initial_schedule(
+ const Scrub::sched_conf_t& app_conf,
+ utime_t scrub_clock_now) const;
/// should we perform deep scrub?
bool is_time_for_deep(
Scrub::scrub_schedule_t ScrubJob::adjust_target_time(
- const sched_params_t& times) const
+ const sched_conf_t& app_conf,
+ const sched_params_t& suggested) const
{
- Scrub::scrub_schedule_t sched_n_dead{
- times.proposed_time, times.proposed_time, times.proposed_time};
+ Scrub::scrub_schedule_t adjusted{
+ suggested.proposed_time, suggested.proposed_time, suggested.proposed_time};
- const auto& conf = cct->_conf;
-
- if (times.is_must == Scrub::must_scrub_t::not_mandatory) {
+ if (suggested.is_must == Scrub::must_scrub_t::not_mandatory) {
// unless explicitly requested, postpone the scrub with a random delay
- double scrub_min_interval = times.min_interval > 0
- ? times.min_interval
- : conf->osd_scrub_min_interval;
- double scrub_max_interval = times.max_interval > 0
- ? times.max_interval
- : conf->osd_scrub_max_interval;
-
- sched_n_dead.scheduled_at += scrub_min_interval;
+ adjusted.scheduled_at += app_conf.shallow_interval;
double r = rand() / (double)RAND_MAX;
- sched_n_dead.scheduled_at +=
- scrub_min_interval * conf->osd_scrub_interval_randomize_ratio * r;
+ adjusted.scheduled_at +=
+ app_conf.shallow_interval * app_conf.interval_randomize_ratio * r;
- if (scrub_max_interval <= 0) {
- sched_n_dead.deadline = utime_t{};
+ if (app_conf.max_shallow) {
+ adjusted.deadline += *app_conf.max_shallow;
} else {
- sched_n_dead.deadline += scrub_max_interval;
+ adjusted.deadline = utime_t{};
+ }
+
+ if (adjusted.not_before < adjusted.scheduled_at) {
+ adjusted.not_before = adjusted.scheduled_at;
}
+
dout(20) << fmt::format(
- "not-must. Was:{:s} {{min:{}/{} max:{}/{} ratio:{}}} "
- "Adjusted:{:s} ({:s})",
- times.proposed_time, fmt::group_digits(times.min_interval),
- fmt::group_digits(conf->osd_scrub_min_interval),
- fmt::group_digits(times.max_interval),
- fmt::group_digits(conf->osd_scrub_max_interval),
- conf->osd_scrub_interval_randomize_ratio,
- sched_n_dead.scheduled_at, sched_n_dead.deadline)
- << dendl;
+ "not-must. Was:{:s} config:{} adjusted:{}",
+ suggested.proposed_time, app_conf, adjusted) << dendl;
}
- // else - no log needed. All relevant data will be logged by the caller
- return sched_n_dead;
+ // else - no log is needed. All relevant data will be logged by the caller
+
+ return adjusted;
}
-// note: some parameters are unused in this commit.
void ScrubJob::init_targets(
const sched_params_t& suggested,
const pg_info_t& info,
const Scrub::sched_conf_t& aconf,
utime_t scrub_clock_now)
{
- auto adjusted = adjust_target_time(suggested);
+ auto adjusted = adjust_target_time(aconf, suggested);
high_priority = suggested.is_must == must_scrub_t::mandatory;
update_schedule(adjusted, true);
}
struct sched_params_t {
utime_t proposed_time{};
- double min_interval{0.0};
- double max_interval{0.0};
must_scrub_t is_must{must_scrub_t::not_mandatory};
};
* random length of time.
* And if delaying the scrub - we calculate, based on pool parameters, a
* deadline we should scrub before.
+ *
+ * @return updated (i.e. - possibly delayed) scrub schedule (schedule,
+ * deadline, not_before)
*/
Scrub::scrub_schedule_t adjust_target_time(
- const Scrub::sched_params_t& recomputed_params) const;
+ const Scrub::sched_conf_t& app_conf,
+ const Scrub::sched_params_t& proposed_schedule) const;
/**
* push the 'not_before' time out by 'delay' seconds, so that this scrub target
struct formatter<Scrub::sched_params_t> {
constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
template <typename FormatContext>
- auto format(const Scrub::sched_params_t& pm, FormatContext& ctx)
+ auto format(const Scrub::sched_params_t& pm, FormatContext& ctx) const
{
return fmt::format_to(
- ctx.out(), "(proposed:{:s} min/max:{:.3f}/{:.3f} must:{:2s})",
- utime_t{pm.proposed_time}, pm.min_interval, pm.max_interval,
- pm.is_must == Scrub::must_scrub_t::mandatory ? "true" : "false");
+ ctx.out(), "proposed:{:s},must:{:c}", pm.proposed_time,
+ pm.is_must == Scrub::must_scrub_t::mandatory ? 'y' : 'n');
}
};
constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
template <typename FormatContext>
- auto format(const Scrub::ScrubJob& sjob, FormatContext& ctx)
+ auto format(const Scrub::ScrubJob& sjob, FormatContext& ctx) const
{
return fmt::format_to(
ctx.out(), "pg[{}] @ nb:{:s} ({:s}) (dl:{:s}) - <{}>",
struct formatter<Scrub::sched_conf_t> {
constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
template <typename FormatContext>
- auto format(const Scrub::sched_conf_t& cf, FormatContext& ctx)
+ auto format(const Scrub::sched_conf_t& cf, FormatContext& ctx) const
{
return fmt::format_to(
ctx.out(),
cf.mandatory_on_invalid);
}
};
+
+template <>
+struct formatter<Scrub::scrub_schedule_t> {
+ constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+ template <typename FormatContext>
+ auto format(const Scrub::scrub_schedule_t& sc, FormatContext& ctx) const
+ {
+ return fmt::format_to(
+ ctx.out(), "nb:{:s}(at:{:s},dl:{:s})", sc.not_before,
+ sc.scheduled_at, sc.deadline);
+ }
+};
} // namespace fmt
virtual pg_scrubbing_status_t get_schedule() const = 0;
- /// notify the scrubber about a scrub failure
- virtual void penalize_next_scrub(Scrub::delay_cause_t cause) = 0;
-
// // perform 'scrub'/'deep_scrub' asok commands
/// ... by faking the "last scrub" stamps