osd/scrub: minor fixes split from main "scrub backend" commit

Reformatting, minor compiler warnings, etc.
Split from "osd/scrub: extract scrub backend code into separate classes"

Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
This commit is contained in:
Ronen Friedman 2021-12-01 11:37:04 +00:00
parent 1fdf23b4bc
commit 7b1c239ccf
16 changed files with 108 additions and 64 deletions

View File

@ -25,7 +25,7 @@ struct SnapRealmInfo {
h.seq = seq_;
h.parent_since = current_parent_since_;
}
inodeno_t ino() const { return inodeno_t(h.ino); }
inodeno_t parent() const { return inodeno_t(h.parent); }
snapid_t seq() const { return snapid_t(h.seq); }
@ -74,6 +74,4 @@ inline std::ostream& operator<<(std::ostream& out, const SnapContext& snapc) {
return out << snapc.seq << "=" << snapc.snaps;
}
//}
#endif

View File

@ -381,7 +381,7 @@ void ECBackend::handle_recovery_push(
if ((get_parent()->pgb_is_primary())) {
ceph_assert(recovery_ops.count(op.soid));
ceph_assert(recovery_ops[op.soid].obc);
if (get_parent()->pg_is_repair())
if (get_parent()->pg_is_repair() || is_repair)
get_parent()->inc_osd_stat_repaired();
get_parent()->on_local_recover(
op.soid,

View File

@ -7473,7 +7473,8 @@ bool OSD::scrub_random_backoff()
bool coin_flip = (rand() / (double)RAND_MAX >=
cct->_conf->osd_scrub_backoff_ratio);
if (!coin_flip) {
dout(20) << "scrub_random_backoff lost coin flip, randomly backing off" << dendl;
dout(20) << "scrub_random_backoff lost coin flip, randomly backing off (ratio: "
<< cct->_conf->osd_scrub_backoff_ratio << ")" << dendl;
return true;
}
return false;

View File

@ -54,8 +54,8 @@
class OSD;
class OSDService;
class OSDShard;
class OSDShardPGSlot;
struct OSDShard;
struct OSDShardPGSlot;
class PG;
struct OpRequest;

View File

@ -14,8 +14,6 @@
* Foundation. See file COPYING.
*
*/
#include "PrimaryLogPG.h"
#include <errno.h>
#include <charconv>
@ -6443,12 +6441,13 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
{
uint64_t ver = op.assert_ver.ver;
tracepoint(osd, do_osd_op_pre_assert_ver, soid.oid.name.c_str(), soid.snap.val, ver);
if (!ver)
if (!ver) {
result = -EINVAL;
else if (ver < oi.user_version)
} else if (ver < oi.user_version) {
result = -ERANGE;
else if (ver > oi.user_version)
} else if (ver > oi.user_version) {
result = -EOVERFLOW;
}
}
break;
@ -12278,6 +12277,8 @@ int PrimaryLogPG::recover_missing(
int priority,
PGBackend::RecoveryHandle *h)
{
dout(10) << __func__ << " sar: " << scrub_after_recovery << dendl;
if (recovery_state.get_missing_loc().is_unfound(soid)) {
dout(7) << __func__ << " " << soid
<< " v " << v

View File

@ -105,3 +105,30 @@ struct fmt::formatter<object_info_t> {
return fmt::format_to(ctx.out(), ")");
}
};
template <>
struct fmt::formatter<pg_t> {
constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
template <typename FormatContext>
auto format(const pg_t& pg, FormatContext& ctx)
{
return fmt::format_to(ctx.out(), "{}.{}", pg.pool(), pg.m_seed);
}
};
template <>
struct fmt::formatter<spg_t> {
constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
template <typename FormatContext>
auto format(const spg_t& spg, FormatContext& ctx)
{
if (shard_id_t::NO_SHARD == spg.shard.id) {
return fmt::format_to(ctx.out(), "{}", spg.pgid);
} else {
return fmt::format_to(ctx.out(), "{}s{}>", spg.pgid, spg.shard.id);
}
}
};

View File

@ -25,7 +25,7 @@
class OSD;
class OSDShard;
struct OSDShard;
namespace ceph::osd::scheduler {

View File

@ -1,7 +1,9 @@
// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
// vim: ts=8 sw=2 smarttab
#include "PrimaryLogScrub.h"
#include "./PrimaryLogScrub.h"
#include <sstream>
#include "common/scrub_types.h"
#include "osd/osd_types_fmt.h"
@ -15,8 +17,6 @@
#undef dout_prefix
#define dout_prefix _prefix(_dout, this)
using std::vector;
template <class T>
static ostream& _prefix(std::ostream* _dout, T* t)
{
@ -122,8 +122,9 @@ void PrimaryLogScrub::_scrub_finish()
return false;
});
if (m_pl_pg->agent_state)
if (m_pl_pg->agent_state) {
m_pl_pg->agent_choose_mode();
}
}
dout(10) << m_mode_desc << " got " << m_scrub_cstat.sum.num_objects << "/"
@ -164,6 +165,7 @@ void PrimaryLogScrub::_scrub_finish()
!info.stats.manifest_stats_invalid) ||
m_scrub_cstat.sum.num_whiteouts != info.stats.stats.sum.num_whiteouts ||
m_scrub_cstat.sum.num_bytes != info.stats.stats.sum.num_bytes) {
m_osds->clog->error() << info.pgid << " " << m_mode_desc << " : stat mismatch, got "
<< m_scrub_cstat.sum.num_objects << "/"
<< info.stats.stats.sum.num_objects << " objects, "

View File

@ -5,10 +5,6 @@
// the './' includes are marked this way to affect clang-format
#include "./pg_scrubber.h"
#include <iostream>
#include <sstream>
#include <vector>
#include "debug.h"
#include "common/errno.h"

View File

@ -485,7 +485,7 @@ ScrubQueue::scrub_schedule_t ScrubQueue::adjust_target_time(
if (times.is_must == ScrubQueue::must_scrub_t::not_mandatory) {
// if not explicitly requested, postpone the scrub with a random delay
// unless explicitly requested, postpone the scrub with a random delay
double scrub_min_interval = times.min_interval > 0
? times.min_interval
: cct->_conf->osd_scrub_min_interval;

View File

@ -23,7 +23,6 @@
#include "scrub_machine.h"
using std::list;
using std::map;
using std::pair;
using std::set;
using std::stringstream;
@ -422,8 +421,8 @@ unsigned int PgScrubber::scrub_requeue_priority(Scrub::scrub_prio_t with_priorit
unsigned int suggested_priority) const
{
if (with_priority == Scrub::scrub_prio_t::high_priority) {
suggested_priority = std::max(suggested_priority,
(unsigned int)m_pg->cct->_conf->osd_client_op_priority);
suggested_priority = std::max(
suggested_priority, (unsigned int)m_pg->get_cct()->_conf->osd_client_op_priority);
}
return suggested_priority;
}
@ -524,7 +523,7 @@ PgScrubber::determine_scrub_time(const requested_scrub_t& request_flags) const
// we do not need the interval data in this case
} else if (m_pg->info.stats.stats_invalid &&
m_pg->cct->_conf->osd_scrub_invalid_stats) {
m_pg->get_cct()->_conf->osd_scrub_invalid_stats) {
res.proposed_time = ceph_clock_now();
res.is_must = ScrubQueue::must_scrub_t::mandatory;
@ -998,7 +997,8 @@ void PgScrubber::on_replica_init()
int PgScrubber::build_primary_map_chunk()
{
epoch_t map_building_since = m_pg->get_osdmap_epoch();
dout(20) << __func__ << ": initiated at epoch " << map_building_since << dendl;
dout(20) << __func__ << ": initiated at epoch " << map_building_since
<< dendl;
auto ret = build_scrub_map_chunk(*m_primary_scrubmap,
m_primary_scrubmap_pos,
@ -1283,7 +1283,7 @@ void PgScrubber::send_preempted_replica()
m_replica_min_epoch, m_pg_whoami);
reply->preempted = true;
::encode(replica_scrubmap, reply->get_data()); // must not skip this
::encode(replica_scrubmap, reply->get_data()); // skipping this crashes the scrubber
m_pg->send_cluster_message(m_pg->get_primary().osd, reply, m_replica_min_epoch, false);
}
@ -1576,6 +1576,9 @@ void PgScrubber::scrub_finish()
// Since we don't know which errors were fixed, we can only clear them
// when every one has been fixed.
if (m_is_repair) {
dout(15) << fmt::format("{}: {} errors. {} errors fixed", __func__,
m_shallow_errors + m_deep_errors, m_fixed_count)
<< dendl;
if (m_fixed_count == m_shallow_errors + m_deep_errors) {
ceph_assert(m_is_deep);
@ -1588,10 +1591,11 @@ void PgScrubber::scrub_finish()
// Deep scrub in order to get corrected error counts
m_pg->scrub_after_recovery = true;
m_pg->m_planned_scrub.req_scrub =
m_pg->m_planned_scrub.req_scrub || m_flags.required;
m_pg->m_planned_scrub.req_scrub || m_flags.required;
dout(20) << __func__ << " Current 'required': " << m_flags.required
<< " Planned 'req_scrub': " << m_pg->m_planned_scrub.req_scrub << dendl;
<< " Planned 'req_scrub': " << m_pg->m_planned_scrub.req_scrub
<< dendl;
} else if (m_shallow_errors || m_deep_errors) {
@ -1599,7 +1603,7 @@ void PgScrubber::scrub_finish()
// possible.
state_set(PG_STATE_FAILED_REPAIR);
dout(10) << __func__ << " " << (m_shallow_errors + m_deep_errors)
<< " error(s) present with no repair possible" << dendl;
<< " error(s) present with no repair possible" << dendl;
}
}

View File

@ -155,7 +155,7 @@ class LocalReservation {
};
/**
* wraps the OSD resource we are using when reserved as a replica by a scrubbing master.
* wraps the OSD resource we are using when reserved as a replica by a scrubbing primary.
*/
class ReservedByRemotePrimary {
const PgScrubber* m_scrubber; ///< we will be using its gen_prefix()
@ -904,7 +904,7 @@ private:
private:
PG* m_pg;
mutable std::mutex m_preemption_lock;
mutable ceph::mutex m_preemption_lock;
bool m_preemptable{false};
bool m_preempted{false};
int m_left;

View File

@ -115,11 +115,11 @@ ScrubMap* ScrubBackend::new_chunk()
return &this_chunk->received_maps[m_pg_whoami];
}
void ScrubBackend::merge_to_master_set()
void ScrubBackend::merge_to_authoritative_set()
{
dout(15) << __func__ << dendl;
ceph_assert(m_pg.is_primary());
ceph_assert(this_chunk->master_set.empty() &&
ceph_assert(this_chunk->authoritative_set.empty() &&
"the scrubber-backend should be empty");
if (g_conf()->subsys.should_gather<ceph_subsys_osd, 15>()) {
@ -132,13 +132,13 @@ void ScrubBackend::merge_to_master_set()
}
}
// Construct the master set of objects
// Construct the authoritative set of objects
for (const auto& map : this_chunk->received_maps) {
std::transform(
map.second.objects.begin(),
map.second.objects.end(),
std::inserter(this_chunk->master_set, this_chunk->master_set.end()),
[](const auto& i) { return i.first; });
std::transform(map.second.objects.begin(),
map.second.objects.end(),
std::inserter(this_chunk->authoritative_set,
this_chunk->authoritative_set.end()),
[](const auto& i) { return i.first; });
}
}
@ -187,7 +187,7 @@ void ScrubBackend::scrub_compare_maps(bool max_reached)
// construct authoritative scrub map for type-specific scrubbing
m_cleaned_meta_map.insert(my_map());
merge_to_master_set();
merge_to_authoritative_set();
// collect some omap statistics into m_omap_stats
omap_checks();
@ -238,7 +238,7 @@ void ScrubBackend::omap_checks()
stringstream wss;
// Iterate through objects and update omap stats
for (const auto& ho : this_chunk->master_set) {
for (const auto& ho : this_chunk->authoritative_set) {
for (const auto& [srd, smap] : this_chunk->received_maps) {
if (srd != m_pg.get_primary()) {
@ -776,8 +776,7 @@ shard_as_auth_t ScrubBackend::possible_auth_shard(const hobject_t& obj,
&shard_info_wrapper::set_obj_size_info_mismatch)) {
errstream << sep(err) << "candidate size " << smap_obj.size << " info size "
<< m_pgbe.be_get_ondisk_size(oi.size)
<< " mismatch";
<< m_pgbe.be_get_ondisk_size(oi.size) << " mismatch";
}
std::optional<uint32_t> digest;
@ -797,13 +796,14 @@ shard_as_auth_t ScrubBackend::possible_auth_shard(const hobject_t& obj,
// re-implementation of PGBackend::be_compare_scrubmaps()
std::optional<std::string> ScrubBackend::compare_smaps()
{
dout(10) << __func__ << ": master-set #: " << this_chunk->master_set.size()
dout(10) << __func__
<< ": authoritative-set #: " << this_chunk->authoritative_set.size()
<< dendl;
std::stringstream errstream;
std::for_each(
this_chunk->master_set.begin(),
this_chunk->master_set.end(),
this_chunk->authoritative_set.begin(),
this_chunk->authoritative_set.end(),
[this, &errstream](const auto& ho) { compare_obj_in_maps(ho, errstream); });
if (errstream.str().empty()) {
@ -892,12 +892,12 @@ void ScrubBackend::compare_obj_in_maps(const hobject_t& ho,
}
std::optional<ScrubBackend::auth_and_obj_errs_t> ScrubBackend::for_empty_auth_list(
std::list<pg_shard_t>&& auths,
std::set<pg_shard_t>&& obj_errors,
shard_to_scrubmap_t::iterator auth,
const hobject_t& ho,
stringstream& errstream)
std::optional<ScrubBackend::auth_and_obj_errs_t>
ScrubBackend::for_empty_auth_list(std::list<pg_shard_t>&& auths,
std::set<pg_shard_t>&& obj_errors,
shard_to_scrubmap_t::iterator auth,
const hobject_t& ho,
stringstream& errstream)
{
if (auths.empty()) {
if (obj_errors.empty()) {
@ -919,7 +919,7 @@ std::optional<ScrubBackend::auth_and_obj_errs_t> ScrubBackend::for_empty_auth_li
}
return ScrubBackend::auth_and_obj_errs_t{std::move(auths),
std::move(obj_errors)};
std::move(obj_errors)};
}
@ -1900,7 +1900,7 @@ void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
ceph::condition_variable my_cond;
ceph::mutex my_lock = ceph::make_mutex("PG::_scan_snaps my_lock");
int e = 0;
bool done;
bool done; // note: initialized to 'false' by C_SafeCond
t.register_on_applied_sync(new C_SafeCond(my_lock, my_cond, &done, &e));
@ -1911,7 +1911,9 @@ void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
} else {
std::unique_lock l{my_lock};
my_cond.wait(l, [&done] { return done; });
ceph_assert(m_pg.osd->store);
}
dout(15) << __func__ << " wait on repair - done" << dendl;
}
}
}

View File

@ -167,7 +167,7 @@ struct scrub_chunk_t {
std::map<pg_shard_t, ScrubMap> received_maps;
/// a collection of all objs mentioned in the maps
std::set<hobject_t> master_set;
std::set<hobject_t> authoritative_set;
utime_t started{ceph_clock_now()};
@ -269,7 +269,15 @@ class ScrubBackend {
ConfigProxy& m_conf;
LogChannelRef clog;
int num_digest_updates_pending{0};
public:
// as used by PgScrubber::final_cstat_update(). consider relocating.
// actually - only filled in by the PG backend, and used by the scrubber.
// We are not handling it. So consider getting it from the Scrubber, or
// creating it by the PG-BE
omap_stat_t m_omap_stats = (const struct omap_stat_t){0};
private:
using auth_and_obj_errs_t =
std::tuple<std::list<pg_shard_t>, ///< the auth-list
@ -291,11 +299,11 @@ class ScrubBackend {
ScrubMap& my_map();
/**
* merge_to_master_set() updates
* merge_to_authoritative_set() updates
* - this_chunk->maps[from] with the replicas' scrub-maps;
* - this_chunk->master_set as a union of all the maps' objects;
* - this_chunk->authoritative_set as a union of all the maps' objects;
*/
void merge_to_master_set();
void merge_to_authoritative_set();
// note: used by both Primary & replicas
static ScrubMap clean_meta_map(ScrubMap& cleaned, bool max_reached);

View File

@ -19,7 +19,6 @@
using namespace std::chrono;
using namespace std::chrono_literals;
namespace sc = boost::statechart;
#define DECLARE_LOCALS \
ScrubMachineListener* scrbr = context<ScrubMachine>().m_scrbr; \
@ -435,6 +434,12 @@ sc::result WaitReplicas::react(const GotReplicas&)
}
}
sc::result WaitReplicas::react(const DigestUpdate&)
{
dout(10) << "WaitReplicas::react(const DigestUpdate&) - too early" << dendl;
return discard_event();
}
// ----------------------- WaitDigestUpdate -----------------------------------
WaitDigestUpdate::WaitDigestUpdate(my_context ctx) : my_base(ctx)

View File

@ -299,11 +299,11 @@ struct WaitReplicas : sc::state<WaitReplicas, ActiveScrubbing> {
using reactions =
mpl::list<sc::custom_reaction<GotReplicas>, // all replicas are accounted for
sc::transition<MapsCompared, WaitDigestUpdate>,
sc::deferral<DigestUpdate> // might arrive before we've reached WDU
sc::custom_reaction<DigestUpdate>
>;
sc::result react(const GotReplicas&);
sc::result react(const DigestUpdate&);
bool all_maps_already_called{false}; // see comment in react code
};