From 1f327081ef6c9fc8c60b896d8c395ac46aef86fc Mon Sep 17 00:00:00 2001 From: Ronen Friedman Date: Sun, 28 Jul 2024 07:37:07 -0500 Subject: [PATCH] osd/scrub: modify after-repair-scrub triggering ... to manipulate the relevant scrub target directly, instead of using the 'planned scrub' flags. The relevant condition flag was moved from the PG and into the scrubber. Signed-off-by: Ronen Friedman --- src/osd/OSD.cc | 5 --- src/osd/OSD.h | 2 -- src/osd/PG.cc | 33 +++--------------- src/osd/PG.h | 16 --------- src/osd/PrimaryLogPG.cc | 7 ++-- src/osd/scheduler/OpSchedulerItem.cc | 9 ----- src/osd/scheduler/OpSchedulerItem.h | 12 ------- src/osd/scrubber/pg_scrubber.cc | 51 +++++++++++++++------------- src/osd/scrubber/pg_scrubber.h | 47 +++++++++++++++++++++++-- src/osd/scrubber/scrub_machine.cc | 8 ----- src/osd/scrubber/scrub_machine.h | 13 ++----- src/osd/scrubber_common.h | 33 ++++++++++-------- 12 files changed, 101 insertions(+), 135 deletions(-) diff --git a/src/osd/OSD.cc b/src/osd/OSD.cc index 0d392d1b6a3..fb3a415a542 100644 --- a/src/osd/OSD.cc +++ b/src/osd/OSD.cc @@ -1819,11 +1819,6 @@ void OSDService::queue_for_scrub(PG* pg, Scrub::scrub_prio_t with_priority) queue_scrub_event_msg_default_cost(pg, with_priority); } -void OSDService::queue_scrub_after_repair(PG* pg, Scrub::scrub_prio_t with_priority) -{ - queue_scrub_event_msg_default_cost(pg, with_priority); -} - void OSDService::queue_for_rep_scrub(PG* pg, Scrub::scrub_prio_t with_priority, unsigned int qu_priority, diff --git a/src/osd/OSD.h b/src/osd/OSD.h index 787827b8ff4..7c9aed7c6ba 100644 --- a/src/osd/OSD.h +++ b/src/osd/OSD.h @@ -514,8 +514,6 @@ public: void queue_for_snap_trim(PG *pg, uint64_t cost); void queue_for_scrub(PG* pg, Scrub::scrub_prio_t with_priority); - void queue_scrub_after_repair(PG* pg, Scrub::scrub_prio_t with_priority); - /// Signals either (a) the end of a sleep period, or (b) a recheck of the availability /// of the primary map being created by the backend. void queue_for_scrub_resched(PG* pg, Scrub::scrub_prio_t with_priority); diff --git a/src/osd/PG.cc b/src/osd/PG.cc index cda893397bd..f7a5033574f 100644 --- a/src/osd/PG.cc +++ b/src/osd/PG.cc @@ -205,7 +205,6 @@ PG::PG(OSDService *o, OSDMapRef curmap, recovery_ops_active(0), backfill_reserving(false), finish_sync_event(NULL), - scrub_after_recovery(false), active_pushes(0), recovery_state( o->cct, @@ -361,7 +360,6 @@ void PG::clear_primary_state() if (m_scrubber) { m_scrubber->on_new_interval(); } - scrub_after_recovery = false; agent_clear(); } @@ -427,27 +425,6 @@ void PG::queue_recovery() } } -void PG::queue_scrub_after_repair() -{ - dout(10) << __func__ << dendl; - ceph_assert(ceph_mutex_is_locked(_lock)); - - m_planned_scrub.must_deep_scrub = true; - m_planned_scrub.check_repair = true; - m_planned_scrub.must_scrub = true; - m_planned_scrub.calculated_to_deep = true; - - if (is_scrub_queued_or_active()) { - dout(10) << __func__ << ": scrubbing already (" - << (is_scrubbing() ? "active)" : "queued)") << dendl; - return; - } - - m_scrubber->set_op_parameters(m_planned_scrub); - dout(15) << __func__ << ": queueing" << dendl; - - osd->queue_scrub_after_repair(this, Scrub::scrub_prio_t::high_priority); -} unsigned PG::get_scrub_priority() { @@ -484,17 +461,15 @@ void PG::_finish_recovery(Context* c) // When recovery is initiated by a repair, that flag is left on state_clear(PG_STATE_REPAIR); if (c == finish_sync_event) { - dout(15) << __func__ << " scrub_after_recovery? " << scrub_after_recovery << dendl; + dout(15) << fmt::format("{}: scrub_after_recovery: {}", __func__, + m_scrubber->is_after_repair_required()) << dendl; finish_sync_event = 0; recovery_state.purge_strays(); publish_stats_to_osd(); - if (scrub_after_recovery) { - dout(10) << "_finish_recovery requeueing for scrub" << dendl; - scrub_after_recovery = false; - queue_scrub_after_repair(); - } + // notify the scrubber that recovery is done. This may trigger a scrub. + m_scrubber->recovery_completed(); } else { dout(10) << "_finish_recovery -- stale" << dendl; } diff --git a/src/osd/PG.h b/src/osd/PG.h index 5f3cd32e023..6bcb77ee5a9 100644 --- a/src/osd/PG.h +++ b/src/osd/PG.h @@ -425,19 +425,6 @@ public: forward_scrub_event(&ScrubPgIF::initiate_regular_scrub, queued, "StartScrub"); } - /** - * a special version of PG::scrub(), which: - * - is initiated after repair, and - * (not true anymore:) - * - is not required to allocate local/remote OSD scrub resources - */ - void recovery_scrub(epoch_t queued, ThreadPool::TPHandle& handle) - { - // a new scrub - forward_scrub_event(&ScrubPgIF::initiate_scrub_after_repair, queued, - "AfterRepairScrub"); - } - void replica_scrub(epoch_t queued, Scrub::act_token_t act_token, ThreadPool::TPHandle& handle); @@ -1219,8 +1206,6 @@ public: // -- scrub -- protected: - bool scrub_after_recovery; - int active_pushes; [[nodiscard]] bool ops_blocked_by_scrub() const; @@ -1354,7 +1339,6 @@ protected: virtual void snap_trimmer_scrub_complete() = 0; void queue_recovery(); - void queue_scrub_after_repair(); unsigned int get_scrub_priority(); bool try_flush_or_schedule_async() override; diff --git a/src/osd/PrimaryLogPG.cc b/src/osd/PrimaryLogPG.cc index 68c3a203bf9..1df284bcfc3 100644 --- a/src/osd/PrimaryLogPG.cc +++ b/src/osd/PrimaryLogPG.cc @@ -12384,7 +12384,10 @@ int PrimaryLogPG::recover_missing( int priority, PGBackend::RecoveryHandle *h) { - dout(10) << __func__ << " sar: " << scrub_after_recovery << dendl; + dout(10) << fmt::format( + "{} sar: {}", __func__, + m_scrubber->is_after_repair_required()) + << dendl; if (recovery_state.get_missing_loc().is_unfound(soid)) { dout(7) << __func__ << " " << soid @@ -12415,7 +12418,7 @@ int PrimaryLogPG::recover_missing( if (!object_missing) { object_stat_sum_t stat_diff; stat_diff.num_objects_recovered = 1; - if (scrub_after_recovery) + if (m_scrubber->is_after_repair_required()) stat_diff.num_objects_repaired = 1; on_global_recover(soid, stat_diff, true); } else { diff --git a/src/osd/scheduler/OpSchedulerItem.cc b/src/osd/scheduler/OpSchedulerItem.cc index ecad656bab9..1a643461888 100644 --- a/src/osd/scheduler/OpSchedulerItem.cc +++ b/src/osd/scheduler/OpSchedulerItem.cc @@ -59,15 +59,6 @@ void PGScrub::run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& ha pg->unlock(); } -void PGScrubAfterRepair::run(OSD* osd, - OSDShard* sdata, - PGRef& pg, - ThreadPool::TPHandle& handle) -{ - pg->recovery_scrub(epoch_queued, handle); - pg->unlock(); -} - void PGScrubResched::run(OSD* osd, OSDShard* sdata, PGRef& pg, diff --git a/src/osd/scheduler/OpSchedulerItem.h b/src/osd/scheduler/OpSchedulerItem.h index 0b28e64158e..d0281cf84e7 100644 --- a/src/osd/scheduler/OpSchedulerItem.h +++ b/src/osd/scheduler/OpSchedulerItem.h @@ -373,18 +373,6 @@ class PGScrubResched : public PGScrubItem { void run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& handle) final; }; -/** - * called when a repair process completes, to initiate scrubbing. No local/remote - * resources are allocated. - */ -class PGScrubAfterRepair : public PGScrubItem { - public: - PGScrubAfterRepair(spg_t pg, epoch_t epoch_queued) - : PGScrubItem{pg, epoch_queued, "PGScrubAfterRepair"} - {} - void run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& handle) final; -}; - class PGScrubPushesUpdate : public PGScrubItem { public: PGScrubPushesUpdate(spg_t pg, epoch_t epoch_queued) diff --git a/src/osd/scrubber/pg_scrubber.cc b/src/osd/scrubber/pg_scrubber.cc index 4b439bf8ec5..6c2dff17bd7 100644 --- a/src/osd/scrubber/pg_scrubber.cc +++ b/src/osd/scrubber/pg_scrubber.cc @@ -67,8 +67,6 @@ ostream& operator<<(ostream& out, const requested_scrub_t& sf) out << " MUST_REPAIR"; if (sf.auto_repair) out << " planned AUTO_REPAIR"; - if (sf.check_repair) - out << " planned CHECK_REPAIR"; if (sf.deep_scrub_on_error) out << " planned DEEP_SCRUB_ON_ERROR"; if (sf.must_deep_scrub) @@ -189,7 +187,7 @@ bool PgScrubber::should_abort() const /* * a note re the checks performed before sending scrub-initiating messages: * - * For those ('StartScrub', 'AfterRepairScrub') scrub-initiation messages that + * For those scrub-initiation messages ('StartScrub') that * possibly were in the queue while the PG changed state and became unavailable * for scrubbing: * @@ -228,22 +226,6 @@ void PgScrubber::advance_token() m_current_token++; } -void PgScrubber::initiate_scrub_after_repair(epoch_t epoch_queued) -{ - dout(10) << fmt::format( - "{}: epoch:{} is PrimaryIdle:{}", __func__, epoch_queued, - m_fsm->is_primary_idle()) - << dendl; - // we may have lost our Primary status while the message languished in the - // queue - if (check_interval(epoch_queued)) { - dout(10) << "scrubber event -->> AfterRepairScrub epoch: " << epoch_queued - << dendl; - m_fsm->process_event(AfterRepairScrub{}); - dout(10) << "scrubber event --<< AfterRepairScrub" << dendl; - } -} - void PgScrubber::send_scrub_unblock(epoch_t epoch_queued) { dout(10) << "scrubber event -->> " << __func__ << " epoch: " << epoch_queued @@ -470,6 +452,7 @@ void PgScrubber::on_new_interval() (is_primary() ? "Primary" : "Replica/other"), is_scrub_active(), is_queued_or_active()) << dendl; + m_after_repair_scrub_required = false; m_fsm->process_event(IntervalChanged{}); // the following asserts were added due to a past bug, where PG flags were // left set in some scenarios. @@ -707,6 +690,30 @@ scrub_level_t PgScrubber::scrub_requested( } +void PgScrubber::recovery_completed() +{ + dout(15) << fmt::format( + "{}: is scrub required? {}", __func__, + m_after_repair_scrub_required) + << dendl; + if (m_after_repair_scrub_required) { + m_after_repair_scrub_required = false; + m_osds->get_scrub_services().dequeue_target(m_pg_id, scrub_level_t::deep); + auto& trgt = m_scrub_job->get_target(scrub_level_t::deep); + trgt.up_urgency_to(urgency_t::after_repair); + trgt.sched_info.schedule.scheduled_at = {0, 0}; + trgt.sched_info.schedule.not_before = ceph_clock_now(); + m_osds->get_scrub_services().enqueue_target(trgt); + } +} + + +bool PgScrubber::is_after_repair_required() const +{ + return m_after_repair_scrub_required; +} + + void PgScrubber::request_rescrubbing(requested_scrub_t& request_flags) { dout(10) << __func__ << " flags: " << request_flags << dendl; @@ -1703,7 +1710,7 @@ void PgScrubber::set_op_parameters(const requested_scrub_t& request) // to discard stale messages from previous aborted scrubs. m_epoch_start = m_pg->get_osdmap_epoch(); - m_flags.check_repair = request.check_repair; + m_flags.check_repair = m_active_target->urgency() == urgency_t::after_repair; m_flags.auto_repair = request.auto_repair || request.need_auto; m_flags.required = request.req_scrub || request.must_scrub; @@ -2016,7 +2023,7 @@ void PgScrubber::scrub_finish() } else if (has_error) { // Deep scrub in order to get corrected error counts - m_pg->scrub_after_recovery = true; + m_after_repair_scrub_required = true; m_planned_scrub.req_scrub = m_planned_scrub.req_scrub || m_flags.required; dout(20) << __func__ << " Current 'required': " << m_flags.required @@ -2185,8 +2192,6 @@ void PgScrubber::on_mid_scrub_abort(Scrub::delay_cause_t issue) m_planned_scrub.need_auto = m_planned_scrub.need_auto || m_flags.auto_repair; m_planned_scrub.deep_scrub_on_error = m_planned_scrub.deep_scrub_on_error || m_flags.deep_scrub_on_error; - m_planned_scrub.check_repair = - m_planned_scrub.check_repair || m_flags.check_repair; // copy the aborted target const auto aborted_target = *m_active_target; diff --git a/src/osd/scrubber/pg_scrubber.h b/src/osd/scrubber/pg_scrubber.h index 0b114e54588..f657dcc690c 100644 --- a/src/osd/scrubber/pg_scrubber.h +++ b/src/osd/scrubber/pg_scrubber.h @@ -150,7 +150,8 @@ struct scrub_flags_t { bool auto_repair{false}; /// this flag indicates that we are scrubbing post repair to verify everything - /// is fixed + /// is fixed (otherwise - PG_STATE_FAILED_REPAIR will be asserted.) + /// Update (July 2024): now reflects an 'after-repair' urgency. bool check_repair{false}; /// checked at the end of the scrub, to possibly initiate a deep-scrub @@ -166,6 +167,35 @@ struct scrub_flags_t { ostream& operator<<(ostream& out, const scrub_flags_t& sf); +namespace fmt { +template <> +struct formatter { + constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); } + template + auto format(scrub_flags_t& sf, FormatContext& ctx) const + { + std::string txt; + bool sep{false}; + if (sf.auto_repair) { + txt = "auto-repair"; + sep = true; + } + if (sf.check_repair) { + txt += sep ? ",check-repair" : "check-repair"; + sep = true; + } + if (sf.deep_scrub_on_error) { + txt += sep ? ",deep-scrub-on-error" : "deep-scrub-on-error"; + sep = true; + } + if (sf.required) { + txt += sep ? ",required" : "required"; + } + return fmt::format_to(ctx.out(), "{}", txt); + } +}; +} // namespace fmt + /** * The part of PG-scrubbing code that isn't state-machine wiring. @@ -195,8 +225,6 @@ class PgScrubber : public ScrubPgIF, void initiate_regular_scrub(epoch_t epoch_queued) final; - void initiate_scrub_after_repair(epoch_t epoch_queued) final; - void send_scrub_resched(epoch_t epoch_queued) final; void active_pushes_notification(epoch_t epoch_queued) final; @@ -267,6 +295,14 @@ class PgScrubber : public ScrubPgIF, scrub_type_t scrub_type, requested_scrub_t& req_flags) final; + /** + * let the scrubber know that a recovery operation has completed. + * This might trigger an 'after repair' scrub. + */ + void recovery_completed() final; + + bool is_after_repair_required() const final; + /** * Reserve local scrub resources (managed by the OSD) * @@ -506,6 +542,11 @@ class PgScrubber : public ScrubPgIF, /// control the order of construction/destruction. std::optional m_scrub_job; + + /// the scrubber has initiated a recovery, and is waiting for the recovery + /// to complete (in order to perform an 'after-repair' scrub) + bool m_after_repair_scrub_required{false}; + ostream& show(ostream& out) const override; public: diff --git a/src/osd/scrubber/scrub_machine.cc b/src/osd/scrubber/scrub_machine.cc index 547a046d5bd..81e9bd7b6d8 100644 --- a/src/osd/scrubber/scrub_machine.cc +++ b/src/osd/scrubber/scrub_machine.cc @@ -159,14 +159,6 @@ sc::result PrimaryIdle::react(const StartScrub&) return transit(); } -sc::result PrimaryIdle::react(const AfterRepairScrub&) -{ - dout(10) << "PrimaryIdle::react(const AfterRepairScrub&)" << dendl; - DECLARE_LOCALS; - scrbr->reset_epoch(); - return transit(); -} - void PrimaryIdle::clear_state(const FullReset&) { dout(10) << "PrimaryIdle::react(const FullReset&): clearing state flags" << dendl; diff --git a/src/osd/scrubber/scrub_machine.h b/src/osd/scrubber/scrub_machine.h index cf8d28c765b..d1edfd37c98 100644 --- a/src/osd/scrubber/scrub_machine.h +++ b/src/osd/scrubber/scrub_machine.h @@ -163,9 +163,6 @@ MEV(RemotesReserved) /// initiate a new scrubbing session (relevant if we are a Primary) MEV(StartScrub) -/// initiate a new scrubbing session. Only triggered at Recovery completion -MEV(AfterRepairScrub) - /// triggered when the PG unblocked an object that was marked for scrubbing. /// Via the PGScrubUnblocked op MEV(Unblocked) @@ -472,11 +469,8 @@ struct PrimaryIdle; * The basic state for an active Primary. Ready to accept a new scrub request. * State managed here: being in the OSD's scrub queue (unless when scrubbing). * - * Scrubbing is triggered by one of the following events: - * - (standard scenario for a Primary): 'StartScrub'. Initiates the OSDs - * resources reservation process. Will be issued by PG::scrub(), following a - * queued "PGScrub" op. - * - a special end-of-recovery Primary scrub event ('AfterRepairScrub'). + * Scrubbing is triggered by a 'StartScrub' event, which is issued by + * PG::scrub(), following a queued "PGScrub" op. */ struct PrimaryActive : sc::state, NamedSimply { @@ -517,13 +511,10 @@ struct PrimaryIdle : sc::state, NamedSimply { using reactions = mpl::list< sc::custom_reaction, - // a scrubbing that was initiated at recovery completion: - sc::custom_reaction, // undoing set_op_params(), if aborted before starting the scrub: sc::in_state_reaction>; sc::result react(const StartScrub&); - sc::result react(const AfterRepairScrub&); }; /** diff --git a/src/osd/scrubber_common.h b/src/osd/scrubber_common.h index 60cb19a9af2..8e2e13c97a6 100644 --- a/src/osd/scrubber_common.h +++ b/src/osd/scrubber_common.h @@ -354,12 +354,6 @@ struct requested_scrub_t { */ bool auto_repair{false}; - /** - * indicating that we are scrubbing post repair to verify everything is fixed. - * Otherwise - PG_STATE_FAILED_REPAIR will be asserted. - */ - bool check_repair{false}; - /** * Used to indicate, both in client-facing listings and internally, that * the planned scrub will be a deep one. @@ -377,10 +371,9 @@ struct fmt::formatter { auto format(const requested_scrub_t& rs, FormatContext& ctx) { return fmt::format_to(ctx.out(), - "(plnd:{}{}{}{}{}{}{}{}{}{})", + "(plnd:{}{}{}{}{}{}{}{}{})", rs.must_repair ? " must_repair" : "", rs.auto_repair ? " auto_repair" : "", - rs.check_repair ? " check_repair" : "", rs.deep_scrub_on_error ? " deep_scrub_on_error" : "", rs.must_deep_scrub ? " must_deep_scrub" : "", rs.must_scrub ? " must_scrub" : "", @@ -409,8 +402,6 @@ struct ScrubPgIF { virtual void initiate_regular_scrub(epoch_t epoch_queued) = 0; - virtual void initiate_scrub_after_repair(epoch_t epoch_queued) = 0; - virtual void send_scrub_resched(epoch_t epoch_queued) = 0; virtual void active_pushes_notification(epoch_t epoch_queued) = 0; @@ -601,6 +592,23 @@ struct ScrubPgIF { */ virtual void update_scrub_job(Scrub::delay_ready_t delay_ready) = 0; + virtual scrub_level_t scrub_requested( + scrub_level_t scrub_level, + scrub_type_t scrub_type, + requested_scrub_t& req_flags) = 0; + + /** + * let the scrubber know that a recovery operation has completed. + * This might trigger an 'after repair' scrub. + */ + virtual void recovery_completed() = 0; + + /** + * m_after_repair_scrub_required is set, and recovery_complete() is + * expected to trigger a deep scrub + */ + virtual bool is_after_repair_required() const = 0; + // --------------- reservations ----------------------------------- @@ -612,11 +620,6 @@ struct ScrubPgIF { */ virtual void handle_scrub_reserve_msgs(OpRequestRef op) = 0; - virtual scrub_level_t scrub_requested( - scrub_level_t scrub_level, - scrub_type_t scrub_type, - requested_scrub_t& req_flags) = 0; - // --------------- debugging via the asok ------------------------------ virtual int asok_debug(std::string_view cmd, -- 2.39.5