ScrubJob is now in the Scrub namespace.
Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
scrubber/osd_scrub.cc
scrubber/osd_scrub_sched.cc
scrubber/PrimaryLogScrub.cc
+ scrubber/scrub_job.cc
scrubber/scrub_machine.cc
scrubber/scrub_resources.cc
scrubber/ScrubStore.cc
f->open_array_section("scrubs");
- std::for_each(to_scrub.cbegin(), to_scrub.cend(), [&f](const ScrubJobRef& j) {
- j->dump(f);
- });
+ std::for_each(
+ to_scrub.cbegin(), to_scrub.cend(),
+ [&f](const Scrub::ScrubJobRef& j) { j->dump(f); });
- std::for_each(penalized.cbegin(),
- penalized.cend(),
- [&f](const ScrubJobRef& j) { j->dump(f); });
+ std::for_each(
+ penalized.cbegin(), penalized.cend(),
+ [&f](const Scrub::ScrubJobRef& j) { j->dump(f); });
f->close_section();
}
// not holding jobs_lock. 'group' is a copy of the actual list.
Scrub::schedule_result_t ScrubQueue::select_from_group(
- ScrubQContainer& group,
+ Scrub::ScrubQContainer& group,
const Scrub::OSDRestrictions& preconds,
utime_t now_is)
{
using namespace ::std::chrono;
using namespace ::std::chrono_literals;
using namespace ::std::literals;
+using qu_state_t = Scrub::qu_state_t;
+using must_scrub_t = Scrub::must_scrub_t;
+using ScrubQContainer = Scrub::ScrubQContainer;
+using sched_params_t = Scrub::sched_params_t;
+using OSDRestrictions = Scrub::OSDRestrictions;
+using ScrubJob = Scrub::ScrubJob;
-// ////////////////////////////////////////////////////////////////////////// //
-// ScrubJob
-
-#define dout_context (cct)
-#define dout_subsys ceph_subsys_osd
-#undef dout_prefix
-#define dout_prefix *_dout << "osd." << whoami << " "
-
-ScrubQueue::ScrubJob::ScrubJob(CephContext* cct, const spg_t& pg, int node_id)
- : RefCountedObject{cct}
- , pgid{pg}
- , whoami{node_id}
- , cct{cct}
-{}
-
-// debug usage only
-ostream& operator<<(ostream& out, const ScrubQueue::ScrubJob& sjob)
-{
- out << sjob.pgid << ", " << sjob.schedule.scheduled_at
- << " dead: " << sjob.schedule.deadline << " - "
- << sjob.registration_state() << " / failure: " << sjob.resources_failure
- << " / pen. t.o.: " << sjob.penalty_timeout
- << " / queue state: " << ScrubQueue::qu_state_text(sjob.state);
-
- return out;
-}
-
-void ScrubQueue::ScrubJob::update_schedule(
- const ScrubQueue::scrub_schedule_t& adjusted)
-{
- schedule = adjusted;
- penalty_timeout = utime_t(0, 0); // helps with debugging
-
- // '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("{}: pg[{}] adjusted: {:s} ({})", __func__, pgid,
- schedule.scheduled_at, registration_state()) << dendl;
-}
-
-std::string ScrubQueue::ScrubJob::scheduling_state(utime_t now_is,
- bool is_deep_expected) const
-{
- // if not in the OSD scheduling queues, not a candidate for scrubbing
- if (state != qu_state_t::registered) {
- return "no scrub is scheduled";
- }
-
- // if the time has passed, we are surely in the queue
- // (note that for now we do not tell client if 'penalized')
- if (now_is > schedule.scheduled_at) {
- // we are never sure that the next scrub will indeed be shallow:
- return fmt::format("queued for {}scrub", (is_deep_expected ? "deep " : ""));
- }
-
- return fmt::format("{}scrub scheduled @ {:s}",
- (is_deep_expected ? "deep " : ""),
- schedule.scheduled_at);
-}
// ////////////////////////////////////////////////////////////////////////// //
}
}
+
/*
* Modify the scrub job state:
* - if 'registered' (as expected): mark as 'unregistering'. The job will be
*
* Note: not holding the jobs lock
*/
-void ScrubQueue::remove_from_osd_queue(ScrubJobRef scrub_job)
+void ScrubQueue::remove_from_osd_queue(Scrub::ScrubJobRef scrub_job)
{
dout(15) << "removing pg[" << scrub_job->pgid << "] from OSD scrub queue"
<< dendl;
if (ret) {
dout(10) << "pg[" << scrub_job->pgid << "] sched-state changed from "
- << qu_state_text(expected_state) << " to "
- << qu_state_text(scrub_job->state) << dendl;
+ << ScrubJob::qu_state_text(expected_state) << " to "
+ << ScrubJob::qu_state_text(scrub_job->state) << dendl;
} else {
// job wasn't in state 'registered' coming in
dout(5) << "removing pg[" << scrub_job->pgid
- << "] failed. State was: " << qu_state_text(expected_state)
+ << "] failed. State was: " << ScrubJob::qu_state_text(expected_state)
<< dendl;
}
}
void ScrubQueue::register_with_osd(
- ScrubJobRef scrub_job,
- const ScrubQueue::sched_params_t& suggested)
+ Scrub::ScrubJobRef scrub_job,
+ const sched_params_t& suggested)
{
qu_state_t state_at_entry = scrub_job->state.load();
dout(20) << fmt::format(
}
// look mommy - no locks!
-void ScrubQueue::update_job(ScrubJobRef scrub_job,
- const ScrubQueue::sched_params_t& suggested)
+void ScrubQueue::update_job(Scrub::ScrubJobRef scrub_job,
+ const sched_params_t& suggested)
{
// adjust the suggested scrub time according to OSD-wide status
auto adjusted = adjust_target_time(suggested);
scrub_job->update_schedule(adjusted);
}
-ScrubQueue::sched_params_t ScrubQueue::determine_scrub_time(
+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;
+ sched_params_t res;
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;
+ res.is_must = Scrub::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;
+ res.is_must = Scrub::must_scrub_t::mandatory;
} else {
res.proposed_time = pg_info.history.last_scrub_stamp;
// remote resources. Move it to the secondary scrub queue.
dout(15) << "moving " << sjob->pgid
- << " state: " << ScrubQueue::qu_state_text(sjob->state) << dendl;
+ << " state: " << ScrubJob::qu_state_text(sjob->state) << dendl;
// determine the penalty time, after which the job should be reinstated
utime_t after = now_is;
return "(unknown)"sv;
}
-std::string_view ScrubQueue::qu_state_text(qu_state_t st)
+std::string_view ScrubJob::qu_state_text(qu_state_t st)
{
switch (st) {
case qu_state_t::not_registered: return "not registered w/ OSD"sv;
namespace {
struct cmp_sched_time_t {
- bool operator()(const ScrubQueue::ScrubJobRef& lhs,
- const ScrubQueue::ScrubJobRef& rhs) const
+ bool operator()(const Scrub::ScrubJobRef& lhs,
+ const Scrub::ScrubJobRef& rhs) const
{
return lhs->schedule.scheduled_at < rhs->schedule.scheduled_at;
}
} // namespace
// called under lock
-ScrubQueue::ScrubQContainer ScrubQueue::collect_ripe_jobs(
+ScrubQContainer ScrubQueue::collect_ripe_jobs(
ScrubQContainer& group,
utime_t time_now)
{
rm_unregistered_jobs(group);
// copy ripe jobs
- ScrubQueue::ScrubQContainer ripes;
+ ScrubQContainer ripes;
ripes.reserve(group.size());
std::copy_if(group.begin(),
}
-ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
+Scrub::scrub_schedule_t ScrubQueue::adjust_target_time(
const sched_params_t& times) const
{
- ScrubQueue::scrub_schedule_t sched_n_dead{
+ Scrub::scrub_schedule_t sched_n_dead{
times.proposed_time, times.proposed_time};
- if (times.is_must == ScrubQueue::must_scrub_t::not_mandatory) {
+ if (times.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
return sched_n_dead;
}
+
// note: called with jobs_lock held
void ScrubQueue::scan_penalized(bool forgive_all, utime_t time_now)
{
}
}
-void ScrubQueue::ScrubJob::dump(ceph::Formatter* f) const
+void ScrubJob::dump(ceph::Formatter* f) const
{
f->open_object_section("scrub");
f->dump_stream("pgid") << pgid;
f->close_section();
}
-ScrubQueue::ScrubQContainer ScrubQueue::list_registered_jobs() const
+ScrubQContainer ScrubQueue::list_registered_jobs() const
{
- ScrubQueue::ScrubQContainer all_jobs;
+ ScrubQContainer all_jobs;
all_jobs.reserve(to_scrub.size() + penalized.size());
dout(20) << " size: " << all_jobs.capacity() << dendl;
*/
// clang-format on
-#include <atomic>
-#include <chrono>
-#include <memory>
#include <optional>
-#include <vector>
-
-#include "common/RefCountedObj.h"
-#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"
-
+#include "scrub_job.h"
class PG;
namespace Scrub {
} // namespace Scrub
+
/**
* the queue of PGs waiting to be scrubbed.
* Main operations are scheduling/unscheduling a PG to be scrubbed at a certain
*/
class ScrubQueue {
public:
- enum class must_scrub_t { not_mandatory, mandatory };
-
- enum class qu_state_t {
- not_registered, // not a primary, thus not considered for scrubbing by this
- // OSD (also the temporary state when just created)
- registered, // in either of the two queues ('to_scrub' or 'penalized')
- unregistering // in the process of being unregistered. Will be finalized
- // under lock
- };
-
ScrubQueue(CephContext* cct, Scrub::ScrubSchedListener& osds);
virtual ~ScrubQueue() = default;
-#include "osd/scrubber/scrub_job.h"
-
friend class TestOSDScrub;
friend class ScrubSchedTestWrapper; ///< unit-tests structure
-
- using ScrubJobRef = ceph::ref_t<ScrubJob>;
- using ScrubQContainer = std::vector<ScrubJobRef>;
-
- static std::string_view qu_state_text(qu_state_t st);
+ using sched_params_t = Scrub::sched_params_t;
/**
* called periodically by the OSD to select the first scrub-eligible PG
* remove the pg from set of PGs to be scanned for scrubbing.
* To be used if we are no longer the PG's primary, or if the PG is removed.
*/
- void remove_from_osd_queue(ScrubJobRef sjob);
+ void remove_from_osd_queue(Scrub::ScrubJobRef sjob);
/**
* @return the list (not std::set!) of all scrub jobs registered
* (apart from PGs in the process of being removed)
*/
- ScrubQContainer list_registered_jobs() const;
+ Scrub::ScrubQContainer list_registered_jobs() const;
/**
* Add the scrub job to the list of jobs (i.e. list of PGs) to be periodically
*
* locking: might lock jobs_lock
*/
- void register_with_osd(ScrubJobRef sjob, const sched_params_t& suggested);
+ void register_with_osd(Scrub::ScrubJobRef sjob, const sched_params_t& suggested);
/**
* modify a scrub-job's scheduled time and deadline
*
* locking: not using the jobs_lock
*/
- void update_job(ScrubJobRef sjob, const sched_params_t& suggested);
+ void update_job(Scrub::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,
*/
mutable ceph::mutex jobs_lock = ceph::make_mutex("ScrubQueue::jobs_lock");
- ScrubQContainer to_scrub; ///< scrub jobs (i.e. PGs) to scrub
- ScrubQContainer penalized; ///< those that failed to reserve remote resources
+ Scrub::ScrubQContainer to_scrub; ///< scrub jobs (i.e. PGs) to scrub
+ Scrub::ScrubQContainer penalized; ///< those that failed to reserve remote resources
bool restore_penalized{false};
double daily_loadavg{0.0};
static inline constexpr auto registered_job = [](const auto& jobref) -> bool {
- return jobref->state == qu_state_t::registered;
+ return jobref->state == Scrub::qu_state_t::registered;
};
static inline constexpr auto invalid_state = [](const auto& jobref) -> bool {
- return jobref->state == qu_state_t::not_registered;
+ return jobref->state == Scrub::qu_state_t::not_registered;
};
/**
* clear dead entries (unregistered, or belonging to removed PGs) from a
* queue. Job state is changed to match new status.
*/
- void rm_unregistered_jobs(ScrubQContainer& group);
+ void rm_unregistered_jobs(Scrub::ScrubQContainer& group);
/**
* the set of all scrub jobs in 'group' which are ready to be scrubbed
*
* Note that the returned container holds independent refs to the
* scrub jobs.
+ * Note also that OSDRestrictions is 1L size, thus copied.
*/
- ScrubQContainer collect_ripe_jobs(ScrubQContainer& group, utime_t time_now);
+ Scrub::ScrubQContainer collect_ripe_jobs(Scrub::ScrubQContainer& group, utime_t time_now);
/// scrub resources management lock (guarding scrubs_local & scrubs_remote)
*
* @return a pair of values: the determined scrub time, and the deadline
*/
- scrub_schedule_t adjust_target_time(
- const sched_params_t& recomputed_params) const;
+ Scrub::scrub_schedule_t adjust_target_time(
+ const Scrub::sched_params_t& recomputed_params) const;
/**
* Look for scrub jobs that have their 'resources_failure' set. These jobs
void move_failed_pgs(utime_t now_is);
Scrub::schedule_result_t select_from_group(
- ScrubQContainer& group,
+ Scrub::ScrubQContainer& group,
const Scrub::OSDRestrictions& preconds,
utime_t now_is);
*/
virtual utime_t time_now() const { return ceph_clock_now(); }
};
-
-template <>
-struct fmt::formatter<ScrubQueue::qu_state_t>
- : fmt::formatter<std::string_view> {
- template <typename FormatContext>
- auto format(const ScrubQueue::qu_state_t& s, FormatContext& ctx)
- {
- auto out = ctx.out();
- out = fmt::formatter<string_view>::format(
- std::string{ScrubQueue::qu_state_text(s)}, ctx);
- return out;
- }
-};
-
-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(),
- "pg[{}] @ {:s} (dl:{:s}) - <{}> / failure: {} / pen. t.o.: {:s} / queue "
- "state: {:.7}",
- sjob.pgid, sjob.schedule.scheduled_at, sjob.schedule.deadline,
- sjob.registration_state(), sjob.resources_failure, sjob.penalty_timeout,
- sjob.state.load(std::memory_order_relaxed));
- }
-};
-
false /* is periodic? unknown, actually */};
}
}
- if (m_scrub_job->state != ScrubQueue::qu_state_t::registered) {
+ if (m_scrub_job->state != Scrub::qu_state_t::registered) {
return pg_scrubbing_status_t{utime_t{},
0,
pg_scrub_sched_status_t::not_queued,
m_fsm = std::make_unique<ScrubMachine>(m_pg, this);
m_fsm->initiate();
- m_scrub_job = ceph::make_ref<ScrubQueue::ScrubJob>(m_osds->cct,
- m_pg->pg_id,
- m_osds->get_nodeid());
+ m_scrub_job = ceph::make_ref<Scrub::ScrubJob>(
+ m_osds->cct, m_pg->pg_id, m_osds->get_nodeid());
}
void PgScrubber::set_scrub_begin_time()
ReplicaReservations::ReplicaReservations(
PG* pg,
pg_shard_t whoami,
- ScrubQueue::ScrubJobRef scrubjob,
+ Scrub::ScrubJobRef scrubjob,
const ConfigProxy& conf)
: m_pg{pg}
, m_acting_set{pg->get_actingset()}
bool m_had_rejections{false};
int m_pending{-1};
const pg_info_t& m_pg_info;
- ScrubQueue::ScrubJobRef m_scrub_job; ///< a ref to this PG's scrub job
+ Scrub::ScrubJobRef m_scrub_job; ///< a ref to this PG's scrub job
const ConfigProxy& m_conf;
// detecting slow peers (see 'slow-secondary' above)
ReplicaReservations(PG* pg,
pg_shard_t whoami,
- ScrubQueue::ScrubJobRef scrubjob,
+ Scrub::ScrubJobRef scrubjob,
const ConfigProxy& conf);
~ReplicaReservations();
virtual void _scrub_clear_state() {}
utime_t m_scrub_reg_stamp; ///< stamp we registered for
- ScrubQueue::ScrubJobRef m_scrub_job; ///< the scrub-job used by the OSD to
+ Scrub::ScrubJobRef m_scrub_job; ///< the scrub-job used by the OSD to
///< schedule us
ostream& show(ostream& out) const override;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "./scrub_job.h"
+
+using qu_state_t = Scrub::qu_state_t;
+using must_scrub_t = Scrub::must_scrub_t;
+using ScrubQContainer = Scrub::ScrubQContainer;
+using sched_params_t = Scrub::sched_params_t;
+using OSDRestrictions = Scrub::OSDRestrictions;
+using ScrubJob = Scrub::ScrubJob;
+
+
+// ////////////////////////////////////////////////////////////////////////// //
+// ScrubJob
+
+#define dout_subsys ceph_subsys_osd
+#undef dout_context
+#define dout_context (cct)
+#undef dout_prefix
+#define dout_prefix _prefix_fn(_dout, this, __func__)
+
+template <class T>
+static std::ostream& _prefix_fn(std::ostream* _dout, T* t, std::string fn = "")
+{
+ return t->gen_prefix(*_dout, fn);
+}
+
+ScrubJob::ScrubJob(CephContext* cct, const spg_t& pg, int node_id)
+ : RefCountedObject{cct}
+ , pgid{pg}
+ , whoami{node_id}
+ , cct{cct}
+ , log_msg_prefix{fmt::format("osd.{}: scrub-job:pg[{}]:", node_id, pgid)}
+{}
+
+// debug usage only
+namespace std {
+ostream& operator<<(ostream& out, const ScrubJob& sjob)
+{
+ return out << fmt::format("{}", sjob);
+}
+} // namespace std
+
+void ScrubJob::update_schedule(const Scrub::scrub_schedule_t& adjusted)
+{
+ schedule = adjusted;
+ penalty_timeout = utime_t(0, 0); // helps with debugging
+
+ // '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: {:s} ({})", schedule.scheduled_at,
+ registration_state())
+ << dendl;
+}
+
+std::string ScrubJob::scheduling_state(utime_t now_is, bool is_deep_expected)
+ const
+{
+ // if not in the OSD scheduling queues, not a candidate for scrubbing
+ if (state != qu_state_t::registered) {
+ return "no scrub is scheduled";
+ }
+
+ // if the time has passed, we are surely in the queue
+ // (note that for now we do not tell client if 'penalized')
+ if (now_is > schedule.scheduled_at) {
+ // we are never sure that the next scrub will indeed be shallow:
+ return fmt::format("queued for {}scrub", (is_deep_expected ? "deep " : ""));
+ }
+
+ return fmt::format(
+ "{}scrub scheduled @ {:s}", (is_deep_expected ? "deep " : ""),
+ schedule.scheduled_at);
+}
+
+std::ostream& ScrubJob::gen_prefix(std::ostream& out, std::string_view fn) const
+{
+ return out << log_msg_prefix << fn << ": ";
+}
// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
// vim: ts=8 sw=2 smarttab
+#pragma once
+
+#include <atomic>
+#include <chrono>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+#include "common/RefCountedObj.h"
+#include "common/ceph_atomic.h"
+#include "include/utime_fmt.h"
+#include "osd/osd_types.h"
+#include "osd/osd_types_fmt.h"
+#include "osd/scrubber_common.h"
+
+
+namespace Scrub {
+
+enum class must_scrub_t { not_mandatory, mandatory };
+
+enum class qu_state_t {
+ not_registered, // not a primary, thus not considered for scrubbing by this
+ // OSD (also the temporary state when just created)
+ registered, // in either of the two queues ('to_scrub' or 'penalized')
+ unregistering // in the process of being unregistered. Will be finalized
+ // under lock
+};
struct scrub_schedule_t {
utime_t scheduled_at{};
utime_t deadline{0, 0};
};
-
struct sched_params_t {
utime_t proposed_time{};
double min_interval{0.0};
double max_interval{0.0};
- must_scrub_t is_must{ScrubQueue::must_scrub_t::not_mandatory};
+ must_scrub_t is_must{must_scrub_t::not_mandatory};
};
- struct ScrubJob final : public RefCountedObject {
-
+class ScrubJob final : public RefCountedObject {
+ public:
/**
* a time scheduled for scrub, and a deadline: The scrub could be delayed
* if system load is too high (but not if after the deadline),or if trying
utime_t get_sched_time() const { return schedule.scheduled_at; }
+ static std::string_view qu_state_text(qu_state_t st);
/**
* relatively low-cost(*) access to the scrub job's state, to be used in
*/
std::string_view state_desc() const
{
- return ScrubQueue::qu_state_text(state.load(std::memory_order_relaxed));
+ return qu_state_text(state.load(std::memory_order_relaxed));
}
- void update_schedule(const ScrubQueue::scrub_schedule_t& adjusted);
+ void update_schedule(const scrub_schedule_t& adjusted);
void dump(ceph::Formatter* f) const;
*/
std::string scheduling_state(utime_t now_is, bool is_deep_expected) const;
- friend std::ostream& operator<<(std::ostream& out, const ScrubJob& pg);
+ std::ostream& gen_prefix(std::ostream& out, std::string_view fn) const;
+ const std::string log_msg_prefix;
+};
+
+using ScrubJobRef = ceph::ref_t<ScrubJob>;
+using ScrubQContainer = std::vector<ScrubJobRef>;
+} // namespace Scrub
+
+namespace std {
+std::ostream& operator<<(std::ostream& out, const Scrub::ScrubJob& pg);
+} // namespace std
+
+namespace fmt {
+template <>
+struct formatter<Scrub::qu_state_t> : formatter<std::string_view> {
+ template <typename FormatContext>
+ auto format(const Scrub::qu_state_t& s, FormatContext& ctx)
+ {
+ auto out = ctx.out();
+ out = fmt::formatter<string_view>::format(
+ std::string{Scrub::ScrubJob::qu_state_text(s)}, ctx);
+ return out;
+ }
+};
+
+template <>
+struct formatter<Scrub::ScrubJob> {
+ constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+ template <typename FormatContext>
+ auto format(const Scrub::ScrubJob& sjob, FormatContext& ctx)
+ {
+ return fmt::format_to(
+ ctx.out(),
+ "pg[{}] @ {:s} (dl:{:s}) - <{}> / failure: {} / pen. t.o.: {:s} / "
+ "queue "
+ "state: {:.7}",
+ sjob.pgid, sjob.schedule.scheduled_at, sjob.schedule.deadline,
+ sjob.registration_state(), sjob.resources_failure, sjob.penalty_timeout,
+ sjob.state.load(std::memory_order_relaxed));
+ }
};
+} // namespace fmt
}
using schedule_result_t = Scrub::schedule_result_t;
-using ScrubJobRef = ScrubQueue::ScrubJobRef;
-using qu_state_t = ScrubQueue::qu_state_t;
+using ScrubJobRef = Scrub::ScrubJobRef;
+using qu_state_t = Scrub::qu_state_t;
+using scrub_schedule_t = Scrub::scrub_schedule_t;
+using ScrubQContainer = Scrub::ScrubQContainer;
/// enabling access into ScrubQueue internals
class ScrubSchedTestWrapper : public ScrubQueue {
std::optional<double> pool_conf_max;
bool is_must;
bool is_need_auto;
- ScrubQueue::scrub_schedule_t initial_schedule;
+ scrub_schedule_t initial_schedule;
};
pg_info_t mocked_pg_info;
pool_opts_t mocked_pool_opts;
requested_scrub_t request_flags;
- ScrubQueue::ScrubJobRef job;
+ ScrubJobRef job;
};
class TestScrubSched : public ::testing::Test {
dyn_data.request_flags.need_auto = sjob_data.is_need_auto;
// create the scrub job
- dyn_data.job = ceph::make_ref<ScrubQueue::ScrubJob>(g_ceph_context,
+ dyn_data.job = ceph::make_ref<Scrub::ScrubJob>(g_ceph_context,
sjob_data.spg,
m_osd_num);
m_scrub_jobs.push_back(dyn_data);
}
void debug_print_jobs(std::string hdr,
- const ScrubQueue::ScrubQContainer& jobs)
+ const ScrubQContainer& jobs)
{
std::cout << fmt::format("{}: time now {}", hdr, m_sched->time_now())
<< std::endl;
std::nullopt, // max scrub delay in pool config
false, // must-scrub
false, // need-auto
- ScrubQueue::scrub_schedule_t{} // initial schedule
+ scrub_schedule_t{} // initial schedule
},
{spg_t{pg_t{4, 1}},
std::nullopt,
true,
false,
- ScrubQueue::scrub_schedule_t{}},
+ scrub_schedule_t{}},
{spg_t{pg_t{7, 1}},
true,
std::nullopt,
false,
false,
- ScrubQueue::scrub_schedule_t{}},
+ scrub_schedule_t{}},
{spg_t{pg_t{5, 1}},
true,
std::nullopt,
false,
false,
- ScrubQueue::scrub_schedule_t{}}};
+ scrub_schedule_t{}}};
} // anonymous namespace