Handle grant/deny messages within the FSM.
One exception at this point: the handling of "granted by everyone"
(due to the technical inconvenience of having to handle the
"0 replicas" case in the FSM state constructor).
Note: after this commit, ScrubMachineListener - an API which is
a subset of the Scrubber API to be used by the Scrubber FSM - does
no longer make sense. The FSM should now have full access to the
scrubber, and that interface will be removed in a subsequent PR.
Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
queue_scrub_event_msg<PGScrubResourcesOK>(pg, with_priority);
}
-void OSDService::queue_for_scrub_denied(PG* pg, Scrub::scrub_prio_t with_priority)
-{
- // Resulting scrub event: 'ReservationFailure'
- queue_scrub_event_msg<PGScrubDenied>(pg, with_priority);
-}
-
void OSDService::queue_for_scrub_resched(PG* pg, Scrub::scrub_prio_t with_priority)
{
// Resulting scrub event: 'InternalSchedScrub'
/// queue the message (-> event) that all replicas have reserved scrub resources for us
void queue_for_scrub_granted(PG* pg, Scrub::scrub_prio_t with_priority);
- /// queue the message (-> event) that some replicas denied our scrub resources request
- void queue_for_scrub_denied(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);
forward_scrub_event(&ScrubPgIF::send_remotes_reserved, queued, "RemotesReserved");
}
- void scrub_send_resources_denied(epoch_t queued, ThreadPool::TPHandle& handle)
- {
- forward_scrub_event(&ScrubPgIF::send_reservation_failure, queued,
- "ReservationFailure");
- }
-
void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle& handle)
{
forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub");
pg->unlock();
}
-void PGScrubDenied::run(OSD* osd,
- OSDShard* sdata,
- PGRef& pg,
- ThreadPool::TPHandle& handle)
-{
- pg->scrub_send_resources_denied(epoch_queued, handle);
- pg->unlock();
-}
-
void PGScrubPushesUpdate::run(OSD* osd,
OSDShard* sdata,
PGRef& pg,
void run(OSD* osd, OSDShard* sdata, PGRef& pg, ThreadPool::TPHandle& handle) final;
};
-/**
- * scrub resources requests denied by replica(s)
- */
-class PGScrubDenied : public PGScrubItem {
- public:
- PGScrubDenied(spg_t pg, epoch_t epoch_queued)
- : PGScrubItem{pg, epoch_queued, "PGScrubDenied"}
- {}
- 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.
dout(10) << "scrubber event --<< " << __func__ << dendl;
}
-void PgScrubber::send_reservation_failure(epoch_t epoch_queued)
-{
- dout(10) << "scrubber event -->> " << __func__ << " epoch: " << epoch_queued
- << dendl;
- if (check_interval(epoch_queued)) { // do not check for 'active'!
- m_fsm->process_event(ReservationFailure{});
- }
- dout(10) << "scrubber event --<< " << __func__ << dendl;
-}
-
void PgScrubber::send_chunk_free(epoch_t epoch_queued)
{
dout(10) << "scrubber event -->> " << __func__ << " epoch: " << epoch_queued
m_osds->send_message_osd_cluster(reply, op->get_req()->get_connection());
}
-
-/// temporary
-void PgScrubber::grant_from_replica(OpRequestRef op, pg_shard_t from)
-{
- dout(10) << fmt::format("{}: {}", __func__, *op->get_req()) << dendl;
- ceph_assert(m_reservations.has_value()); // the FSM should know
- m_reservations->handle_reserve_grant(op, from);
-}
-
-
-/// temporary
-void PgScrubber::reject_from_replica(OpRequestRef op, pg_shard_t from)
-{
- dout(10) << fmt::format("{}: {}", __func__, *op->get_req()) << dendl;
- ceph_assert(m_reservations.has_value()); // the FSM should know
- m_reservations->handle_reserve_reject(op, from);
-}
-
-
void PgScrubber::handle_scrub_reserve_release(OpRequestRef op)
{
dout(10) << __func__ << " " << *op->get_req() << dendl;
m_fsm->process_event(FullReset{});
}
-void PgScrubber::discard_replica_reservations()
-{
- dout(10) << __func__ << dendl;
- if (m_reservations.has_value()) {
- m_reservations->discard_remote_reservations();
- }
-}
-
-void PgScrubber::clear_scrub_reservations()
-{
- dout(10) << __func__ << dendl;
- m_reservations.reset(); // the remote reservations
- m_local_osd_resource.reset(); // the local reservation
-}
-
bool PgScrubber::set_reserving_now() {
return m_osds->get_scrub_services().set_reserving_now(m_pg_id,
ceph_clock_now());
});
}
-void PgScrubber::reserve_replicas()
-{
- dout(10) << __func__ << dendl;
- m_reservations.emplace(*this);
-}
-
void PgScrubber::cleanup_on_finish()
{
dout(10) << __func__ << dendl;
state_clear(PG_STATE_SCRUBBING);
state_clear(PG_STATE_DEEP_SCRUB);
- clear_scrub_reservations();
+ m_local_osd_resource.reset();
requeue_waiting();
reset_internal_state();
state_clear(PG_STATE_REPAIR);
- clear_scrub_reservations();
+ m_local_osd_resource.reset();
requeue_waiting();
reset_internal_state();
void handle_scrub_reserve_request(OpRequestRef op);
void handle_scrub_reserve_release(OpRequestRef op);
- void discard_replica_reservations() final;
- void clear_scrub_reservations() final; // PG::clear... fwds to here
// managing scrub op registration
void send_preempted_replica() final;
void send_remotes_reserved(epoch_t epoch_queued) final;
- void send_reservation_failure(epoch_t epoch_queued) final;
/**
* does the PG have newer updates than what we (the scrubber) know?
int build_replica_map_chunk() final;
- void reserve_replicas() final;
-
bool set_reserving_now() final;
void clear_reserving_now() final;
void log_cluster_warning(const std::string& warning) const final;
- // temporary interface to handle forwarded reservation messages:
- void grant_from_replica(OpRequestRef op, pg_shard_t from) final;
- void reject_from_replica(OpRequestRef op, pg_shard_t from) final;
-
protected:
bool state_test(uint64_t m) const { return m_pg->state_test(m); }
void state_set(uint64_t m) { m_pg->state_set(m); }
epoch_t m_last_aborted{}; // last time we've noticed a request to abort
- // 'optional', as 'ReplicaReservations' & 'LocalReservation' are
+ // 'optional', as 'LocalReservation' is
// 'RAII-designed' to guarantee un-reserving when deleted.
- std::optional<Scrub::ReplicaReservations> m_reservations;
std::optional<Scrub::LocalReservation> m_local_osd_resource;
void cleanup_on_finish(); // scrub_clear_state() as called for a Primary when
Session::~Session()
{
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
+ m_reservations.reset();
// note the interaction between clearing the 'queued' flag and two
// other states: the snap-mapper and the scrubber internal state.
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(10) << "Session::react(const IntervalChanged&)" << dendl;
- /// \todo (future commit): the reservations will be local to this state
- scrbr->discard_replica_reservations();
+ m_reservations->discard_remote_reservations();
return transit<NotActive>();
}
dout(10) << "-- state -->> ReservingReplicas" << dendl;
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
- scrbr->reserve_replicas();
+ // initiate the reservation process
+ context<Session>().m_reservations.emplace(*scrbr);
- auto timeout = scrbr->get_pg_cct()->_conf.get_val<
- std::chrono::milliseconds>("osd_scrub_reservation_timeout");
+ auto timeout = scrbr->get_pg_cct()->_conf.get_val<milliseconds>(
+ "osd_scrub_reservation_timeout");
if (timeout.count() > 0) {
// Start a timer to handle case where the replicas take a long time to
// ack the reservation. See ReservationTimeout handler below.
sc::result ReservingReplicas::react(const ReplicaGrant& ev)
{
- // for now - route the message back to the scrubber, as it holds the
- // ReplicaReservations object
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(10) << "ReservingReplicas::react(const ReplicaGrant&)" << dendl;
- scrbr->grant_from_replica(ev.m_op, ev.m_from);
+ context<Session>().m_reservations->handle_reserve_grant(ev.m_op, ev.m_from);
return discard_event();
}
sc::result ReservingReplicas::react(const ReplicaReject& ev)
{
- // for now - route the message back to the scrubber, as it holds the
- // ReplicaReservations object
DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
dout(10) << "ReservingReplicas::react(const ReplicaReject&)" << dendl;
- scrbr->reject_from_replica(ev.m_op, ev.m_from);
- return discard_event();
+ // manipulate the 'next to reserve' iterator to exclude
+ // the rejecting replica from the set of replicas requiring release
+ context<Session>().m_reservations->verify_rejections_source(
+ ev.m_op, ev.m_from);
+
+ // set 'reservation failure' as the scrub termination cause (affecting
+ // the rescheduling of this PG)
+ scrbr->flag_reservations_failure();
+
+ // 'Session' state dtor stops the scrubber
+ return transit<NotActive>();
}
sc::result ReservingReplicas::react(const ReservationTimeout&)
dout(10) << "ReservingReplicas::react(const ReservationTimeout&)" << dendl;
const auto msg = fmt::format(
- "PgScrubber: {} timeout on reserving replicas (since {})",
- scrbr->get_spgid(), entered_at);
- dout(5) << msg << dendl;
- scrbr->get_clog()->warn() << "osd." << scrbr->get_whoami() << " " << msg;
+ "osd.{} PgScrubber: {} timeout on reserving replicas (since {})",
+ scrbr->get_whoami(), scrbr->get_spgid(), entered_at);
+ dout(1) << msg << dendl;
+ scrbr->get_clog()->warn() << msg;
// cause the scrubber to stop the scrub session, marking 'reservation
// failure' as the cause (affecting future scheduling)
return transit<NotActive>();
}
-sc::result ReservingReplicas::react(const ReservationFailure&)
-{
- DECLARE_LOCALS; // 'scrbr' & 'pg_id' aliases
- dout(10) << "ReservingReplicas::react(const ReservationFailure&)" << dendl;
- return transit<NotActive>();
-}
-
// ----------------------- ActiveScrubbing -----------------------------------
ActiveScrubbing::ActiveScrubbing(my_context ctx)
#include "osd/scrubber_common.h"
#include "scrub_machine_lstnr.h"
+#include "scrub_reservations.h"
/// a wrapper that sets the FSM state description used by the
/// PgScrubber
/// all replicas have granted our reserve request
MEV(RemotesReserved)
-/// a reservation request has failed
-MEV(ReservationFailure)
-
/// reservations have timed out
MEV(ReservationTimeout)
sc::custom_reaction<IntervalChanged>>;
sc::result react(const IntervalChanged&);
+
+ /// managing the scrub session's reservations (optional, as
+ /// it's an RAII wrapper around the state of 'holding reservations')
+ std::optional<ReplicaReservations> m_reservations{std::nullopt};
};
struct ReservingReplicas : sc::state<ReservingReplicas, Session>,
NamedSimply {
explicit ReservingReplicas(my_context ctx);
~ReservingReplicas();
- using reactions = mpl::list<
- // all replicas granted our resources request
- sc::custom_reaction<ReplicaGrant>,
+ using reactions = mpl::list<sc::custom_reaction<ReplicaGrant>,
sc::custom_reaction<ReplicaReject>,
sc::transition<RemotesReserved, ActiveScrubbing>,
- sc::custom_reaction<ReservationTimeout>,
- sc::custom_reaction<ReservationFailure>>;
+ sc::custom_reaction<ReservationTimeout>>;
ceph::coarse_real_clock::time_point entered_at =
ceph::coarse_real_clock::now();
sc::result react(const ReplicaReject&);
sc::result react(const ReservationTimeout&);
-
- /// at least one replica denied us the scrub resources we've requested
- sc::result react(const ReservationFailure&);
};
/**
* \file the PgScrubber interface used by the scrub FSM
*/
+#include "common/LogClient.h"
#include "common/version.h"
#include "include/Context.h"
#include "osd/osd_types.h"
+#include "osd/scrubber_common.h"
-struct ScrubMachineListener;
+class PG;
namespace Scrub {
*/
virtual void maps_compare_n_cleanup() = 0;
- /**
- * order the PgScrubber to initiate the process of reserving replicas' scrub
- * resources.
- */
- virtual void reserve_replicas() = 0;
-
- /// discard replica reservations without sending a message (on interval)
- virtual void discard_replica_reservations() = 0;
-
virtual void set_scrub_begin_time() = 0;
virtual void set_scrub_duration() = 0;
/// sending cluster-log warnings
virtual void log_cluster_warning(const std::string& msg) const = 0;
- // temporary interface (to be discarded in a follow-up commit)
- // to handle replica reservation messages routed thru the FSM
- virtual void grant_from_replica(OpRequestRef op, pg_shard_t from) = 0;
- virtual void reject_from_replica(OpRequestRef op, pg_shard_t from) = 0;
-
// temporary interface (to be discarded in a follow-up PR)
/// set the 'resources_failure' flag in the scrub-job object
virtual void flag_reservations_failure() = 0;
}
}
-// temporary comment: the part of handle_reserve_reject() that will not
-// be delegated to the FSM in the following commits
void ReplicaReservations::verify_rejections_source(
OpRequestRef op,
pg_shard_t from)
}
}
-// to be delegated to the FSM in the following commits
-void ReplicaReservations::handle_reserve_reject(
- OpRequestRef op,
- pg_shard_t from)
-{
- verify_rejections_source(op, from);
- release_all();
- m_scrubber.flag_reservations_failure();
- m_osds->queue_for_scrub_denied(m_pg, scrub_prio_t::low_priority);
-}
-
std::optional<pg_shard_t> ReplicaReservations::get_last_sent() const
{
if (m_next_to_request == m_sorted_secondaries.cbegin()) {
*/
void verify_rejections_source(OpRequestRef op, pg_shard_t from);
- void handle_reserve_reject(OpRequestRef op, pg_shard_t from);
-
/**
* Notifies implementation that it is no longer responsible for releasing
* tracked remote reservations.
class ReplicaReservations;
}
-/// Facilitating scrub-realated object access to private PG data
+/// Facilitating scrub-related object access to private PG data
class ScrubberPasskey {
private:
friend class Scrub::ReplicaReservations;
*/
virtual void send_remotes_reserved(epoch_t epoch_queued) = 0;
- /**
- * triggers the 'ReservationFailure' (at least one replica denied us the
- * requested resources) state-machine event
- */
- virtual void send_reservation_failure(epoch_t epoch_queued) = 0;
-
virtual void cleanup_store(ObjectStore::Transaction* t) = 0;
virtual bool get_store_errors(const scrub_ls_arg_t& arg,
// --------------- reservations -----------------------------------
- /**
- * "forget" all replica reservations. No messages are sent to the
- * previously-reserved.
- *
- * Used upon interval change. The replicas' state is guaranteed to
- * be reset separately by the interval-change event.
- */
- virtual void discard_replica_reservations() = 0;
-
- /**
- * clear both local and OSD-managed resource reservation flags
- */
- virtual void clear_scrub_reservations() = 0;
-
/**
* Reserve local scrub resources (managed by the OSD)
*