// vim: ts=8 sw=2 smarttab
#include "./osd_scrub_sched.h"
-#include "include/utime_fmt.h"
#include "osd/OSD.h"
-#include "osd/osd_types_fmt.h"
#include "pg_scrubber.h"
#undef dout_context
#define dout_context (cct)
#undef dout_prefix
-#define dout_prefix \
- *_dout << "osd." << osd_service.whoami << " scrub-queue::" << __func__ << " "
+#define dout_prefix \
+ *_dout << "osd." << osd_service.get_nodeid() << " scrub-queue::" << __func__ \
+ << " "
-ScrubQueue::ScrubQueue(CephContext* cct, OSDService& osds)
+ScrubQueue::ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds)
: cct{cct}
, osd_service{osds}
{
std::optional<double> ScrubQueue::update_load_average()
{
- int hb_interval = cct->_conf->osd_heartbeat_interval;
+ int hb_interval = conf()->osd_heartbeat_interval;
int n_samples = 60 * 24 * 24;
if (hb_interval > 1) {
n_samples /= hb_interval;
scrub_job->update_schedule(adjusted);
}
+ScrubQueue::sched_params_t ScrubQueue::determine_scrub_time(
+ const requested_scrub_t& request_flags,
+ const pg_info_t& pg_info,
+ const pool_opts_t pool_conf) const
+{
+ ScrubQueue::sched_params_t res;
+
+ //dout(15) << fmt::format(": requested_scrub_t: {}", request_flags) << dendl; // RRR
+ dout(15) << ": requested_scrub_t: {}" << request_flags << dendl;
+
+
+ if (request_flags.must_scrub || request_flags.need_auto) {
+
+ // Set the smallest time that isn't utime_t()
+ res.proposed_time = PgScrubber::scrub_must_stamp();
+ res.is_must = ScrubQueue::must_scrub_t::mandatory;
+ // we do not need the interval data in this case
+
+ } else if (pg_info.stats.stats_invalid &&
+ conf()->osd_scrub_invalid_stats) {
+ res.proposed_time = time_now();
+ res.is_must = ScrubQueue::must_scrub_t::mandatory;
+
+ } else {
+ res.proposed_time = 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: {} hist: {} v: {}/{} must: {} pool-min: {}",
+ res.proposed_time,
+ pg_info.history.last_scrub_stamp,
+ (bool)pg_info.stats.stats_invalid,
+ conf()->osd_scrub_invalid_stats,
+ (res.is_must == must_scrub_t::mandatory ? "y" : "n"),
+ res.min_interval)
+ << dendl;
+ return res;
+}
+
+
// used under jobs_lock
void ScrubQueue::move_failed_pgs(utime_t now_is)
{
// determine the penalty time, after which the job should be reinstated
utime_t after = now_is;
- after += cct->_conf->osd_scrub_sleep * 2 + utime_t{300'000ms};
+ after += conf()->osd_scrub_sleep * 2 + utime_t{300'000ms};
// note: currently - not taking 'deadline' into account when determining
// 'penalty_timeout'.
dout(10) << " reg./pen. sizes: " << to_scrub.size() << " / "
<< penalized.size() << dendl;
- utime_t now_is = ceph_clock_now();
+ utime_t now_is = time_now();
preconds.time_permit = scrub_time_permit(now_is);
preconds.load_is_low = scrub_load_below_threshold();
if (g_conf()->subsys.should_gather<ceph_subsys_osd, 20>()) {
dout(20) << "min t: " << times.min_interval
- << " osd: " << cct->_conf->osd_scrub_min_interval
+ << " osd: " << conf()->osd_scrub_min_interval
<< " max t: " << times.max_interval
- << " osd: " << cct->_conf->osd_scrub_max_interval << dendl;
+ << " osd: " << conf()->osd_scrub_max_interval << dendl;
dout(20) << "at " << sched_n_dead.scheduled_at << " ratio "
- << cct->_conf->osd_scrub_interval_randomize_ratio << dendl;
+ << conf()->osd_scrub_interval_randomize_ratio << dendl;
}
if (times.is_must == ScrubQueue::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
- : cct->_conf->osd_scrub_min_interval;
+ : conf()->osd_scrub_min_interval;
double scrub_max_interval = times.max_interval > 0
? times.max_interval
- : cct->_conf->osd_scrub_max_interval;
+ : conf()->osd_scrub_max_interval;
sched_n_dead.scheduled_at += scrub_min_interval;
double r = rand() / (double)RAND_MAX;
sched_n_dead.scheduled_at +=
- scrub_min_interval * cct->_conf->osd_scrub_interval_randomize_ratio * r;
+ scrub_min_interval * conf()->osd_scrub_interval_randomize_ratio * r;
if (scrub_max_interval <= 0) {
sched_n_dead.deadline = utime_t{};
double ScrubQueue::scrub_sleep_time(bool must_scrub) const
{
- double regular_sleep_period = cct->_conf->osd_scrub_sleep;
+ double regular_sleep_period = conf()->osd_scrub_sleep;
- if (must_scrub || scrub_time_permit(ceph_clock_now())) {
+ if (must_scrub || scrub_time_permit(time_now())) {
return regular_sleep_period;
}
// relevant if scrubbing started during allowed time, but continued into
// forbidden hours
- double extended_sleep = cct->_conf->osd_scrub_extended_sleep;
+ double extended_sleep = conf()->osd_scrub_extended_sleep;
dout(20) << "w/ extended sleep (" << extended_sleep << ")" << dendl;
return std::max(extended_sleep, regular_sleep_period);
}
// allow scrub if below configured threshold
long cpus = sysconf(_SC_NPROCESSORS_ONLN);
double loadavg_per_cpu = cpus > 0 ? loadavgs[0] / cpus : loadavgs[0];
- if (loadavg_per_cpu < cct->_conf->osd_scrub_load_threshold) {
+ if (loadavg_per_cpu < conf()->osd_scrub_load_threshold) {
dout(20) << "loadavg per cpu " << loadavg_per_cpu << " < max "
- << cct->_conf->osd_scrub_load_threshold << " = yes" << dendl;
+ << conf()->osd_scrub_load_threshold << " = yes" << dendl;
return true;
}
}
dout(20) << "loadavg " << loadavgs[0] << " >= max "
- << cct->_conf->osd_scrub_load_threshold << " and ( >= daily_loadavg "
+ << conf()->osd_scrub_load_threshold << " and ( >= daily_loadavg "
<< daily_loadavg << " or >= 15m avg " << loadavgs[2] << ") = no"
<< dendl;
return false;
time_t tt = now.sec();
localtime_r(&tt, &bdt);
- bool day_permit = isbetween_modulo(cct->_conf->osd_scrub_begin_week_day,
- cct->_conf->osd_scrub_end_week_day,
+ bool day_permit = isbetween_modulo(conf()->osd_scrub_begin_week_day,
+ conf()->osd_scrub_end_week_day,
bdt.tm_wday);
if (!day_permit) {
dout(20) << "should run between week day "
- << cct->_conf->osd_scrub_begin_week_day << " - "
- << cct->_conf->osd_scrub_end_week_day << " now " << bdt.tm_wday
+ << conf()->osd_scrub_begin_week_day << " - "
+ << conf()->osd_scrub_end_week_day << " now " << bdt.tm_wday
<< " - no" << dendl;
return false;
}
- bool time_permit = isbetween_modulo(cct->_conf->osd_scrub_begin_hour,
- cct->_conf->osd_scrub_end_hour,
+ bool time_permit = isbetween_modulo(conf()->osd_scrub_begin_hour,
+ conf()->osd_scrub_end_hour,
bdt.tm_hour);
- dout(20) << "should run between " << cct->_conf->osd_scrub_begin_hour << " - "
- << cct->_conf->osd_scrub_end_hour << " now (" << bdt.tm_hour
+ dout(20) << "should run between " << conf()->osd_scrub_begin_hour << " - "
+ << conf()->osd_scrub_end_hour << " now (" << bdt.tm_hour
<< ") = " << (time_permit ? "yes" : "no") << dendl;
return time_permit;
}
// inc_scrubs_local() failures
std::lock_guard lck{resource_lock};
- if (scrubs_local + scrubs_remote < cct->_conf->osd_max_scrubs) {
+ if (scrubs_local + scrubs_remote < conf()->osd_max_scrubs) {
return true;
}
dout(20) << " == false. " << scrubs_local << " local + " << scrubs_remote
- << " remote >= max " << cct->_conf->osd_max_scrubs << dendl;
+ << " remote >= max " << conf()->osd_max_scrubs << dendl;
return false;
}
{
std::lock_guard lck{resource_lock};
- if (scrubs_local + scrubs_remote < cct->_conf->osd_max_scrubs) {
+ if (scrubs_local + scrubs_remote < conf()->osd_max_scrubs) {
++scrubs_local;
return true;
}
dout(20) << ": " << scrubs_local << " local + " << scrubs_remote
- << " remote >= max " << cct->_conf->osd_max_scrubs << dendl;
+ << " remote >= max " << conf()->osd_max_scrubs << dendl;
return false;
}
{
std::lock_guard lck{resource_lock};
dout(20) << ": " << scrubs_local << " -> " << (scrubs_local - 1) << " (max "
- << cct->_conf->osd_max_scrubs << ", remote " << scrubs_remote << ")"
+ << conf()->osd_max_scrubs << ", remote " << scrubs_remote << ")"
<< dendl;
--scrubs_local;
{
std::lock_guard lck{resource_lock};
- if (scrubs_local + scrubs_remote < cct->_conf->osd_max_scrubs) {
+ if (scrubs_local + scrubs_remote < conf()->osd_max_scrubs) {
dout(20) << ": " << scrubs_remote << " -> " << (scrubs_remote + 1)
- << " (max " << cct->_conf->osd_max_scrubs << ", local "
+ << " (max " << conf()->osd_max_scrubs << ", local "
<< scrubs_local << ")" << dendl;
++scrubs_remote;
return true;
}
dout(20) << ": " << scrubs_local << " local + " << scrubs_remote
- << " remote >= max " << cct->_conf->osd_max_scrubs << dendl;
+ << " remote >= max " << conf()->osd_max_scrubs << dendl;
return false;
}
{
std::lock_guard lck{resource_lock};
dout(20) << ": " << scrubs_remote << " -> " << (scrubs_remote - 1) << " (max "
- << cct->_conf->osd_max_scrubs << ", local " << scrubs_local << ")"
+ << conf()->osd_max_scrubs << ", local " << scrubs_local << ")"
<< dendl;
--scrubs_remote;
ceph_assert(scrubs_remote >= 0);
std::lock_guard lck{resource_lock};
f->dump_int("scrubs_local", scrubs_local);
f->dump_int("scrubs_remote", scrubs_remote);
- f->dump_int("osd_max_scrubs", cct->_conf->osd_max_scrubs);
+ f->dump_int("osd_max_scrubs", conf()->osd_max_scrubs);
}
#include "common/ceph_atomic.h"
#include "osd/osd_types.h"
#include "osd/scrubber_common.h"
-
+#include "include/utime_fmt.h"
+#include "osd/osd_types_fmt.h"
#include "utime.h"
class PG;
preconditions // time, configuration, etc.
};
+// the OSD services provided to the scrub scheduler
+class ScrubSchedListener {
+ public:
+ virtual int get_nodeid() const = 0; // returns the OSD number ('whoami')
+
+ /**
+ * A callback used by the ScrubQueue object to initiate a scrub on a specific
+ * PG.
+ *
+ * The request might fail for multiple reasons, as ScrubQueue cannot by its
+ * own check some of the PG-specific preconditions and those are checked here.
+ * See attempt_t definition.
+ *
+ * @return a Scrub::attempt_t detailing either a success, or the failure
+ * reason.
+ */
+ virtual schedule_result_t initiate_a_scrub(
+ spg_t pgid,
+ bool allow_requested_repair_only) = 0;
+
+ virtual ~ScrubSchedListener() {}
+};
+
} // namespace Scrub
/**
// under lock
};
- ScrubQueue(CephContext* cct, OSDService& osds);
+ ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds);
+ virtual ~ScrubQueue() = default;
struct scrub_schedule_t {
utime_t scheduled_at{};
*/
void update_job(ScrubJobRef sjob, const sched_params_t& suggested);
+ sched_params_t determine_scrub_time(const requested_scrub_t& request_flags,
+ const pg_info_t& pg_info,
+ const pool_opts_t pool_conf) const;
+
public:
void dump_scrubs(ceph::Formatter* f) const;
private:
CephContext* cct;
- OSDService& osd_service;
+ Scrub::ScrubSchedListener& osd_service;
+
+#ifdef WITH_SEASTAR
+ auto& conf() const { return local_conf(); }
+#else
+ auto& conf() const { return cct->_conf; }
+#endif
/**
* jobs_lock protects the job containers and the relevant scrub-jobs state
ScrubQContainer& group,
const Scrub::ScrubPreconds& preconds,
utime_t now_is);
+
+protected: // used by the unit-tests
+ /**
+ * unit-tests will override this function to return a mock time
+ */
+ virtual utime_t time_now() const { return ceph_clock_now(); }
+};
+
+template <>
+struct fmt::formatter<ScrubQueue::ScrubJob> {
+ constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+ template <typename FormatContext>
+ auto format(const ScrubQueue::ScrubJob& sjob, FormatContext& ctx)
+ {
+ return fmt::format_to(
+ ctx.out(),
+ "{}, {} dead: {} - {} / failure: {} / pen. t.o.: {} / queue state: {}",
+ sjob.pgid,
+ sjob.schedule.scheduled_at,
+ sjob.schedule.deadline,
+ sjob.registration_state(),
+ sjob.resources_failure,
+ sjob.penalty_timeout,
+ ScrubQueue::qu_state_text(sjob.state));
+ }
};
<< dendl;
if (is_primary()) {
- auto suggested = determine_scrub_time(request_flags);
+ auto suggested = m_osds->get_scrub_services().determine_scrub_time(
+ request_flags,
+ m_pg->info,
+ m_pg->get_pgpool().info.opts);
m_osds->get_scrub_services().register_with_osd(m_scrub_job, suggested);
} else {
m_osds->get_scrub_services().remove_from_osd_queue(m_scrub_job);
}
if (is_primary() && m_scrub_job) {
- auto suggested = determine_scrub_time(request_flags);
+ auto suggested = m_osds->get_scrub_services().determine_scrub_time(
+ request_flags,
+ m_pg->info,
+ m_pg->get_pgpool().info.opts);
m_osds->get_scrub_services().update_job(m_scrub_job, suggested);
}
dout(15) << __func__ << " done " << registration_state() << dendl;
}
-ScrubQueue::sched_params_t PgScrubber::determine_scrub_time(
- const requested_scrub_t& request_flags) const
-{
- ScrubQueue::sched_params_t res;
-
- if (!is_primary()) {
- return res; // with ok_to_scrub set to 'false'
- }
-
- if (request_flags.must_scrub || request_flags.need_auto) {
-
- // Set the smallest time that isn't utime_t()
- res.proposed_time = PgScrubber::scrub_must_stamp();
- res.is_must = ScrubQueue::must_scrub_t::mandatory;
- // we do not need the interval data in this case
-
- } else if (m_pg->info.stats.stats_invalid &&
- m_pg->get_cct()->_conf->osd_scrub_invalid_stats) {
- res.proposed_time = ceph_clock_now();
- res.is_must = ScrubQueue::must_scrub_t::mandatory;
-
- } else {
- res.proposed_time = m_pg->info.history.last_scrub_stamp;
- res.min_interval =
- m_pg->get_pgpool().info.opts.value_or(pool_opts_t::SCRUB_MIN_INTERVAL, 0.0);
- res.max_interval =
- m_pg->get_pgpool().info.opts.value_or(pool_opts_t::SCRUB_MAX_INTERVAL, 0.0);
- }
-
- dout(15) << __func__ << " suggested: " << res.proposed_time
- << " hist: " << m_pg->info.history.last_scrub_stamp
- << " v:" << m_pg->info.stats.stats_invalid << " / "
- << m_pg->cct->_conf->osd_scrub_invalid_stats << " must:"
- << (res.is_must == ScrubQueue::must_scrub_t::mandatory ? "y" : "n")
- << " pool min: " << res.min_interval << dendl;
- return res;
-}
-
void PgScrubber::scrub_requested(scrub_level_t scrub_level,
scrub_type_t scrub_type,
requested_scrub_t& req_flags)