}
// This has already started, so go on to the next scrub job
- if (pg->is_scrub_active()) {
+ 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;
info_struct_v(0),
pgmeta_oid(p.make_pgmeta_oid()),
stat_queue_item(this),
- scrub_queued(false),
recovery_queued(false),
recovery_ops_active(0),
backfill_reserving(false),
m_planned_scrub.check_repair = true;
m_planned_scrub.must_scrub = true;
- if (is_scrubbing()) {
- dout(10) << __func__ << ": scrubbing already" << dendl;
- return;
- }
- if (scrub_queued) {
- dout(10) << __func__ << ": already queued" << dendl;
+ 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;
- scrub_queued = true;
+ m_scrubber->set_queued_or_active();
osd->queue_scrub_after_repair(this, Scrub::scrub_prio_t::high_priority);
}
<< (is_clean() ? " <clean>" : " <not-clean>") << dendl;
ceph_assert(ceph_mutex_is_locked(_lock));
- if (!is_primary() || !is_active() || !is_clean()) {
- return Scrub::schedule_result_t::bad_pg_state;
+ if (is_scrub_queued_or_active()) {
+ return Scrub::schedule_result_t::already_started;
}
- if (scrub_queued) {
- // only applicable to the very first time a scrub event is queued
- // (until handled and posted to the scrub FSM)
- dout(10) << __func__ << ": already queued" << dendl;
- return Scrub::schedule_result_t::already_started;
+ if (!is_primary() || !is_active() || !is_clean()) {
+ return Scrub::schedule_result_t::bad_pg_state;
}
// analyse the combination of the requested scrub flags, the osd/pool configuration
m_scrubber->set_op_parameters(m_planned_scrub);
dout(10) << __func__ << ": queueing" << dendl;
-
- scrub_queued = true;
+ m_scrubber->set_queued_or_active();
osd->queue_for_scrub(this, Scrub::scrub_prio_t::low_priority);
return Scrub::schedule_result_t::scrub_initiated;
}
plpg_on_role_change();
}
-void PG::on_new_interval() {
- dout(20) << __func__ << " scrub_queued was " << scrub_queued << " flags: " << m_planned_scrub << dendl;
- scrub_queued = false;
+void PG::on_new_interval()
+{
projected_last_update = eversion_t();
cancel_recovery();
- if (m_scrubber) {
- m_scrubber->on_maybe_registration_change(m_planned_scrub);
- }
+
+ assert(m_scrubber);
+ // log some scrub data before we react to the interval
+ dout(20) << __func__ << (is_scrub_queued_or_active() ? " scrubbing " : " ")
+ << "flags: " << m_planned_scrub << dendl;
+
+ m_scrubber->on_maybe_registration_change(m_planned_scrub);
}
epoch_t PG::oldest_stored_osdmap() {
void PG::forward_scrub_event(ScrubAPI fn, epoch_t epoch_queued, std::string_view desc)
{
dout(20) << __func__ << ": " << desc << " queued at: " << epoch_queued << dendl;
- if (is_active() && m_scrubber) {
+ ceph_assert(m_scrubber);
+ if (is_active()) {
((*m_scrubber).*fn)(epoch_queued);
} else {
// pg might be in the process of being deleted
{
dout(20) << __func__ << ": " << desc << " queued: " << epoch_queued
<< " token: " << act_token << dendl;
- if (is_active() && m_scrubber) {
+ ceph_assert(m_scrubber);
+ if (is_active()) {
((*m_scrubber).*fn)(epoch_queued, act_token);
} else {
// pg might be in the process of being deleted
void PG::replica_scrub(OpRequestRef op, ThreadPool::TPHandle& handle)
{
dout(10) << __func__ << " (op)" << dendl;
- if (m_scrubber)
- m_scrubber->replica_scrub_op(op);
+ ceph_assert(m_scrubber);
+ m_scrubber->replica_scrub_op(op);
}
void PG::replica_scrub(epoch_t epoch_queued,
{
dout(10) << __func__ << " queued at: " << epoch_queued
<< (is_primary() ? " (primary)" : " (replica)") << dendl;
- scrub_queued = false;
forward_scrub_event(&ScrubPgIF::send_start_replica, epoch_queued, act_token,
"StartReplica/nw");
}
/// scrubbing state for both Primary & replicas
bool is_scrub_active() const { return m_scrubber->is_scrub_active(); }
+ /// set when the scrub request is queued, and reset after scrubbing fully
+ /// cleaned up.
+ bool is_scrub_queued_or_active() const { return m_scrubber->is_queued_or_active(); }
+
public:
// -- members --
const coll_t coll;
void scrub(epoch_t queued, ThreadPool::TPHandle& handle)
{
// a new scrub
- scrub_queued = false;
forward_scrub_event(&ScrubPgIF::initiate_regular_scrub, queued, "StartScrub");
}
void recovery_scrub(epoch_t queued, ThreadPool::TPHandle& handle)
{
// a new scrub
- scrub_queued = false;
forward_scrub_event(&ScrubPgIF::initiate_scrub_after_repair, queued,
"AfterRepairScrub");
}
Scrub::act_token_t act_token,
ThreadPool::TPHandle& handle)
{
- scrub_queued = false;
forward_scrub_event(&ScrubPgIF::send_sched_replica, queued, act_token,
"SchedReplica");
}
void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle& handle)
{
- scrub_queued = false;
forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub");
}
/* You should not use these items without taking their respective queue locks
* (if they have one) */
xlist<PG*>::item stat_queue_item;
- bool scrub_queued;
bool recovery_queued;
int recovery_ops_active;
bool PrimaryLogPG::SnapTrimmer::permit_trim() {
return
pg->is_clean() &&
- !pg->m_scrubber->is_scrub_active() &&
+ !pg->is_scrub_queued_or_active() &&
!pg->snap_trimq.empty();
}
ldout(pg->cct, 10) << "NotTrimming not clean or nothing to trim" << dendl;
return discard_event();
}
- if (pg->m_scrubber->is_scrub_active()) {
+ if (pg->is_scrub_queued_or_active()) {
ldout(pg->cct, 10) << " scrubbing, will requeue snap_trimmer after" << dendl;
return transit< WaitScrub >();
} else {
#include "messages/MOSDScrubReserve.h"
#include "osd/OSD.h"
+#include "osd/osd_types_fmt.h"
#include "ScrubStore.h"
#include "scrub_machine.h"
reset_epoch(epoch_queued);
m_fsm->process_event(StartScrub{});
dout(10) << "scrubber event --<< StartScrub" << dendl;
+ } else {
+ clear_queued_or_active();
}
}
reset_epoch(epoch_queued);
m_fsm->process_event(AfterRepairScrub{});
dout(10) << "scrubber event --<< AfterRepairScrub" << dendl;
+ } else {
+ clear_queued_or_active();
}
}
return;
}
+ if (is_queued_or_active()) {
+ // this is bug!
+ // Somehow, we have received a new scrub request from our Primary, before
+ // having finished with the previous one. Did we go through an interval
+ // change without reseting the FSM? Possible responses:
+ // - crashing (the original assert_not_active() implemented that one), or
+ // - trying to recover:
+ // - (logging enough information to debug this scenario)
+ // - reset the FSM.
+ m_osds->clog->warn() << fmt::format(
+ "{}: error: a second scrub-op received while handling the previous one",
+ __func__);
+
+ scrub_clear_state();
+ m_osds->clog->warn() << fmt::format(
+ "{}: after a reset. Now handling the new OP", __func__);
+ }
+ // make sure the FSM is at NotActive
+ m_fsm->assert_not_active();
+
replica_scrubmap = ScrubMap{};
replica_scrubmap_pos = ScrubMapBuilder{};
replica_scrubmap_pos.reset();
- // make sure the FSM is at NotActive
- m_fsm->assert_not_active();
-
- m_osds->queue_for_rep_scrub(m_pg, m_replica_request_priority, m_flags.priority,
- m_current_token);
+ set_queued_or_active();
+ m_osds->queue_for_rep_scrub(m_pg, m_replica_request_priority,
+ m_flags.priority, m_current_token);
}
void PgScrubber::set_op_parameters(requested_scrub_t& request)
m_osds->get_scrub_services().clear_reserving_now();
}
+void PgScrubber::set_queued_or_active()
+{
+ m_queued_or_active = true;
+}
+
+void PgScrubber::clear_queued_or_active()
+{
+ m_queued_or_active = false;
+}
+
+bool PgScrubber::is_queued_or_active() const
+{
+ return m_queued_or_active;
+}
[[nodiscard]] bool PgScrubber::scrub_process_inconsistent()
{
<< ". deep_scrub_on_error: " << m_flags.deep_scrub_on_error << dendl;
ceph_assert(m_pg->is_locked());
+ ceph_assert(is_queued_or_active());
m_pg->m_planned_scrub = requested_scrub_t{};
void PgScrubber::on_digest_updates()
{
- dout(10) << __func__ << " #pending: " << num_digest_updates_pending << " pending? "
- << num_digest_updates_pending
- << (m_end.is_max() ? " <last chunk> " : " <mid chunk> ") << dendl;
+ dout(10) << __func__ << " #pending: " << num_digest_updates_pending
+ << (m_end.is_max() ? " <last chunk>" : " <mid chunk>")
+ << (is_queued_or_active() ? "" : " ** not marked as scrubbing **")
+ << dendl;
if (num_digest_updates_pending > 0) {
// do nothing for now. We will be called again when new updates arrive
// got all updates, and finished with this chunk. Any more?
if (m_end.is_max()) {
-
- scrub_finish();
m_osds->queue_scrub_is_finished(m_pg);
-
} else {
// go get a new chunk (via "requeue")
preemption_data.reset();
m_sleep_started_at = utime_t{};
m_active = false;
+ clear_queued_or_active();
}
// note that only applicable to the Replica:
void scrub_clear_state() final;
+ bool is_queued_or_active() const final;
+
/**
* add to scrub statistics, but only if the soid is below the scrub start
*/
void on_digest_updates() final;
+ void scrub_finish() final;
+
ScrubMachineListener::MsgAndEpoch
prep_replica_map_msg(Scrub::PreemptionNoted was_preempted) final;
[[nodiscard]] bool was_epoch_changed() const final;
+ void set_queued_or_active() final;
+ void clear_queued_or_active() final;
+
void mark_local_map_ready() final;
[[nodiscard]] bool are_all_maps_available() const final;
void cleanup_on_finish(); // scrub_clear_state() as called for a Primary when
// Active->NotActive
- /// the part that actually finalizes a scrub
- void scrub_finish();
-
protected:
PG* const m_pg;
bool m_active{false};
+ /**
+ * a flag designed to prevent the initiation of a second scrub on a PG for which scrubbing
+ * has been initiated.
+ *
+ * set once scrubbing was initiated (i.e. - even before the FSM event that
+ * will trigger a state-change out of Inactive was handled), and only reset
+ * once the FSM is back in Inactive.
+ * In other words - its ON period encompasses:
+ * - the time period covered today by 'queued', and
+ * - the time when m_active is set, and
+ * - all the time from scrub_finish() calling update_stats() till the
+ * FSM handles the 'finished' event
+ *
+ * Compared with 'm_active', this flag is asserted earlier and remains ON for longer.
+ */
+ bool m_queued_or_active{false};
+
eversion_t m_subset_last_update{};
std::unique_ptr<Scrub::Store> m_store;
NotActive::NotActive(my_context ctx) : my_base(ctx)
{
dout(10) << "-- state -->> NotActive" << dendl;
+ DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
+ scrbr->clear_queued_or_active();
}
sc::result NotActive::react(const StartScrub&)
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(15) << __func__ << dendl;
scrbr->unreserve_replicas();
+ scrbr->clear_queued_or_active();
}
/*
WaitDigestUpdate::WaitDigestUpdate(my_context ctx) : my_base(ctx)
{
+ DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(10) << "-- state -->> Act/WaitDigestUpdate" << dendl;
+
// perform an initial check: maybe we already
// have all the updates we need:
// (note that DigestUpdate is usually an external event)
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(10) << "WaitDigestUpdate::react(const ScrubFinished&)" << dendl;
scrbr->set_scrub_duration();
+ scrbr->scrub_finish();
return transit<NotActive>();
}
virtual void on_digest_updates() = 0;
+ /// the part that actually finalizes a scrub
+ virtual void scrub_finish() = 0;
+
/**
* Prepare a MOSDRepScrubMap message carrying the requested scrub map
* @param was_preempted - were we preempted?
virtual void 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;
+ virtual void clear_queued_or_active() = 0;
+
/**
* the FSM interface into the "are we waiting for maps, either our own or from
* replicas" state.
*/
[[nodiscard]] virtual bool is_scrub_active() const = 0;
+ /**
+ * 'true' until after the FSM processes the 'scrub-finished' event,
+ * and scrubbing is completely cleaned-up.
+ *
+ * In other words - holds longer than is_scrub_active(), thus preventing
+ * a rescrubbing of the same PG while the previous scrub has not fully
+ * terminated.
+ */
+ [[nodiscard]] virtual bool is_queued_or_active() const = 0;
+
+ /**
+ * Manipulate the 'scrubbing request has been queued, or - we are
+ * actually scrubbing' Scrubber's flag
+ */
+ virtual void set_queued_or_active() = 0;
+ virtual void clear_queued_or_active() = 0;
+
/// are we waiting for resource reservation grants form our replicas?
[[nodiscard]] virtual bool is_reserving() const = 0;