using namespace ::std::chrono;
using namespace ::std::chrono_literals;
using namespace ::std::literals;
+using schedule_result_t = Scrub::schedule_result_t;
#define dout_subsys ceph_subsys_osd
return false;
}
+
void OsdScrub::initiate_scrub(bool is_recovery_active)
{
if (scrub_random_backoff()) {
utime_t scrub_time = ceph_clock_now();
dout(10) << fmt::format(
- "{}: time now:{}, recover is active?:{}", __func__,
- scrub_time, is_recovery_active)
+ "time now:{}, recover is active?:{}", scrub_time,
+ is_recovery_active)
<< dendl;
// check the OSD-wide environment conditions (scrub resources, time, etc.).
}
}
- auto was_started = select_pg_and_scrub(*env_restrictions);
- dout(20) << fmt::format(
- "scrub scheduling done ({})",
- ScrubQueue::attempt_res_text(was_started))
- << dendl;
+ // at this phase of the refactoring: minimal changes to the
+ // queue interface used here: we ask for a list of
+ // eligible targets (based on the known restrictions).
+ // We try all elements of this list until a (possibly temporary) success.
+ auto candidates = m_queue.ready_to_scrub(*env_restrictions, scrub_time);
+ auto res{schedule_result_t::none_ready};
+ for (const auto& candidate : candidates) {
+ dout(20) << fmt::format("initiating scrub on pg[{}]", candidate) << dendl;
+
+ // we have a candidate to scrub. But we may fail when trying to initiate that
+ // scrub. For some failures - we can continue with the next candidate. For
+ // others - we should stop trying to scrub at this tick.
+ res = m_osd_svc.initiate_a_scrub(
+ candidate, env_restrictions->allow_requested_repair_only);
+ switch (res) {
+ case schedule_result_t::scrub_initiated:
+ // the happy path. We are done
+ dout(20) << fmt::format("scrub initiated for pg[{}]", candidate.pgid)
+ << dendl;
+ break;
+
+ case schedule_result_t::already_started:
+ case schedule_result_t::preconditions:
+ case schedule_result_t::bad_pg_state:
+ // continue with the next job
+ dout(20) << fmt::format(
+ "pg[{}] failed (state/cond/started)", candidate.pgid)
+ << dendl;
+ break;
+
+ case schedule_result_t::no_such_pg:
+ // The pg is no longer there
+ dout(20) << fmt::format("pg[{}] failed (no PG)", candidate.pgid)
+ << dendl;
+ // \todo better handling of this case
+ break;
+
+ case schedule_result_t::no_local_resources:
+ // failure to secure local resources. No point in trying the other
+ // PGs at this time. Note that this is not the same as replica resources
+ // failure!
+ dout(20) << "failed (local resources)" << dendl;
+ break;
+
+ case schedule_result_t::none_ready:
+ // can't happen. Just for the compiler.
+ dout(5) << fmt::format(
+ "failed!! (possible bug. pg[{}])", candidate.pgid)
+ << dendl;
+ break;
+ }
+
+ if (res == schedule_result_t::no_local_resources) {
+ break;
+ }
+
+ if (res == schedule_result_t::scrub_initiated) {
+ // note: in the full implementation: we need to dequeue the target
+ // at this time
+ break;
+ }
+ }
}
#undef dout_prefix
#define dout_prefix _prefix(_dout, whoami, get_osdmap_epoch())
-Scrub::schedule_result_t OSDService::initiate_a_scrub(spg_t pgid,
+
+schedule_result_t OSDService::initiate_a_scrub(spg_t pgid,
bool allow_requested_repair_only)
{
dout(20) << __func__ << " trying " << pgid << dendl;
// the PG was dequeued in the short timespan between creating the candidates list
// (collect_ripe_jobs()) and here
dout(5) << __func__ << " pg " << pgid << " not found" << dendl;
- return Scrub::schedule_result_t::no_such_pg;
+ return schedule_result_t::no_such_pg;
}
// This has already started, so go on to the next scrub job
if (pg->is_scrub_queued_or_active()) {
pg->unlock();
dout(20) << __func__ << ": already in progress pgid " << pgid << dendl;
- return Scrub::schedule_result_t::already_started;
+ return schedule_result_t::already_started;
}
// Skip other kinds of scrubbing if only explicitly requested repairing is allowed
if (allow_requested_repair_only && !pg->get_planned_scrub().must_repair) {
pg->unlock();
dout(10) << __func__ << " skip " << pgid
<< " because repairing is not explicitly requested on it" << dendl;
- return Scrub::schedule_result_t::preconditions;
+ return schedule_result_t::preconditions;
}
auto scrub_attempt = pg->sched_scrub();
f->close_section();
}
-/**
- * a note regarding 'to_scrub_copy':
- * 'to_scrub_copy' is a sorted set of all the ripe jobs from to_copy.
- * As we usually expect to refer to only the first job in this set, we could
- * consider an alternative implementation:
- * - have collect_ripe_jobs() return the copied set without sorting it;
- * - loop, performing:
- * - use std::min_element() to find a candidate;
- * - try that one. If not suitable, discard from 'to_scrub_copy'
- */
-Scrub::schedule_result_t ScrubQueue::select_pg_and_scrub(
- Scrub::OSDRestrictions preconds)
-{
- dout(10) << " reg./pen. sizes: " << to_scrub.size() << " / "
- << penalized.size() << dendl;
-
- utime_t now_is = time_now();
-
- // create a list of candidates (copying, as otherwise creating a deadlock):
- // - possibly restore penalized
- // - (if we didn't handle directly) remove invalid jobs
- // - create a copy of the to_scrub (possibly up to first not-ripe)
- // - same for the penalized (although that usually be a waste)
- // unlock, then try the lists
-
- std::unique_lock lck{jobs_lock};
-
- // pardon all penalized jobs that have deadlined (or were updated)
- scan_penalized(restore_penalized, now_is);
- restore_penalized = false;
-
- // remove the 'updated' flag from all entries
- std::for_each(to_scrub.begin(),
- to_scrub.end(),
- [](const auto& jobref) -> void { jobref->updated = false; });
-
- // add failed scrub attempts to the penalized list
- move_failed_pgs(now_is);
-
- // collect all valid & ripe jobs from the two lists. Note that we must copy,
- // as when we use the lists we will not be holding jobs_lock (see
- // explanation above)
-
- auto to_scrub_copy = collect_ripe_jobs(to_scrub, now_is);
- auto penalized_copy = collect_ripe_jobs(penalized, now_is);
- lck.unlock();
-
- // try the regular queue first
- auto res = select_from_group(to_scrub_copy, preconds, now_is);
-
- // in the sole scenario in which we've gone over all ripe jobs without success
- // - we will try the penalized
- if (res == Scrub::schedule_result_t::none_ready && !penalized_copy.empty()) {
- res = select_from_group(penalized_copy, preconds, now_is);
- dout(10) << "tried the penalized. Res: "
- << ScrubQueue::attempt_res_text(res) << dendl;
- restore_penalized = true;
- }
-
- dout(15) << dendl;
- return res;
-}
-
-
-// not holding jobs_lock. 'group' is a copy of the actual list.
-Scrub::schedule_result_t ScrubQueue::select_from_group(
- Scrub::ScrubQContainer& group,
- Scrub::OSDRestrictions preconds,
- utime_t now_is)
-{
- dout(15) << "jobs #: " << group.size() << dendl;
-
- for (auto& candidate : group) {
-
- // we expect the first job in the list to be a good candidate (if any)
-
- dout(20) << "try initiating scrub for " << candidate->pgid << dendl;
-
- if (preconds.only_deadlined && (candidate->schedule.deadline.is_zero() ||
- candidate->schedule.deadline >= now_is)) {
- dout(15) << " not scheduling scrub for " << candidate->pgid << " due to "
- << (preconds.time_permit ? "high load" : "time not permitting")
- << dendl;
- continue;
- }
-
- // we have a candidate to scrub. We turn to the OSD to verify that the PG
- // configuration allows the specified type of scrub, and to initiate the
- // scrub.
- switch (osd_service.initiate_a_scrub(
- candidate->pgid, preconds.allow_requested_repair_only)) {
-
- case Scrub::schedule_result_t::scrub_initiated:
- // the happy path. We are done
- dout(20) << " initiated for " << candidate->pgid << dendl;
- return Scrub::schedule_result_t::scrub_initiated;
-
- case Scrub::schedule_result_t::already_started:
- case Scrub::schedule_result_t::preconditions:
- case Scrub::schedule_result_t::bad_pg_state:
- // continue with the next job
- dout(20) << "failed (state/cond/started) " << candidate->pgid << dendl;
- break;
-
- case Scrub::schedule_result_t::no_such_pg:
- // The pg is no longer there
- dout(20) << "failed (no pg) " << candidate->pgid << dendl;
- break;
-
- case Scrub::schedule_result_t::no_local_resources:
- // failure to secure local resources. No point in trying the other
- // PGs at this time. Note that this is not the same as replica resources
- // failure!
- dout(20) << "failed (local) " << candidate->pgid << dendl;
- return Scrub::schedule_result_t::no_local_resources;
-
- case Scrub::schedule_result_t::none_ready:
- // can't happen. Just for the compiler.
- dout(5) << "failed !!! " << candidate->pgid << dendl;
- return Scrub::schedule_result_t::none_ready;
- }
- }
-
- dout(20) << " returning 'none ready'" << dendl;
- return Scrub::schedule_result_t::none_ready;
-}
-
// ////////////////////////////////////////////////////////////////////////// //
// CPU load tracking and related
{
return m_queue.list_registered_jobs();
}
-
-Scrub::schedule_result_t OsdScrub::select_pg_and_scrub(
- Scrub::OSDRestrictions preconds)
-{
- return m_queue.select_pg_and_scrub(preconds);
-}
}
// clang-format on
+std::vector<ScrubTargetId> ScrubQueue::ready_to_scrub(
+ OSDRestrictions restrictions, // note: 4B in size! (copy)
+ utime_t scrub_tick)
+{
+ dout(10) << fmt::format(
+ " @{:s}: reg./pen. sizes: {} / {} ({})", scrub_tick,
+ to_scrub.size(), penalized.size(), restrictions)
+ << dendl;
+ // create a list of candidates (copying, as otherwise creating a deadlock):
+ // - possibly restore penalized
+ // - (if we didn't handle directly) remove invalid jobs
+ // - create a copy of the to_scrub (possibly up to first not-ripe)
+ // - same for the penalized (although that usually be a waste)
+ // unlock, then try the lists
+
+ std::unique_lock lck{jobs_lock};
+
+ // pardon all penalized jobs that have deadlined (or were updated)
+ scan_penalized(restore_penalized, scrub_tick);
+ restore_penalized = false;
+
+ // remove the 'updated' flag from all entries
+ std::for_each(
+ to_scrub.begin(), to_scrub.end(),
+ [](const auto& jobref) -> void { jobref->updated = false; });
+
+ // add failed scrub attempts to the penalized list
+ move_failed_pgs(scrub_tick);
+
+ // collect all valid & ripe jobs from the two lists. Note that we must copy,
+ // as when we use the lists we will not be holding jobs_lock (see
+ // explanation above)
+
+ // and in this step 1 of the refactoring (Aug 2023): the set returned must be
+ // transformed into a vector of targets (which, in this phase, are
+ // the PG id-s).
+ auto to_scrub_copy = collect_ripe_jobs(to_scrub, restrictions, scrub_tick);
+ auto penalized_copy = collect_ripe_jobs(penalized, restrictions, scrub_tick);
+ lck.unlock();
+
+ std::vector<ScrubTargetId> all_ready;
+ std::transform(
+ to_scrub_copy.cbegin(), to_scrub_copy.cend(),
+ std::back_inserter(all_ready),
+ [](const auto& jobref) -> ScrubTargetId { return jobref->pgid; });
+ // not bothering to handle the "reached the penalized - so all should be
+ // forgiven" case, as the penalty queue is destined to be removed in a
+ // followup PR.
+ std::transform(
+ penalized_copy.cbegin(), penalized_copy.cend(),
+ std::back_inserter(all_ready),
+ [](const auto& jobref) -> ScrubTargetId { return jobref->pgid; });
+ return all_ready;
+}
+
// must be called under lock
void ScrubQueue::rm_unregistered_jobs(ScrubQContainer& group)
// called under lock
ScrubQContainer ScrubQueue::collect_ripe_jobs(
- ScrubQContainer& group,
- utime_t time_now)
+ ScrubQContainer& group,
+ OSDRestrictions restrictions,
+ utime_t time_now)
{
- rm_unregistered_jobs(group);
+ auto filtr = [time_now, restrictions](const auto& jobref) -> bool {
+ return jobref->schedule.scheduled_at <= time_now &&
+ (!restrictions.only_deadlined ||
+ (!jobref->schedule.deadline.is_zero() &&
+ jobref->schedule.deadline <= time_now));
+ };
- // copy ripe jobs
+ rm_unregistered_jobs(group);
+ // copy ripe jobs (unless prohibited by 'restrictions')
ScrubQContainer ripes;
ripes.reserve(group.size());
- std::copy_if(group.begin(),
- group.end(),
- std::back_inserter(ripes),
- [time_now](const auto& jobref) -> bool {
- return jobref->schedule.scheduled_at <= time_now;
- });
+ std::copy_if(group.begin(), group.end(), std::back_inserter(ripes), filtr);
std::sort(ripes.begin(), ripes.end(), cmp_sched_time_t{});
if (g_conf()->subsys.should_gather<ceph_subsys_osd, 20>()) {
for (const auto& jobref : group) {
- if (jobref->schedule.scheduled_at > time_now) {
- dout(20) << " not ripe: " << jobref->pgid << " @ "
- << jobref->schedule.scheduled_at << dendl;
+ if (!filtr(jobref)) {
+ dout(20) << fmt::format(
+ " not ripe: {} @ {:s}", jobref->pgid,
+ jobref->schedule.scheduled_at)
+ << dendl;
}
}
}
using sched_params_t = Scrub::sched_params_t;
/**
- * called periodically by the OSD to select the first scrub-eligible PG
- * and scrub it.
+ * returns the list of all scrub targets that are ready to be scrubbed.
+ * Note that the following changes are expected in the near future (as part
+ * of the scheduling refactoring):
+ * - only one target will be requested by the OsdScrub (the OSD's sub-object
+ * that initiates scrubs);
+ * - that target would name a PG X scrub type;
*
- * Selection is affected by:
- * - time of day: scheduled scrubbing might be configured to only happen
- * during certain hours;
- * - same for days of the week, and for the system load;
- *
- * @param preconds: what types of scrub are allowed, given system status &
- * config. Some of the preconditions are calculated here.
- * @return Scrub::attempt_t::scrubbing if a scrub session was successfully
- * initiated. Otherwise - the failure cause.
- *
- * locking: locks jobs_lock
+ * @param restrictions: what types of scrub are allowed, given system status
+ * & config. Some of the preconditions are calculated here.
*/
- Scrub::schedule_result_t select_pg_and_scrub(Scrub::OSDRestrictions preconds);
+ std::vector<ScrubTargetId> ready_to_scrub(
+ Scrub::OSDRestrictions restrictions, // 4B! copy
+ utime_t scrub_tick);
/**
* Translate attempt_ values into readable text
* scrub jobs.
* Note also that OSDRestrictions is 1L size, thus copied.
*/
- Scrub::ScrubQContainer collect_ripe_jobs(Scrub::ScrubQContainer& group, utime_t time_now);
-
+ Scrub::ScrubQContainer collect_ripe_jobs(
+ Scrub::ScrubQContainer& group,
+ Scrub::OSDRestrictions restrictions,
+ utime_t time_now);
/**
* The scrubbing of PGs might be delayed if the scrubbed chunk of objects is
*/
void move_failed_pgs(utime_t now_is);
- Scrub::schedule_result_t select_from_group(
- Scrub::ScrubQContainer& group,
- Scrub::OSDRestrictions preconds,
- utime_t now_is);
-
protected: // used by the unit-tests
/**
* unit-tests will override this function to return a mock time