Merge pull request #42780 from ronen-fr/wip-ronenf-unique-scrub

osd/scrub: mark PG as being scrubbed, from scrub initiation to Inactive state

Reviewed-by: Samuel Just <sjust@redhat.com>
Reviewed-by: Neha Ojha <nojha@redhat.com>
This commit is contained in:
Ronen Friedman 2021-11-18 16:08:07 +02:00 committed by GitHub
commit df658feae2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 197 additions and 101 deletions

View File

@ -7542,7 +7542,7 @@ Scrub::schedule_result_t OSDService::initiate_a_scrub(spg_t pgid,
}
// 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;

View File

@ -201,7 +201,6 @@ PG::PG(OSDService *o, OSDMapRef curmap,
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),
@ -433,19 +432,16 @@ void PG::queue_scrub_after_repair()
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);
}
@ -633,8 +629,8 @@ void PG::release_backoffs(const hobject_t& begin, const hobject_t& end)
auto q = p->second.begin();
while (q != p->second.end()) {
dout(20) << __func__ << " checking " << *q << dendl;
int r = cmp((*q)->begin, begin);
if (r == 0 || (r > 0 && (*q)->end < end)) {
int rr = cmp((*q)->begin, begin);
if (rr == 0 || (rr > 0 && (*q)->end < end)) {
bv.push_back(*q);
q = p->second.erase(q);
} else {
@ -760,7 +756,7 @@ void PG::set_probe_targets(const set<pg_shard_t> &probe_set)
void PG::send_cluster_message(
int target, MessageRef m,
epoch_t epoch, bool share_map_update=false)
epoch_t epoch, bool share_map_update)
{
ConnectionRef con = osd->get_con_osd_cluster(
target, get_osdmap_epoch());
@ -818,13 +814,12 @@ void PG::publish_stats_to_osd()
if (!is_primary())
return;
if (m_scrubber) {
recovery_state.update_stats_wo_resched(
[scrubber = m_scrubber.get()](pg_history_t& hist,
pg_stat_t& info) mutable -> void {
info.scrub_sched_status = scrubber->get_schedule();
});
}
ceph_assert(m_scrubber);
recovery_state.update_stats_wo_resched(
[scrubber = m_scrubber.get()](pg_history_t& hist,
pg_stat_t& info) mutable -> void {
info.scrub_sched_status = scrubber->get_schedule();
});
std::lock_guard l{pg_stats_publish_lock};
auto stats =
@ -1337,18 +1332,16 @@ Scrub::schedule_result_t PG::sched_scrub()
<< (is_active() ? ") <active>" : ") <not-active>")
<< (is_clean() ? " <clean>" : " <not-clean>") << dendl;
ceph_assert(ceph_mutex_is_locked(_lock));
ceph_assert(m_scrubber);
if (is_scrub_queued_or_active()) {
return Scrub::schedule_result_t::already_started;
}
if (!is_primary() || !is_active() || !is_clean()) {
return Scrub::schedule_result_t::bad_pg_state;
}
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;
}
// analyse 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.
@ -1379,8 +1372,7 @@ Scrub::schedule_result_t PG::sched_scrub()
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;
}
@ -1485,14 +1477,16 @@ bool PG::verify_periodic_scrub_mode(bool allow_deep_scrub,
std::optional<requested_scrub_t> PG::verify_scrub_mode() const
{
bool allow_regular_scrub = !(get_osdmap()->test_flag(CEPH_OSDMAP_NOSCRUB) ||
pool.info.has_flag(pg_pool_t::FLAG_NOSCRUB));
bool allow_deep_scrub = allow_regular_scrub &&
!(get_osdmap()->test_flag(CEPH_OSDMAP_NODEEP_SCRUB) ||
pool.info.has_flag(pg_pool_t::FLAG_NODEEP_SCRUB));
bool has_deep_errors = (info.stats.stats.sum.num_deep_scrub_errors > 0);
bool try_to_auto_repair = (cct->_conf->osd_scrub_auto_repair &&
get_pgbackend()->auto_repair_supported());
const bool allow_regular_scrub =
!(get_osdmap()->test_flag(CEPH_OSDMAP_NOSCRUB) ||
pool.info.has_flag(pg_pool_t::FLAG_NOSCRUB));
const bool allow_deep_scrub =
allow_regular_scrub &&
!(get_osdmap()->test_flag(CEPH_OSDMAP_NODEEP_SCRUB) ||
pool.info.has_flag(pg_pool_t::FLAG_NODEEP_SCRUB));
const bool has_deep_errors = (info.stats.stats.sum.num_deep_scrub_errors > 0);
const bool try_to_auto_repair = (cct->_conf->osd_scrub_auto_repair &&
get_pgbackend()->auto_repair_supported());
dout(10) << __func__ << " pg: " << info.pgid
<< " allow: " << allow_regular_scrub << "/" << allow_deep_scrub
@ -1509,18 +1503,21 @@ std::optional<requested_scrub_t> PG::verify_scrub_mode() const
upd_flags.auto_repair = false;
if (upd_flags.must_scrub && !upd_flags.must_deep_scrub && has_deep_errors) {
osd->clog->error() << "osd." << osd->whoami << " pg " << info.pgid
<< " Regular scrub request, deep-scrub details will be lost";
osd->clog->error()
<< "osd." << osd->whoami << " pg " << info.pgid
<< " Regular scrub request, deep-scrub details will be lost";
}
if (!upd_flags.must_scrub) {
// All periodic scrub handling goes here because must_scrub is
// always set for must_deep_scrub and must_repair.
bool can_start_periodic =
verify_periodic_scrub_mode(allow_deep_scrub, try_to_auto_repair,
allow_regular_scrub, has_deep_errors, upd_flags);
const bool can_start_periodic = verify_periodic_scrub_mode(
allow_deep_scrub, try_to_auto_repair, allow_regular_scrub,
has_deep_errors, upd_flags);
if (!can_start_periodic) {
// "I don't want no scrub"
dout(20) << __func__ << ": no periodic scrubs allowed" << dendl;
return std::nullopt;
}
}
@ -1549,9 +1546,8 @@ void PG::on_info_history_change()
{
dout(20) << __func__ << " for a " << (is_primary() ? "Primary" : "non-primary") <<dendl;
if (m_scrubber) {
m_scrubber->on_maybe_registration_change(m_planned_scrub);
}
ceph_assert(m_scrubber);
m_scrubber->on_maybe_registration_change(m_planned_scrub);
}
void PG::reschedule_scrub()
@ -1559,7 +1555,8 @@ void PG::reschedule_scrub()
dout(20) << __func__ << " for a " << (is_primary() ? "Primary" : "non-primary") <<dendl;
// we are assuming no change in primary status
if (is_primary() && m_scrubber) {
if (is_primary()) {
ceph_assert(m_scrubber);
m_scrubber->update_scrub_job(m_planned_scrub);
}
}
@ -1567,18 +1564,17 @@ void PG::reschedule_scrub()
void PG::on_primary_status_change(bool was_primary, bool now_primary)
{
// make sure we have a working scrubber when becoming a primary
ceph_assert(m_scrubber || !now_primary);
if ((was_primary != now_primary) && m_scrubber) {
if (was_primary != now_primary) {
ceph_assert(m_scrubber);
m_scrubber->on_primary_change(m_planned_scrub);
}
}
void PG::scrub_requested(scrub_level_t scrub_level, scrub_type_t scrub_type)
{
if (m_scrubber) {
m_scrubber->scrub_requested(scrub_level, scrub_type, m_planned_scrub);
}
ceph_assert(m_scrubber);
m_scrubber->scrub_requested(scrub_level, scrub_type, m_planned_scrub);
}
void PG::clear_ready_to_merge() {
@ -1598,14 +1594,17 @@ void PG::on_role_change() {
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() {
@ -2101,7 +2100,8 @@ void PG::repair_object(
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
@ -2117,7 +2117,8 @@ void PG::forward_scrub_event(ScrubSafeAPI fn,
{
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
@ -2130,8 +2131,8 @@ void PG::forward_scrub_event(ScrubSafeAPI fn,
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,
@ -2140,7 +2141,6 @@ 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");
}

View File

@ -181,13 +181,20 @@ public:
const pg_shard_t pg_whoami;
const spg_t pg_id;
/// the 'scrubber'. Will be allocated in the derivative (PrimaryLogPG) ctor,
/// and be removed only in the PrimaryLogPG destructor.
std::unique_ptr<ScrubPgIF> m_scrubber;
/// flags detailing scheduling/operation characteristics of the next scrub
requested_scrub_t m_planned_scrub;
/// 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;
@ -384,7 +391,6 @@ public:
void scrub(epoch_t queued, ThreadPool::TPHandle& handle)
{
// a new scrub
scrub_queued = false;
forward_scrub_event(&ScrubPgIF::initiate_regular_scrub, queued, "StartScrub");
}
@ -397,7 +403,6 @@ public:
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");
}
@ -410,7 +415,6 @@ public:
Scrub::act_token_t act_token,
ThreadPool::TPHandle& handle)
{
scrub_queued = false;
forward_scrub_event(&ScrubPgIF::send_sched_replica, queued, act_token,
"SchedReplica");
}
@ -428,7 +432,6 @@ public:
void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle& handle)
{
scrub_queued = false;
forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub");
}
@ -808,7 +811,6 @@ protected:
/* 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;

View File

@ -12435,7 +12435,7 @@ void PrimaryLogPG::_applied_recovered_object(ObjectContextRef obc)
// requeue an active chunky scrub waiting on recovery ops
if (!recovery_state.is_deleting() && active_pushes == 0 &&
m_scrubber->is_scrub_active()) {
is_scrub_active()) {
osd->queue_scrub_pushes_update(this, is_scrub_blocking_ops());
}
@ -12449,7 +12449,7 @@ void PrimaryLogPG::_applied_recovered_object_replica()
// requeue an active scrub waiting on recovery ops
if (!recovery_state.is_deleting() && active_pushes == 0 &&
m_scrubber->is_scrub_active()) {
is_scrub_active()) {
osd->queue_scrub_replica_pushes(this, m_scrubber->replica_op_priority());
}
@ -15407,10 +15407,10 @@ bool PrimaryLogPG::already_complete(eversion_t v)
void PrimaryLogPG::do_replica_scrub_map(OpRequestRef op)
{
dout(15) << __func__ << " is scrub active? " << m_scrubber->is_scrub_active() << dendl;
dout(15) << __func__ << " is scrub active? " << is_scrub_active() << dendl;
op->mark_started();
if (!m_scrubber->is_scrub_active()) {
if (!is_scrub_active()) {
dout(10) << __func__ << " scrub isn't active" << dendl;
return;
}
@ -15501,7 +15501,7 @@ void PrimaryLogPG::SnapTrimmer::log_exit(const char *state_name, utime_t enter_t
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();
}
@ -15537,7 +15537,7 @@ boost::statechart::result PrimaryLogPG::NotTrimming::react(const KickTrim&)
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 {

View File

@ -549,7 +549,7 @@ void PrimaryLogScrub::scrub_snapshot_metadata(ScrubMap& scrubmap,
++num_digest_updates_pending;
ctx->register_on_success([this]() {
dout(20) << "updating scrub digest " << num_digest_updates_pending << dendl;
if (--num_digest_updates_pending <= 0) {
if ((num_digest_updates_pending >= 1) && (--num_digest_updates_pending == 0)) {
m_osds->queue_scrub_digest_update(m_pl_pg, m_pl_pg->is_scrub_blocking_ops());
}
});

View File

@ -17,6 +17,7 @@
#include "messages/MOSDScrubReserve.h"
#include "osd/OSD.h"
#include "osd/osd_types_fmt.h"
#include "ScrubStore.h"
#include "scrub_machine.h"
@ -186,6 +187,8 @@ void PgScrubber::initiate_regular_scrub(epoch_t epoch_queued)
reset_epoch(epoch_queued);
m_fsm->process_event(StartScrub{});
dout(10) << "scrubber event --<< StartScrub" << dendl;
} else {
clear_queued_or_active();
}
}
@ -198,6 +201,8 @@ void PgScrubber::initiate_scrub_after_repair(epoch_t epoch_queued)
reset_epoch(epoch_queued);
m_fsm->process_event(AfterRepairScrub{});
dout(10) << "scrubber event --<< AfterRepairScrub" << dendl;
} else {
clear_queued_or_active();
}
}
@ -1249,23 +1254,26 @@ Scrub::preemption_t& PgScrubber::get_preemptor()
}
/*
* Process note: called for the arriving "give me your map, replica!" request. Unlike
* the original implementation, we do not requeue the Op waiting for
* Process note: called for the arriving "give me your map, replica!" request.
* Unlike the original implementation, we do not requeue the Op waiting for
* updates. Instead - we trigger the FSM.
*/
void PgScrubber::replica_scrub_op(OpRequestRef op)
{
op->mark_started();
auto msg = op->get_req<MOSDRepScrub>();
dout(10) << __func__ << " pg:" << m_pg->pg_id << " Msg: map_epoch:" << msg->map_epoch
<< " min_epoch:" << msg->min_epoch << " deep?" << msg->deep << dendl;
dout(10) << __func__ << " pg:" << m_pg->pg_id
<< " Msg: map_epoch:" << msg->map_epoch
<< " min_epoch:" << msg->min_epoch << " deep?" << msg->deep << dendl;
// are we still processing a previous scrub-map request without noticing that the
// interval changed? won't see it here, but rather at the reservation stage.
// are we still processing a previous scrub-map request without noticing that
// the interval changed? won't see it here, but rather at the reservation
// stage.
if (msg->map_epoch < m_pg->info.history.same_interval_since) {
dout(10) << "replica_scrub_op discarding old replica_scrub from " << msg->map_epoch
<< " < " << m_pg->info.history.same_interval_since << dendl;
dout(10) << "replica_scrub_op discarding old replica_scrub from "
<< msg->map_epoch << " < "
<< m_pg->info.history.same_interval_since << dendl;
// is there a general sync issue? are we holding a stale reservation?
// not checking now - assuming we will actively react to interval change.
@ -1273,6 +1281,26 @@ void PgScrubber::replica_scrub_op(OpRequestRef op)
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{};
@ -1282,8 +1310,9 @@ void PgScrubber::replica_scrub_op(OpRequestRef op)
m_max_end = msg->end;
m_is_deep = msg->deep;
m_interval_start = m_pg->info.history.same_interval_since;
m_replica_request_priority = msg->high_priority ? Scrub::scrub_prio_t::high_priority
: Scrub::scrub_prio_t::low_priority;
m_replica_request_priority = msg->high_priority
? Scrub::scrub_prio_t::high_priority
: Scrub::scrub_prio_t::low_priority;
m_flags.priority = msg->priority ? msg->priority : m_pg->get_scrub_priority();
preemption_data.reset();
@ -1291,11 +1320,9 @@ void PgScrubber::replica_scrub_op(OpRequestRef op)
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)
@ -1674,6 +1701,20 @@ void PgScrubber::clear_reserving_now()
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()
{
@ -1726,6 +1767,7 @@ void PgScrubber::scrub_finish()
<< ". 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{};
@ -1881,9 +1923,10 @@ void PgScrubber::scrub_finish()
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
@ -1892,10 +1935,7 @@ void PgScrubber::on_digest_updates()
// 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();
@ -2200,6 +2240,7 @@ void PgScrubber::reset_internal_state()
m_sleep_started_at = utime_t{};
m_active = false;
clear_queued_or_active();
}
// note that only applicable to the Replica:

View File

@ -324,6 +324,8 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
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
*/
@ -392,6 +394,8 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
void on_digest_updates() final;
void scrub_finish() final;
ScrubMachineListener::MsgAndEpoch
prep_replica_map_msg(Scrub::PreemptionNoted was_preempted) final;
@ -424,6 +428,9 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
[[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;
@ -497,7 +504,7 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
// ----- methods used to verify the relevance of incoming events:
/**
* is the incoming event still relevant, and should be processed?
* is the incoming event still relevant and should be forwarded to the FSM?
*
* It isn't if:
* - (1) we are no longer 'actively scrubbing'; or
@ -506,7 +513,7 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
* - (3) the message epoch is from a previous interval; or
* - (4) the 'abort' configuration flags were set.
*
* For (1) & (2) - teh incoming message is discarded, w/o further action.
* For (1) & (2) - the incoming message is discarded, w/o further action.
*
* For (3): (see check_interval() for a full description) if we have not reacted yet
* to this specific new interval, we do now:
@ -562,9 +569,6 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
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;
@ -592,6 +596,7 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
const pg_shard_t m_pg_whoami; ///< a local copy of m_pg->pg_whoami;
epoch_t m_interval_start{0}; ///< interval's 'from' of when scrubbing was first scheduled
/*
* the exact epoch when the scrubbing actually started (started here - cleared checks
* for no-scrub conf). Incoming events are verified against this, with stale events
@ -612,6 +617,23 @@ class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
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;

View File

@ -92,6 +92,8 @@ std::ostream& ScrubMachine::gen_prefix(std::ostream& out) const
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&)
@ -157,6 +159,7 @@ ActiveScrubbing::~ActiveScrubbing()
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(15) << __func__ << dendl;
scrbr->unreserve_replicas();
scrbr->clear_queued_or_active();
}
/*
@ -436,7 +439,9 @@ sc::result WaitReplicas::react(const GotReplicas&)
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)
@ -453,7 +458,6 @@ sc::result WaitDigestUpdate::react(const DigestUpdate&)
// - finish the scrubbing of the current chunk, and:
// - send NextChunk, or
// - send ScrubFinished
scrbr->on_digest_updates();
return discard_event();
}
@ -463,6 +467,7 @@ sc::result WaitDigestUpdate::react(const ScrubFinished&)
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(10) << "WaitDigestUpdate::react(const ScrubFinished&)" << dendl;
scrbr->set_scrub_duration();
scrbr->scrub_finish();
return transit<NotActive>();
}

View File

@ -90,7 +90,7 @@ MEV(IntLocalMapDone)
MEV(DigestUpdate) ///< external. called upon success of a MODIFY op. See
///< scrub_snapshot_metadata()
MEV(MapsCompared) ///< (Crimson) maps_compare_n_cleanup() transactions are done
MEV(MapsCompared) ///< maps_compare_n_cleanup() transactions are done
MEV(StartReplica) ///< initiating replica scrub.

View File

@ -103,6 +103,9 @@ struct ScrubMachineListener {
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?
@ -161,6 +164,12 @@ struct ScrubMachineListener {
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.

View File

@ -178,6 +178,23 @@ struct ScrubPgIF {
*/
[[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;