#include <fmt/format.h>
#include "common/hobject.h"
+#include "include/object_fmt.h"
#include "include/types_fmt.h"
#include "msg/msg_fmt.h"
void
inconsistent_obj_wrapper::set_auth_missing(const hobject_t& hoid,
- const map<pg_shard_t, ScrubMap*>& maps,
+ const map<pg_shard_t, ScrubMap>& maps,
map<pg_shard_t, shard_info_wrapper> &shard_map,
int &shallow_errors, int &deep_errors,
const pg_shard_t &primary)
{
for (auto pg_map : maps) {
- auto oid_object = pg_map.second->objects.find(hoid);
+ auto oid_object = pg_map.second.objects.find(hoid);
shard_map[pg_map.first].primary = (pg_map.first == primary);
- if (oid_object == pg_map.second->objects.end())
+ if (oid_object == pg_map.second.objects.end())
shard_map[pg_map.first].set_missing();
else
shard_map[pg_map.first].set_object(oid_object->second);
}
void add_shard(const pg_shard_t& pgs, const shard_info_wrapper& shard);
void set_auth_missing(const hobject_t& hoid,
- const std::map<pg_shard_t, ScrubMap*>&,
+ const std::map<pg_shard_t, ScrubMap>&,
std::map<pg_shard_t, shard_info_wrapper>&,
int &shallow_errors, int &deep_errors,
const pg_shard_t &primary);
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+#pragma once
+
+/**
+ * \file fmtlib formatters for some object.h structs
+ */
+#include <fmt/format.h>
+
+#include "object.h"
+
+
+template <>
+struct fmt::formatter<snapid_t> {
+
+ constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+ template <typename FormatContext>
+ auto format(const snapid_t& snp, FormatContext& ctx)
+ {
+ if (snp == CEPH_NOSNAP) {
+ return fmt::format_to(ctx.out(), "head");
+ }
+ if (snp == CEPH_SNAPDIR) {
+ return fmt::format_to(ctx.out(), "snapdir");
+ }
+ return fmt::format_to(ctx.out(), "{:x}", snp.val);
+ }
+};
return fmt::format_to(ctx.out(), "}}");
}
};
+
+template <class A>
+struct fmt::formatter<std::list<A>> {
+ constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+ template <typename FormatContext>
+ auto format(const std::list<A>& l, FormatContext& ctx)
+ {
+ std::string_view sep = "";
+ for (const auto& e : l) {
+ fmt::format_to(ctx.out(), "{}{}", sep, e);
+ sep = ",";
+ }
+ return ctx.out();
+ }
+};
+
+template <class A>
+struct fmt::formatter<std::set<A>> {
+ constexpr auto parse(format_parse_context& ctx) { return ctx.begin(); }
+
+ template <typename FormatContext>
+ auto format(const std::set<A>& l, FormatContext& ctx)
+ {
+ std::string_view sep = "";
+ for (const auto& e : l) {
+ fmt::format_to(ctx.out(), "{}{}", sep, e);
+ sep = ",";
+ }
+ return ctx.out();
+ }
+};
scrubber/PrimaryLogScrub.cc
scrubber/scrub_machine.cc
scrubber/ScrubStore.cc
+ scrubber/scrub_backend.cc
Watch.cc
Session.cc
SnapMapper.cc
}
-void PG::_repair_oinfo_oid(ScrubMap &smap)
-{
- for (map<hobject_t, ScrubMap::object>::reverse_iterator i = smap.objects.rbegin();
- i != smap.objects.rend();
- ++i) {
- const hobject_t &hoid = i->first;
- ScrubMap::object &o = i->second;
-
- bufferlist bl;
- if (o.attrs.find(OI_ATTR) == o.attrs.end()) {
- continue;
- }
- bl.push_back(o.attrs[OI_ATTR]);
- object_info_t oi;
- try {
- oi.decode(bl);
- } catch(...) {
- continue;
- }
- if (oi.soid != hoid) {
- ObjectStore::Transaction t;
- OSDriver::OSTransaction _t(osdriver.get_transaction(&t));
- osd->clog->error() << "osd." << osd->whoami
- << " found object info error on pg "
- << info.pgid
- << " oid " << hoid << " oid in object info: "
- << oi.soid
- << "...repaired";
- // Fix object info
- oi.soid = hoid;
- bl.clear();
- encode(oi, bl, get_osdmap()->get_features(CEPH_ENTITY_TYPE_OSD, nullptr));
-
- bufferptr bp(bl.c_str(), bl.length());
- o.attrs[OI_ATTR] = bp;
-
- t.setattr(coll, ghobject_t(hoid), OI_ATTR, bl);
- int r = osd->store->queue_transaction(ch, std::move(t));
- if (r != 0) {
- derr << __func__ << ": queue_transaction got " << cpp_strerror(r)
- << dendl;
- }
- }
- }
-}
-
-void PG::repair_object(
- const hobject_t &soid,
- const list<pair<ScrubMap::object, pg_shard_t> > &ok_peers,
- const set<pg_shard_t> &bad_peers)
-{
- set<pg_shard_t> ok_shards;
- for (auto &&peer: ok_peers) ok_shards.insert(peer.second);
-
- dout(10) << "repair_object " << soid
- << " bad_peers osd.{" << bad_peers << "},"
- << " ok_peers osd.{" << ok_shards << "}" << dendl;
-
- const ScrubMap::object &po = ok_peers.back().first;
- eversion_t v;
- object_info_t oi;
- try {
- bufferlist bv;
- if (po.attrs.count(OI_ATTR)) {
- bv.push_back(po.attrs.find(OI_ATTR)->second);
- }
- auto bliter = bv.cbegin();
- decode(oi, bliter);
- } catch (...) {
- dout(0) << __func__ << ": Need version of replica, bad object_info_t: "
- << soid << dendl;
- ceph_abort();
- }
-
- if (bad_peers.count(get_primary())) {
- // We should only be scrubbing if the PG is clean.
- ceph_assert(waiting_for_unreadable_object.empty());
- dout(10) << __func__ << ": primary = " << get_primary() << dendl;
- }
-
- /* No need to pass ok_peers, they must not be missing the object, so
- * force_object_missing will add them to missing_loc anyway */
- recovery_state.force_object_missing(bad_peers, soid, oi.version);
-}
-
void PG::forward_scrub_event(ScrubAPI fn, epoch_t epoch_queued, std::string_view desc)
{
dout(20) << __func__ << ": " << desc << " queued at: " << epoch_queued << dendl;
typedef OpRequest::Ref OpRequestRef;
class DynamicPerfStats;
class PgScrubber;
+class ScrubBackend;
namespace Scrub {
class Store;
friend struct NamedState;
friend class PeeringState;
friend class PgScrubber;
+ friend class ScrubBackend;
public:
const pg_shard_t pg_whoami;
int active_pushes;
- 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);
-
[[nodiscard]] bool ops_blocked_by_scrub() const;
[[nodiscard]] Scrub::scrub_prio_t is_scrub_blocking_ops() const;
- void _repair_oinfo_oid(ScrubMap &map);
void _scan_rollback_obs(const std::vector<ghobject_t> &rollback_obs);
/**
* returns true if [begin, end) is good to scrub at this time
pos.next_object();
return 0;
}
-
-bool PGBackend::be_compare_scrub_objects(
- pg_shard_t auth_shard,
- const ScrubMap::object &auth,
- const object_info_t& auth_oi,
- const ScrubMap::object &candidate,
- shard_info_wrapper &shard_result,
- inconsistent_obj_wrapper &obj_result,
- ostream &errorstream,
- bool has_snapset)
-{
- enum { CLEAN, FOUND_ERROR } error = CLEAN;
- if (auth.digest_present && candidate.digest_present) {
- if (auth.digest != candidate.digest) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "data_digest 0x" << std::hex << candidate.digest
- << " != data_digest 0x" << auth.digest << std::dec
- << " from shard " << auth_shard;
- obj_result.set_data_digest_mismatch();
- }
- }
- if (auth.omap_digest_present && candidate.omap_digest_present) {
- if (auth.omap_digest != candidate.omap_digest) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "omap_digest 0x" << std::hex << candidate.omap_digest
- << " != omap_digest 0x" << auth.omap_digest << std::dec
- << " from shard " << auth_shard;
- obj_result.set_omap_digest_mismatch();
- }
- }
- if (parent->get_pool().is_replicated()) {
- if (auth_oi.is_data_digest() && candidate.digest_present) {
- if (auth_oi.data_digest != candidate.digest) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "data_digest 0x" << std::hex << candidate.digest
- << " != data_digest 0x" << auth_oi.data_digest << std::dec
- << " from auth oi " << auth_oi;
- shard_result.set_data_digest_mismatch_info();
- }
- }
- if (auth_oi.is_omap_digest() && candidate.omap_digest_present) {
- if (auth_oi.omap_digest != candidate.omap_digest) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "omap_digest 0x" << std::hex << candidate.omap_digest
- << " != omap_digest 0x" << auth_oi.omap_digest << std::dec
- << " from auth oi " << auth_oi;
- shard_result.set_omap_digest_mismatch_info();
- }
- }
- }
- if (candidate.stat_error)
- return error == FOUND_ERROR;
- if (!shard_result.has_info_missing()
- && !shard_result.has_info_corrupted()) {
- bufferlist can_bl, auth_bl;
- auto can_attr = candidate.attrs.find(OI_ATTR);
- auto auth_attr = auth.attrs.find(OI_ATTR);
-
- ceph_assert(auth_attr != auth.attrs.end());
- ceph_assert(can_attr != candidate.attrs.end());
-
- can_bl.push_back(can_attr->second);
- auth_bl.push_back(auth_attr->second);
- if (!can_bl.contents_equal(auth_bl)) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- obj_result.set_object_info_inconsistency();
- errorstream << "object info inconsistent ";
- }
- }
- if (has_snapset) {
- if (!shard_result.has_snapset_missing()
- && !shard_result.has_snapset_corrupted()) {
- bufferlist can_bl, auth_bl;
- auto can_attr = candidate.attrs.find(SS_ATTR);
- auto auth_attr = auth.attrs.find(SS_ATTR);
-
- ceph_assert(auth_attr != auth.attrs.end());
- ceph_assert(can_attr != candidate.attrs.end());
-
- can_bl.push_back(can_attr->second);
- auth_bl.push_back(auth_attr->second);
- if (!can_bl.contents_equal(auth_bl)) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- obj_result.set_snapset_inconsistency();
- errorstream << "snapset inconsistent ";
- }
- }
- }
- if (parent->get_pool().is_erasure()) {
- if (!shard_result.has_hinfo_missing()
- && !shard_result.has_hinfo_corrupted()) {
- bufferlist can_bl, auth_bl;
- auto can_hi = candidate.attrs.find(ECUtil::get_hinfo_key());
- auto auth_hi = auth.attrs.find(ECUtil::get_hinfo_key());
-
- ceph_assert(auth_hi != auth.attrs.end());
- ceph_assert(can_hi != candidate.attrs.end());
-
- can_bl.push_back(can_hi->second);
- auth_bl.push_back(auth_hi->second);
- if (!can_bl.contents_equal(auth_bl)) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- obj_result.set_hinfo_inconsistency();
- errorstream << "hinfo inconsistent ";
- }
- }
- }
- uint64_t oi_size = be_get_ondisk_size(auth_oi.size);
- if (oi_size != candidate.size) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "size " << candidate.size
- << " != size " << oi_size
- << " from auth oi " << auth_oi;
- shard_result.set_size_mismatch_info();
- }
- if (auth.size != candidate.size) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "size " << candidate.size
- << " != size " << auth.size
- << " from shard " << auth_shard;
- obj_result.set_size_mismatch();
- }
- // If the replica is too large and we didn't already count it for this object
- //
- if (candidate.size > cct->_conf->osd_max_object_size
- && !obj_result.has_size_too_large()) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "size " << candidate.size
- << " > " << cct->_conf->osd_max_object_size
- << " is too large";
- obj_result.set_size_too_large();
- }
- for (map<string,bufferptr>::const_iterator i = auth.attrs.begin();
- i != auth.attrs.end();
- ++i) {
- // We check system keys seperately
- if (i->first == OI_ATTR || i->first[0] != '_')
- continue;
- if (!candidate.attrs.count(i->first)) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "attr name mismatch '" << i->first << "'";
- obj_result.set_attr_name_mismatch();
- } else if (candidate.attrs.find(i->first)->second.cmp(i->second)) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "attr value mismatch '" << i->first << "'";
- obj_result.set_attr_value_mismatch();
- }
- }
- for (map<string,bufferptr>::const_iterator i = candidate.attrs.begin();
- i != candidate.attrs.end();
- ++i) {
- // We check system keys seperately
- if (i->first == OI_ATTR || i->first[0] != '_')
- continue;
- if (!auth.attrs.count(i->first)) {
- if (error != CLEAN)
- errorstream << ", ";
- error = FOUND_ERROR;
- errorstream << "attr name mismatch '" << i->first << "'";
- obj_result.set_attr_name_mismatch();
- }
- }
- return error == FOUND_ERROR;
-}
-
-static int dcount(const object_info_t &oi)
-{
- int count = 0;
- if (oi.is_data_digest())
- count++;
- if (oi.is_omap_digest())
- count++;
- return count;
-}
-
-map<pg_shard_t, ScrubMap *>::const_iterator
- PGBackend::be_select_auth_object(
- const hobject_t &obj,
- const map<pg_shard_t,ScrubMap*> &maps,
- object_info_t *auth_oi,
- map<pg_shard_t, shard_info_wrapper> &shard_map,
- bool &digest_match,
- spg_t pgid,
- ostream &errorstream)
-{
- eversion_t auth_version;
-
- // Create list of shards with primary first so it will be auth copy all
- // other things being equal.
- list<pg_shard_t> shards;
- for (map<pg_shard_t, ScrubMap *>::const_iterator j = maps.begin();
- j != maps.end();
- ++j) {
- if (j->first == get_parent()->whoami_shard())
- continue;
- shards.push_back(j->first);
- }
- shards.push_front(get_parent()->whoami_shard());
-
- map<pg_shard_t, ScrubMap *>::const_iterator auth = maps.end();
- digest_match = true;
- for (auto &l : shards) {
- ostringstream shard_errorstream;
- bool error = false;
- map<pg_shard_t, ScrubMap *>::const_iterator j = maps.find(l);
- map<hobject_t, ScrubMap::object>::iterator i =
- j->second->objects.find(obj);
- if (i == j->second->objects.end()) {
- continue;
- }
- auto& shard_info = shard_map[j->first];
- if (j->first == get_parent()->whoami_shard())
- shard_info.primary = true;
- if (i->second.read_error) {
- shard_info.set_read_error();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a read error";
- }
- if (i->second.ec_hash_mismatch) {
- shard_info.set_ec_hash_mismatch();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had an ec hash mismatch";
- }
- if (i->second.ec_size_mismatch) {
- shard_info.set_ec_size_mismatch();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had an ec size mismatch";
- }
-
- object_info_t oi;
- bufferlist bl;
- map<string, bufferptr>::iterator k;
- SnapSet ss;
- bufferlist ss_bl, hk_bl;
-
- if (i->second.stat_error) {
- shard_info.set_stat_error();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a stat error";
- // With stat_error no further checking
- // We don't need to also see a missing_object_info_attr
- goto out;
- }
-
- // We won't pick an auth copy if the snapset is missing or won't decode.
- ceph_assert(!obj.is_snapdir());
- if (obj.is_head()) {
- k = i->second.attrs.find(SS_ATTR);
- if (k == i->second.attrs.end()) {
- shard_info.set_snapset_missing();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a missing snapset key";
- } else {
- ss_bl.push_back(k->second);
- try {
- auto bliter = ss_bl.cbegin();
- decode(ss, bliter);
- } catch (...) {
- // invalid snapset, probably corrupt
- shard_info.set_snapset_corrupted();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a corrupt snapset";
- }
- }
- }
-
- if (parent->get_pool().is_erasure()) {
- ECUtil::HashInfo hi;
- k = i->second.attrs.find(ECUtil::get_hinfo_key());
- if (k == i->second.attrs.end()) {
- shard_info.set_hinfo_missing();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a missing hinfo key";
- } else {
- hk_bl.push_back(k->second);
- try {
- auto bliter = hk_bl.cbegin();
- decode(hi, bliter);
- } catch (...) {
- // invalid snapset, probably corrupt
- shard_info.set_hinfo_corrupted();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a corrupt hinfo";
- }
- }
- }
-
- k = i->second.attrs.find(OI_ATTR);
- if (k == i->second.attrs.end()) {
- // no object info on object, probably corrupt
- shard_info.set_info_missing();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a missing info key";
- goto out;
- }
- bl.push_back(k->second);
- try {
- auto bliter = bl.cbegin();
- decode(oi, bliter);
- } catch (...) {
- // invalid object info, probably corrupt
- shard_info.set_info_corrupted();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate had a corrupt info";
- goto out;
- }
-
- // This is automatically corrected in PG::_repair_oinfo_oid()
- ceph_assert(oi.soid == obj);
-
- if (i->second.size != be_get_ondisk_size(oi.size)) {
- shard_info.set_obj_size_info_mismatch();
- if (error)
- shard_errorstream << ", ";
- error = true;
- shard_errorstream << "candidate size " << i->second.size << " info size "
- << oi.size << " mismatch";
- }
-
- // digest_match will only be true if computed digests are the same
- if (auth_version != eversion_t()
- && auth->second->objects[obj].digest_present
- && i->second.digest_present
- && auth->second->objects[obj].digest != i->second.digest) {
- digest_match = false;
- dout(10) << __func__ << " digest_match = false, " << obj << " data_digest 0x" << std::hex << i->second.digest
- << " != data_digest 0x" << auth->second->objects[obj].digest << std::dec
- << dendl;
- }
-
- // Don't use this particular shard due to previous errors
- // XXX: For now we can't pick one shard for repair and another's object info or snapset
- if (shard_info.errors)
- goto out;
-
- if (auth_version == eversion_t() || oi.version > auth_version ||
- (oi.version == auth_version && dcount(oi) > dcount(*auth_oi))) {
- auth = j;
- *auth_oi = oi;
- auth_version = oi.version;
- }
-
-out:
- if (error)
- errorstream << pgid.pgid << " shard " << l << " soid " << obj
- << " : " << shard_errorstream.str() << "\n";
- // Keep scanning other shards
- }
- dout(10) << __func__ << ": selecting osd " << auth->first
- << " for obj " << obj
- << " with oi " << *auth_oi
- << dendl;
- return auth;
-}
-
-void PGBackend::be_compare_scrubmaps(
- const map<pg_shard_t,ScrubMap*> &maps,
- const set<hobject_t> &master_set,
- bool repair,
- map<hobject_t, set<pg_shard_t>> &missing,
- map<hobject_t, set<pg_shard_t>> &inconsistent,
- map<hobject_t, list<pg_shard_t>> &authoritative,
- map<hobject_t, pair<std::optional<uint32_t>,
- std::optional<uint32_t>>> &missing_digest,
- int &shallow_errors, int &deep_errors,
- Scrub::Store *store,
- const spg_t& pgid,
- const vector<int> &acting,
- ostream &errorstream)
-{
- utime_t now = ceph_clock_now();
-
- // Check maps against master set and each other
- for (set<hobject_t>::const_iterator k = master_set.begin();
- k != master_set.end();
- ++k) {
- object_info_t auth_oi;
- map<pg_shard_t, shard_info_wrapper> shard_map;
-
- inconsistent_obj_wrapper object_error{*k};
-
- bool digest_match;
- map<pg_shard_t, ScrubMap *>::const_iterator auth =
- be_select_auth_object(*k, maps, &auth_oi, shard_map, digest_match,
- pgid, errorstream);
-
- list<pg_shard_t> auth_list;
- set<pg_shard_t> object_errors;
- if (auth == maps.end()) {
- object_error.set_version(0);
- object_error.set_auth_missing(*k, maps, shard_map, shallow_errors,
- deep_errors, get_parent()->whoami_shard());
- if (object_error.has_deep_errors())
- ++deep_errors;
- else if (object_error.has_shallow_errors())
- ++shallow_errors;
- store->add_object_error(k->pool, object_error);
- errorstream << pgid.pgid << " soid " << *k
- << " : failed to pick suitable object info\n";
- continue;
- }
- object_error.set_version(auth_oi.user_version);
- ScrubMap::object& auth_object = auth->second->objects[*k];
- set<pg_shard_t> cur_missing;
- set<pg_shard_t> cur_inconsistent;
- bool fix_digest = false;
-
- for (auto j = maps.cbegin(); j != maps.cend(); ++j) {
- if (j == auth)
- shard_map[auth->first].selected_oi = true;
- if (j->second->objects.count(*k)) {
- shard_map[j->first].set_object(j->second->objects[*k]);
- // Compare
- stringstream ss;
- bool found = be_compare_scrub_objects(auth->first,
- auth_object,
- auth_oi,
- j->second->objects[*k],
- shard_map[j->first],
- object_error,
- ss,
- k->has_snapset());
-
- dout(20) << __func__ << (repair ? " repair " : " ") << (parent->get_pool().is_replicated() ? "replicated " : "")
- << (j == auth ? "auth" : "") << "shards " << shard_map.size() << (digest_match ? " digest_match " : " ")
- << (shard_map[j->first].only_data_digest_mismatch_info() ? "'info mismatch info'" : "")
- << dendl;
- // If all replicas match, but they don't match object_info we can
- // repair it by using missing_digest mechanism
- if (repair && parent->get_pool().is_replicated() && j == auth && shard_map.size() > 1
- && digest_match && shard_map[j->first].only_data_digest_mismatch_info()
- && auth_object.digest_present) {
- // Set in missing_digests
- fix_digest = true;
- // Clear the error
- shard_map[j->first].clear_data_digest_mismatch_info();
- errorstream << pgid << " soid " << *k << " : repairing object info data_digest" << "\n";
- }
- // Some errors might have already been set in be_select_auth_object()
- if (shard_map[j->first].errors != 0) {
- cur_inconsistent.insert(j->first);
- if (shard_map[j->first].has_deep_errors())
- ++deep_errors;
- else
- ++shallow_errors;
- // Only true if be_compare_scrub_objects() found errors and put something
- // in ss.
- if (found)
- errorstream << pgid << " shard " << j->first << " soid " << *k
- << " : " << ss.str() << "\n";
- } else if (found) {
- // Track possible shard to use as authoritative, if needed
- // There are errors, without identifying the shard
- object_errors.insert(j->first);
- errorstream << pgid << " soid " << *k << " : " << ss.str() << "\n";
- } else {
- // XXX: The auth shard might get here that we don't know
- // that it has the "correct" data.
- auth_list.push_back(j->first);
- }
- } else {
- cur_missing.insert(j->first);
- shard_map[j->first].set_missing();
- shard_map[j->first].primary = (j->first == get_parent()->whoami_shard());
- // Can't have any other errors if there is no information available
- ++shallow_errors;
- errorstream << pgid << " shard " << j->first << " " << *k << " : missing\n";
- }
- object_error.add_shard(j->first, shard_map[j->first]);
- }
-
- if (auth_list.empty()) {
- if (object_errors.empty()) {
- errorstream << pgid.pgid << " soid " << *k
- << " : failed to pick suitable auth object\n";
- goto out;
- }
- // Object errors exist and nothing in auth_list
- // Prefer the auth shard otherwise take first from list.
- pg_shard_t shard;
- if (object_errors.count(auth->first)) {
- shard = auth->first;
- } else {
- shard = *(object_errors.begin());
- }
- auth_list.push_back(shard);
- object_errors.erase(shard);
- }
- // At this point auth_list is populated, so we add the object errors shards
- // as inconsistent.
- cur_inconsistent.insert(object_errors.begin(), object_errors.end());
- if (!cur_missing.empty()) {
- missing[*k] = cur_missing;
- }
- if (!cur_inconsistent.empty()) {
- inconsistent[*k] = cur_inconsistent;
- }
-
- if (fix_digest) {
- std::optional<uint32_t> data_digest, omap_digest;
- ceph_assert(auth_object.digest_present);
- data_digest = auth_object.digest;
- if (auth_object.omap_digest_present) {
- omap_digest = auth_object.omap_digest;
- }
- missing_digest[*k] = make_pair(data_digest, omap_digest);
- }
- if (!cur_inconsistent.empty() || !cur_missing.empty()) {
- authoritative[*k] = auth_list;
- } else if (!fix_digest && parent->get_pool().is_replicated()) {
- enum {
- NO = 0,
- MAYBE = 1,
- FORCE = 2,
- } update = NO;
-
- if (auth_object.digest_present && !auth_oi.is_data_digest()) {
- dout(20) << __func__ << " missing data digest on " << *k << dendl;
- update = MAYBE;
- }
- if (auth_object.omap_digest_present && !auth_oi.is_omap_digest()) {
- dout(20) << __func__ << " missing omap digest on " << *k << dendl;
- update = MAYBE;
- }
-
- // recorded digest != actual digest?
- if (auth_oi.is_data_digest() && auth_object.digest_present &&
- auth_oi.data_digest != auth_object.digest) {
- ceph_assert(shard_map[auth->first].has_data_digest_mismatch_info());
- errorstream << pgid << " recorded data digest 0x"
- << std::hex << auth_oi.data_digest << " != on disk 0x"
- << auth_object.digest << std::dec << " on " << auth_oi.soid
- << "\n";
- if (repair)
- update = FORCE;
- }
- if (auth_oi.is_omap_digest() && auth_object.omap_digest_present &&
- auth_oi.omap_digest != auth_object.omap_digest) {
- ceph_assert(shard_map[auth->first].has_omap_digest_mismatch_info());
- errorstream << pgid << " recorded omap digest 0x"
- << std::hex << auth_oi.omap_digest << " != on disk 0x"
- << auth_object.omap_digest << std::dec
- << " on " << auth_oi.soid << "\n";
- if (repair)
- update = FORCE;
- }
-
- if (update != NO) {
- utime_t age = now - auth_oi.local_mtime;
- if (update == FORCE ||
- age > cct->_conf->osd_deep_scrub_update_digest_min_age) {
- std::optional<uint32_t> data_digest, omap_digest;
- if (auth_object.digest_present) {
- data_digest = auth_object.digest;
- dout(20) << __func__ << " will update data digest on " << *k << dendl;
- }
- if (auth_object.omap_digest_present) {
- omap_digest = auth_object.omap_digest;
- dout(20) << __func__ << " will update omap digest on " << *k << dendl;
- }
- missing_digest[*k] = make_pair(data_digest, omap_digest);
- } else {
- dout(20) << __func__ << " missing digest but age " << age
- << " < " << cct->_conf->osd_deep_scrub_update_digest_min_age
- << " on " << *k << dendl;
- }
- }
- }
-out:
- if (object_error.has_deep_errors())
- ++deep_errors;
- else if (object_error.has_shallow_errors())
- ++shallow_errors;
- if (object_error.errors || object_error.union_shards.errors) {
- store->add_object_error(k->pool, object_error);
- }
- }
-}
-
-void PGBackend::be_omap_checks(const map<pg_shard_t,ScrubMap*> &maps,
- const set<hobject_t> &master_set,
- omap_stat_t& omap_stats,
- ostream &warnstream) const
-{
- bool needs_omap_check = false;
- for (const auto& map : maps) {
- if (map.second->has_large_omap_object_errors || map.second->has_omap_keys) {
- needs_omap_check = true;
- break;
- }
- }
-
- if (!needs_omap_check) {
- return; // Nothing to do
- }
-
- // Iterate through objects and update omap stats
- for (const auto& k : master_set) {
- for (const auto& map : maps) {
- if (map.first != get_parent()->primary_shard()) {
- // Only set omap stats for the primary
- continue;
- }
- auto it = map.second->objects.find(k);
- if (it == map.second->objects.end())
- continue;
- ScrubMap::object& obj = it->second;
- omap_stats.omap_bytes += obj.object_omap_bytes;
- omap_stats.omap_keys += obj.object_omap_keys;
- if (obj.large_omap_object_found) {
- pg_t pg;
- auto osdmap = get_osdmap();
- osdmap->map_to_pg(k.pool, k.oid.name, k.get_key(), k.nspace, &pg);
- pg_t mpg = osdmap->raw_pg_to_pg(pg);
- omap_stats.large_omap_objects++;
- warnstream << "Large omap object found. Object: " << k
- << " PG: " << pg << " (" << mpg << ")"
- << " Key count: " << obj.large_omap_object_key_count
- << " Size (bytes): " << obj.large_omap_object_value_size
- << '\n';
- break;
- }
- }
- }
-}
int be_scan_list(
ScrubMap &map,
ScrubMapBuilder &pos);
- bool be_compare_scrub_objects(
- pg_shard_t auth_shard,
- const ScrubMap::object &auth,
- const object_info_t& auth_oi,
- const ScrubMap::object &candidate,
- shard_info_wrapper& shard_error,
- inconsistent_obj_wrapper &result,
- std::ostream &errorstream,
- bool has_snapset);
- std::map<pg_shard_t, ScrubMap *>::const_iterator be_select_auth_object(
- const hobject_t &obj,
- const std::map<pg_shard_t,ScrubMap*> &maps,
- object_info_t *auth_oi,
- std::map<pg_shard_t, shard_info_wrapper> &shard_map,
- bool &digest_match,
- spg_t pgid,
- std::ostream &errorstream);
- void be_compare_scrubmaps(
- const std::map<pg_shard_t,ScrubMap*> &maps,
- const std::set<hobject_t> &master_set,
- bool repair,
- std::map<hobject_t, std::set<pg_shard_t>> &missing,
- std::map<hobject_t, std::set<pg_shard_t>> &inconsistent,
- std::map<hobject_t, std::list<pg_shard_t>> &authoritative,
- std::map<hobject_t, std::pair<std::optional<uint32_t>,
- std::optional<uint32_t>>> &missing_digest,
- int &shallow_errors, int &deep_errors,
- Scrub::Store *store,
- const spg_t& pgid,
- const std::vector<int> &acting,
- std::ostream &errorstream);
+
virtual uint64_t be_get_ondisk_size(
uint64_t logical_size) = 0;
virtual int be_deep_scrub(
ScrubMap &map,
ScrubMapBuilder &pos,
ScrubMap::object &o) = 0;
- void be_omap_checks(
- const std::map<pg_shard_t,ScrubMap*> &maps,
- const std::set<hobject_t> &master_set,
- omap_stat_t& omap_stats,
- std::ostream &warnstream) const;
static PGBackend *build_pg_backend(
const pg_pool_t &pool,
return pgid.calc_name(buf, "");
}
+std::string spg_t::calc_name_sring() const
+{
+ char buf[spg_t::calc_name_buf_size];
+ buf[spg_t::calc_name_buf_size - 1] = '\0';
+ return string{calc_name(buf + spg_t::calc_name_buf_size - 1, "")};
+}
+
ostream& operator<<(ostream& out, const spg_t &pg)
{
char buf[spg_t::calc_name_buf_size];
static const uint8_t calc_name_buf_size = pg_t::calc_name_buf_size + 4; // 36 + len('s') + len("255");
char *calc_name(char *buf, const char *suffix_backwords) const;
+ // and a (limited) version that uses an internal buffer:
+ std::string calc_name_sring() const;
bool parse(const char *s);
bool parse(const std::string& s) {
void init_pg_ondisk(
ceph::os::Transaction& t, spg_t pgid, const pg_pool_t *pool);
-// omap specific stats
-struct omap_stat_t {
- int large_omap_objects;
- int64_t omap_bytes;
- int64_t omap_keys;
-};
-
// filter for pg listings
class PGLSFilter {
CephContext* cct;
#include "common/hobject_fmt.h"
#include "osd/osd_types.h"
+#include "include/types_fmt.h"
template <>
struct fmt::formatter<osd_reqid_t> {
return true;
}
+// forwarders used by the scrubber backend
+
+/// \todo combine the multiple transactions into a single one
+void PrimaryLogScrub::submit_digest_fixes(const digests_fixes_t& fixes)
+{
+ num_digest_updates_pending += fixes.size();
+ dout(10) << __func__
+ << ": num_digest_updates_pending: " << num_digest_updates_pending
+ << dendl;
+
+ for (auto& [obj, dgs] : fixes) {
+
+ ObjectContextRef obc = m_pl_pg->get_object_context(obj, false);
+ if (!obc) {
+ m_osds->clog->error() << m_pg_id << " " << m_mode_desc
+ << " cannot get object context for object " << obj;
+ num_digest_updates_pending--;
+ continue;
+ }
+ if (obc->obs.oi.soid != obj) {
+ m_osds->clog->error()
+ << m_pg_id << " " << m_mode_desc << " " << obj
+ << " : object has a valid oi attr with a mismatched name, "
+ << " obc->obs.oi.soid: " << obc->obs.oi.soid;
+ num_digest_updates_pending--;
+ continue;
+ }
+
+ PrimaryLogPG::OpContextUPtr ctx = m_pl_pg->simple_opc_create(obc);
+ ctx->at_version = m_pl_pg->get_next_version();
+ ctx->mtime = utime_t(); // do not update mtime
+ if (dgs.first) {
+ ctx->new_obs.oi.set_data_digest(*dgs.first);
+ } else {
+ ctx->new_obs.oi.clear_data_digest();
+ }
+ if (dgs.second) {
+ ctx->new_obs.oi.set_omap_digest(*dgs.second);
+ } else {
+ ctx->new_obs.oi.clear_omap_digest();
+ }
+ m_pl_pg->finish_ctx(ctx.get(), pg_log_entry_t::MODIFY);
+
+
+ ctx->register_on_success([this]() {
+ dout(20) << "updating scrub digest " << num_digest_updates_pending
+ << dendl;
+ if (--num_digest_updates_pending <= 0) {
+ m_osds->queue_scrub_digest_update(m_pl_pg,
+ m_pl_pg->is_scrub_blocking_ops());
+ }
+ });
+
+ m_pl_pg->simple_opc_submit(std::move(ctx));
+ }
+}
+
+
+void PrimaryLogScrub::add_to_stats(const object_stat_sum_t& stat)
+{
+ m_scrub_cstat.add(stat);
+}
+
+
void PrimaryLogScrub::_scrub_finish()
{
auto& info = m_pg->get_pg_info(ScrubberPasskey{}); ///< a temporary alias
dout(10) << __func__
<< " info stats: " << (info.stats.stats_invalid ? "invalid" : "valid")
+ << " m_is_repair: " << m_is_repair
<< dendl;
if (info.stats.stats_invalid) {
m_pl_pg->object_contexts.clear();
}
-static bool doing_clones(const std::optional<SnapSet>& snapset,
- const vector<snapid_t>::reverse_iterator& curclone)
-{
- return snapset && curclone != snapset->clones.rend();
-}
-
-void PrimaryLogScrub::log_missing(int missing,
- const std::optional<hobject_t>& head,
- LogChannelRef clog,
- const spg_t& pgid,
- const char* func,
- bool allow_incomplete_clones)
-{
- ceph_assert(head);
- if (allow_incomplete_clones) {
- dout(20) << func << " " << m_mode_desc << " " << pgid << " " << *head << " skipped "
- << missing << " clone(s) in cache tier" << dendl;
- } else {
- clog->info() << m_mode_desc << " " << pgid << " " << *head << " : " << missing
- << " missing clone(s)";
- }
-}
-
-int PrimaryLogScrub::process_clones_to(const std::optional<hobject_t>& head,
- const std::optional<SnapSet>& snapset,
- LogChannelRef clog,
- const spg_t& pgid,
- bool allow_incomplete_clones,
- std::optional<snapid_t> target,
- vector<snapid_t>::reverse_iterator* curclone,
- inconsistent_snapset_wrapper& e)
-{
- ceph_assert(head);
- ceph_assert(snapset);
- int missing_count = 0;
-
- // NOTE: clones are in descending order, thus **curclone > target test here
- hobject_t next_clone(*head);
- while (doing_clones(snapset, *curclone) && (!target || **curclone > *target)) {
-
- ++missing_count;
- // it is okay to be missing one or more clones in a cache tier.
- // skip higher-numbered clones in the list.
- if (!allow_incomplete_clones) {
- next_clone.snap = **curclone;
- clog->error() << m_mode_desc << " " << pgid << " " << *head << " : expected clone "
- << next_clone << " " << m_missing << " missing";
- ++m_shallow_errors;
- e.set_clone_missing(next_clone.snap);
- }
- // Clones are descending
- ++(*curclone);
- }
- return missing_count;
-}
-
-/*
- * Validate consistency of the object info and snap sets.
- *
- * We are sort of comparing 2 lists. The main loop is on objmap.objects. But
- * the comparison of the objects is against multiple snapset.clones. There are
- * multiple clone lists and in between lists we expect head.
- *
- * Example
- *
- * objects expected
- * ======= =======
- * obj1 snap 1 head, unexpected obj1 snap 1
- * obj2 head head, match
- * [SnapSet clones 6 4 2 1]
- * obj2 snap 7 obj2 snap 6, unexpected obj2 snap 7
- * obj2 snap 6 obj2 snap 6, match
- * obj2 snap 4 obj2 snap 4, match
- * obj3 head obj2 snap 2 (expected), obj2 snap 1 (expected), match
- * [Snapset clones 3 1]
- * obj3 snap 3 obj3 snap 3 match
- * obj3 snap 1 obj3 snap 1 match
- * obj4 head head, match
- * [Snapset clones 4]
- * EOL obj4 snap 4, (expected)
- */
-void PrimaryLogScrub::scrub_snapshot_metadata(ScrubMap& scrubmap,
- const missing_map_t& missing_digest)
-{
- dout(10) << __func__ << " num stat obj " << m_pl_pg->info.stats.stats.sum.num_objects
- << dendl;
-
- auto& info = m_pl_pg->info;
- const PGPool& pool = m_pl_pg->pool;
- bool allow_incomplete_clones = pool.info.allow_incomplete_clones();
-
- std::optional<snapid_t> all_clones; // Unspecified snapid_t or std::nullopt
-
- // traverse in reverse order.
- std::optional<hobject_t> head;
- std::optional<SnapSet> snapset; // If initialized so will head (above)
- vector<snapid_t>::reverse_iterator curclone; // Defined only if snapset initialized
- int missing = 0;
- inconsistent_snapset_wrapper soid_error, head_error;
- int soid_error_count = 0;
-
- for (auto p = scrubmap.objects.rbegin(); p != scrubmap.objects.rend(); ++p) {
-
- const hobject_t& soid = p->first;
- ceph_assert(!soid.is_snapdir());
- soid_error = inconsistent_snapset_wrapper{soid};
- object_stat_sum_t stat;
- std::optional<object_info_t> oi;
-
- stat.num_objects++;
-
- if (soid.nspace == m_pl_pg->cct->_conf->osd_hit_set_namespace)
- stat.num_objects_hit_set_archive++;
-
- if (soid.is_snap()) {
- // it's a clone
- stat.num_object_clones++;
- }
-
- // basic checks.
- if (p->second.attrs.count(OI_ATTR) == 0) {
- oi = std::nullopt;
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid << " : no '"
- << OI_ATTR << "' attr";
- ++m_shallow_errors;
- soid_error.set_info_missing();
- } else {
- bufferlist bv;
- bv.push_back(p->second.attrs[OI_ATTR]);
- try {
- oi = object_info_t(bv);
- } catch (ceph::buffer::error& e) {
- oi = std::nullopt;
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid
- << " : can't decode '" << OI_ATTR << "' attr " << e.what();
- ++m_shallow_errors;
- soid_error.set_info_corrupted();
- soid_error.set_info_missing(); // Not available too
- }
- }
-
- if (oi) {
- if (m_pl_pg->pgbackend->be_get_ondisk_size(oi->size) != p->second.size) {
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid
- << " : on disk size (" << p->second.size
- << ") does not match object info size (" << oi->size
- << ") adjusted for ondisk to ("
- << m_pl_pg->pgbackend->be_get_ondisk_size(oi->size) << ")";
- soid_error.set_size_mismatch();
- ++m_shallow_errors;
- }
-
- dout(20) << m_mode_desc << " " << soid << " " << *oi << dendl;
-
- // A clone num_bytes will be added later when we have snapset
- if (!soid.is_snap()) {
- stat.num_bytes += oi->size;
- }
- if (soid.nspace == m_pl_pg->cct->_conf->osd_hit_set_namespace)
- stat.num_bytes_hit_set_archive += oi->size;
-
- if (oi->is_dirty())
- ++stat.num_objects_dirty;
- if (oi->is_whiteout())
- ++stat.num_whiteouts;
- if (oi->is_omap())
- ++stat.num_objects_omap;
- if (oi->is_cache_pinned())
- ++stat.num_objects_pinned;
- if (oi->has_manifest())
- ++stat.num_objects_manifest;
- }
-
- // Check for any problems while processing clones
- if (doing_clones(snapset, curclone)) {
- std::optional<snapid_t> target;
- // Expecting an object with snap for current head
- if (soid.has_snapset() || soid.get_head() != head->get_head()) {
-
- dout(10) << __func__ << " " << m_mode_desc << " " << info.pgid << " new object " << soid
- << " while processing " << *head << dendl;
-
- target = all_clones;
- } else {
- ceph_assert(soid.is_snap());
- target = soid.snap;
- }
-
- // Log any clones we were expecting to be there up to target
- // This will set missing, but will be a no-op if snap.soid == *curclone.
- missing +=
- process_clones_to(head, snapset, m_osds->clog, info.pgid,
- allow_incomplete_clones, target, &curclone, head_error);
- }
-
- bool expected;
- // Check doing_clones() again in case we ran process_clones_to()
- if (doing_clones(snapset, curclone)) {
- // A head would have processed all clones above
- // or all greater than *curclone.
- ceph_assert(soid.is_snap() && *curclone <= soid.snap);
-
- // After processing above clone snap should match the expected curclone
- expected = (*curclone == soid.snap);
- } else {
- // If we aren't doing clones any longer, then expecting head
- expected = soid.has_snapset();
- }
- if (!expected) {
- // If we couldn't read the head's snapset, just ignore clones
- if (head && !snapset) {
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid
- << " : clone ignored due to missing snapset";
- } else {
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid
- << " : is an unexpected clone";
- }
- ++m_shallow_errors;
- soid_error.set_headless();
- m_store->add_snap_error(pool.id, soid_error);
- ++soid_error_count;
- if (head && soid.get_head() == head->get_head())
- head_error.set_clone(soid.snap);
- continue;
- }
-
- // new snapset?
- if (soid.has_snapset()) {
-
- if (missing) {
- log_missing(missing, head, m_osds->clog, info.pgid, __func__,
- pool.info.allow_incomplete_clones());
- }
-
- // Save previous head error information
- if (head && (head_error.errors || soid_error_count))
- m_store->add_snap_error(pool.id, head_error);
- // Set this as a new head object
- head = soid;
- missing = 0;
- head_error = soid_error;
- soid_error_count = 0;
-
- dout(20) << __func__ << " " << m_mode_desc << " new head " << head << dendl;
-
- if (p->second.attrs.count(SS_ATTR) == 0) {
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid << " : no '"
- << SS_ATTR << "' attr";
- ++m_shallow_errors;
- snapset = std::nullopt;
- head_error.set_snapset_missing();
- } else {
- bufferlist bl;
- bl.push_back(p->second.attrs[SS_ATTR]);
- auto blp = bl.cbegin();
- try {
- snapset = SnapSet(); // Initialize optional<> before decoding into it
- decode(*snapset, blp);
- head_error.ss_bl.push_back(p->second.attrs[SS_ATTR]);
- } catch (ceph::buffer::error& e) {
- snapset = std::nullopt;
- m_osds->clog->error()
- << m_mode_desc << " " << info.pgid << " " << soid << " : can't decode '" << SS_ATTR
- << "' attr " << e.what();
- ++m_shallow_errors;
- head_error.set_snapset_corrupted();
- }
- }
-
- if (snapset) {
- // what will be next?
- curclone = snapset->clones.rbegin();
-
- if (!snapset->clones.empty()) {
- dout(20) << " snapset " << *snapset << dendl;
- if (snapset->seq == 0) {
- m_osds->clog->error()
- << m_mode_desc << " " << info.pgid << " " << soid << " : snaps.seq not set";
- ++m_shallow_errors;
- head_error.set_snapset_error();
- }
- }
- }
- } else {
- ceph_assert(soid.is_snap());
- ceph_assert(head);
- ceph_assert(snapset);
- ceph_assert(soid.snap == *curclone);
-
- dout(20) << __func__ << " " << m_mode_desc << " matched clone " << soid << dendl;
-
- if (snapset->clone_size.count(soid.snap) == 0) {
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid
- << " : is missing in clone_size";
- ++m_shallow_errors;
- soid_error.set_size_mismatch();
- } else {
- if (oi && oi->size != snapset->clone_size[soid.snap]) {
- m_osds->clog->error()
- << m_mode_desc << " " << info.pgid << " " << soid << " : size " << oi->size
- << " != clone_size " << snapset->clone_size[*curclone];
- ++m_shallow_errors;
- soid_error.set_size_mismatch();
- }
-
- if (snapset->clone_overlap.count(soid.snap) == 0) {
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid
- << " : is missing in clone_overlap";
- ++m_shallow_errors;
- soid_error.set_size_mismatch();
- } else {
- // This checking is based on get_clone_bytes(). The first 2 asserts
- // can't happen because we know we have a clone_size and
- // a clone_overlap. Now we check that the interval_set won't
- // cause the last assert.
- uint64_t size = snapset->clone_size.find(soid.snap)->second;
- const interval_set<uint64_t>& overlap =
- snapset->clone_overlap.find(soid.snap)->second;
- bool bad_interval_set = false;
- for (interval_set<uint64_t>::const_iterator i = overlap.begin();
- i != overlap.end(); ++i) {
- if (size < i.get_len()) {
- bad_interval_set = true;
- break;
- }
- size -= i.get_len();
- }
-
- if (bad_interval_set) {
- m_osds->clog->error() << m_mode_desc << " " << info.pgid << " " << soid
- << " : bad interval_set in clone_overlap";
- ++m_shallow_errors;
- soid_error.set_size_mismatch();
- } else {
- stat.num_bytes += snapset->get_clone_bytes(soid.snap);
- }
- }
- }
-
- // what's next?
- ++curclone;
- if (soid_error.errors) {
- m_store->add_snap_error(pool.id, soid_error);
- ++soid_error_count;
- }
- }
- m_scrub_cstat.add(stat);
- }
-
- if (doing_clones(snapset, curclone)) {
- dout(10) << __func__ << " " << m_mode_desc << " " << info.pgid
- << " No more objects while processing " << *head << dendl;
-
- missing +=
- process_clones_to(head, snapset, m_osds->clog, info.pgid,
- allow_incomplete_clones, all_clones, &curclone, head_error);
- }
-
- // There could be missing found by the test above or even
- // before dropping out of the loop for the last head.
- if (missing) {
- log_missing(missing, head, m_osds->clog, info.pgid, __func__,
- allow_incomplete_clones);
- }
- if (head && (head_error.errors || soid_error_count))
- m_store->add_snap_error(pool.id, head_error);
-
- dout(20) << __func__ << " - " << missing << " (" << missing_digest.size() << ") missing"
- << dendl;
- for (auto p = missing_digest.begin(); p != missing_digest.end(); ++p) {
-
- ceph_assert(!p->first.is_snapdir());
- dout(10) << __func__ << " recording digests for " << p->first << dendl;
-
- ObjectContextRef obc = m_pl_pg->get_object_context(p->first, false);
- if (!obc) {
- m_osds->clog->error() << info.pgid << " " << m_mode_desc
- << " cannot get object context for object " << p->first;
- continue;
- }
- if (obc->obs.oi.soid != p->first) {
- m_osds->clog->error() << info.pgid << " " << m_mode_desc << " " << p->first
- << " : object has a valid oi attr with a mismatched name, "
- << " obc->obs.oi.soid: " << obc->obs.oi.soid;
- continue;
- }
- PrimaryLogPG::OpContextUPtr ctx = m_pl_pg->simple_opc_create(obc);
- ctx->at_version = m_pl_pg->get_next_version();
- ctx->mtime = utime_t(); // do not update mtime
- if (p->second.first) {
- ctx->new_obs.oi.set_data_digest(*p->second.first);
- } else {
- ctx->new_obs.oi.clear_data_digest();
- }
- if (p->second.second) {
- ctx->new_obs.oi.set_omap_digest(*p->second.second);
- } else {
- ctx->new_obs.oi.clear_omap_digest();
- }
- m_pl_pg->finish_ctx(ctx.get(), pg_log_entry_t::MODIFY);
-
- ++num_digest_updates_pending;
- ctx->register_on_success([this]() {
- dout(20) << "updating scrub digest " << num_digest_updates_pending << dendl;
- 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());
- }
- });
-
- m_pl_pg->simple_opc_submit(std::move(ctx));
- }
-
- dout(10) << __func__ << " (" << m_mode_desc << ") finish" << dendl;
-}
-
PrimaryLogScrub::PrimaryLogScrub(PrimaryLogPG* pg) : PgScrubber{pg}, m_pl_pg{pg} {}
void PrimaryLogScrub::_scrub_clear_state()
void stats_of_handled_objects(const object_stat_sum_t& delta_stats,
const hobject_t& soid) final;
- private:
- // we know our PG is actually a PrimaryLogPG. Let's alias the pointer to that object:
- PrimaryLogPG* const m_pl_pg;
-
- /**
- * Validate consistency of the object info and snap sets.
- */
- void scrub_snapshot_metadata(ScrubMap& map, const missing_map_t& missing_digest) final;
+ // the interface used by the scrubber-backend:
- void log_missing(int missing,
- const std::optional<hobject_t>& head,
- LogChannelRef clog,
- const spg_t& pgid,
- const char* func,
- bool allow_incomplete_clones);
+ void add_to_stats(const object_stat_sum_t& stat) final;
- int process_clones_to(const std::optional<hobject_t>& head,
- const std::optional<SnapSet>& snapset,
- LogChannelRef clog,
- const spg_t& pgid,
- bool allow_incomplete_clones,
- std::optional<snapid_t> target,
- std::vector<snapid_t>::reverse_iterator* curclone,
- inconsistent_snapset_wrapper& snap_error);
+ void submit_digest_fixes(const digests_fixes_t& fixes) final;
+ private:
+ // we know our PG is actually a PrimaryLogPG. Let's alias the pointer to that object:
+ PrimaryLogPG* const m_pl_pg;
// handle our part in stats collection
object_stat_collection_t m_scrub_cstat;
#include "osd/OSD.h"
#include "osd/osd_types_fmt.h"
#include "ScrubStore.h"
+#include "scrub_backend.h"
#include "scrub_machine.h"
using std::list;
*/
bool PgScrubber::select_range()
{
- m_primary_scrubmap = ScrubMap{};
- m_received_maps.clear();
+ m_primary_scrubmap = m_be->new_chunk();
/* get the start and end of our scrub chunk
*
dout(10) << __func__ << " start same_interval:" << m_interval_start << dendl;
+ m_be = std::make_unique<ScrubBackend>(
+ *this,
+ *(m_pg->get_pgbackend()),
+ *m_pg,
+ m_pg_whoami,
+ m_is_repair,
+ m_is_deep ? scrub_level_t::deep : scrub_level_t::shallow,
+ m_pg->get_acting_recovery_backfill());
+
// create a new store
{
ObjectStore::Transaction t;
void PgScrubber::on_replica_init()
{
+ m_be = std::make_unique<ScrubBackend>(
+ *this,
+ *(m_pg->get_pgbackend()),
+ *m_pg,
+ m_pg_whoami,
+ m_is_repair,
+ m_is_deep ? scrub_level_t::deep : scrub_level_t::shallow);
m_active = true;
++m_sessions_counter;
}
-void PgScrubber::_scan_snaps(ScrubMap& smap)
-{
- hobject_t head;
- SnapSet snapset;
-
- // Test qa/standalone/scrub/osd-scrub-snaps.sh greps for the strings
- // in this function
- dout(15) << "_scan_snaps starts" << dendl;
-
- for (auto i = smap.objects.rbegin(); i != smap.objects.rend(); ++i) {
-
- const hobject_t& hoid = i->first;
- ScrubMap::object& o = i->second;
-
- dout(20) << __func__ << " " << hoid << dendl;
-
- ceph_assert(!hoid.is_snapdir());
- if (hoid.is_head()) {
- // parse the SnapSet
- bufferlist bl;
- if (o.attrs.find(SS_ATTR) == o.attrs.end()) {
- continue;
- }
- bl.push_back(o.attrs[SS_ATTR]);
- auto p = bl.cbegin();
- try {
- decode(snapset, p);
- } catch (...) {
- continue;
- }
- head = hoid.get_head();
- continue;
- }
-
- if (hoid.snap < CEPH_MAXSNAP) {
- // check and if necessary fix snap_mapper
- if (hoid.get_head() != head) {
- derr << __func__ << " no head for " << hoid << " (have " << head << ")" << dendl;
- continue;
- }
- set<snapid_t> obj_snaps;
- auto p = snapset.clone_snaps.find(hoid.snap);
- if (p == snapset.clone_snaps.end()) {
- derr << __func__ << " no clone_snaps for " << hoid << " in " << snapset << dendl;
- continue;
- }
- obj_snaps.insert(p->second.begin(), p->second.end());
- set<snapid_t> cur_snaps;
- int r = m_pg->snap_mapper.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();
- }
- m_pg->osd->clog->error()
- << "osd." << m_pg->osd->whoami << " found snap mapper error on pg "
- << m_pg->info.pgid << " oid " << hoid << " snaps in mapper: " << cur_snaps
- << ", oi: " << obj_snaps << "...repaired";
- } else {
- m_pg->osd->clog->error()
- << "osd." << m_pg->osd->whoami << " found snap mapper error on pg "
- << m_pg->info.pgid << " 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;
-
- 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; });
- }
- }
- }
- }
- }
-}
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;
- auto ret = build_scrub_map_chunk(m_primary_scrubmap, m_primary_scrubmap_pos, m_start,
- m_end, m_is_deep);
+ auto ret = build_scrub_map_chunk(*m_primary_scrubmap,
+ m_primary_scrubmap_pos,
+ m_start,
+ m_end,
+ m_is_deep);
if (ret == -EINPROGRESS) {
// reschedule another round of asking the backend to collect the scrub data
return ret;
}
+
int PgScrubber::build_replica_map_chunk()
{
dout(10) << __func__ << " interval start: " << m_interval_start
<< " current token: " << m_current_token << " epoch: " << m_epoch_start
<< " deep: " << m_is_deep << dendl;
+ ceph_assert(m_be);
+
auto ret = build_scrub_map_chunk(replica_scrubmap, replica_scrubmap_pos, m_start, m_end,
m_is_deep);
case 0: {
// finished!
- m_cleaned_meta_map.clear_from(m_start);
- m_cleaned_meta_map.insert(replica_scrubmap);
- auto for_meta_scrub = clean_meta_map();
- _scan_snaps(for_meta_scrub);
+
+ m_be->replica_clean_meta(replica_scrubmap, m_end.is_max(), m_start);
// the local map has been created. Send it to the primary.
// Note: once the message reaches the Primary, it may ask us for another
// finish
dout(20) << __func__ << " finishing" << dendl;
ceph_assert(pos.done());
- m_pg->_repair_oinfo_oid(map);
+ m_be->repair_oinfo_oid(map);
dout(20) << __func__ << " done, got " << map.objects.size() << " items" << dendl;
return 0;
}
-/*
- * Process:
- * Building a map of objects suitable for snapshot validation.
- * The data in m_cleaned_meta_map is the left over partial items that need to
- * be completed before they can be processed.
- *
- * Snapshots in maps precede the head object, which is why we are scanning backwards.
- */
-ScrubMap PgScrubber::clean_meta_map()
-{
- ScrubMap for_meta_scrub;
-
- if (m_end.is_max() || m_cleaned_meta_map.objects.empty()) {
- m_cleaned_meta_map.swap(for_meta_scrub);
- } else {
- auto iter = m_cleaned_meta_map.objects.end();
- --iter; // not empty, see 'if' clause
- auto begin = m_cleaned_meta_map.objects.begin();
- if (iter->first.has_snapset()) {
- ++iter;
- } else {
- while (iter != begin) {
- auto next = iter--;
- if (next->first.get_head() != iter->first.get_head()) {
- ++iter;
- break;
- }
- }
- }
- for_meta_scrub.objects.insert(begin, iter);
- m_cleaned_meta_map.objects.erase(begin, iter);
- }
-
- return for_meta_scrub;
-}
void PgScrubber::run_callbacks()
{
void PgScrubber::maps_compare_n_cleanup()
{
- scrub_compare_maps();
+ m_pg->add_objects_scrubbed_count(m_be->get_primary_scrubmap().objects.size());
+ m_be->scrub_compare_maps(m_end.is_max());
m_start = m_end;
run_callbacks();
requeue_waiting();
preemption_data.reset();
preemption_data.force_preemptability(msg->allow_preemption);
- replica_scrubmap_pos.reset();
+ replica_scrubmap_pos.reset(); // needed? RRR
set_queued_or_active();
m_osds->queue_for_rep_scrub(m_pg, m_replica_request_priority,
m_flags.deep_scrub_on_error = request.deep_scrub_on_error;
}
-void PgScrubber::scrub_compare_maps()
-{
- dout(10) << __func__ << " has maps, analyzing" << dendl;
-
- // construct authoritative scrub map for type-specific scrubbing
- m_cleaned_meta_map.insert(m_primary_scrubmap);
- map<hobject_t, pair<std::optional<uint32_t>, std::optional<uint32_t>>> missing_digest;
-
- map<pg_shard_t, ScrubMap*> maps;
- maps[m_pg_whoami] = &m_primary_scrubmap;
-
- for (const auto& i : m_pg->get_acting_recovery_backfill()) {
- if (i == m_pg_whoami)
- continue;
- dout(2) << __func__ << " replica " << i << " has "
- << m_received_maps[i].objects.size() << " items" << dendl;
- maps[i] = &m_received_maps[i];
- }
-
- set<hobject_t> master_set;
-
- // Construct master set
- for (const auto& map : maps) {
- for (const auto& i : map.second->objects) {
- master_set.insert(i.first);
- }
- }
-
- stringstream ss;
- m_pg->get_pgbackend()->be_omap_checks(maps, master_set, m_omap_stats, ss);
-
- if (!ss.str().empty()) {
- m_osds->clog->warn(ss);
- }
-
- if (m_pg->recovery_state.get_acting_recovery_backfill().size() > 1) {
-
- dout(10) << __func__ << " comparing replica scrub maps" << dendl;
-
- // Map from object with errors to good peer
- map<hobject_t, list<pg_shard_t>> authoritative;
-
- dout(2) << __func__ << ": primary (" << m_pg->get_primary() << ") has "
- << m_primary_scrubmap.objects.size() << " items" << dendl;
- m_pg->add_objects_scrubbed_count(m_primary_scrubmap.objects.size());
-
- ss.str("");
- ss.clear();
-
- m_pg->get_pgbackend()->be_compare_scrubmaps(
- maps, master_set, m_is_repair, m_missing, m_inconsistent,
- authoritative, missing_digest, m_shallow_errors, m_deep_errors, m_store.get(),
- m_pg->info.pgid, m_pg->recovery_state.get_acting(), ss);
-
- if (!ss.str().empty()) {
- m_osds->clog->error(ss);
- }
-
- for (auto& i : authoritative) {
- list<pair<ScrubMap::object, pg_shard_t>> good_peers;
- for (list<pg_shard_t>::const_iterator j = i.second.begin(); j != i.second.end();
- ++j) {
- good_peers.emplace_back(maps[*j]->objects[i.first], *j);
- }
- m_authoritative.emplace(i.first, good_peers);
- }
-
- for (auto i = authoritative.begin(); i != authoritative.end(); ++i) {
- m_cleaned_meta_map.objects.erase(i->first);
- m_cleaned_meta_map.objects.insert(
- *(maps[i->second.back()]->objects.find(i->first)));
- }
- }
-
- auto for_meta_scrub = clean_meta_map();
-
- // ok, do the pg-type specific scrubbing
-
- // (Validates consistency of the object info and snap sets)
- scrub_snapshot_metadata(for_meta_scrub, missing_digest);
-
- // Called here on the primary can use an authoritative map if it isn't the primary
- _scan_snaps(for_meta_scrub);
-
- if (!m_store->empty()) {
-
- if (m_is_repair) {
- dout(10) << __func__ << ": discarding scrub results" << dendl;
- m_store->flush(nullptr);
- } else {
- dout(10) << __func__ << ": updating scrub object" << dendl;
- ObjectStore::Transaction t;
- m_store->flush(&t);
- m_pg->osd->store->queue_transaction(m_pg->ch, std::move(t), nullptr);
- }
- }
-}
ScrubMachineListener::MsgAndEpoch PgScrubber::prep_replica_map_msg(
PreemptionNoted was_preempted)
return;
}
- auto p = const_cast<bufferlist&>(m->get_data()).cbegin();
-
- m_received_maps[m->from].decode(p, m_pg->info.pgid.pool());
- dout(15) << "map version is " << m_received_maps[m->from].valid_through << dendl;
+ // note: we check for active() before map_from_replica() is called. Thus, we
+ // know m_be is initialized
+ m_be->decode_received_map(m->from, *m);
auto [is_ok, err_txt] = m_maps_status.mark_arriving_map(m->from);
if (!is_ok) {
return m_queued_or_active;
}
-[[nodiscard]] bool PgScrubber::scrub_process_inconsistent()
-{
- dout(10) << __func__ << ": checking authoritative (mode="
- << m_mode_desc << ", auth remaining #: " << m_authoritative.size()
- << ")" << dendl;
-
- // authoritative only store objects which are missing or inconsistent.
- if (!m_authoritative.empty()) {
-
- stringstream ss;
- ss << m_pg->info.pgid << " " << m_mode_desc << " " << m_missing.size() << " missing, "
- << m_inconsistent.size() << " inconsistent objects";
- dout(2) << ss.str() << dendl;
- m_osds->clog->error(ss);
-
- if (m_is_repair) {
- 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
- state_set(PG_STATE_REPAIR);
- update_op_mode_text();
-
- for (const auto& [hobj, shrd_list] : m_authoritative) {
-
- auto missing_entry = m_missing.find(hobj);
-
- if (missing_entry != m_missing.end()) {
- m_pg->repair_object(hobj, shrd_list, missing_entry->second);
- m_fixed_count += missing_entry->second.size();
- }
-
- if (m_inconsistent.count(hobj)) {
- m_pg->repair_object(hobj, shrd_list, m_inconsistent[hobj]);
- m_fixed_count += m_inconsistent[hobj].size();
- }
- }
- }
- }
- return (!m_authoritative.empty() && m_is_repair);
-}
-
/*
* note: only called for the Primary.
*/
update_op_mode_text();
}
- bool do_auto_scrub = false;
+ m_be->update_repair_status(m_is_repair);
// if a regular scrub had errors within the limit, do a deep scrub to auto repair
+ bool do_auto_scrub = false;
if (m_flags.deep_scrub_on_error && !m_authoritative.empty() &&
m_authoritative.size() <= m_pg->cct->_conf->osd_scrub_auto_repair_num_errors) {
ceph_assert(!m_is_deep);
// type-specific finish (can tally more errors)
_scrub_finish();
- bool has_error = scrub_process_inconsistent();
+ // note that the PG_STATE_REPAIR might have changed above
+ m_fixed_count += m_be->scrub_process_inconsistent();
+ bool has_error = !m_authoritative.empty() && m_is_repair;
{
stringstream oss;
ObjectStore::Transaction t;
m_pg->recovery_state.update_stats(
[this](auto& history, auto& stats) {
- dout(10) << "m_pg->recovery_state.update_stats()" << dendl;
- utime_t now = ceph_clock_now();
- history.last_scrub = m_pg->recovery_state.get_info().last_update;
- history.last_scrub_stamp = now;
- if (m_is_deep) {
- history.last_deep_scrub = m_pg->recovery_state.get_info().last_update;
- history.last_deep_scrub_stamp = now;
- }
-
- if (m_is_deep) {
- if ((m_shallow_errors == 0) && (m_deep_errors == 0))
- history.last_clean_scrub_stamp = now;
- stats.stats.sum.num_shallow_scrub_errors = m_shallow_errors;
- stats.stats.sum.num_deep_scrub_errors = m_deep_errors;
- stats.stats.sum.num_large_omap_objects = m_omap_stats.large_omap_objects;
- stats.stats.sum.num_omap_bytes = m_omap_stats.omap_bytes;
- stats.stats.sum.num_omap_keys = m_omap_stats.omap_keys;
- dout(25) << "scrub_finish shard " << m_pg_whoami
- << " num_omap_bytes = " << stats.stats.sum.num_omap_bytes
- << " num_omap_keys = " << stats.stats.sum.num_omap_keys << dendl;
- } else {
- stats.stats.sum.num_shallow_scrub_errors = m_shallow_errors;
- // XXX: last_clean_scrub_stamp doesn't mean the pg is not inconsistent
- // because of deep-scrub errors
- if (m_shallow_errors == 0)
- history.last_clean_scrub_stamp = now;
- }
- stats.stats.sum.num_scrub_errors = stats.stats.sum.num_shallow_scrub_errors +
- stats.stats.sum.num_deep_scrub_errors;
- if (m_flags.check_repair) {
- m_flags.check_repair = false;
- if (m_pg->info.stats.stats.sum.num_scrub_errors) {
- state_set(PG_STATE_FAILED_REPAIR);
- dout(10) << "scrub_finish " << m_pg->info.stats.stats.sum.num_scrub_errors
- << " error(s) still present after re-scrub" << dendl;
- }
- }
- return true;
+ dout(10) << "m_pg->recovery_state.update_stats() errors:"
+ << m_shallow_errors << "/" << m_deep_errors << " deep? "
+ << m_is_deep << dendl;
+ utime_t now = ceph_clock_now();
+ history.last_scrub = m_pg->recovery_state.get_info().last_update;
+ history.last_scrub_stamp = now;
+ if (m_is_deep) {
+ history.last_deep_scrub = m_pg->recovery_state.get_info().last_update;
+ history.last_deep_scrub_stamp = now;
+ }
+
+ if (m_is_deep) {
+ if ((m_shallow_errors == 0) && (m_deep_errors == 0)) {
+ history.last_clean_scrub_stamp = now;
+ }
+ stats.stats.sum.num_shallow_scrub_errors = m_shallow_errors;
+ stats.stats.sum.num_deep_scrub_errors = m_deep_errors;
+ auto omap_stats = m_be->this_scrub_omapstats();
+ stats.stats.sum.num_large_omap_objects =
+ omap_stats.large_omap_objects;
+ stats.stats.sum.num_omap_bytes = omap_stats.omap_bytes;
+ stats.stats.sum.num_omap_keys = omap_stats.omap_keys;
+ dout(19) << "scrub_finish shard " << m_pg_whoami
+ << " num_omap_bytes = " << stats.stats.sum.num_omap_bytes
+ << " num_omap_keys = " << stats.stats.sum.num_omap_keys
+ << dendl;
+ } else {
+ stats.stats.sum.num_shallow_scrub_errors = m_shallow_errors;
+ // XXX: last_clean_scrub_stamp doesn't mean the pg is not inconsistent
+ // because of deep-scrub errors
+ if (m_shallow_errors == 0) {
+ history.last_clean_scrub_stamp = now;
+ }
+ }
+
+ stats.stats.sum.num_scrub_errors =
+ stats.stats.sum.num_shallow_scrub_errors +
+ stats.stats.sum.num_deep_scrub_errors;
+
+ if (m_flags.check_repair) {
+ m_flags.check_repair = false;
+ if (m_pg->info.stats.stats.sum.num_scrub_errors) {
+ state_set(PG_STATE_FAILED_REPAIR);
+ dout(10) << "scrub_finish "
+ << m_pg->info.stats.stats.sum.num_scrub_errors
+ << " error(s) still present after re-scrub" << dendl;
+ }
+ }
+ return true;
},
&t);
int tr = m_osds->store->queue_transaction(m_pg->ch, std::move(t), nullptr);
void PgScrubber::on_digest_updates()
{
- dout(10) << __func__ << " #pending: " << num_digest_updates_pending
+ 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;
requeue_waiting();
reset_internal_state();
+ m_pg->publish_stats_to_osd();
m_flags = scrub_flags_t{};
// type-specific state clear
// type-specific state clear
_scrub_clear_state();
+ m_pg->publish_stats_to_osd();
}
void PgScrubber::replica_handling_done()
preemption_data.reset();
m_maps_status.reset();
- m_received_maps.clear();
m_start = hobject_t{};
m_end = hobject_t{};
m_shallow_errors = 0;
m_deep_errors = 0;
m_fixed_count = 0;
- m_omap_stats = (const struct omap_stat_t){0};
run_callbacks();
- m_inconsistent.clear();
- m_missing.clear();
m_authoritative.clear();
num_digest_updates_pending = 0;
- m_primary_scrubmap = ScrubMap{};
m_primary_scrubmap_pos.reset();
replica_scrubmap = ScrubMap{};
replica_scrubmap_pos.reset();
- m_cleaned_meta_map = ScrubMap{};
m_needs_sleep = true;
m_sleep_started_at = utime_t{};
m_active = false;
clear_queued_or_active();
++m_sessions_counter;
+ m_be.reset();
}
// note that only applicable to the Replica:
#pragma once
+// clang-format off
+/*
+
+Main Scrubber interfaces:
+
+┌──────────────────────────────────────────────┬────┐
+│ │ │
+│ │ │
+│ PG │ │
+│ │ │
+│ │ │
+├──────────────────────────────────────────────┘ │
+│ │
+│ PrimaryLogPG │
+└────────────────────────────────┬──────────────────┘
+ │
+ │
+ │ ownes & uses
+ │
+ │
+ │
+┌────────────────────────────────▼──────────────────┐
+│ ScrubPgIf │
+└───────────────────────────▲───────────────────────┘
+ │
+ │
+ │implements
+ │
+ │
+ │
+┌───────────────────────────┴───────────────┬───────┐
+│ │ │
+│ PgScrubber │ │
+│ │ │
+│ │ ├───────┐
+├───────────────────────────────────────────┘ │ │
+│ │ │
+│ PrimaryLogScrub │ │
+└─────┬───────────────────┬─────────────────────────┘ │
+ │ │ implements
+ │ ownes & uses │ │
+ │ │ ┌─────────────────────────▼──────┐
+ │ │ │ ScrubMachineListener │
+ │ │ └─────────▲──────────────────────┘
+ │ │ │
+ │ │ │
+ │ ▼ │
+ │ ┌────────────────────────────────┴───────┐
+ │ │ │
+ │ │ ScrubMachine │
+ │ │ │
+ │ └────────────────────────────────────────┘
+ │
+ ┌───▼─────────────────────────────────┐
+ │ │
+ │ ScrubStore │
+ │ │
+ └─────────────────────────────────────┘
+
+*/
+// clang-format on
+
+
#include <cassert>
#include <chrono>
#include <memory>
#include <vector>
#include "osd/PG.h"
-#include "ScrubStore.h"
-#include "scrub_machine_lstnr.h"
#include "osd/scrubber_common.h"
-#include "osd_scrub_sched.h"
-class Callback;
+#include "ScrubStore.h"
+#include "osd_scrub_sched.h"
+#include "scrub_backend.h"
+#include "scrub_machine_lstnr.h"
namespace Scrub {
class ScrubMachine;
public:
explicit PgScrubber(PG* pg);
+ friend class ScrubBackend; // will be replaced by a limited interface
+
// ------------------ the I/F exposed to the PG (ScrubPgIF) -------------
/// are we waiting for resource reservation grants form our replicas?
ostream& show(ostream& out) const override;
public:
- // -------------------------------------------------------------------------------------------
+ // ------------------ the I/F used by the ScrubBackend (not named yet) -------------
+
+ // note: the reason we must have these forwarders, is because of the
+ // artificial PG vs. PrimaryLogPG distinction. Some of the services used
+ // by the scrubber backend are PrimaryLog-specific.
+
+ virtual void add_to_stats(const object_stat_sum_t& stat)
+ {
+ ceph_assert(0 && "expecting a PrimaryLogScrub object");
+ }
+
+ virtual void submit_digest_fixes(const digests_fixes_t& fixes)
+ {
+ ceph_assert(0 && "expecting a PrimaryLogScrub object");
+ }
+
+ // -------------------------------------------------------------------------------------
friend ostream& operator<<(ostream& out, const PgScrubber& scrubber);
void requeue_waiting() const { m_pg->requeue_ops(m_pg->waiting_for_scrub); }
- void _scan_snaps(ScrubMap& smap);
-
- ScrubMap clean_meta_map();
-
/**
* mark down some parameters of the initiated scrub:
* - the epoch when started;
epoch_t m_last_aborted{}; // last time we've noticed a request to abort
- /**
- * return true if any inconsistency/missing is repaired, false otherwise
- */
- [[nodiscard]] bool scrub_process_inconsistent();
-
- void scrub_compare_maps();
-
bool m_needs_sleep{true}; ///< should we sleep before being rescheduled? always
///< 'true', unless we just got out of a sleep period
*/
virtual void _scrub_finish() {}
- /**
- * Validate consistency of the object info and snap sets.
- */
- virtual void scrub_snapshot_metadata(ScrubMap& map, const missing_map_t& missing_digest)
- {}
-
// common code used by build_primary_map_chunk() and build_replica_map_chunk():
int build_scrub_map_chunk(ScrubMap& map, // primary or replica?
ScrubMapBuilder& pos,
int m_deep_errors{0};
int m_fixed_count{0};
- /// Maps from objects with errors to missing peers
- HobjToShardSetMapping m_missing;
-
protected:
/**
* 'm_is_deep' - is the running scrub a deep one?
void message_all_replicas(int32_t opcode, std::string_view op_text);
hobject_t m_max_end; ///< Largest end that may have been sent to replicas
- ScrubMap m_primary_scrubmap;
+ ScrubMap* m_primary_scrubmap{nullptr}; ///< the map is owned by the ScrubBackend
ScrubMapBuilder m_primary_scrubmap_pos;
- std::map<pg_shard_t, ScrubMap> m_received_maps;
-
- /// Cleaned std::map pending snap metadata scrub
- ScrubMap m_cleaned_meta_map;
-
void _request_scrub_map(pg_shard_t replica,
eversion_t version,
hobject_t start,
Scrub::MapsCollectionStatus m_maps_status;
- omap_stat_t m_omap_stats = (const struct omap_stat_t){0};
-
- /// Maps from objects with errors to inconsistent peers
- HobjToShardSetMapping m_inconsistent;
/// Maps from object with errors to good peers
std::map<hobject_t, std::list<std::pair<ScrubMap::object, pg_shard_t>>> m_authoritative;
ScrubMapBuilder replica_scrubmap_pos;
ScrubMap replica_scrubmap;
+ // the backend, handling the details of comparing maps & fixing objects
+ std::unique_ptr<ScrubBackend> m_be;
+
/**
* we mark the request priority as it arrived. It influences the queuing priority
* when we wait for local updates
--- /dev/null
+// -*- m_mode_desc:C++; tab-width:8; c-basic-offset:2; indent-tabs-m_mode_desc:t
+// -*- vim: ts=2 sw=2 smarttab
+
+#include "./scrub_backend.h"
+
+#include <algorithm>
+
+#include "common/debug.h"
+
+#include "include/utime_fmt.h"
+#include "messages/MOSDRepScrubMap.h"
+#include "osd/ECUtil.h"
+#include "osd/OSD.h"
+#include "osd/PG.h"
+#include "osd/PrimaryLogPG.h"
+#include "osd/osd_types_fmt.h"
+
+#include "pg_scrubber.h"
+
+using std::list;
+using std::pair;
+using std::set;
+using std::stringstream;
+using std::vector;
+using namespace Scrub;
+using namespace std::chrono;
+using namespace std::chrono_literals;
+using namespace std::literals;
+
+#define dout_context (m_scrubber.m_osds->cct)
+#define dout_subsys ceph_subsys_osd
+#undef dout_prefix
+
+#define dout_prefix ScrubBackend::logger_prefix(_dout, this)
+
+std::ostream& ScrubBackend::logger_prefix(std::ostream* out,
+ const ScrubBackend* t)
+{
+ return t->m_scrubber.gen_prefix(*out) << " b.e.: ";
+}
+
+// ////////////////////////////////////////////////////////////////////////// //
+
+// for a Primary
+ScrubBackend::ScrubBackend(PgScrubber& scrubber,
+ PGBackend& backend,
+ PG& pg,
+ pg_shard_t i_am,
+ bool repair,
+ scrub_level_t shallow_or_deep,
+ const std::set<pg_shard_t>& acting)
+ : m_scrubber{scrubber}
+ , m_pgbe{backend}
+ , m_pg{pg}
+ , m_pg_whoami{i_am}
+ , m_repair{repair}
+ , m_depth{shallow_or_deep}
+ , m_pg_id{scrubber.m_pg_id}
+ , m_conf{m_scrubber.get_pg_cct()->_conf}
+ , clog{m_scrubber.m_osds->clog}
+{
+ m_formatted_id = m_pg_id.calc_name_sring();
+
+ m_acting_but_me.reserve(acting.size());
+ std::copy_if(acting.begin(),
+ acting.end(),
+ std::back_inserter(m_acting_but_me),
+ [i_am](const pg_shard_t& shard) { return shard != i_am; });
+
+ m_is_replicated = m_pg.get_pool().info.is_replicated();
+ m_mode_desc =
+ (m_repair ? "repair"sv
+ : (m_depth == scrub_level_t::deep ? "deep-scrub"sv : "scrub"sv));
+}
+
+// for a Replica
+ScrubBackend::ScrubBackend(PgScrubber& scrubber,
+ PGBackend& backend,
+ PG& pg,
+ pg_shard_t i_am,
+ bool repair,
+ scrub_level_t shallow_or_deep)
+ : m_scrubber{scrubber}
+ , m_pgbe{backend}
+ , m_pg{pg}
+ , m_pg_whoami{i_am}
+ , m_repair{repair}
+ , m_depth{shallow_or_deep}
+ , m_pg_id{scrubber.m_pg_id}
+ , m_conf{m_scrubber.get_pg_cct()->_conf}
+ , clog{m_scrubber.m_osds->clog}
+{
+ m_formatted_id = m_pg_id.calc_name_sring();
+ m_is_replicated = m_pg.get_pool().info.is_replicated();
+ m_mode_desc =
+ (m_repair ? "repair"sv
+ : (m_depth == scrub_level_t::deep ? "deep-scrub"sv : "scrub"sv));
+}
+
+void ScrubBackend::update_repair_status(bool should_repair)
+{
+ dout(15) << __func__
+ << ": repair state set to :" << (should_repair ? "true" : "false")
+ << dendl;
+ m_repair = should_repair;
+ m_mode_desc =
+ (m_repair ? "repair"sv
+ : (m_depth == scrub_level_t::deep ? "deep-scrub"sv : "scrub"sv));
+}
+
+ScrubMap* ScrubBackend::new_chunk()
+{
+ dout(15) << __func__ << dendl;
+ this_chunk.emplace(m_pg_whoami);
+ return &this_chunk->received_maps[m_pg_whoami];
+}
+
+void ScrubBackend::merge_to_master_set()
+{
+ dout(15) << __func__ << dendl;
+ ceph_assert(m_pg.is_primary());
+ ceph_assert(this_chunk->master_set.empty() &&
+ "the scrubber-backend should be empty");
+
+ if (g_conf()->subsys.should_gather<ceph_subsys_osd, 15>()) {
+ for (const auto& rpl : m_acting_but_me) {
+ dout(15) << fmt::format("{}: replica {} has {} items",
+ __func__,
+ rpl,
+ this_chunk->received_maps[rpl].objects.size())
+ << dendl;
+ }
+ }
+
+ // Construct the master 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; });
+ }
+}
+
+ScrubMap& ScrubBackend::my_map()
+{
+ return this_chunk->received_maps[m_pg_whoami];
+}
+
+void ScrubBackend::decode_received_map(pg_shard_t from,
+ const MOSDRepScrubMap& msg)
+{
+ auto p = const_cast<bufferlist&>(msg.get_data()).cbegin();
+ this_chunk->received_maps[from].decode(p, m_pg.pool.id);
+
+ dout(15) << __func__ << ": decoded map from : " << from
+ << ": versions: " << this_chunk->received_maps[from].valid_through
+ << " / " << msg.get_map_epoch() << dendl;
+}
+
+
+void ScrubBackend::replica_clean_meta(ScrubMap& repl_map,
+ bool max_reached,
+ const hobject_t& start)
+{
+ dout(15) << __func__ << ": REPL META # " << m_cleaned_meta_map.objects.size()
+ << " objects" << dendl;
+ ceph_assert(!m_cleaned_meta_map.objects.size());
+ 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);
+}
+
+
+// /////////////////////////////////////////////////////////////////////////////
+//
+// comparing the maps
+//
+// /////////////////////////////////////////////////////////////////////////////
+
+void ScrubBackend::scrub_compare_maps(bool max_reached)
+{
+ dout(10) << __func__ << " has maps, analyzing" << dendl;
+ ceph_assert(m_pg.is_primary());
+
+ // construct authoritative scrub map for type-specific scrubbing
+
+ m_cleaned_meta_map.insert(my_map());
+ merge_to_master_set();
+
+ // collect some omap statistics into m_omap_stats
+ omap_checks();
+
+ update_authoritative();
+ auto for_meta_scrub = clean_meta_map(m_cleaned_meta_map, max_reached);
+
+ // ok, do the pg-type specific scrubbing
+
+ // (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);
+ }
+ }
+}
+
+void ScrubBackend::omap_checks()
+{
+ const bool needs_omap_check = std::any_of(
+ this_chunk->received_maps.begin(),
+ this_chunk->received_maps.end(),
+ [](const auto& m) -> bool {
+ return m.second.has_large_omap_object_errors || m.second.has_omap_keys;
+ });
+
+ if (!needs_omap_check) {
+ return; // Nothing to do
+ }
+
+ stringstream wss;
+
+ // Iterate through objects and update omap stats
+ for (const auto& ho : this_chunk->master_set) {
+
+ for (const auto& [srd, smap] : this_chunk->received_maps) {
+ if (srd != m_pg.get_primary()) {
+ // Only set omap stats for the primary
+ continue;
+ }
+
+ auto it = smap.objects.find(ho);
+ if (it == smap.objects.end()) {
+ continue;
+ }
+
+ const ScrubMap::object& smap_obj = it->second;
+ m_omap_stats.omap_bytes += smap_obj.object_omap_bytes;
+ m_omap_stats.omap_keys += smap_obj.object_omap_keys;
+ if (smap_obj.large_omap_object_found) {
+ auto osdmap = m_pg.get_osdmap();
+ pg_t pg;
+ osdmap->map_to_pg(ho.pool, ho.oid.name, ho.get_key(), ho.nspace, &pg);
+ pg_t mpg = osdmap->raw_pg_to_pg(pg);
+ m_omap_stats.large_omap_objects++;
+ wss << "Large omap object found. Object: " << ho << " PG: " << pg
+ << " (" << mpg << ")"
+ << " Key count: " << smap_obj.large_omap_object_key_count
+ << " Size (bytes): " << smap_obj.large_omap_object_value_size
+ << '\n';
+ break;
+ }
+ }
+ }
+
+ if (!wss.str().empty()) {
+ dout(5) << __func__ << ": " << wss.str() << dendl;
+ clog->warn(wss);
+ }
+}
+
+/*
+ * update_authoritative() updates:
+ #
+ * - m_scrubber.m_authoritative: adds obj-> list of pairs < scrub-map, shard>
+ *
+ * - m_cleaned_meta_map: replaces [obj] entry with:
+ * the relevant object in the scrub-map of the "selected" (back-most) peer
+ */
+void ScrubBackend::update_authoritative()
+{
+ dout(10) << __func__ << dendl;
+
+ if (m_acting_but_me.empty()) {
+ return;
+ }
+
+ auto maybe_err_msg = compare_smaps();
+ if (maybe_err_msg) {
+ dout(5) << __func__ << ": " << *maybe_err_msg << dendl;
+ clog->error() << *maybe_err_msg;
+ }
+
+ /// \todo try replacing with algorithm-based code
+
+ // update the scrubber object's m_authoritative with the list of good
+ // peers for each object (i.e. the ones that are in this_chunks's auth list)
+ for (auto& [obj, peers] : this_chunk->authoritative) {
+
+ list<pair<ScrubMap::object, pg_shard_t>> good_peers;
+
+ for (auto& peer : peers) {
+ good_peers.emplace_back(this_chunk->received_maps[peer].objects[obj],
+ peer);
+ }
+
+ m_scrubber.m_authoritative.emplace(obj, good_peers);
+ }
+
+ for (const auto& [obj, peers] : this_chunk->authoritative) {
+ m_cleaned_meta_map.objects.erase(obj);
+ m_cleaned_meta_map.objects.insert(
+ *(this_chunk->received_maps[peers.back()].objects.find(obj)));
+ }
+}
+
+void ScrubBackend::repair_oinfo_oid(ScrubMap& smap)
+{
+ for (auto i = smap.objects.rbegin(); i != smap.objects.rend(); ++i) {
+
+ const hobject_t& hoid = i->first;
+ ScrubMap::object& o = i->second;
+
+ if (o.attrs.find(OI_ATTR) == o.attrs.end()) {
+ continue;
+ }
+ bufferlist bl;
+ bl.push_back(o.attrs[OI_ATTR]);
+ object_info_t oi;
+ try {
+ oi.decode(bl);
+ } catch (...) {
+ continue;
+ }
+
+ if (oi.soid != hoid) {
+ ObjectStore::Transaction t;
+ OSDriver::OSTransaction _t(m_pg.osdriver.get_transaction(&t));
+
+ clog->error() << "osd." << m_pg_whoami
+ << " found object info error on pg " << m_pg.info.pgid
+ << " oid " << hoid << " oid in object info: " << oi.soid
+ << "...repaired";
+ // Fix object info
+ oi.soid = hoid;
+ bl.clear();
+ encode(oi,
+ bl,
+ m_pg.get_osdmap()->get_features(CEPH_ENTITY_TYPE_OSD, nullptr));
+
+ bufferptr bp(bl.c_str(), bl.length());
+ o.attrs[OI_ATTR] = bp;
+
+ t.setattr(m_pg.coll, ghobject_t(hoid), OI_ATTR, bl);
+ int r = m_pg.osd->store->queue_transaction(m_pg.ch, std::move(t));
+ if (r != 0) {
+ derr << __func__ << ": queue_transaction got " << cpp_strerror(r)
+ << dendl;
+ }
+ }
+ }
+}
+
+int ScrubBackend::scrub_process_inconsistent()
+{
+ dout(20) << fmt::format("{}: {} (m_repair:{}) good peers tbl #: {}",
+ __func__,
+ m_mode_desc,
+ m_repair,
+ m_scrubber.m_authoritative.size())
+ << dendl;
+
+ // authoritative only store objects which are missing or inconsistent.
+ if (m_scrubber.m_authoritative.empty()) {
+ return 0;
+ }
+
+ // some tests expect an error message that does not contain the __func__ and
+ // PG:
+ auto err_msg = fmt::format("{} {} {} missing, {} inconsistent objects",
+ m_formatted_id,
+ m_mode_desc,
+ m_missing.size(),
+ m_inconsistent.size());
+
+ dout(2) << err_msg << dendl;
+ clog->error() << fmt::to_string(err_msg);
+
+ 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) {
+
+ auto missing_entry = m_missing.find(hobj);
+
+ if (missing_entry != m_missing.end()) {
+ repair_object(hobj, shrd_list, missing_entry->second);
+ fixed_cnt += missing_entry->second.size();
+ }
+
+ if (m_inconsistent.count(hobj)) {
+ repair_object(hobj, shrd_list, m_inconsistent[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 set<pg_shard_t>& bad_peers)
+{
+ if (g_conf()->subsys.should_gather<ceph_subsys_osd, 20>()) {
+ // log the good peers
+ set<pg_shard_t> ok_shards; // the shards from the ok_peers list
+ for (const auto& peer : ok_peers) {
+ ok_shards.insert(peer.second);
+ }
+ dout(10) << fmt::format(
+ "repair_object {} bad_peers osd.{{{}}}, ok_peers osd.{{{}}}",
+ soid,
+ bad_peers,
+ ok_shards)
+ << dendl;
+ }
+
+ const ScrubMap::object& po = ok_peers.back().first;
+
+ object_info_t oi;
+ try {
+ bufferlist bv;
+ if (po.attrs.count(OI_ATTR)) {
+ bv.push_back(po.attrs.find(OI_ATTR)->second);
+ }
+ auto bliter = bv.cbegin();
+ decode(oi, bliter);
+ } catch (...) {
+ dout(0) << __func__
+ << ": Need version of replica, bad object_info_t: " << soid
+ << dendl;
+ ceph_abort();
+ }
+
+ if (bad_peers.count(m_pg.get_primary())) {
+ // We should only be scrubbing if the PG is clean.
+ ceph_assert(m_pg.waiting_for_unreadable_object.empty());
+ dout(10) << __func__ << ": primary = " << m_pg.get_primary() << dendl;
+ }
+
+ // No need to pass ok_peers, they must not be missing the object, so
+ // force_object_missing will add them to missing_loc anyway
+ m_pg.recovery_state.force_object_missing(bad_peers, soid, oi.version);
+}
+
+
+// /////////////////////////////////////////////////////////////////////////////
+//
+// components formerly of PGBackend::be_compare_scrubmaps()
+//
+// /////////////////////////////////////////////////////////////////////////////
+
+using usable_t = shard_as_auth_t::usable_t;
+
+
+static inline int dcount(const object_info_t& oi)
+{
+ return (oi.is_data_digest() ? 1 : 0) + (oi.is_omap_digest() ? 1 : 0);
+}
+
+auth_selection_t ScrubBackend::select_auth_object(const hobject_t& ho,
+ stringstream& errstream)
+{
+ // Create a list of shards (with the Primary first, so that it will be
+ // auth-copy, all other things being equal)
+
+ /// \todo: consider sorting the candidate shards by the conditions for
+ /// selecting best auth source below. Then - stopping on the first one
+ /// that is auth eligible.
+ /// This creates an issue with 'digest_match' that should be handled.
+ std::list<pg_shard_t> shards;
+ for (const auto& [srd, smap] : this_chunk->received_maps) {
+ if (srd != m_pg_whoami) {
+ shards.push_back(srd);
+ }
+ }
+ shards.push_front(m_pg_whoami);
+
+ auth_selection_t ret_auth;
+ ret_auth.auth = this_chunk->received_maps.end();
+ eversion_t auth_version;
+
+ for (auto& l : shards) {
+
+ auto shard_ret = possible_auth_shard(ho, l, ret_auth.shard_map);
+
+ // digest_match will only be true if computed digests are the same
+ if (auth_version != eversion_t() &&
+ ret_auth.auth->second.objects[ho].digest_present &&
+ shard_ret.digest.has_value() &&
+ ret_auth.auth->second.objects[ho].digest != *shard_ret.digest) {
+
+ ret_auth.digest_match = false;
+ dout(10) << __func__ << " digest_match = false, " << ho
+ << " data_digest 0x" << std::hex
+ << ret_auth.auth->second.objects[ho].digest
+ << " != data_digest 0x" << *shard_ret.digest << std::dec
+ << dendl;
+ }
+
+ if (shard_ret.possible_auth == shard_as_auth_t::usable_t::not_usable) {
+
+ // Don't use this particular shard due to previous errors
+ // XXX: For now we can't pick one shard for repair and another's object
+ // info or snapset
+
+ errstream << m_pg_id.pgid << " shard " << l << " soid " << ho << " : "
+ << shard_ret.error_text << "\n";
+
+ } else {
+
+ dout(30) << __func__ << " consider using " << l
+ << " srv: " << shard_ret.oi.version
+ << " oi soid: " << shard_ret.oi.soid << dendl;
+
+ // consider using this shard as authoritative. Is it more recent?
+
+ if (auth_version == eversion_t() || shard_ret.oi.version > auth_version ||
+ (shard_ret.oi.version == auth_version &&
+ dcount(shard_ret.oi) > dcount(ret_auth.auth_oi))) {
+
+ dout(30) << __func__ << " using " << l << " moved auth oi " << std::hex
+ << (uint64_t)(&ret_auth.auth_oi) << " <-> "
+ << (uint64_t)(&shard_ret.oi) << std::dec << dendl;
+
+ ret_auth.auth = shard_ret.auth_iter;
+ ret_auth.auth_shard = ret_auth.auth->first;
+ ret_auth.auth_oi = shard_ret.oi;
+ auth_version = shard_ret.oi.version;
+ ret_auth.is_auth_available = true;
+ }
+ }
+ }
+
+ dout(10) << fmt::format("{}: selecting osd {} for obj {} with oi {}",
+ __func__,
+ ret_auth.auth_shard,
+ ho,
+ ret_auth.auth_oi)
+ << dendl;
+
+ return ret_auth;
+}
+
+using set_sinfo_err_t = void (shard_info_wrapper::*)();
+
+inline static const char* sep(bool& prev_err)
+{
+ if (prev_err) {
+ return ", ";
+ } else {
+ prev_err = true;
+ return "";
+ }
+}
+
+// retval: should we continue with the tests
+static inline bool dup_error_cond(bool& prev_err,
+ bool continue_on_err,
+ bool pred,
+ shard_info_wrapper& si,
+ set_sinfo_err_t sete,
+ std::string_view msg,
+ stringstream& errstream)
+{
+ if (pred) {
+ (si.*sete)();
+ errstream << sep(prev_err) << msg;
+ return continue_on_err;
+ }
+ return true;
+}
+
+/**
+ * calls a shard_info_wrapper function, but only if the error predicate is
+ * true.
+ * Returns a copy of the error status.
+ */
+static inline bool test_error_cond(bool error_pred,
+ shard_info_wrapper& si,
+ set_sinfo_err_t sete)
+{
+ if (error_pred) {
+ (si.*sete)();
+ }
+ return error_pred;
+}
+
+shard_as_auth_t ScrubBackend::possible_auth_shard(const hobject_t& obj,
+ const pg_shard_t& srd,
+ shard_info_map_t& shard_map)
+{
+ // 'maps' (called with this_chunk->maps originaly): this_chunk->maps
+ // 'auth_oi' (called with 'auth_oi', which wasn't initialized at call site)
+ // - will probably need to create and return
+ // 'shard_map' - the one created in select_auth_object()
+ // - used to access the 'shard_info'
+
+ const auto j = this_chunk->received_maps.find(srd);
+ const auto& j_shard = j->first;
+ const auto& j_smap = j->second;
+ auto i = j_smap.objects.find(obj);
+ if (i == j_smap.objects.end()) {
+ return shard_as_auth_t{""s};
+ }
+ const auto& smap_obj = i->second;
+
+ auto& shard_info = shard_map[j_shard];
+ if (j_shard == m_pg_whoami) {
+ shard_info.primary = true;
+ }
+
+ stringstream errstream; // for this shard
+
+ bool err{false};
+ dup_error_cond(err,
+ true,
+ smap_obj.read_error,
+ shard_info,
+ &shard_info_wrapper::set_read_error,
+ "candidate had a read error"sv,
+ errstream);
+ dup_error_cond(err,
+ true,
+ smap_obj.ec_hash_mismatch,
+ shard_info,
+ &shard_info_wrapper::set_ec_hash_mismatch,
+ "candidate had an ec hash mismatch"sv,
+ errstream);
+ dup_error_cond(err,
+ true,
+ smap_obj.ec_size_mismatch,
+ shard_info,
+ &shard_info_wrapper::set_ec_size_mismatch,
+ "candidate had an ec size mismatch"sv,
+ errstream);
+
+ if (!dup_error_cond(err,
+ false,
+ smap_obj.stat_error,
+ shard_info,
+ &shard_info_wrapper::set_stat_error,
+ "candidate had a stat error"sv,
+ errstream)) {
+ // With stat_error no further checking
+ // We don't need to also see a missing_object_info_attr
+ return shard_as_auth_t{errstream.str()};
+ }
+
+ // We won't pick an auth copy if the snapset is missing or won't decode.
+ ceph_assert(!obj.is_snapdir());
+
+ if (obj.is_head()) {
+ auto k = smap_obj.attrs.find(SS_ATTR);
+ if (dup_error_cond(err,
+ false,
+ (k == smap_obj.attrs.end()),
+ shard_info,
+ &shard_info_wrapper::set_snapset_missing,
+ "candidate had a missing snapset key"sv,
+ errstream)) {
+ bufferlist ss_bl;
+ SnapSet snapset;
+ ss_bl.push_back(k->second);
+ try {
+ auto bliter = ss_bl.cbegin();
+ decode(snapset, bliter);
+ } catch (...) {
+ // invalid snapset, probably corrupt
+ dup_error_cond(err,
+ false,
+ true,
+ shard_info,
+ &shard_info_wrapper::set_snapset_corrupted,
+ "candidate had a corrupt snapset"sv,
+ errstream);
+ }
+ } else {
+ // debug@dev only
+ dout(30) << __func__ << " missing snap addr: " << std::hex
+ << (uint64_t)(&smap_obj)
+ << " shard_info: " << (uint64_t)(&shard_info)
+ << " er: " << shard_info.errors << std::dec << dendl;
+ }
+ }
+
+ if (!m_is_replicated) {
+ auto k = smap_obj.attrs.find(ECUtil::get_hinfo_key());
+ if (dup_error_cond(err,
+ false,
+ (k == smap_obj.attrs.end()),
+ shard_info,
+ &shard_info_wrapper::set_hinfo_missing,
+ "candidate had a missing hinfo key"sv,
+ errstream)) {
+ bufferlist hk_bl;
+ ECUtil::HashInfo hi;
+ hk_bl.push_back(k->second);
+ try {
+ auto bliter = hk_bl.cbegin();
+ decode(hi, bliter);
+ } catch (...) {
+ dup_error_cond(err,
+ false,
+ true,
+ shard_info,
+ &shard_info_wrapper::set_hinfo_corrupted,
+ "candidate had a corrupt hinfo"sv,
+ errstream);
+ }
+ }
+ }
+
+ object_info_t oi;
+
+ {
+ auto k = smap_obj.attrs.find(OI_ATTR);
+ if (!dup_error_cond(err,
+ false,
+ (k == smap_obj.attrs.end()),
+ shard_info,
+ &shard_info_wrapper::set_info_missing,
+ "candidate had a missing info key"sv,
+ errstream)) {
+ // no object info on object, probably corrupt
+ return shard_as_auth_t{errstream.str()};
+ }
+
+ bufferlist bl;
+ bl.push_back(k->second);
+ try {
+ auto bliter = bl.cbegin();
+ decode(oi, bliter);
+ } catch (...) {
+ // invalid object info, probably corrupt
+ if (!dup_error_cond(err,
+ false,
+ true,
+ shard_info,
+ &shard_info_wrapper::set_info_corrupted,
+ "candidate had a corrupt info"sv,
+ errstream)) {
+ return shard_as_auth_t{errstream.str()};
+ }
+ }
+ }
+
+ // This is automatically corrected in repair_oinfo_oid()
+ ceph_assert(oi.soid == obj);
+
+ if (test_error_cond(smap_obj.size != m_pgbe.be_get_ondisk_size(oi.size),
+ shard_info,
+ &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";
+ }
+
+ std::optional<uint32_t> digest;
+ if (smap_obj.digest_present) {
+ digest = smap_obj.digest;
+ }
+
+ if (shard_info.errors) {
+ ceph_assert(err);
+ return shard_as_auth_t{errstream.str(), digest};
+ }
+
+ ceph_assert(!err);
+ return shard_as_auth_t{oi, j, errstream.str(), 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()
+ << dendl;
+
+ std::stringstream errstream;
+ std::for_each(
+ this_chunk->master_set.begin(),
+ this_chunk->master_set.end(),
+ [this, &errstream](const auto& ho) { compare_obj_in_maps(ho, errstream); });
+
+ if (errstream.str().empty()) {
+ return std::nullopt;
+ } else {
+ return errstream.str();
+ }
+}
+
+void ScrubBackend::compare_obj_in_maps(const hobject_t& ho,
+ stringstream& errstream)
+{
+ // clear per-object data:
+ this_chunk->cur_inconsistent.clear();
+ this_chunk->cur_missing.clear();
+ this_chunk->fix_digest = false;
+
+ auto auth_res = select_auth_object(ho, errstream);
+
+ inconsistent_obj_wrapper object_error{ho};
+ if (!auth_res.is_auth_available) {
+ // no auth selected
+ object_error.set_version(0);
+ object_error.set_auth_missing(ho,
+ this_chunk->received_maps,
+ auth_res.shard_map,
+ m_scrubber.m_shallow_errors,
+ m_scrubber.m_deep_errors,
+ m_pg_whoami);
+
+ if (object_error.has_deep_errors()) {
+ ++m_scrubber.m_deep_errors;
+ } else if (object_error.has_shallow_errors()) {
+ ++m_scrubber.m_shallow_errors;
+ }
+
+ m_scrubber.m_store->add_object_error(ho.pool, object_error);
+ errstream << m_scrubber.m_pg_id.pgid << " soid " << ho
+ << " : failed to pick suitable object info\n";
+ return;
+ }
+
+ auto& auth = auth_res.auth;
+
+ // an auth source was selected
+
+ object_error.set_version(auth_res.auth_oi.user_version);
+ ScrubMap::object& auth_object = auth->second.objects[ho];
+ ceph_assert(!this_chunk->fix_digest);
+
+ auto [auths, objerrs] =
+ 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),
+ auth,
+ ho,
+ errstream);
+
+ if (opt_ers.has_value()) {
+
+ // At this point auth_list is populated, so we add the object error
+ // shards as inconsistent.
+ inconsistents(ho,
+ auth_object,
+ auth_res.auth_oi,
+ std::move(*opt_ers),
+ errstream);
+ } else {
+
+ // both the auth & errs containers are empty
+ errstream << m_scrubber.m_pg_id.pgid << " soid " << ho
+ << " : empty auth list\n";
+ }
+
+ if (object_error.has_deep_errors()) {
+ ++m_scrubber.m_deep_errors;
+ } else if (object_error.has_shallow_errors()) {
+ ++m_scrubber.m_shallow_errors;
+ }
+
+ if (object_error.errors || object_error.union_shards.errors) {
+ m_scrubber.m_store->add_object_error(ho.pool, object_error);
+ }
+}
+
+
+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()) {
+ errstream << m_pg_id << " soid " << ho
+ << " : failed to pick suitable auth object\n";
+ return std::nullopt;
+ }
+ // Object errors exist and nothing in auth_list
+ // Prefer the auth shard, otherwise take first from list.
+ pg_shard_t shard;
+ if (obj_errors.count(auth->first)) {
+ shard = auth->first;
+ } else {
+ shard = *(obj_errors.begin());
+ }
+
+ auths.push_back(shard);
+ obj_errors.erase(shard);
+ }
+
+ return ScrubBackend::auth_and_obj_errs_t{std::move(auths),
+ std::move(obj_errors)};
+}
+
+
+/// \todo replace the errstream with a member of this_chunk. Better be a
+/// fmt::buffer. Then - we can use it directly in should_fix_digest()
+void ScrubBackend::inconsistents(const hobject_t& ho,
+ ScrubMap::object& auth_object,
+ object_info_t& auth_oi,
+ 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);
+
+ this_chunk->cur_inconsistent.insert(object_errors.begin(),
+ object_errors.end()); // merge?
+
+ dout(15) << fmt::format(
+ "{}: object errors #: {} auth list #: {} cur_missing #: {} "
+ "cur_incon #: {}",
+ __func__,
+ object_errors.size(),
+ auth_list.size(),
+ this_chunk->cur_missing.size(),
+ this_chunk->cur_inconsistent.size())
+ << dendl;
+
+
+ if (!this_chunk->cur_missing.empty()) {
+ m_missing[ho] = this_chunk->cur_missing;
+ }
+ if (!this_chunk->cur_inconsistent.empty()) {
+ m_inconsistent[ho] = this_chunk->cur_inconsistent;
+ }
+
+ if (this_chunk->fix_digest) {
+
+ ceph_assert(auth_object.digest_present);
+ std::optional<uint32_t> data_digest{auth_object.digest};
+
+ std::optional<uint32_t> omap_digest;
+ if (auth_object.omap_digest_present) {
+ omap_digest = auth_object.omap_digest;
+ }
+ this_chunk->missing_digest[ho] = make_pair(data_digest, omap_digest);
+ }
+
+ if (!this_chunk->cur_inconsistent.empty() ||
+ !this_chunk->cur_missing.empty()) {
+
+ this_chunk->authoritative[ho] = auth_list;
+
+ } else if (!this_chunk->fix_digest && m_is_replicated) {
+
+ auto is_to_fix =
+ should_fix_digest(ho, auth_object, auth_oi, m_repair, errstream);
+
+ switch (is_to_fix) {
+
+ case digest_fixing_t::no:
+ break;
+
+ case digest_fixing_t::if_aged: {
+ utime_t age = this_chunk->started - auth_oi.local_mtime;
+
+ // \todo find out 'age_limit' only once
+ const auto age_limit =
+ m_scrubber.get_pg_cct()->_conf->osd_deep_scrub_update_digest_min_age;
+
+ if (age <= age_limit) {
+ dout(20) << __func__ << ": missing digest but age (" << age
+ << ") < conf (" << age_limit << ") on " << ho << dendl;
+ break;
+ }
+ }
+
+ [[fallthrough]];
+
+ case digest_fixing_t::force:
+
+ std::optional<uint32_t> data_digest;
+ if (auth_object.digest_present) {
+ data_digest = auth_object.digest;
+ dout(20) << __func__ << ": will update data digest on " << ho
+ << dendl;
+ }
+
+ std::optional<uint32_t> omap_digest;
+ if (auth_object.omap_digest_present) {
+ omap_digest = auth_object.omap_digest;
+ dout(20) << __func__ << ": will update omap digest on " << ho
+ << dendl;
+ }
+ this_chunk->missing_digest[ho] = make_pair(data_digest, omap_digest);
+ break;
+ }
+ }
+}
+
+/// \todo consider changing to use format() and to return the strings
+ScrubBackend::digest_fixing_t ScrubBackend::should_fix_digest(
+ const hobject_t& ho,
+ const ScrubMap::object& auth_object,
+ const object_info_t& auth_oi,
+ bool repair_flag,
+ stringstream& errstream)
+{
+ digest_fixing_t update{digest_fixing_t::no};
+
+ if (auth_object.digest_present && !auth_oi.is_data_digest()) {
+ dout(15) << __func__ << " missing data digest on " << ho << dendl;
+ update = digest_fixing_t::if_aged;
+ }
+
+ if (auth_object.omap_digest_present && !auth_oi.is_omap_digest()) {
+ dout(15) << __func__ << " missing omap digest on " << ho << dendl;
+ update = digest_fixing_t::if_aged;
+ }
+
+ // recorded digest != actual digest?
+ if (auth_oi.is_data_digest() && auth_object.digest_present &&
+ auth_oi.data_digest != auth_object.digest) {
+ errstream << m_pg_id << " recorded data digest 0x" << std::hex
+ << auth_oi.data_digest << " != on disk 0x" << auth_object.digest
+ << std::dec << " on " << auth_oi.soid << "\n";
+ if (repair_flag)
+ update = digest_fixing_t::force;
+ }
+
+ if (auth_oi.is_omap_digest() && auth_object.omap_digest_present &&
+ auth_oi.omap_digest != auth_object.omap_digest) {
+ errstream << m_pg_id << " recorded omap digest 0x" << std::hex
+ << auth_oi.omap_digest << " != on disk 0x"
+ << auth_object.omap_digest << std::dec << " on " << auth_oi.soid
+ << "\n";
+ if (repair_flag)
+ update = digest_fixing_t::force;
+ }
+
+ return update;
+}
+
+ScrubBackend::auth_and_obj_errs_t ScrubBackend::match_in_shards(
+ const hobject_t& ho,
+ auth_selection_t& auth_sel,
+ inconsistent_obj_wrapper& obj_result,
+ stringstream& errstream)
+{
+ std::list<pg_shard_t> auth_list; // out "param" to
+ std::set<pg_shard_t> object_errors; // be returned
+
+ for (auto& [srd, smap] : this_chunk->received_maps) {
+
+ if (srd == auth_sel.auth_shard) {
+ auth_sel.shard_map[auth_sel.auth_shard].selected_oi = true;
+ }
+
+ if (smap.objects.count(ho)) {
+
+ // the scrub-map has our object
+ auth_sel.shard_map[srd].set_object(smap.objects[ho]);
+
+ // Compare
+ stringstream ss;
+ const auto& auth_object = auth_sel.auth->second.objects[ho];
+ const bool discrep_found = compare_obj_details(auth_sel.auth_shard,
+ auth_object,
+ auth_sel.auth_oi,
+ smap.objects[ho],
+ auth_sel.shard_map[srd],
+ obj_result,
+ ss,
+ ho.has_snapset());
+
+ dout(20) << fmt::format(
+ "{}: {} {} {} shards: {} {} {}",
+ __func__,
+ (m_repair ? " repair " : " "),
+ (m_is_replicated ? "replicated " : ""),
+ (srd == auth_sel.auth_shard ? "auth" : ""),
+ auth_sel.shard_map.size(),
+ (auth_sel.digest_match ? " digest_match " : " "),
+ (auth_sel.shard_map[srd].only_data_digest_mismatch_info()
+ ? "'info mismatch info'"
+ : ""))
+ << dendl;
+
+ // If all replicas match, but they don't match object_info we can
+ // repair it by using missing_digest mechanism
+ if (m_repair && m_is_replicated && (srd == auth_sel.auth_shard) &&
+ auth_sel.shard_map.size() > 1 && auth_sel.digest_match &&
+ auth_sel.shard_map[srd].only_data_digest_mismatch_info() &&
+ auth_object.digest_present) {
+ // Set in missing_digests
+ this_chunk->fix_digest = true;
+ // Clear the error
+ auth_sel.shard_map[srd].clear_data_digest_mismatch_info();
+ errstream << m_pg_id << " soid " << ho
+ << " : repairing object info data_digest"
+ << "\n";
+ }
+
+ // Some errors might have already been set in select_auth_object()
+ if (auth_sel.shard_map[srd].errors != 0) {
+
+ this_chunk->cur_inconsistent.insert(srd);
+ if (auth_sel.shard_map[srd].has_deep_errors()) {
+ ++m_scrubber.m_deep_errors;
+ } else {
+ ++m_scrubber.m_shallow_errors;
+ }
+
+ if (discrep_found) {
+ // Only true if compare_obj_details() found errors and put something
+ // in ss
+ errstream << m_pg_id << " shard " << srd << " soid " << ho << " : "
+ << ss.str() << "\n";
+ }
+
+ } else if (discrep_found) {
+
+ // Track possible shards to use as authoritative, if needed
+
+ // There are errors, without identifying the shard
+ object_errors.insert(srd);
+ errstream << m_pg_id << " soid " << ho << " : " << ss.str() << "\n";
+
+ } else {
+
+ // XXX: The auth shard might get here that we don't know
+ // that it has the "correct" data.
+ auth_list.push_back(srd);
+ }
+
+ } else {
+
+ this_chunk->cur_missing.insert(srd);
+ auth_sel.shard_map[srd].set_missing();
+ auth_sel.shard_map[srd].primary = (srd == m_pg_whoami);
+
+ // Can't have any other errors if there is no information available
+ ++m_scrubber.m_shallow_errors;
+ errstream << m_pg_id << " shard " << srd << " " << ho << " : missing\n";
+ }
+ obj_result.add_shard(srd, auth_sel.shard_map[srd]);
+
+ dout(30) << __func__ << ": soid " << ho << " : " << errstream.str()
+ << dendl;
+ }
+
+ dout(15) << __func__ << ": auth_list: " << auth_list
+ << " #: " << auth_list.size()
+ << "; obj-errs#: " << object_errors.size() << dendl;
+
+ dout(15) << fmt::format("{}: auth_list: {} #: {}; obj-errs#: {}",
+ __func__,
+ auth_list,
+ auth_list.size(),
+ object_errors.size())
+ << dendl;
+ return {auth_list, object_errors};
+}
+
+// == PGBackend::be_compare_scrub_objects()
+bool ScrubBackend::compare_obj_details(pg_shard_t auth_shard,
+ const ScrubMap::object& auth,
+ const object_info_t& auth_oi,
+ const ScrubMap::object& candidate,
+ shard_info_wrapper& shard_result,
+ inconsistent_obj_wrapper& obj_result,
+ stringstream& errstream,
+ bool has_snapset)
+{
+ fmt::memory_buffer out;
+ bool error{false};
+
+ // ------------------------------------------------------------------------
+
+ if (auth.digest_present && candidate.digest_present &&
+ auth.digest != candidate.digest) {
+ format_to(out,
+ "data_digest {:#x} != data_digest {:#x} from shard {}",
+ candidate.digest,
+ auth.digest,
+ auth_shard);
+ error = true;
+ obj_result.set_data_digest_mismatch();
+ }
+
+ if (auth.omap_digest_present && candidate.omap_digest_present &&
+ auth.omap_digest != candidate.omap_digest) {
+ format_to(out,
+ "{}omap_digest {:#x} != omap_digest {:#x} from shard {}",
+ sep(error),
+ candidate.omap_digest,
+ auth.omap_digest,
+ auth_shard);
+ obj_result.set_omap_digest_mismatch();
+ }
+
+ // for replicated:
+ if (m_is_replicated) {
+ if (auth_oi.is_data_digest() && candidate.digest_present &&
+ auth_oi.data_digest != candidate.digest) {
+ format_to(out,
+ "{}data_digest {:#x} != data_digest {:#x} from auth oi {}",
+ sep(error),
+ candidate.digest,
+ auth_oi.data_digest,
+ auth_oi);
+ shard_result.set_data_digest_mismatch_info();
+ }
+
+ // for replicated:
+ if (auth_oi.is_omap_digest() && candidate.omap_digest_present &&
+ auth_oi.omap_digest != candidate.omap_digest) {
+ format_to(out,
+ "{}omap_digest {:#x} != omap_digest {:#x} from auth oi {}",
+ sep(error),
+ candidate.omap_digest,
+ auth_oi.omap_digest,
+ auth_oi);
+ shard_result.set_omap_digest_mismatch_info();
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ if (candidate.stat_error) {
+ if (error) {
+ errstream << fmt::to_string(out);
+ }
+ return error;
+ }
+
+ // ------------------------------------------------------------------------
+
+ if (!shard_result.has_info_missing() && !shard_result.has_info_corrupted()) {
+
+ auto can_attr = candidate.attrs.find(OI_ATTR);
+ ceph_assert(can_attr != candidate.attrs.end());
+ bufferlist can_bl;
+ can_bl.push_back(can_attr->second);
+
+ auto auth_attr = auth.attrs.find(OI_ATTR);
+ ceph_assert(auth_attr != auth.attrs.end());
+ bufferlist auth_bl;
+ auth_bl.push_back(auth_attr->second);
+
+ if (!can_bl.contents_equal(auth_bl)) {
+ format_to(out, "{}object info inconsistent ", sep(error));
+ obj_result.set_object_info_inconsistency();
+ }
+ }
+
+ if (has_snapset) {
+ if (!shard_result.has_snapset_missing() &&
+ !shard_result.has_snapset_corrupted()) {
+
+ auto can_attr = candidate.attrs.find(SS_ATTR);
+ ceph_assert(can_attr != candidate.attrs.end());
+ bufferlist can_bl;
+ can_bl.push_back(can_attr->second);
+
+ auto auth_attr = auth.attrs.find(SS_ATTR);
+ ceph_assert(auth_attr != auth.attrs.end());
+ bufferlist auth_bl;
+ auth_bl.push_back(auth_attr->second);
+
+ if (!can_bl.contents_equal(auth_bl)) {
+ format_to(out, "{}snapset inconsistent ", sep(error));
+ obj_result.set_snapset_inconsistency();
+ }
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ if (!m_is_replicated) {
+ if (!shard_result.has_hinfo_missing() &&
+ !shard_result.has_hinfo_corrupted()) {
+
+ auto can_hi = candidate.attrs.find(ECUtil::get_hinfo_key());
+ ceph_assert(can_hi != candidate.attrs.end());
+ bufferlist can_bl;
+ can_bl.push_back(can_hi->second);
+
+ auto auth_hi = auth.attrs.find(ECUtil::get_hinfo_key());
+ ceph_assert(auth_hi != auth.attrs.end());
+ bufferlist auth_bl;
+ auth_bl.push_back(auth_hi->second);
+
+ if (!can_bl.contents_equal(auth_bl)) {
+ format_to(out, "{}hinfo inconsistent ", sep(error));
+ obj_result.set_hinfo_inconsistency();
+ }
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ // sizes:
+
+ uint64_t oi_size = m_pgbe.be_get_ondisk_size(auth_oi.size);
+ if (oi_size != candidate.size) {
+ format_to(out,
+ "{}size {} != size {} from auth oi {}",
+ sep(error),
+ candidate.size,
+ oi_size,
+ auth_oi);
+ shard_result.set_size_mismatch_info();
+ }
+
+ if (auth.size != candidate.size) {
+ format_to(out,
+ "{}size {} != size {} from shard {}",
+ sep(error),
+ candidate.size,
+ auth.size,
+ auth_shard);
+ obj_result.set_size_mismatch();
+ }
+
+ // If the replica is too large and we didn't already count it for this object
+
+ if (candidate.size > m_conf->osd_max_object_size &&
+ !obj_result.has_size_too_large()) {
+
+ format_to(out,
+ "{}size {} > {} is too large",
+ sep(error),
+ candidate.size,
+ m_conf->osd_max_object_size);
+ obj_result.set_size_too_large();
+ }
+
+ // ------------------------------------------------------------------------
+
+ // comparing the attributes:
+
+ for (const auto& [k, v] : auth.attrs) {
+ if (k == OI_ATTR || k[0] != '_') {
+ // We check system keys separately
+ continue;
+ }
+
+ auto cand = candidate.attrs.find(k);
+ if (cand == candidate.attrs.end()) {
+ format_to(out, "{}attr name mismatch '{}'", sep(error), k);
+ obj_result.set_attr_name_mismatch();
+ } else if (cand->second.cmp(v)) {
+ format_to(out, "{}attr value mismatch '{}'", sep(error), k);
+ obj_result.set_attr_value_mismatch();
+ }
+ }
+
+ for (const auto& [k, v] : candidate.attrs) {
+ if (k == OI_ATTR || k[0] != '_') {
+ // We check system keys separately
+ continue;
+ }
+
+ auto in_auth = auth.attrs.find(k);
+ if (in_auth == auth.attrs.end()) {
+ format_to(out, "{}attr name mismatch '{}'", sep(error), k);
+ obj_result.set_attr_name_mismatch();
+ }
+ }
+
+ if (error) {
+ errstream << fmt::to_string(out);
+ }
+ return error;
+}
+
+static inline bool doing_clones(
+ const std::optional<SnapSet>& snapset,
+ const vector<snapid_t>::reverse_iterator& curclone)
+{
+ return snapset && curclone != snapset->clones.rend();
+}
+
+// /////////////////////////////////////////////////////////////////////////////
+//
+// final checking & fixing - scrub_snapshot_metadata()
+//
+// /////////////////////////////////////////////////////////////////////////////
+
+/*
+ * Validate consistency of the object info and snap sets.
+ *
+ * We are sort of comparing 2 lists. The main loop is on objmap.objects. But
+ * the comparison of the objects is against multiple snapset.clones. There are
+ * multiple clone lists and in between lists we expect head.
+ *
+ * Example
+ *
+ * objects expected
+ * ======= =======
+ * obj1 snap 1 head, unexpected obj1 snap 1
+ * obj2 head head, match
+ * [SnapSet clones 6 4 2 1]
+ * obj2 snap 7 obj2 snap 6, unexpected obj2 snap 7
+ * obj2 snap 6 obj2 snap 6, match
+ * obj2 snap 4 obj2 snap 4, match
+ * obj3 head obj2 snap 2 (expected), obj2 snap 1 (expected), match
+ * [Snapset clones 3 1]
+ * obj3 snap 3 obj3 snap 3 match
+ * obj3 snap 1 obj3 snap 1 match
+ * obj4 head head, match
+ * [Snapset clones 4]
+ * EOL obj4 snap 4, (expected)
+ */
+void ScrubBackend::scrub_snapshot_metadata(ScrubMap& map)
+{
+ dout(10) << __func__ << " num stat obj "
+ << m_pg.info.stats.stats.sum.num_objects << dendl;
+
+ auto& info = m_pg.info;
+ const PGPool& pool = m_pg.pool;
+ bool allow_incomplete_clones = pool.info.allow_incomplete_clones();
+
+ std::optional<snapid_t> all_clones; // Unspecified snapid_t or std::nullopt
+
+ // traverse in reverse order.
+ std::optional<hobject_t> head;
+ std::optional<SnapSet> snapset; // If initialized so will head (above)
+ vector<snapid_t>::reverse_iterator
+ curclone; // Defined only if snapset initialized
+ int missing = 0;
+ inconsistent_snapset_wrapper soid_error, head_error;
+ int soid_error_count = 0;
+
+ for (auto p = map.objects.rbegin(); p != map.objects.rend(); ++p) {
+
+ const hobject_t& soid = p->first;
+ ceph_assert(!soid.is_snapdir());
+ soid_error = inconsistent_snapset_wrapper{soid};
+ object_stat_sum_t stat;
+
+ stat.num_objects++;
+
+ if (soid.nspace == m_conf->osd_hit_set_namespace)
+ stat.num_objects_hit_set_archive++;
+
+ if (soid.is_snap()) {
+ // it's a clone
+ stat.num_object_clones++;
+ }
+
+ // basic checks.
+ std::optional<object_info_t> oi;
+ if (!p->second.attrs.count(OI_ATTR)) {
+ oi = std::nullopt;
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : no '" << OI_ATTR << "' attr";
+ ++m_scrubber.m_shallow_errors;
+ soid_error.set_info_missing();
+ } else {
+ bufferlist bv;
+ bv.push_back(p->second.attrs[OI_ATTR]);
+ try {
+ oi = object_info_t(bv);
+ } catch (ceph::buffer::error& e) {
+ oi = std::nullopt;
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : can't decode '" << OI_ATTR << "' attr "
+ << e.what();
+ ++m_scrubber.m_shallow_errors;
+ soid_error.set_info_corrupted();
+ soid_error.set_info_missing(); // Not available too
+ }
+ }
+
+ if (oi) {
+ if (m_pgbe.be_get_ondisk_size(oi->size) != p->second.size) {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : on disk size (" << p->second.size
+ << ") does not match object info size (" << oi->size
+ << ") adjusted for ondisk to ("
+ << m_pgbe.be_get_ondisk_size(oi->size) << ")";
+ soid_error.set_size_mismatch();
+ ++m_scrubber.m_shallow_errors;
+ }
+
+ dout(20) << m_mode_desc << " " << soid << " " << *oi << dendl;
+
+ // A clone num_bytes will be added later when we have snapset
+ if (!soid.is_snap()) {
+ stat.num_bytes += oi->size;
+ }
+ if (soid.nspace == m_conf->osd_hit_set_namespace)
+ stat.num_bytes_hit_set_archive += oi->size;
+
+ if (oi->is_dirty())
+ ++stat.num_objects_dirty;
+ if (oi->is_whiteout())
+ ++stat.num_whiteouts;
+ if (oi->is_omap())
+ ++stat.num_objects_omap;
+ if (oi->is_cache_pinned())
+ ++stat.num_objects_pinned;
+ if (oi->has_manifest())
+ ++stat.num_objects_manifest;
+ }
+
+ // Check for any problems while processing clones
+ if (doing_clones(snapset, curclone)) {
+ std::optional<snapid_t> target;
+ // Expecting an object with snap for current head
+ if (soid.has_snapset() || soid.get_head() != head->get_head()) {
+
+ dout(10) << __func__ << " " << m_mode_desc << " " << info.pgid
+ << " new object " << soid << " while processing " << *head
+ << dendl;
+
+ target = all_clones;
+ } else {
+ ceph_assert(soid.is_snap());
+ target = soid.snap;
+ }
+
+ // Log any clones we were expecting to be there up to target
+ // This will set missing, but will be a no-op if snap.soid == *curclone.
+ missing += process_clones_to(head,
+ snapset, /*clog, info.pgid, m_mode_desc,*/
+ allow_incomplete_clones,
+ target,
+ &curclone,
+ head_error);
+ }
+
+ bool expected;
+ // Check doing_clones() again in case we ran process_clones_to()
+ if (doing_clones(snapset, curclone)) {
+ // A head would have processed all clones above
+ // or all greater than *curclone.
+ ceph_assert(soid.is_snap() && *curclone <= soid.snap);
+
+ // After processing above clone snap should match the expected curclone
+ expected = (*curclone == soid.snap);
+ } else {
+ // If we aren't doing clones any longer, then expecting head
+ expected = soid.has_snapset();
+ }
+ if (!expected) {
+ // If we couldn't read the head's snapset, just ignore clones
+ if (head && !snapset) {
+ clog->error() << m_mode_desc << " " << info.pgid << " TTTTT:" << m_pg_id
+ << " " << soid
+ << " : clone ignored due to missing snapset";
+ } else {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : is an unexpected clone";
+ }
+ ++m_scrubber.m_shallow_errors;
+ soid_error.set_headless();
+ m_scrubber.m_store->add_snap_error(pool.id, soid_error);
+ ++soid_error_count;
+ if (head && soid.get_head() == head->get_head())
+ head_error.set_clone(soid.snap);
+ continue;
+ }
+
+ // new snapset?
+ if (soid.has_snapset()) {
+
+ if (missing) {
+ log_missing(missing,
+ head,
+ __func__,
+ pool.info.allow_incomplete_clones());
+ }
+
+ // Save previous head error information
+ if (head && (head_error.errors || soid_error_count))
+ m_scrubber.m_store->add_snap_error(pool.id, head_error);
+ // Set this as a new head object
+ head = soid;
+ missing = 0;
+ head_error = soid_error;
+ soid_error_count = 0;
+
+ dout(20) << __func__ << " " << m_mode_desc << " new head " << head
+ << dendl;
+
+ if (p->second.attrs.count(SS_ATTR) == 0) {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : no '" << SS_ATTR << "' attr";
+ ++m_scrubber.m_shallow_errors;
+ snapset = std::nullopt;
+ head_error.set_snapset_missing();
+ } else {
+ bufferlist bl;
+ bl.push_back(p->second.attrs[SS_ATTR]);
+ auto blp = bl.cbegin();
+ try {
+ snapset = SnapSet(); // Initialize optional<> before decoding into it
+ decode(*snapset, blp);
+ head_error.ss_bl.push_back(p->second.attrs[SS_ATTR]);
+ } catch (ceph::buffer::error& e) {
+ snapset = std::nullopt;
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : can't decode '" << SS_ATTR << "' attr "
+ << e.what();
+ ++m_scrubber.m_shallow_errors;
+ head_error.set_snapset_corrupted();
+ }
+ }
+
+ if (snapset) {
+ // what will be next?
+ curclone = snapset->clones.rbegin();
+
+ if (!snapset->clones.empty()) {
+ dout(20) << " snapset " << *snapset << dendl;
+ if (snapset->seq == 0) {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : snaps.seq not set";
+ ++m_scrubber.m_shallow_errors;
+ head_error.set_snapset_error();
+ }
+ }
+ }
+ } else {
+ ceph_assert(soid.is_snap());
+ ceph_assert(head);
+ ceph_assert(snapset);
+ ceph_assert(soid.snap == *curclone);
+
+ dout(20) << __func__ << " " << m_mode_desc << " matched clone " << soid
+ << dendl;
+
+ if (snapset->clone_size.count(soid.snap) == 0) {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : is missing in clone_size";
+ ++m_scrubber.m_shallow_errors;
+ soid_error.set_size_mismatch();
+ } else {
+ if (oi && oi->size != snapset->clone_size[soid.snap]) {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : size " << oi->size << " != clone_size "
+ << snapset->clone_size[*curclone];
+ ++m_scrubber.m_shallow_errors;
+ soid_error.set_size_mismatch();
+ }
+
+ if (snapset->clone_overlap.count(soid.snap) == 0) {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : is missing in clone_overlap";
+ ++m_scrubber.m_shallow_errors;
+ soid_error.set_size_mismatch();
+ } else {
+ // This checking is based on get_clone_bytes(). The first 2 asserts
+ // can't happen because we know we have a clone_size and
+ // a clone_overlap. Now we check that the interval_set won't
+ // cause the last assert.
+ uint64_t size = snapset->clone_size.find(soid.snap)->second;
+ const interval_set<uint64_t>& overlap =
+ snapset->clone_overlap.find(soid.snap)->second;
+ bool bad_interval_set = false;
+ for (interval_set<uint64_t>::const_iterator i = overlap.begin();
+ i != overlap.end();
+ ++i) {
+ if (size < i.get_len()) {
+ bad_interval_set = true;
+ break;
+ }
+ size -= i.get_len();
+ }
+
+ if (bad_interval_set) {
+ clog->error() << m_mode_desc << " " << info.pgid << " " << soid
+ << " : bad interval_set in clone_overlap";
+ ++m_scrubber.m_shallow_errors;
+ soid_error.set_size_mismatch();
+ } else {
+ stat.num_bytes += snapset->get_clone_bytes(soid.snap);
+ }
+ }
+ }
+
+ // what's next?
+ ++curclone;
+ if (soid_error.errors) {
+ m_scrubber.m_store->add_snap_error(pool.id, soid_error);
+ ++soid_error_count;
+ }
+ }
+ m_scrubber.add_to_stats(stat);
+ }
+
+ if (doing_clones(snapset, curclone)) {
+ dout(10) << __func__ << " " << m_mode_desc << " " << info.pgid
+ << " No more objects while processing " << *head << dendl;
+
+ missing += process_clones_to(head,
+ snapset,
+ allow_incomplete_clones,
+ all_clones,
+ &curclone,
+ head_error);
+ }
+
+ // There could be missing found by the test above or even
+ // before dropping out of the loop for the last head.
+
+ 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);
+
+ // fix data/omap digests
+ digests_fixes_t digest_fixes{this_chunk->missing_digest.size()};
+ std::transform(
+ this_chunk->missing_digest.begin(),
+ this_chunk->missing_digest.end(),
+ digest_fixes.begin(),
+ [](const auto& p) { return std::make_pair(p.first, p.second); });
+
+ m_scrubber.submit_digest_fixes(digest_fixes);
+
+ dout(10) << __func__ << " (" << m_mode_desc << ") finish" << dendl;
+}
+
+int ScrubBackend::process_clones_to(
+ const std::optional<hobject_t>& head,
+ const std::optional<SnapSet>& snapset,
+ bool allow_incomplete_clones,
+ std::optional<snapid_t> target,
+ vector<snapid_t>::reverse_iterator* curclone,
+ inconsistent_snapset_wrapper& e)
+{
+ ceph_assert(head);
+ ceph_assert(snapset);
+ int missing_count = 0;
+
+ // NOTE: clones are in descending order, thus **curclone > target test here
+ hobject_t next_clone(*head);
+ while (doing_clones(snapset, *curclone) &&
+ (!target || **curclone > *target)) {
+
+ ++missing_count;
+ // it is okay to be missing one or more clones in a cache tier.
+ // skip higher-numbered clones in the list.
+ if (!allow_incomplete_clones) {
+ next_clone.snap = **curclone;
+ clog->error() << m_mode_desc << " " << m_pg_id << " " << *head
+ << " : expected clone " << next_clone << " " << m_missing
+ << " missing";
+ ++m_scrubber.m_shallow_errors;
+ e.set_clone_missing(next_clone.snap);
+ }
+ // Clones are descending
+ ++(*curclone);
+ }
+ return missing_count;
+}
+
+void ScrubBackend::log_missing(int missing,
+ const std::optional<hobject_t>& head,
+ const char* logged_func_name,
+ bool allow_incomplete_clones)
+{
+ ceph_assert(head);
+ if (allow_incomplete_clones) {
+ dout(20) << logged_func_name << " " << m_mode_desc << " " << m_pg_id << " "
+ << *head << " skipped " << missing << " clone(s) in cache tier"
+ << dendl;
+ } else {
+ clog->info() << m_mode_desc << " " << m_pg_id << " " << *head << " : "
+ << missing << " missing clone(s)";
+ }
+}
+
+
+// ////////////////////////////////////////////////////////////////////////////////
+
+void ScrubBackend::scan_snaps(ScrubMap& smap)
+{
+ hobject_t head;
+ SnapSet snapset;
+
+ // Test qa/standalone/scrub/osd-scrub-snaps.sh greps for the strings
+ // in this function
+ dout(15) << "_scan_snaps starts" << dendl;
+
+ for (auto i = smap.objects.rbegin(); i != smap.objects.rend(); ++i) {
+
+ const hobject_t& hoid = i->first;
+ ScrubMap::object& o = i->second;
+
+ dout(20) << __func__ << " " << hoid << dendl;
+
+ ceph_assert(!hoid.is_snapdir());
+
+ if (hoid.is_head()) {
+ // parse the SnapSet
+ bufferlist bl;
+ if (o.attrs.find(SS_ATTR) == o.attrs.end()) {
+ continue;
+ }
+ bl.push_back(o.attrs[SS_ATTR]);
+ auto p = bl.cbegin();
+ try {
+ decode(snapset, p);
+ } catch (...) {
+ continue;
+ }
+ head = hoid.get_head();
+ continue;
+ }
+
+ /// \todo document why guaranteed to have initialized 'head' at this point
+
+ if (hoid.snap < CEPH_MAXSNAP) {
+
+ if (hoid.get_head() != head) {
+ derr << __func__ << " no head for " << hoid << " (have " << head << ")"
+ << dendl;
+ continue;
+ }
+
+ scan_object_snaps(hoid, o, snapset);
+ }
+ }
+}
+
+void ScrubBackend::scan_object_snaps(const hobject_t& hoid,
+ ScrubMap::object& scrmap_obj,
+ const SnapSet& snapset)
+{
+ // check and if necessary fix snap_mapper
+
+ auto p = snapset.clone_snaps.find(hoid.snap);
+ if (p == snapset.clone_snaps.end()) {
+ derr << __func__ << " no clone_snaps for " << hoid << " in " << snapset
+ << dendl;
+ return;
+ }
+ 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);
+ 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;
+
+ 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; });
+ }
+ }
+ }
+}
+
+
+/*
+ * Process:
+ * Building a map of objects suitable for snapshot validation.
+ * The data in m_cleaned_meta_map is the leftover partial items that need to
+ * be completed before they can be processed.
+ *
+ * Snapshots in maps precede the head object, which is why we are scanning
+ * backwards.
+ */
+ScrubMap ScrubBackend::clean_meta_map(ScrubMap& cleaned, bool max_reached)
+{
+ ScrubMap for_meta_scrub;
+
+ if (max_reached || cleaned.objects.empty()) {
+ cleaned.swap(for_meta_scrub);
+ } else {
+ auto iter = cleaned.objects.end();
+ --iter; // not empty, see 'if' clause
+ auto begin = cleaned.objects.begin();
+ if (iter->first.has_snapset()) {
+ ++iter;
+ } else {
+ while (iter != begin) {
+ auto next = iter--;
+ if (next->first.get_head() != iter->first.get_head()) {
+ ++iter;
+ break;
+ }
+ }
+ }
+ for_meta_scrub.objects.insert(begin, iter);
+ cleaned.objects.erase(begin, iter);
+ }
+
+ return for_meta_scrub;
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+#pragma once
+
+// clang-format off
+/*
+ +------------------------+
+ | |
+ | PgScrubber |
+ | |-----------------------------+
+ | | |
+ +------------------------+ | ownes & uses
+ | PrimaryLogScrub | |
+ +------------------------+ |
+ |
+ |
+ v
+ +-------------------------------------------+
+ |ScrubBackend |
+ +----------------+ |============ |
+ | this_chunk | | |
+ | (scrub_chunk_t)|<-------| + decode_received_map() |
+ +----------------+ | + scrub_compare_maps() |
+ | + scan_snaps() |
+ | ..... |
+ | |
+ | |
+ +--------------------/-------------\--------+
+ --/ / \
+ --/ | |
+ --/ / \
+ -/ uses | uses |
+ uses --/ / \
+ --/ / |
+ --/ | \
+ v v v
+ PgBackend PG/PrimaryLogPG OSD Services
+
+
+*/
+// clang-format on
+
+#include <fmt/core.h>
+#include <fmt/format.h>
+
+#include <string_view>
+
+#include "common/LogClient.h"
+#include "common/scrub_types.h"
+
+struct ScrubMap;
+
+class PG;
+class PgScrubber;
+class PGBackend;
+
+
+using data_omap_digests_t =
+ std::pair<std::optional<uint32_t>, std::optional<uint32_t>>;
+
+/// a list of fixes to be performed on objects' digests
+using digests_fixes_t = std::vector<std::pair<hobject_t, data_omap_digests_t>>;
+
+using shard_info_map_t = std::map<pg_shard_t, shard_info_wrapper>;
+using shard_to_scrubmap_t = std::map<pg_shard_t, ScrubMap>;
+
+
+/// omap-specific stats
+struct omap_stat_t {
+ int large_omap_objects{0};
+ int64_t omap_bytes{0};
+ int64_t omap_keys{0};
+};
+
+
+/**
+ * A structure used internally by select_auth_object()
+ *
+ * Conveys the usability of a specific shard as an auth source.
+ */
+struct shard_as_auth_t {
+ enum class usable_t : uint8_t { not_usable, usable };
+
+ // the ctor used when the shard should not be considered as auth
+ explicit shard_as_auth_t(std::string err_msg)
+ : possible_auth{usable_t::not_usable}
+ , error_text{err_msg}
+ , oi{}
+ , auth_iter{}
+ , digest{std::nullopt}
+ {}
+
+ shard_as_auth_t(std::string err_msg, std::optional<uint32_t> data_digest)
+ : possible_auth{usable_t::not_usable}
+ , error_text{err_msg}
+ , oi{}
+ , auth_iter{}
+ , digest{data_digest}
+ {}
+
+ // possible auth candidate
+ shard_as_auth_t(const object_info_t& anoi,
+ shard_to_scrubmap_t::iterator it,
+ std::string err_msg,
+ std::optional<uint32_t> data_digest)
+ : possible_auth{usable_t::usable}
+ , error_text{err_msg}
+ , oi{anoi}
+ , auth_iter{it}
+ , digest{data_digest}
+ {}
+
+
+ usable_t possible_auth;
+ std::string error_text;
+ object_info_t oi;
+ shard_to_scrubmap_t::iterator auth_iter;
+ std::optional<uint32_t> digest;
+ // when used for Crimson, we'll probably want to return 'digest_match' (and
+ // other in/out arguments) via this struct
+};
+
+struct auth_selection_t {
+ shard_to_scrubmap_t::iterator auth; ///< an iter into one of this_chunk->maps
+ pg_shard_t auth_shard; // set to auth->first
+ object_info_t auth_oi;
+ shard_info_map_t shard_map;
+ bool is_auth_available{false}; ///< managed to select an auth' source?
+ bool digest_match{true}; ///< do all (existing) digests match?
+};
+
+// note: some scrub tests are sensitive to the specific format of
+// auth_selection_t listing in the logs
+template <>
+struct fmt::formatter<auth_selection_t> {
+ template <typename ParseContext>
+ constexpr auto parse(ParseContext& ctx)
+ {
+ return ctx.begin();
+ }
+
+ template <typename FormatContext>
+ auto format(auth_selection_t const& aus, FormatContext& ctx)
+ {
+ return fmt::format_to(ctx.out(),
+ " {{AU-S: {}->{:x} OI({:x}:{}) {} dm:{}}} ",
+ aus.auth->first,
+ (uint64_t)(&aus.auth->second),
+ (uint64_t)(&aus.auth_oi),
+ aus.auth_oi,
+ aus.shard_map.size(),
+ aus.digest_match);
+ }
+};
+
+/**
+ * the back-end data that is per-chunk
+ *
+ * Created by the Scrubber after all replicas' maps have arrived.
+ */
+struct scrub_chunk_t {
+
+ explicit scrub_chunk_t(pg_shard_t i_am) { received_maps[i_am] = ScrubMap{}; }
+
+ /// the working set of scrub maps: the received maps, plus
+ /// Primary's own map.
+ std::map<pg_shard_t, ScrubMap> received_maps;
+
+ /// a collection of all objs mentioned in the maps
+ std::set<hobject_t> master_set;
+
+ utime_t started{ceph_clock_now()};
+
+ std::map<hobject_t, data_omap_digests_t> missing_digest;
+
+ /// Map from object with errors to good peers
+ std::map<hobject_t, std::list<pg_shard_t>> authoritative;
+
+
+ // these must be reset for each element:
+
+ std::set<pg_shard_t> cur_missing;
+ std::set<pg_shard_t> cur_inconsistent;
+ bool fix_digest{false};
+};
+
+
+/**
+ * ScrubBackend wraps the data and operations required for the back-end part of
+ * the scrubbing (i.e. for comparing the maps and fixing objects).
+ *
+ * Created anew upon each initiation of a scrub session.
+ */
+class ScrubBackend {
+ public:
+ // Primary constructor
+ ScrubBackend(PgScrubber& scrubber,
+ PGBackend& backend,
+ PG& pg,
+ pg_shard_t i_am,
+ bool repair,
+ scrub_level_t shallow_or_deep,
+ const std::set<pg_shard_t>& acting);
+
+ // Replica constructor: no primary map
+ ScrubBackend(PgScrubber& scrubber,
+ PGBackend& backend,
+ PG& pg,
+ pg_shard_t i_am,
+ bool repair,
+ scrub_level_t shallow_or_deep);
+
+ friend class PgScrubber;
+
+ /**
+ * reset the per-chunk data structure (scrub_chunk_t).
+ * Create an empty scrub-map for this shard, and place it
+ * in the appropriate entry in 'received_maps'.
+ *
+ * @returns a pointer to the newly created ScrubMap.
+ */
+ ScrubMap* new_chunk();
+
+ /**
+ * sets Backend's m_repair flag (setting m_mode_desc to a corresponding
+ * string)
+ */
+ void update_repair_status(bool should_repair);
+
+ void replica_clean_meta(ScrubMap& smap,
+ bool max_reached,
+ const hobject_t& start);
+
+ /**
+ * decode the arriving MOSDRepScrubMap message, placing the replica's
+ * scrub-map into received_maps[from].
+ *
+ * @param from replica
+ */
+ void decode_received_map(pg_shard_t from, const MOSDRepScrubMap& msg);
+
+ void scrub_compare_maps(bool max_reached);
+
+ int scrub_process_inconsistent();
+
+ void repair_oinfo_oid(ScrubMap& smap);
+
+ const omap_stat_t& this_scrub_omapstats() const { return m_omap_stats; }
+
+ std::ostream& logger_prefix(std::ostream* _dout, const ScrubBackend* t);
+
+ private:
+ // set/constructed at the ctor():
+ PgScrubber& m_scrubber;
+ PGBackend& m_pgbe;
+ PG& m_pg;
+ const pg_shard_t m_pg_whoami;
+ bool m_repair;
+ const scrub_level_t m_depth;
+ const spg_t m_pg_id;
+ std::vector<pg_shard_t> m_acting_but_me; // primary only
+ bool m_is_replicated{true};
+ std::string_view m_mode_desc;
+ std::string m_formatted_id;
+ /// collecting some scrub-session-wide omap stats
+ omap_stat_t m_omap_stats;
+
+ // 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
+ >;
+
+ std::optional<scrub_chunk_t> this_chunk;
+
+ /// Maps from objects with errors to missing peers
+ HobjToShardSetMapping m_missing; // used by scrub_process_inconsistent()
+
+ /// Maps from objects with errors to inconsistent peers
+ HobjToShardSetMapping m_inconsistent; // used by scrub_process_inconsistent()
+
+ /// Cleaned std::map pending snap metadata scrub
+ ScrubMap m_cleaned_meta_map{};
+
+ /// a reference to the primary map
+ ScrubMap& my_map();
+
+ /**
+ * merge_to_master_set() updates
+ * - this_chunk->maps[from] with the replicas' scrub-maps;
+ * - this_chunk->master_set as a union of all the maps' objects;
+ */
+ void merge_to_master_set();
+
+ // note: used by both Primary & replicas
+ static ScrubMap clean_meta_map(ScrubMap& cleaned, bool max_reached);
+
+ std::optional<std::string> compare_smaps();
+
+ void compare_obj_in_maps(const hobject_t& ho, std::stringstream& errstream);
+
+ void omap_checks();
+
+ std::optional<auth_and_obj_errs_t> 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,
+ std::stringstream& errstream);
+
+ auth_and_obj_errs_t match_in_shards(const hobject_t& ho,
+ auth_selection_t& auth_sel,
+ inconsistent_obj_wrapper& obj_result,
+ std::stringstream& errstream);
+
+ // returns: true if a discrepancy was found
+ bool compare_obj_details(pg_shard_t auth_shard,
+ const ScrubMap::object& auth,
+ const object_info_t& auth_oi,
+ const ScrubMap::object& candidate,
+ shard_info_wrapper& shard_result,
+ inconsistent_obj_wrapper& obj_result,
+ 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);
+
+ /**
+ * An auxiliary used by select_auth_object() to test a specific shard
+ * as a possible auth candidate.
+ * @param ho the hobject for which we are looking for an auth source
+ * @param srd the candidate shard
+ * @param shard_map [out] a collection of shard_info-s per shard.
+ * possible_auth_shard() might set error flags in the relevant (this shard's)
+ * entry.
+ */
+ shard_as_auth_t possible_auth_shard(const hobject_t& ho,
+ const pg_shard_t& srd,
+ shard_info_map_t& shard_map);
+
+ auth_selection_t select_auth_object(const hobject_t& ho,
+ std::stringstream& errstream);
+
+
+ enum class digest_fixing_t { no, if_aged, force };
+
+ /*
+ * an aux used by inconsistents() to determine whether to fix the digest
+ */
+ [[nodiscard]] digest_fixing_t should_fix_digest(
+ const hobject_t& ho,
+ const ScrubMap::object& auth_object,
+ const object_info_t& auth_oi,
+ bool repair_flag,
+ std::stringstream& errstream);
+
+ void inconsistents(const hobject_t& ho,
+ ScrubMap::object& auth_object,
+ object_info_t& auth_oi, // consider moving to object
+ auth_and_obj_errs_t&& auth_n_errs,
+ std::stringstream& errstream);
+
+ int process_clones_to(const std::optional<hobject_t>& head,
+ const std::optional<SnapSet>& snapset,
+ bool allow_incomplete_clones,
+ std::optional<snapid_t> target,
+ std::vector<snapid_t>::reverse_iterator* curclone,
+ inconsistent_snapset_wrapper& e);
+
+ /**
+ * Validate consistency of the object info and snap sets.
+ */
+ void scrub_snapshot_metadata(ScrubMap& map);
+
+ /**
+ * Updates the "global" (i.e. - not 'per-chunk') databases:
+ * - in m_authoritative: a list of good peers for each "problem" object in
+ * the current chunk;
+ * - in m_cleaned_meta_map: a "cleaned" version of the object (the one from
+ * the selected shard).
+ */
+ void update_authoritative();
+
+ void log_missing(int missing,
+ const std::optional<hobject_t>& head,
+ const char* logged_func_name,
+ bool allow_incomplete_clones);
+
+ void scan_snaps(ScrubMap& smap);
+
+ void scan_object_snaps(const hobject_t& hoid,
+ ScrubMap::object& scrmap_obj,
+ const SnapSet& snapset);
+};