ceph_assert(results.empty());
}
+void Store::add_error(int64_t pool, const inconsistent_obj_wrapper& e)
+{
+ add_object_error(pool, e);
+}
+
void Store::add_object_error(int64_t pool, const inconsistent_obj_wrapper& e)
{
bufferlist bl;
results[to_object_key(pool, e.object)] = bl;
}
+void Store::add_error(int64_t pool, const inconsistent_snapset_wrapper& e)
+{
+ add_snap_error(pool, e);
+}
+
void Store::add_snap_error(int64_t pool, const inconsistent_snapset_wrapper& e)
{
bufferlist bl;
const coll_t& coll);
void add_object_error(int64_t pool, const inconsistent_obj_wrapper& e);
void add_snap_error(int64_t pool, const inconsistent_snapset_wrapper& e);
+
+ // and a variant-friendly interface:
+ void add_error(int64_t pool, const inconsistent_obj_wrapper& e);
+ void add_error(int64_t pool, const inconsistent_snapset_wrapper& e);
+
bool empty() const;
void flush(ObjectStore::Transaction *);
void cleanup(ObjectStore::Transaction *);
using std::list;
using std::pair;
-using std::set;
using std::stringstream;
using std::vector;
using namespace Scrub;
case 0: {
// finished!
- m_be->replica_clean_meta(replica_scrubmap, m_end.is_max(), m_start);
+ auto required_fixes = m_be->replica_clean_meta(
+ replica_scrubmap, m_end.is_max(), m_start, *this);
+ // actuate snap-mapper changes:
+ apply_snap_mapper_fixes(required_fixes);
// the local map has been created. Send it to the primary.
// Note: once the message reaches the Primary, it may ask us for another
- // chunk - and we better be done with the current scrub. Thus - the preparation of
- // the reply message is separate, and we clear the scrub state before actually
- // sending it.
+ // chunk - and we better be done with the current scrub. Thus - the
+ // preparation of the reply message is separate, and we clear the scrub
+ // state before actually sending it.
auto reply = prep_replica_map_msg(PreemptionNoted::no_preemption);
replica_handling_done();
}
}
+void PgScrubber::persist_scrub_results(inconsistent_objs_t&& all_errors)
+{
+ dout(10) << __func__ << " " << all_errors.size() << " errors" << dendl;
+
+ for (auto& e : all_errors) {
+ std::visit([this](auto& e) { m_store->add_error(m_pg->pool.id, e); }, e);
+ }
+
+ ObjectStore::Transaction t;
+ m_store->flush(&t);
+ m_osds->store->queue_transaction(m_pg->ch, std::move(t), nullptr);
+}
+
+void PgScrubber::apply_snap_mapper_fixes(
+ const std::vector<snap_mapper_fix_t>& fix_list)
+{
+ dout(15) << __func__ << " " << fix_list.size() << " fixes" << dendl;
+
+ if (fix_list.empty()) {
+ return;
+ }
+
+ ObjectStore::Transaction t;
+ OSDriver::OSTransaction t_drv(m_pg->osdriver.get_transaction(&t));
+
+ for (auto& [fix_op, hoid, snaps, bogus_snaps] : fix_list) {
+
+ if (fix_op == snap_mapper_op_t::update) {
+
+ // must remove the existing snap-set before inserting the correct one
+ if (auto r = m_pg->snap_mapper.remove_oid(hoid, &t_drv); r < 0) {
+
+ derr << __func__ << ": remove_oid returned " << cpp_strerror(r)
+ << dendl;
+ ceph_abort();
+ }
+
+ m_osds->clog->error() << fmt::format(
+ "osd.{} found snap mapper error on pg {} oid {} snaps in mapper: {}, "
+ "oi: "
+ "{} ...repaired",
+ m_pg_whoami, m_pg_id, hoid, bogus_snaps, snaps);
+
+ } else {
+
+ m_osds->clog->error() << fmt::format(
+ "osd.{} found snap mapper error on pg {} oid {} snaps missing in "
+ "mapper, should be: {} ...repaired",
+ m_pg_whoami, m_pg_id, hoid, snaps);
+ }
+
+ // now - insert the correct snap-set
+
+ m_pg->snap_mapper.add_oid(hoid, snaps, &t_drv);
+ }
+
+ // wait for repair to apply to avoid confusing other bits of the system.
+ {
+ dout(15) << __func__ << " wait on repair!" << dendl;
+
+ ceph::condition_variable my_cond;
+ ceph::mutex my_lock = ceph::make_mutex("PG::_scan_snaps my_lock");
+ int e = 0;
+ bool done{false};
+
+ t.register_on_applied_sync(new C_SafeCond(my_lock, my_cond, &done, &e));
+
+ if (e = m_pg->osd->store->queue_transaction(m_pg->ch, std::move(t));
+ e != 0) {
+ derr << __func__ << ": queue_transaction got " << cpp_strerror(e)
+ << dendl;
+ } else {
+ std::unique_lock l{my_lock};
+ my_cond.wait(l, [&done] { return done; });
+ ceph_assert(m_pg->osd->store); // RRR why?
+ }
+ dout(15) << __func__ << " wait on repair - done" << dendl;
+ }
+}
+
void PgScrubber::maps_compare_n_cleanup()
{
m_pg->add_objects_scrubbed_count(m_be->get_primary_scrubmap().objects.size());
- m_be->scrub_compare_maps(m_end.is_max());
+
+ auto required_fixes = m_be->scrub_compare_maps(m_end.is_max(), *this);
+ if (!required_fixes.inconsistent_objs.empty()) {
+ if (state_test(PG_STATE_REPAIR)) {
+ dout(10) << __func__ << ": discarding scrub results (repairing)" << dendl;
+ } else {
+ // perform the ordered scrub-store I/O:
+ persist_scrub_results(std::move(required_fixes.inconsistent_objs));
+ }
+ }
+
+ // actuate snap-mapper changes:
+ apply_snap_mapper_fixes(required_fixes.snap_fix_list);
+
+ auto chunk_err_counts = m_be->get_error_counts();
+ m_shallow_errors += chunk_err_counts.shallow_errors;
+ m_deep_errors += chunk_err_counts.deep_errors;
+
m_start = m_end;
run_callbacks();
requeue_waiting();
* am forced to strongly decouple the state-machine implementation details from
* the actual scrubbing code.
*/
-class PgScrubber : public ScrubPgIF, public ScrubMachineListener {
-
+class PgScrubber : public ScrubPgIF,
+ public ScrubMachineListener,
+ public SnapMapperAccessor {
public:
explicit PgScrubber(PG* pg);
utime_t scrub_begin_stamp;
std::ostream& gen_prefix(std::ostream& out) const final;
+ // fetching the snap-set for a given object (used by the scrub-backend)
+ int get_snaps(const hobject_t& hoid, std::set<snapid_t>* snaps_set) const final
+ {
+ return m_pg->snap_mapper.get_snaps(hoid, snaps_set);
+ }
+
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); }
Scrub::MapsCollectionStatus m_maps_status;
-
- /// Maps from object with errors to good peers
- std::map<hobject_t, std::list<std::pair<ScrubMap::object, pg_shard_t>>> m_authoritative;
+ void persist_scrub_results(inconsistent_objs_t&& all_errors);
+ void apply_snap_mapper_fixes(const std::vector<snap_mapper_fix_t>& fix_list);
// ------------ members used if we are a replica
}
-void ScrubBackend::replica_clean_meta(ScrubMap& repl_map,
- bool max_reached,
- const hobject_t& start)
+std::vector<snap_mapper_fix_t> ScrubBackend::replica_clean_meta(
+ ScrubMap& repl_map,
+ bool max_reached,
+ const hobject_t& start,
+ SnapMapperAccessor& snaps_getter)
{
dout(15) << __func__ << ": REPL META # " << m_cleaned_meta_map.objects.size()
<< " objects" << dendl;
m_cleaned_meta_map.clear_from(start); // RRR how can this be required?
m_cleaned_meta_map.insert(repl_map);
auto for_meta_scrub = clean_meta_map(m_cleaned_meta_map, max_reached);
- scan_snaps(for_meta_scrub);
+ return scan_snaps(for_meta_scrub, snaps_getter);
}
//
// /////////////////////////////////////////////////////////////////////////////
-void ScrubBackend::scrub_compare_maps(bool max_reached)
+objs_fix_list_t ScrubBackend::scrub_compare_maps(
+ bool max_reached,
+ SnapMapperAccessor& snaps_getter)
{
dout(10) << __func__ << " has maps, analyzing" << dendl;
ceph_assert(m_pg.is_primary());
// (Validates consistency of the object info and snap sets)
scrub_snapshot_metadata(for_meta_scrub);
- // Called here on the primary. Can use an authoritative map if it isn't the
- // primary
- scan_snaps(for_meta_scrub);
-
- if (!m_scrubber.m_store->empty()) {
-
- if (m_scrubber.state_test(PG_STATE_REPAIR)) {
- dout(10) << __func__ << ": discarding scrub results" << dendl;
- m_scrubber.m_store->flush(nullptr);
-
- } else {
-
- dout(10) << __func__ << ": updating scrub object" << dendl;
- ObjectStore::Transaction t;
- m_scrubber.m_store->flush(&t);
- m_scrubber.m_osds->store->queue_transaction(m_pg.ch,
- std::move(t),
- nullptr);
- }
- }
+ return objs_fix_list_t{std::move(this_chunk->m_inconsistent_objs),
+ scan_snaps(for_meta_scrub, snaps_getter)};
}
void ScrubBackend::omap_checks()
m_missing.size(),
m_inconsistent.size());
- dout(2) << err_msg << dendl;
+ dout(4) << err_msg << dendl;
clog->error() << err_msg;
+ ceph_assert(m_repair);
int fixed_cnt{0};
- if (m_repair) {
- m_scrubber.state_clear(PG_STATE_CLEAN);
- // we know we have a problem, so it's OK to set the user-visible flag
- // even if we only reached here via auto-repair
- m_scrubber.state_set(PG_STATE_REPAIR);
- m_scrubber.update_op_mode_text();
- for (const auto& [hobj, shrd_list] : m_scrubber.m_authoritative) {
+ for (const auto& [hobj, shrd_list] : m_auth_peers) {
auto missing_entry = m_missing.find(hobj);
fixed_cnt += m_inconsistent[hobj].size();
}
}
- }
return fixed_cnt;
}
void ScrubBackend::repair_object(
const hobject_t& soid,
- const list<pair<ScrubMap::object, pg_shard_t>>& ok_peers,
+ const auth_peers_t& ok_peers,
const set<pg_shard_t>& bad_peers)
{
if (g_conf()->subsys.should_gather<ceph_subsys_osd, 20>()) {
match_in_shards(ho, auth_res, object_error, errstream);
auto opt_ers =
- for_empty_auth_list(std::forward<std::list<pg_shard_t>>(auths),
- std::forward<std::set<pg_shard_t>>(objerrs),
+ for_empty_auth_list(std::move(auths),
+ std::move(objerrs),
auth,
ho,
errstream);
}
if (object_error.errors || object_error.union_shards.errors) {
- m_scrubber.m_store->add_object_error(ho.pool, object_error);
+ this_chunk->m_inconsistent_objs.push_back(std::move(object_error));
}
if (errstream.str().empty()) {
auth_and_obj_errs_t&& auth_n_errs,
stringstream& errstream)
{
- auto& object_errors = std::get<1>(auth_n_errs);
- auto& auth_list = std::get<0>(auth_n_errs);
+ auto& object_errors = auth_n_errs.object_errors;
+ auto& auth_list = auth_n_errs.auth_list;
this_chunk->cur_inconsistent.insert(object_errors.begin(),
object_errors.end()); // merge?
// what's next?
++curclone;
if (soid_error.errors) {
- m_scrubber.m_store->add_snap_error(pool.id, soid_error);
+ this_chunk->m_inconsistent_objs.push_back(std::move(soid_error));
++soid_error_count;
}
}
if (missing) {
log_missing(missing, head, __func__, allow_incomplete_clones);
}
- if (head && (head_error.errors || soid_error_count))
- m_scrubber.m_store->add_snap_error(pool.id, head_error);
+ if (head && (head_error.errors || soid_error_count)) {
+ this_chunk->m_inconsistent_objs.push_back(std::move(head_error));
+ }
// fix data/omap digests
m_scrubber.submit_digest_fixes(this_chunk->missing_digest);
// ////////////////////////////////////////////////////////////////////////////////
-void ScrubBackend::scan_snaps(ScrubMap& smap)
+std::vector<snap_mapper_fix_t> ScrubBackend::scan_snaps(
+ ScrubMap& smap,
+ SnapMapperAccessor& snaps_getter)
{
+ std::vector<snap_mapper_fix_t> out_orders;
hobject_t head;
SnapSet snapset;
continue;
}
- scan_object_snaps(hoid, o, snapset);
+ auto maybe_fix_order = scan_object_snaps(hoid, snapset, snaps_getter);
+ if (maybe_fix_order) {
+ out_orders.push_back(std::move(*maybe_fix_order));
+ }
+
}
}
+
+ dout(15) << __func__ << " " << out_orders.size() << " fix orders" << dendl;
+ return out_orders;
}
-void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
- ScrubMap::object& scrmap_obj,
- const SnapSet& snapset)
+std::optional<snap_mapper_fix_t> ScrubBackend::scan_object_snaps(
+ const hobject_t& hoid,
+ const SnapSet& snapset,
+ SnapMapperAccessor& snaps_getter)
{
// check and if necessary fix snap_mapper
if (p == snapset.clone_snaps.end()) {
derr << __func__ << " no clone_snaps for " << hoid << " in " << snapset
<< dendl;
- return;
+ return std::nullopt;
}
set<snapid_t> obj_snaps{p->second.begin(), p->second.end()};
set<snapid_t> cur_snaps;
- int r = m_pg.snap_mapper.get_snaps(hoid, &cur_snaps);
+ int r = snaps_getter.get_snaps(hoid, &cur_snaps);
if (r != 0 && r != -ENOENT) {
derr << __func__ << ": get_snaps returned " << cpp_strerror(r) << dendl;
ceph_abort();
}
if (r == -ENOENT || cur_snaps != obj_snaps) {
- ObjectStore::Transaction t;
- OSDriver::OSTransaction _t(m_pg.osdriver.get_transaction(&t));
- if (r == 0) {
- r = m_pg.snap_mapper.remove_oid(hoid, &_t);
- if (r != 0) {
- derr << __func__ << ": remove_oid returned " << cpp_strerror(r)
- << dendl;
- ceph_abort();
- }
- clog->error() << "osd." << m_pg_whoami
- << " found snap mapper error on pg " << m_pg_id << " oid "
- << hoid << " snaps in mapper: " << cur_snaps
- << ", oi: " << obj_snaps << "...repaired";
- } else {
- clog->error() << "osd." << m_pg_whoami
- << " found snap mapper error on pg " << m_pg_id << " oid "
- << hoid << " snaps missing in mapper"
- << ", should be: " << obj_snaps << " was " << cur_snaps
- << " r " << r << "...repaired";
- }
- m_pg.snap_mapper.add_oid(hoid, obj_snaps, &_t);
-
- // wait for repair to apply to avoid confusing other bits of the system.
- {
- dout(15) << __func__ << " wait on repair!" << dendl;
- ceph::condition_variable my_cond;
- ceph::mutex my_lock = ceph::make_mutex("PG::_scan_snaps my_lock");
- int e = 0;
- bool done; // note: initialized to 'false' by C_SafeCond
-
- t.register_on_applied_sync(new C_SafeCond(my_lock, my_cond, &done, &e));
-
- e = m_pg.osd->store->queue_transaction(m_pg.ch, std::move(t));
- if (e != 0) {
- derr << __func__ << ": queue_transaction got " << cpp_strerror(e)
- << dendl;
- } 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;
- }
+ // add this object to the list of snapsets that needs fixing. Note
+ // that we also collect the existing (bogus) list, as legacy log lines show those
+ snap_mapper_op_t fixing_op =
+ (r == -ENOENT ? snap_mapper_op_t::add : snap_mapper_op_t::update);
+ return snap_mapper_fix_t{fixing_op, hoid, obj_snaps, cur_snaps};
}
+ return std::nullopt;
}
-
/*
* Process:
* Building a map of objects suitable for snapshot validation.
using shard_info_map_t = std::map<pg_shard_t, shard_info_wrapper>;
using shard_to_scrubmap_t = std::map<pg_shard_t, ScrubMap>;
+using auth_peers_t = std::vector<std::pair<ScrubMap::object, pg_shard_t>>;
+
+using wrapped_err_t =
+ std::variant<inconsistent_obj_wrapper, inconsistent_snapset_wrapper>;
+using inconsistent_objs_t = std::vector<wrapped_err_t>;
/// omap-specific stats
struct omap_stat_t {
};
+/*
+ * snaps-related aux structures:
+ * the scrub-backend scans the snaps associated with each scrubbed object, and
+ * fixes corrupted snap-sets.
+ * The actual access to the PG's snap_mapper, and the actual I/O transactions,
+ * are performed by the main PgScrubber object.
+ * the following aux structures are used to facilitate the required exchanges:
+ * - pre-fix snap-sets are accessed by the scrub-backend, and:
+ * - a list of fix-orders (either insert or replace operations) are returned
+ */
+
+struct SnapMapperAccessor {
+ virtual int get_snaps(const hobject_t& hoid,
+ std::set<snapid_t>* snaps_set) const = 0;
+ virtual ~SnapMapperAccessor() = default;
+};
+
+enum class snap_mapper_op_t {
+ add,
+ update,
+};
+
+struct snap_mapper_fix_t {
+ snap_mapper_op_t op;
+ hobject_t hoid;
+ std::set<snapid_t> snaps;
+ std::set<snapid_t> wrong_snaps; // only collected & returned for logging sake
+};
+
+// and - as the main scrub-backend entry point - scrub_compare_maps() - must
+// be able to return both a list of snap fixes and a list of inconsistent
+// objects:
+struct objs_fix_list_t {
+ inconsistent_objs_t inconsistent_objs;
+ std::vector<snap_mapper_fix_t> snap_fix_list;
+};
+
/**
* A structure used internally by select_auth_object()
*
// note: 'if' chain, as hard to consistently (on all compilers) avoid some
// warnings for a switch plus multiple return paths
if (as_auth.possible_auth == shard_as_auth_t::usable_t::not_usable) {
- return format_to(
- ctx.out(), "{{shard-not-usable:{}}}", as_auth.error_text);
+ return format_to(ctx.out(),
+ "{{shard-not-usable:{}}}",
+ as_auth.error_text);
}
if (as_auth.possible_auth == shard_as_auth_t::usable_t::not_found) {
return format_to(ctx.out(), "{{shard-not-found}}");
/// Map from object with errors to good peers
std::map<hobject_t, std::list<pg_shard_t>> authoritative;
+ inconsistent_objs_t m_inconsistent_objs;
+
// these must be reset for each element:
*/
void update_repair_status(bool should_repair);
- void replica_clean_meta(ScrubMap& smap,
- bool max_reached,
- const hobject_t& start);
+ std::vector<snap_mapper_fix_t> replica_clean_meta(
+ ScrubMap& smap,
+ bool max_reached,
+ const hobject_t& start,
+ SnapMapperAccessor& snaps_getter);
/**
* decode the arriving MOSDRepScrubMap message, placing the replica's
*/
void decode_received_map(pg_shard_t from, const MOSDRepScrubMap& msg);
- void scrub_compare_maps(bool max_reached);
+ objs_fix_list_t scrub_compare_maps(bool max_reached,
+ SnapMapperAccessor& snaps_getter);
int scrub_process_inconsistent();
/// collecting some scrub-session-wide omap stats
omap_stat_t m_omap_stats;
+ /// Mapping from object with errors to good peers
+ std::map<hobject_t, auth_peers_t> m_auth_peers;
+
// shorthands:
ConfigProxy& m_conf;
LogChannelRef clog;
private:
- using auth_and_obj_errs_t =
- std::tuple<std::list<pg_shard_t>, ///< the auth-list
- std::set<pg_shard_t> ///< object_errors
- >;
+
+ struct auth_and_obj_errs_t {
+ std::list<pg_shard_t> auth_list;
+ std::set<pg_shard_t> object_errors;
+ };
std::optional<scrub_chunk_t> this_chunk;
std::stringstream& errorstream,
bool has_snapset);
-
- void repair_object(
- const hobject_t& soid,
- const std::list<std::pair<ScrubMap::object, pg_shard_t>>& ok_peers,
- const std::set<pg_shard_t>& bad_peers);
+ void repair_object(const hobject_t& soid,
+ const auth_peers_t& ok_peers,
+ const std::set<pg_shard_t>& bad_peers);
/**
* An auxiliary used by select_auth_object() to test a specific shard
const char* logged_func_name,
bool allow_incomplete_clones);
- void scan_snaps(ScrubMap& smap);
+ /**
+ * returns a list of snaps "fix orders"
+ */
+ std::vector<snap_mapper_fix_t> scan_snaps(
+ ScrubMap& smap,
+ SnapMapperAccessor& snaps_getter);
+
+ /**
+ * an aux used by scan_snaps(), possibly returning a fix-order
+ * for a specific hobject.
+ */
+ std::optional<snap_mapper_fix_t> scan_object_snaps(
+ const hobject_t& hoid,
+ const SnapSet& snapset,
+ SnapMapperAccessor& snaps_getter);
- void scan_object_snaps(const hobject_t& hoid,
- ScrubMap::object& scrmap_obj,
- const SnapSet& snapset);
+ // accessing the PG backend for this translation service
+ uint64_t logical_to_ondisk_size(uint64_t logical_size) const;
};