Reformatting, minor compiler warnings, etc.
Split from "osd/scrub: extract scrub backend code into separate classes"
Signed-off-by: Ronen Friedman <rfriedma@redhat.com>
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); }
return out << snapc.seq << "=" << snapc.snaps;
}
-//}
-
#endif
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,
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;
class OSD;
class OSDService;
-class OSDShard;
-class OSDShardPGSlot;
+struct OSDShard;
+struct OSDShardPGSlot;
class PG;
struct OpRequest;
* Foundation. See file COPYING.
*
*/
-#include "PrimaryLogPG.h"
-
#include <errno.h>
#include <charconv>
{
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;
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
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);
+ }
+ }
+};
class OSD;
-class OSDShard;
+struct OSDShard;
namespace ceph::osd::scheduler {
// -*- 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"
#undef dout_prefix
#define dout_prefix _prefix(_dout, this)
-using std::vector;
-
template <class T>
static ostream& _prefix(std::ostream* _dout, T* t)
{
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 << "/"
!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, "
// 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"
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;
#include "scrub_machine.h"
using std::list;
-using std::map;
using std::pair;
using std::set;
using std::stringstream;
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;
}
// 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;
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,
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);
}
// 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);
// 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) {
// 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;
}
}
};
/**
- * 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()
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;
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>()) {
}
}
- // 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; });
}
}
// 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();
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()) {
&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;
// 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()) {
}
-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()) {
}
return ScrubBackend::auth_and_obj_errs_t{std::move(auths),
- std::move(obj_errors)};
+ std::move(obj_errors)};
}
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));
} 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;
}
}
}
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()};
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
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);
using namespace std::chrono;
using namespace std::chrono_literals;
-namespace sc = boost::statechart;
#define DECLARE_LOCALS \
ScrubMachineListener* scrbr = context<ScrubMachine>().m_scrbr; \
}
}
+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)
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
};