<< dendl;
ceph_assert(ceph_mutex_is_locked(_lock));
+ // recheck PG status (as the PG was unlocked for a time after being selected
+ // for scrubbing)
if (!is_primary() || !is_active() || !is_clean()) {
dout(10) << __func__ << ": cannot scrub (not a clean and active primary)"
<< dendl;
+ m_scrubber->penalize_next_scrub(Scrub::delay_cause_t::pg_state);
return schedule_result_t::target_specific_failure;
}
<< ": skipping this PG as repairing was not explicitly "
"requested for it"
<< dendl;
+ m_scrubber->penalize_next_scrub(Scrub::delay_cause_t::scrub_params);
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;
+ m_scrubber->penalize_next_scrub(Scrub::delay_cause_t::pg_state);
return schedule_result_t::target_specific_failure;
}
// (due to configuration or priority issues)
// The reason was already reported by the callee.
dout(10) << __func__ << ": failed to initiate a scrub" << dendl;
+ m_scrubber->penalize_next_scrub(Scrub::delay_cause_t::scrub_params);
return schedule_result_t::target_specific_failure;
}
// be retried by the OSD later on.
if (!m_scrubber->reserve_local()) {
dout(10) << __func__ << ": failed to reserve locally" << dendl;
+ m_scrubber->penalize_next_scrub(Scrub::delay_cause_t::local_resources);
return schedule_result_t::osd_wide_failure;
}
m_queue.update_job(sjob, suggested, reset_notbefore);
}
+void OsdScrub::delay_on_failure(
+ Scrub::ScrubJobRef 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::register_with_osd(
Scrub::ScrubJobRef sjob,
const Scrub::sched_params_t& suggested)
void clear_reserving_now(spg_t reserving_id);
+ /**
+ * 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::ScrubJobRef 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
*/
scrub_job->update_schedule(adjusted, reset_nb);
}
+
+void ScrubQueue::delay_on_failure(
+ Scrub::ScrubJobRef 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);
+}
+
+
sched_params_t ScrubQueue::determine_scrub_time(
const requested_scrub_t& request_flags,
const pg_info_t& pg_info,
for (const auto& jobref : group) {
if (!filtr(jobref)) {
dout(20) << fmt::format(
- " not ripe: {} @ {:s} ({:s})", jobref->pgid,
- jobref->schedule.not_before,
- jobref->schedule.scheduled_at)
+ " not eligible: {} @ {:s} ({:s},{:s})", jobref->pgid,
+ jobref->schedule.not_before,
+ jobref->schedule.scheduled_at, jobref->last_issue)
<< dendl;
}
}
const sched_params_t& suggested,
bool reset_notbefore);
+ void delay_on_failure(
+ Scrub::ScrubJobRef sjob,
+ std::chrono::seconds delay,
+ Scrub::delay_cause_t delay_cause,
+ utime_t now_is);
+
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;
void PgScrubber::flag_reservations_failure()
{
- m_scrub_job->resources_failure = true;
+ dout(10) << __func__ << dendl;
+ // delay the next invocation of the scrubber on this target
+ penalize_next_scrub(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()
{
dout(10) << __func__ << " #pending: " << num_digest_updates_pending << " "
PG* get_pg() const final { return m_pg; }
PerfCounters& get_counters_set() const final;
- // temporary interface (to be discarded in a follow-up PR)
- /// set the 'resources_failure' flag in the scrub-job object
+ /// delay next retry of this PG after a replica reservation failure
void flag_reservations_failure();
scrubber_callback_cancel_token_t schedule_callback_after(
void scrub_finish() final;
+ void penalize_next_scrub(Scrub::delay_cause_t cause) final;
+
ScrubMachineListener::MsgAndEpoch prep_replica_map_msg(
Scrub::PreemptionNoted was_preempted) final;
void ScrubJob::update_schedule(
const Scrub::scrub_schedule_t& adjusted,
- bool reset_nb)
+ bool reset_failure_penalty)
{
- dout(15)
- << fmt::format(
- "was: nb:{:s}({:s}). Called with: rest?{} nb:{:s} ({:s}) ({})",
- schedule.not_before, schedule.scheduled_at, reset_nb,
- adjusted.not_before, adjusted.scheduled_at, registration_state())
- << dendl;
+ dout(15) << fmt::format(
+ "was: nb:{:s}({:s}). Called with: rest?{} {:s} ({})",
+ schedule.not_before, schedule.scheduled_at,
+ reset_failure_penalty, adjusted.scheduled_at,
+ registration_state())
+ << dendl;
schedule.scheduled_at = adjusted.scheduled_at;
schedule.deadline = adjusted.deadline;
- if (reset_nb || schedule.not_before < schedule.scheduled_at) {
+ if (reset_failure_penalty || (schedule.not_before < schedule.scheduled_at)) {
schedule.not_before = schedule.scheduled_at;
}
- // 'updated' is changed here while not holding jobs_lock. That's OK, as
- // the (atomic) flag will only be cleared by select_pg_and_scrub() after
- // scan_penalized() is called and the job was moved to the to_scrub queue.
updated = true;
dout(10) << fmt::format(
"adjusted: nb:{:s} ({:s}) ({})", schedule.not_before,
<< dendl;
}
+void ScrubJob::delay_on_failure(
+ 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;
+}
+
std::string ScrubJob::scheduling_state(utime_t now_is, bool is_deep_expected)
const
{
*/
std::atomic_bool in_queues{false};
- /// last scrub attempt failed to secure replica resources
- bool resources_failure{false};
+ /// how the last attempt to scrub this PG ended
+ delay_cause_t last_issue{delay_cause_t::none};
/**
* 'updated' is a temporary flag, used to create a barrier after
const scrub_schedule_t& adjusted,
bool reset_failure_penalty);
+ /**
+ * 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(
+ std::chrono::seconds delay,
+ delay_cause_t delay_cause,
+ utime_t scrub_clock_now);
+
void dump(ceph::Formatter* f) const;
/*
}
};
+template <>
+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)
+ {
+ 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");
+ }
+};
+
+
template <>
struct formatter<Scrub::ScrubJob> {
constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
auto format(const Scrub::ScrubJob& sjob, FormatContext& ctx)
{
return fmt::format_to(
- ctx.out(),
- "pg[{}] @ {:s} ({:s}) (dl:{:s}) - <{}> / failure: {} / queue state: "
- "{:.7}",
+ ctx.out(), "pg[{}] @ nb:{:s} ({:s}) (dl:{:s}) - <{}> queue state:{:.7}",
sjob.pgid, sjob.schedule.not_before, sjob.schedule.scheduled_at,
sjob.schedule.deadline, sjob.registration_state(),
- sjob.resources_failure, sjob.state.load(std::memory_order_relaxed));
+ sjob.state.load(std::memory_order_relaxed));
}
};
// if the begin-time stamp was not set 'off' (as done if the scrubbing
// completed successfully), we use it now to set the 'failed scrub' duration.
if (session.m_session_started_at != ScrubTimePoint{}) {
+ // delay the next invocation of the scrubber on this target
+ scrbr->penalize_next_scrub(Scrub::delay_cause_t::aborted);
+
auto logged_duration = ScrubClock::now() - session.m_session_started_at;
session.m_perf_set->tinc(scrbcnt_failed_elapsed, logged_duration);
session.m_perf_set->inc(scrbcnt_failed);
/// the part that actually finalizes a scrub
virtual void scrub_finish() = 0;
+ /// notify the scrubber about a scrub failure
+ /// (note: temporary implementation)
+ virtual void penalize_next_scrub(Scrub::delay_cause_t cause) = 0;
+
/**
* Prepare a MOSDRepScrubMap message carrying the requested scrub map
* @param was_preempted - were we preempted?
/// sending cluster-log warnings
virtual void log_cluster_warning(const std::string& msg) const = 0;
- // temporary interface (to be discarded in a follow-up PR)
- /// set the 'resources_failure' flag in the scrub-job object
+ /// delay next retry of this PG after a replica reservation failure
virtual void flag_reservations_failure() = 0;
/// is this scrub more than just regular periodic scrub?
};
} // namespace fmt
+namespace Scrub {
+
+/**
+ * the result of the last attempt to schedule a scrub for a specific PG.
+ * The enum value itself is mostly used for logging purposes.
+ */
+enum class delay_cause_t {
+ none, ///< scrub attempt was successful
+ replicas, ///< failed to reserve replicas
+ flags, ///< noscrub or nodeep-scrub
+ pg_state, ///< e.g. snap-trimming
+ restricted_time, ///< time restrictions or busy CPU
+ local_resources, ///< too many scrubbing PGs
+ aborted, ///< scrub was aborted w/ unspecified reason
+ interval, ///< the interval had ended mid-scrub
+ scrub_params, ///< the specific scrub type is not allowed
+};
+} // namespace Scrub
+
+namespace fmt {
+// clang-format off
+template <>
+struct formatter<Scrub::delay_cause_t> : ::fmt::formatter<std::string_view> {
+ template <typename FormatContext>
+ auto format(Scrub::delay_cause_t cause, FormatContext& ctx)
+ {
+ using enum Scrub::delay_cause_t;
+ std::string_view desc;
+ switch (cause) {
+ case none: desc = "ok"; break;
+ case replicas: desc = "replicas"; break;
+ case flags: desc = "noscrub"; break;
+ case pg_state: desc = "pg-state"; break;
+ case restricted_time: desc = "time/load"; break;
+ case local_resources: desc = "local-cnt"; break;
+ case aborted: desc = "aborted"; break;
+ case interval: desc = "interval"; break;
+ case scrub_params: desc = "scrub-mode"; break;
+ // better to not have a default case, so that the compiler will warn
+ }
+ return ::fmt::formatter<string_view>::format(desc, ctx);
+ }
+};
+// clang-format on
+} // namespace fmt
+
+
namespace Scrub {
/// PG services used by the scrubber backend
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