summaryrefslogtreecommitdiffstats
path: root/src/osd/scrubber
diff options
context:
space:
mode:
Diffstat (limited to 'src/osd/scrubber')
-rw-r--r--src/osd/scrubber/osd_scrub.cc16
-rw-r--r--src/osd/scrubber/osd_scrub.h9
-rw-r--r--src/osd/scrubber/osd_scrub_sched.cc31
-rw-r--r--src/osd/scrubber/osd_scrub_sched.h43
-rw-r--r--src/osd/scrubber/pg_scrubber.cc420
-rw-r--r--src/osd/scrubber/pg_scrubber.h45
-rw-r--r--src/osd/scrubber/scrub_machine.cc50
-rw-r--r--src/osd/scrubber/scrub_machine.h19
-rw-r--r--src/osd/scrubber/scrub_machine_lstnr.h12
9 files changed, 448 insertions, 197 deletions
diff --git a/src/osd/scrubber/osd_scrub.cc b/src/osd/scrubber/osd_scrub.cc
index 48f978b18e3..b1b063c484e 100644
--- a/src/osd/scrubber/osd_scrub.cc
+++ b/src/osd/scrubber/osd_scrub.cc
@@ -171,12 +171,6 @@ Scrub::OSDRestrictions OsdScrub::restrictions_on_scrubbing(
<< dendl;
env_conditions.high_priority_only = true;
- } else if (m_queue.is_reserving_now()) {
- // if there is a PG that is just now trying to reserve scrub replica
- // resources - we should wait and not initiate a new scrub
- dout(10) << "scrub resources reservation in progress" << dendl;
- env_conditions.high_priority_only = true;
-
} else if (is_recovery_active && !conf->osd_scrub_during_recovery) {
if (conf->osd_repair_during_recovery) {
dout(15)
@@ -482,13 +476,3 @@ int OsdScrub::get_blocked_pgs_count() const
{
return m_queue.get_blocked_pgs_count();
}
-
-bool OsdScrub::set_reserving_now(spg_t reserving_id, utime_t now_is)
-{
- return m_queue.set_reserving_now(reserving_id, now_is);
-}
-
-void OsdScrub::clear_reserving_now(spg_t reserving_id)
-{
- m_queue.clear_reserving_now(reserving_id);
-}
diff --git a/src/osd/scrubber/osd_scrub.h b/src/osd/scrubber/osd_scrub.h
index cd1158d4723..41f5122681c 100644
--- a/src/osd/scrubber/osd_scrub.h
+++ b/src/osd/scrubber/osd_scrub.h
@@ -135,15 +135,6 @@ class OsdScrub {
bool high_priority_scrub) const;
/**
- * No new scrub session will start while a scrub was initiated on a PG,
- * and that PG is trying to acquire replica resources.
- * \retval false if the flag was already set (due to a race)
- */
- bool set_reserving_now(spg_t reserving_id, utime_t now_is);
-
- 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.
*/
diff --git a/src/osd/scrubber/osd_scrub_sched.cc b/src/osd/scrubber/osd_scrub_sched.cc
index 1d0bf614c9b..079e2a7e7ae 100644
--- a/src/osd/scrubber/osd_scrub_sched.cc
+++ b/src/osd/scrubber/osd_scrub_sched.cc
@@ -361,34 +361,3 @@ int ScrubQueue::get_blocked_pgs_count() const
{
return blocked_scrubs_cnt;
}
-
-// ////////////////////////////////////////////////////////////////////////// //
-// ScrubQueue - maintaining the 'some PG is reserving' flag
-
-bool ScrubQueue::set_reserving_now(spg_t reserving_id, utime_t now_is)
-{
- std::unique_lock l{reserving_lock};
-
- if (!reserving_pg.has_value()) {
- reserving_pg = reserving_id;
- reserving_since = now_is;
- return true;
- }
- ceph_assert(reserving_id != *reserving_pg);
- return false;
-}
-
-void ScrubQueue::clear_reserving_now(spg_t was_reserving_id)
-{
- std::unique_lock l{reserving_lock};
- if (reserving_pg && (*reserving_pg == was_reserving_id)) {
- reserving_pg.reset();
- }
- // otherwise - ignore silently
-}
-
-bool ScrubQueue::is_reserving_now() const
-{
- // no lock needed, as set_reserving_now() will recheck
- return reserving_pg.has_value();
-}
diff --git a/src/osd/scrubber/osd_scrub_sched.h b/src/osd/scrubber/osd_scrub_sched.h
index 140c1428889..75708af7bcf 100644
--- a/src/osd/scrubber/osd_scrub_sched.h
+++ b/src/osd/scrubber/osd_scrub_sched.h
@@ -84,7 +84,6 @@ ScrubQueue interfaces (main functions):
- can_inc_scrubs()
- {inc/dec}_scrubs_{local/remote}()
- dump_scrub_reservations()
- - {set/clear/is}_reserving_now()
<2> - environment conditions:
@@ -117,13 +116,6 @@ namespace Scrub {
using namespace ::std::literals;
-/// possible outcome when trying to select a PG and scrub it
-enum class schedule_result_t {
- scrub_initiated, // successfully started a scrub
- target_specific_failure, // failed to scrub this specific target
- osd_wide_failure // failed to scrub any target
-};
-
// the OSD services provided to the scrub scheduler
class ScrubSchedListener {
public:
@@ -238,30 +230,6 @@ class ScrubQueue {
public:
void dump_scrubs(ceph::Formatter* f) const;
- /**
- * No new scrub session will start while a scrub was initiated on a PG,
- * and that PG is trying to acquire replica resources.
- *
- * \todo replace the atomic bool with a regular bool protected by a
- * common OSD-service lock. Or better still - once PR#53263 is merged,
- * remove this flag altogether.
- */
-
- /**
- * set_reserving_now()
- * \returns 'false' if the flag was already set
- * (which is a possible result of a race between the check in OsdScrub and
- * the initiation of a scrub by some other PG)
- */
- bool set_reserving_now(spg_t reserving_id, utime_t now_is);
-
- /**
- * silently ignore attempts to clear the flag if it was not set by
- * the named pg.
- */
- void clear_reserving_now(spg_t reserving_id);
- bool is_reserving_now() const;
-
/// counting the number of PGs stuck while scrubbing, waiting for objects
void mark_pg_scrub_blocked(spg_t blocked_pg);
void clear_pg_scrub_blocked(spg_t blocked_pg);
@@ -332,17 +300,6 @@ class ScrubQueue {
std::atomic_int_fast16_t blocked_scrubs_cnt{0};
/**
- * One of the OSD's primary PGs is in the initial phase of a scrub,
- * trying to secure its replicas' resources. We will refrain from initiating
- * any other scrub sessions until this one is done.
- *
- * \todo replace the local lock with regular osd-service locking
- */
- ceph::mutex reserving_lock = ceph::make_mutex("ScrubQueue::reserving_lock");
- std::optional<spg_t> reserving_pg;
- utime_t reserving_since;
-
- /**
* If the scrub job was not explicitly requested, we postpone it by some
* random length of time.
* And if delaying the scrub - we calculate, based on pool parameters, a
diff --git a/src/osd/scrubber/pg_scrubber.cc b/src/osd/scrubber/pg_scrubber.cc
index e1604222c2c..471677318b1 100644
--- a/src/osd/scrubber/pg_scrubber.cc
+++ b/src/osd/scrubber/pg_scrubber.cc
@@ -876,8 +876,11 @@ int PgScrubber::get_whoami() const
* - m_max_end
* - end
* - start
+ * returns:
+ * - std::nullopt if the range is blocked
+ * - otherwise, the number of objects in the selected range
*/
-bool PgScrubber::select_range()
+std::optional<uint64_t> PgScrubber::select_range()
{
m_be->new_chunk();
@@ -959,7 +962,7 @@ bool PgScrubber::select_range()
// we'll be requeued by whatever made us unavailable for scrub
dout(10) << __func__ << ": scrub blocked somewhere in range "
<< "[" << m_start << ", " << candidate_end << ")" << dendl;
- return false;
+ return std::nullopt;
}
m_end = candidate_end;
@@ -972,20 +975,20 @@ bool PgScrubber::select_range()
// debug: be 'blocked' if told so by the 'pg scrub_debug block' asok command
if (m_debug_blockrange > 0) {
m_debug_blockrange--;
- return false;
+ return std::nullopt;
}
- return true;
+ return objects.size();
}
void PgScrubber::select_range_n_notify()
{
get_counters_set().inc(scrbcnt_chunks_selected);
-
- if (select_range()) {
+ auto num_chunk_objects = select_range();
+ if (num_chunk_objects.has_value()) {
// the next chunk to handle is not blocked
dout(20) << __func__ << ": selection OK" << dendl;
- m_osds->queue_scrub_chunk_free(m_pg, Scrub::scrub_prio_t::low_priority);
-
+ auto cost = get_scrub_cost(num_chunk_objects.value());
+ m_osds->queue_scrub_chunk_free(m_pg, Scrub::scrub_prio_t::low_priority, cost);
} else {
// we will wait for the objects range to become available for scrubbing
dout(10) << __func__ << ": selected chunk is busy" << dendl;
@@ -994,6 +997,28 @@ void PgScrubber::select_range_n_notify()
}
}
+uint64_t PgScrubber::get_scrub_cost(uint64_t num_chunk_objects)
+{
+ const auto& conf = m_pg->get_cct()->_conf;
+ if (op_queue_type_t::WeightedPriorityQueue == m_osds->osd->osd_op_queue_type()) {
+ // if the osd_op_queue is WPQ, we will use the default osd_scrub_cost value
+ return conf->osd_scrub_cost;
+ }
+ uint64_t cost = 0;
+ double scrub_metadata_cost = m_osds->get_cost_per_io();
+ if (m_is_deep) {
+ auto pg_avg_object_size = m_pg->get_average_object_size();
+ cost = conf->osd_scrub_event_cost + (num_chunk_objects
+ * (scrub_metadata_cost + pg_avg_object_size));
+ dout(20) << fmt::format("{} : deep-scrub cost = {}", __func__, cost) << dendl;
+ return cost;
+ } else {
+ cost = conf->osd_scrub_event_cost + (num_chunk_objects * scrub_metadata_cost);
+ dout(20) << fmt::format("{} : shallow-scrub cost = {}", __func__, cost) << dendl;
+ return cost;
+ }
+}
+
bool PgScrubber::write_blocked_by_scrub(const hobject_t& soid)
{
if (soid < m_start || soid >= m_end) {
@@ -1574,10 +1599,15 @@ void PgScrubber::replica_scrub_op(OpRequestRef op)
set_queued_or_active();
advance_token();
+ const auto& conf = m_pg->get_cct()->_conf;
+ const int max_from_conf = size_from_conf(
+ m_is_deep, conf, "osd_scrub_chunk_max", "osd_shallow_scrub_chunk_max");
+ auto cost = get_scrub_cost(max_from_conf);
m_osds->queue_for_rep_scrub(m_pg,
m_replica_request_priority,
m_flags.priority,
- m_current_token);
+ m_current_token,
+ cost);
}
void PgScrubber::set_op_parameters(const requested_scrub_t& request)
@@ -1743,17 +1773,6 @@ void PgScrubber::handle_scrub_reserve_msgs(OpRequestRef op)
}
}
-
-bool PgScrubber::set_reserving_now() {
- return m_osds->get_scrub_services().set_reserving_now(m_pg_id,
- ceph_clock_now());
-}
-
-void PgScrubber::clear_reserving_now()
-{
- m_osds->get_scrub_services().clear_reserving_now(m_pg_id);
-}
-
void PgScrubber::set_queued_or_active()
{
m_queued_or_active = true;
@@ -2043,6 +2062,89 @@ 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);
+}
+
+
+Scrub::schedule_result_t PgScrubber::start_scrub_session(
+ Scrub::OSDRestrictions osd_restrictions,
+ Scrub::ScrubPGPreconds pg_cond,
+ const requested_scrub_t& requested_flags)
+{
+ 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;
+ return schedule_result_t::target_specific_failure;
+ }
+
+ // for all other failures - we must reinstate our entry in the Scrub Queue
+ if (!is_primary() || !m_pg->is_active() || !m_pg->is_clean()) {
+ dout(10) << __func__ << ": cannot scrub (not a clean and active primary)"
+ << dendl;
+ requeue_penalized(Scrub::delay_cause_t::pg_state);
+ return schedule_result_t::target_specific_failure;
+ }
+
+ if (state_test(PG_STATE_SNAPTRIM) || state_test(PG_STATE_SNAPTRIM_WAIT)) {
+ // note that the trimmer checks scrub status when setting 'snaptrim_wait'
+ // (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);
+ return schedule_result_t::target_specific_failure;
+ }
+
+ // analyze the combination of the requested scrub flags, the osd/pool
+ // configuration and the PG status to determine whether we should scrub
+ // now, and what type of scrub should that be.
+ 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);
+ return schedule_result_t::target_specific_failure;
+ }
+
+ // if only explicitly requested repairing is allowed - skip other types
+ // of scrubbing
+ if (osd_restrictions.allow_requested_repair_only &&
+ !updated_flags->must_repair) {
+ dout(10) << __func__
+ << ": skipping this PG as repairing was not explicitly "
+ "requested for it"
+ << dendl;
+ requeue_penalized(Scrub::delay_cause_t::scrub_params);
+ return schedule_result_t::target_specific_failure;
+ }
+
+ // try to reserve the local OSD resources. If failing: no harm. We will
+ // 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);
+ return schedule_result_t::osd_wide_failure;
+ }
+
+ // can commit to the updated flags now, as nothing will stop the scrub
+ m_planned_scrub = *updated_flags;
+
+ // An interrupted recovery repair could leave this set.
+ state_clear(PG_STATE_REPAIR);
+
+ set_op_parameters(m_planned_scrub);
+
+ // 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 schedule_result_t::scrub_initiated;
+}
+
+
/*
* note that the flags-set fetched from the PG (m_pg->m_planned_scrub)
* is cleared once scrubbing starts; Some of the values dumped here are
@@ -2475,6 +2577,284 @@ void PgScrubber::update_scrub_stats(ceph::coarse_real_clock::time_point now_is)
}
+bool PgScrubber::is_time_for_deep(
+ Scrub::ScrubPGPreconds pg_cond,
+ const requested_scrub_t& planned) const
+{
+ const auto last_deep = m_pg->info.history.last_deep_scrub_stamp; // shorthand
+ dout(10) << fmt::format(
+ "{}: pg_cond:({}) need-auto?{} last_deep_scrub_stamp:{}",
+ __func__, pg_cond, planned.need_auto, last_deep)
+ << dendl;
+
+ if (!pg_cond.allow_deep)
+ return false;
+
+ if (planned.need_auto) {
+ dout(10) << __func__ << ": need repair after scrub errors" << dendl;
+ return true;
+ }
+
+ const auto sched_conf = populate_config_params();
+ const auto next_deep = last_deep + sched_conf.deep_interval;
+ const auto timenow = ceph_clock_now();
+ if (timenow >= next_deep) {
+ dout(20) << fmt::format(
+ "{}: now ({}) >= time for deep ({})", __func__, timenow,
+ next_deep)
+ << dendl;
+ return true;
+ }
+
+ if (pg_cond.has_deep_errors) {
+ // note: the text below is matched by 'standalone' tests
+ get_clog()->info() << fmt::format(
+ "osd.{} pg {} Deep scrub errors, upgrading scrub to deep-scrub",
+ get_whoami(), m_pg_id);
+ return true;
+ }
+
+ // we only flip coins if 'allow_shallow_scrub' is asserted. Otherwise - as
+ // this function is called often, we will probably be deep-scrubbing most of
+ // the time.
+ if (pg_cond.allow_shallow) {
+ const bool deep_coin_flip =
+ random_bool_with_probability(sched_conf.deep_randomize_ratio);
+ if (deep_coin_flip) {
+ dout(10) << fmt::format(
+ "{}: scrub upgraded to deep (coin flip)", __func__)
+ << dendl;
+ return true;
+ }
+ }
+
+ return false;
+}
+
+
+/*
+ clang-format off
+
+ Request details | none | no-scrub | no-scrub+no-deep | no-deep
+ ------------------------------------------------------------------------
+ ------------------------------------------------------------------------
+ initiated | shallow | shallow | shallow | shallow
+ ------------------------------------------------------------------------
+ init. + t.f.deep | deep | deep | shallow | shallow
+ ------------------------------------------------------------------------
+ initiated deep | deep | deep | deep | deep
+ ------------------------------------------------------------------------
+
+ clang-format on
+*/
+std::optional<requested_scrub_t> PgScrubber::validate_initiated_scrub(
+ Scrub::ScrubPGPreconds pg_cond,
+ bool time_for_deep,
+ const requested_scrub_t& planned) const
+{
+ requested_scrub_t upd_flags{planned};
+
+ upd_flags.time_for_deep = time_for_deep;
+ upd_flags.deep_scrub_on_error = false;
+ upd_flags.auto_repair = false;
+
+ if (upd_flags.must_deep_scrub) {
+ upd_flags.calculated_to_deep = true;
+ } else if (
+ upd_flags.time_for_deep && pg_cond.allow_deep) {
+ upd_flags.calculated_to_deep = true;
+ } else {
+ upd_flags.calculated_to_deep = false;
+ if (pg_cond.has_deep_errors) {
+ get_clog()->error() << fmt::format(
+ "osd.{} pg {} Regular scrub request, deep-scrub details will be lost",
+ get_whoami(), m_pg_id);
+ }
+ }
+
+ if (pg_cond.can_autorepair) {
+ // for shallow scrubs: rescrub if errors found
+ // for deep: turn 'auto-repair' on
+ if (upd_flags.calculated_to_deep) {
+ dout(10) << fmt::format(
+ "{}: performing an auto-repair deep scrub", __func__)
+ << dendl;
+ upd_flags.auto_repair = true;
+ } else {
+ dout(10) << fmt::format(
+ "{}: will perform an auto-repair deep scrub if errors "
+ "are found",
+ __func__)
+ << dendl;
+ upd_flags.deep_scrub_on_error = true;
+ }
+ }
+
+ return upd_flags;
+}
+
+/*
+ clang-format off
+
+ for periodic scrubs:
+
+ Periodic type | none | no-scrub | no-scrub+no-deep | no-deep
+ ------------------------------------------------------------------------
+ ------------------------------------------------------------------------
+ periodic | shallow | x | x | shallow
+ ------------------------------------------------------------------------
+ periodic + t.f.deep| deep | deep | x | shallow
+ ------------------------------------------------------------------------
+
+ clang-format on
+*/
+std::optional<requested_scrub_t> PgScrubber::validate_periodic_mode(
+ Scrub::ScrubPGPreconds pg_cond,
+ bool time_for_deep,
+ const requested_scrub_t& planned) const
+
+{
+ ceph_assert(!planned.must_deep_scrub && !planned.must_repair);
+
+ if (!pg_cond.allow_deep && pg_cond.has_deep_errors) {
+ get_clog()->error() << fmt::format(
+ "osd.{} pg {} Regular scrub skipped due to deep-scrub errors and "
+ "nodeep-scrub set",
+ get_whoami(), m_pg_id);
+ return std::nullopt; // no scrubbing
+ }
+
+ requested_scrub_t upd_flags{planned};
+
+ upd_flags.time_for_deep = time_for_deep;
+ upd_flags.deep_scrub_on_error = false;
+ upd_flags.auto_repair = false;
+ upd_flags.calculated_to_deep = false;
+
+ dout(20) << fmt::format(
+ "{}: allowed:{}/{} t.f.d:{} req:{}", __func__,
+ pg_cond.allow_shallow, pg_cond.allow_deep,
+ upd_flags.time_for_deep, planned)
+ << dendl;
+
+ // should we perform a shallow scrub?
+ if (pg_cond.allow_shallow) {
+ if (!upd_flags.time_for_deep || !pg_cond.allow_deep) {
+ if (pg_cond.can_autorepair) {
+ dout(10) << __func__
+ << ": auto repair with scrubbing, rescrub if errors found"
+ << dendl;
+ upd_flags.deep_scrub_on_error = true;
+ }
+ dout(20) << __func__ << " will do shallow scrub (time_for_deep = "
+ << upd_flags.time_for_deep << ")" << dendl;
+ return upd_flags;
+ }
+ // else - either deep-scrub or nothing
+ }
+
+ if (upd_flags.time_for_deep) {
+ if (pg_cond.allow_deep) {
+ if (pg_cond.can_autorepair) {
+ dout(20) << __func__ << ": auto repair with deep scrubbing" << dendl;
+ upd_flags.auto_repair = true;
+ }
+ upd_flags.calculated_to_deep = true;
+ dout(20) << fmt::format("{}: final: {}", __func__, upd_flags) << dendl;
+ return upd_flags;
+ }
+ if (pg_cond.allow_shallow) {
+ dout(20) << fmt::format("{}: final:{}", __func__, upd_flags) << dendl;
+ return upd_flags;
+ }
+ // else - no scrubbing
+ }
+
+ return std::nullopt; // no scrubbing
+}
+
+
+/*
+ From docs.ceph.com (osd-internals/scrub):
+
+ clang-format off
+
+ Desired no-scrub flags & scrub type interactions:
+
+ Periodic type | none | no-scrub | no-scrub+no-deep | no-deep
+ ------------------------------------------------------------------------
+ ------------------------------------------------------------------------
+ periodic | shallow | x | x | shallow
+ ------------------------------------------------------------------------
+ periodic + t.f.deep| deep | deep | x | shallow
+ ------------------------------------------------------------------------
+ initiated | shallow | shallow | shallow | shallow
+ ------------------------------------------------------------------------
+ init. + t.f.deep | deep | deep | shallow | shallow
+ ------------------------------------------------------------------------
+ initiated deep | deep | deep | deep | deep
+ ------------------------------------------------------------------------
+
+ "periodic" - if !must_scrub && !must_deep_scrub;
+ "initiated deep" - if must_scrub && must_deep_scrub;
+ "initiated" - if must_scrub && !must_deep_scrub;
+
+ clang-format on
+*/
+/*
+ * The returned flags collection (requested_scrub_t) is based on
+ * m_planned_scrub with the following modifications:
+ *
+ * - calculated_to_deep will be set to shallow or deep, depending on the
+ * scrub type (according to the decision table above);
+ * - deep_scrub_on_error will be determined;
+ * - same for auto_repair;
+ * - time_for_deep will be set to true if the scrub is periodic and the
+ * time for a deep scrub has been reached (+ some other conditions);
+ * and
+ * - need_auto is cleared
+ */
+std::optional<requested_scrub_t> PgScrubber::validate_scrub_mode(
+ Scrub::OSDRestrictions osd_restrictions,
+ Scrub::ScrubPGPreconds pg_cond) const
+{
+ dout(10) << fmt::format(
+ "{}: osd_restrictions:{} pg_cond:{}", __func__,
+ osd_restrictions, pg_cond)
+ << 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()) {
+ // 'initiated' scrubs
+ dout(10) << fmt::format(
+ "{}: initiated (\"must\") scrub (target:{} pg:{})",
+ __func__, *m_scrub_job, pg_cond)
+ << dendl;
+ upd_flags =
+ validate_initiated_scrub(pg_cond, time_for_deep, m_planned_scrub);
+
+ } else {
+ // -------- a periodic scrub
+ dout(10) << fmt::format(
+ "{}: periodic target:{} pg:{}", __func__, *m_scrub_job,
+ pg_cond)
+ << dendl;
+ upd_flags = validate_periodic_mode(pg_cond, time_for_deep, m_planned_scrub);
+ if (!upd_flags) {
+ dout(20) << __func__ << ": no periodic scrubs allowed" << dendl;
+ return std::nullopt;
+ }
+ }
+
+ dout(10) << fmt::format("{}: next scrub flags: {}", __func__, *upd_flags)
+ << dendl;
+ upd_flags->need_auto = false;
+ return upd_flags;
+}
+
+
// ///////////////////// preemption_data_t //////////////////////////////////
PgScrubber::preemption_data_t::preemption_data_t(PG* pg) : m_pg{pg}
diff --git a/src/osd/scrubber/pg_scrubber.h b/src/osd/scrubber/pg_scrubber.h
index 78e8ba90d44..7a50093c2d1 100644
--- a/src/osd/scrubber/pg_scrubber.h
+++ b/src/osd/scrubber/pg_scrubber.h
@@ -187,6 +187,11 @@ class PgScrubber : public ScrubPgIF,
/// are we waiting for resource reservation grants form our replicas?
[[nodiscard]] bool is_reserving() const final;
+ Scrub::schedule_result_t start_scrub_session(
+ Scrub::OSDRestrictions osd_restrictions,
+ Scrub::ScrubPGPreconds,
+ const requested_scrub_t& requested_flags) final;
+
void initiate_regular_scrub(epoch_t epoch_queued) final;
void initiate_scrub_after_repair(epoch_t epoch_queued) final;
@@ -456,9 +461,6 @@ class PgScrubber : public ScrubPgIF,
int build_replica_map_chunk() final;
- bool set_reserving_now() final;
- void clear_reserving_now() final;
-
[[nodiscard]] bool was_epoch_changed() const final;
void set_queued_or_active() final;
@@ -561,6 +563,11 @@ class PgScrubber : public ScrubPgIF,
// '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).
+ void requeue_penalized(Scrub::delay_cause_t cause);
+
// ----- methods used to verify the relevance of incoming events:
/**
@@ -729,6 +736,8 @@ class PgScrubber : public ScrubPgIF,
/// Returns epoch of current osdmap
epoch_t get_osdmap_epoch() const { return get_osdmap()->get_epoch(); }
+ uint64_t get_scrub_cost(uint64_t num_chunk_objects);
+
// collected statistics
int m_shallow_errors{0};
int m_deep_errors{0};
@@ -793,6 +802,31 @@ class PgScrubber : public ScrubPgIF,
Scrub::sched_params_t determine_scrub_time(
const pool_opts_t& pool_conf) const;
+ /// should we perform deep scrub?
+ bool is_time_for_deep(
+ Scrub::ScrubPGPreconds pg_cond,
+ const requested_scrub_t& planned) const;
+
+ /**
+ * Validate the various 'next scrub' flags against configuration
+ * and scrub-related timestamps.
+ *
+ * @returns an updated copy of the m_planned_flags (or nothing if no scrubbing)
+ */
+ std::optional<requested_scrub_t> validate_scrub_mode(
+ Scrub::OSDRestrictions osd_restrictions,
+ Scrub::ScrubPGPreconds pg_cond) const;
+
+ std::optional<requested_scrub_t> validate_periodic_mode(
+ Scrub::ScrubPGPreconds pg_cond,
+ bool time_for_deep,
+ const requested_scrub_t& planned) const;
+
+ std::optional<requested_scrub_t> validate_initiated_scrub(
+ Scrub::ScrubPGPreconds pg_cond,
+ bool time_for_deep,
+ const requested_scrub_t& planned) const;
+
/*
* Select a range of objects to scrub.
*
@@ -802,8 +836,11 @@ class PgScrubber : public ScrubPgIF,
* - handling some head/clones issues
*
* The selected range is set directly into 'm_start' and 'm_end'
+ *
+ * Returns std::nullopt if the range is busy otherwise returns the
+ * number of objects in the range.
*/
- bool select_range();
+ std::optional<uint64_t> select_range();
std::list<Context*> m_callbacks;
diff --git a/src/osd/scrubber/scrub_machine.cc b/src/osd/scrubber/scrub_machine.cc
index acdddbd18eb..ca0ff522278 100644
--- a/src/osd/scrubber/scrub_machine.cc
+++ b/src/osd/scrubber/scrub_machine.cc
@@ -183,15 +183,6 @@ Session::Session(my_context ctx)
dout(10) << "-- state -->> PrimaryActive/Session" << dendl;
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
- // while we've checked the 'someone is reserving' flag before queueing
- // the start-scrub event, it's possible that the flag was set in the meantime.
- // Handling this case here requires adding a new sub-state, and the
- // complication of reporting a failure to the caller in a new failure
- // path. On the other hand - ignoring an ongoing reservation on rare
- // occasions will cause no harm.
- // We choose ignorance.
- std::ignore = scrbr->set_reserving_now();
-
m_perf_set = &scrbr->get_counters_set();
m_perf_set->inc(scrbcnt_started);
}
@@ -235,18 +226,7 @@ ReservingReplicas::ReservingReplicas(my_context ctx)
*scrbr, context<PrimaryActive>().last_request_sent_nonce,
*session.m_perf_set);
- if (session.m_reservations->get_last_sent()) {
- // the 1'st reservation request was sent
-
- auto timeout = scrbr->get_pg_cct()->_conf.get_val<milliseconds>(
- "osd_scrub_reservation_timeout");
- if (timeout.count() > 0) {
- // Start a timer to handle case where the replicas take a long time to
- // ack the reservation. See ReservationTimeout handler below.
- m_timeout_token =
- machine.schedule_timer_event_after<ReservationTimeout>(timeout);
- }
- } else {
+ if (!session.m_reservations->get_last_sent()) {
// no replicas to reserve
dout(10) << "no replicas to reserve" << dendl;
// can't transit directly from here
@@ -254,14 +234,6 @@ ReservingReplicas::ReservingReplicas(my_context ctx)
}
}
-ReservingReplicas::~ReservingReplicas()
-{
- DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
- // it's OK to try and clear the flag even if we don't hold it
- // (the flag remembers the actual holder)
- scrbr->clear_reserving_now();
-}
-
sc::result ReservingReplicas::react(const ReplicaGrant& ev)
{
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
@@ -305,26 +277,6 @@ sc::result ReservingReplicas::react(const ReplicaReject& ev)
return transit<PrimaryIdle>();
}
-sc::result ReservingReplicas::react(const ReservationTimeout&)
-{
- DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
- auto& session = context<Session>();
- dout(10) << "ReservingReplicas::react(const ReservationTimeout&)" << dendl;
- ceph_assert(session.m_reservations);
-
- session.m_reservations->log_failure_and_duration(scrbcnt_resrv_timed_out);
-
- const auto msg = fmt::format(
- "osd.{} PgScrubber: {} timeout on reserving replicas (since {})",
- scrbr->get_whoami(), scrbr->get_spgid(), entered_at);
- dout(1) << msg << dendl;
- scrbr->get_clog()->warn() << msg;
-
- // cause the scrubber to stop the scrub session, marking 'reservation
- // failure' as the cause (affecting future scheduling)
- scrbr->flag_reservations_failure();
- return transit<PrimaryIdle>();
-}
// ----------------------- ActiveScrubbing -----------------------------------
diff --git a/src/osd/scrubber/scrub_machine.h b/src/osd/scrubber/scrub_machine.h
index b9f60481674..cf8d28c765b 100644
--- a/src/osd/scrubber/scrub_machine.h
+++ b/src/osd/scrubber/scrub_machine.h
@@ -160,9 +160,6 @@ VALUE_EVENT(ReserverGranted, AsyncScrubResData);
/// all replicas have granted our reserve request
MEV(RemotesReserved)
-/// reservations have timed out
-MEV(ReservationTimeout)
-
/// initiate a new scrubbing session (relevant if we are a Primary)
MEV(StartScrub)
@@ -565,25 +562,21 @@ struct Session : sc::state<Session, PrimaryActive, ReservingReplicas>,
ScrubTimePoint m_session_started_at{ScrubClock::now()};
};
-struct ReservingReplicas : sc::state<ReservingReplicas, Session>,
- NamedSimply {
+struct ReservingReplicas : sc::state<ReservingReplicas, Session>, NamedSimply {
explicit ReservingReplicas(my_context ctx);
- ~ReservingReplicas();
- using reactions = mpl::list<sc::custom_reaction<ReplicaGrant>,
- sc::custom_reaction<ReplicaReject>,
- sc::transition<RemotesReserved, ActiveScrubbing>,
- sc::custom_reaction<ReservationTimeout>>;
+ ~ReservingReplicas() = default;
+ using reactions = mpl::list<
+ sc::custom_reaction<ReplicaGrant>,
+ sc::custom_reaction<ReplicaReject>,
+ sc::transition<RemotesReserved, ActiveScrubbing>>;
ScrubTimePoint entered_at = ScrubClock::now();
- ScrubMachine::timer_event_token_t m_timeout_token;
/// a "raw" event carrying a peer's grant response
sc::result react(const ReplicaGrant&);
/// a "raw" event carrying a peer's denial response
sc::result react(const ReplicaReject&);
-
- sc::result react(const ReservationTimeout&);
};
diff --git a/src/osd/scrubber/scrub_machine_lstnr.h b/src/osd/scrubber/scrub_machine_lstnr.h
index ea893ba81f0..85c518c402f 100644
--- a/src/osd/scrubber/scrub_machine_lstnr.h
+++ b/src/osd/scrubber/scrub_machine_lstnr.h
@@ -206,18 +206,6 @@ struct ScrubMachineListener {
virtual void set_scrub_duration(std::chrono::milliseconds duration) = 0;
/**
- * No new scrub session will start while a scrub was initiate on a PG,
- * and that PG is trying to acquire replica resources.
- * set_reserving_now()/clear_reserving_now() let's the OSD scrub-queue know
- * we are busy reserving.
- *
- * set_reserving_now() returns 'false' if there already is a PG in the
- * reserving stage of the scrub session.
- */
- virtual bool set_reserving_now() = 0;
- virtual void clear_reserving_now() = 0;
-
- /**
* Manipulate the 'I am being scrubbed now' Scrubber's flag
*/
virtual void set_queued_or_active() = 0;