common/linux_version.c
common/TracepointProvider.cc
common/Cycles.cc
+ common/scrub_types.cc
osdc/Striper.cc
osdc/Objecter.cc
common/Graylog.cc
osd/OpRequest.cc
common/TrackedOp.cc
osd/SnapMapper.cc
+ osd/ScrubStore.cc
osd/osd_types.cc
osd/ECUtil.cc
objclass/class_api.cc
common/Cycles.cc \
common/ContextCompletion.cc \
common/TracepointProvider.cc \
- common/PluginRegistry.cc
+ common/PluginRegistry.cc \
+ common/scrub_types.cc
common/PluginRegistry.cc: ./ceph_ver.h
common/TracepointProvider.h \
common/event_socket.h \
common/PluginRegistry.h \
+ common/scrub_types.h \
common/ceph_time.h \
common/ceph_timer.h \
common/align.h \
--- /dev/null
+#include "scrub_types.h"
+
+using namespace librados;
+
+void object_id_wrapper::encode(bufferlist& bl) const
+{
+ ENCODE_START(1, 1, bl);
+ ::encode(name, bl);
+ ::encode(nspace, bl);
+ ::encode(locator, bl);
+ ::encode(snap, bl);
+ ENCODE_FINISH(bl);
+}
+
+void object_id_wrapper::decode(bufferlist::iterator& bp)
+{
+ DECODE_START(1, bp);
+ ::decode(name, bp);
+ ::decode(nspace, bp);
+ ::decode(locator, bp);
+ ::decode(snap, bp);
+ DECODE_FINISH(bp);
+}
+
+static void encode(const object_id_t& obj, bufferlist& bl)
+{
+ reinterpret_cast<const object_id_wrapper&>(obj).encode(bl);
+}
+
+void shard_info_wrapper::set_object(const ScrubMap::object& object)
+{
+ for (auto attr : object.attrs) {
+ bufferlist bl;
+ bl.push_back(attr.second);
+ attrs.insert(std::make_pair(attr.first, std::move(bl)));
+ }
+ size = object.size;
+ if (object.omap_digest_present) {
+ omap_digest_present = true;
+ omap_digest = object.omap_digest;
+ }
+ if (object.digest_present) {
+ data_digest_present = true;
+ data_digest = object.digest;
+ }
+ if (object.read_error) {
+ errors |= SHARD_READ_ERR;
+ }
+ if (object.stat_error) {
+ errors |= SHARD_STAT_ERR;
+ }
+}
+
+void shard_info_wrapper::encode(bufferlist& bl) const
+{
+ ENCODE_START(1, 1, bl);
+ ::encode(errors, bl);
+ if (has_shard_missing()) {
+ return;
+ }
+ ::encode(attrs, bl);
+ ::encode(size, bl);
+ ::encode(omap_digest_present, bl);
+ ::encode(omap_digest, bl);
+ ::encode(data_digest_present, bl);
+ ::encode(data_digest, bl);
+ ENCODE_FINISH(bl);
+}
+
+void shard_info_wrapper::decode(bufferlist::iterator& bp)
+{
+ DECODE_START(1, bp);
+ ::decode(errors, bp);
+ if (has_shard_missing()) {
+ return;
+ }
+ ::decode(attrs, bp);
+ ::decode(size, bp);
+ ::decode(omap_digest_present, bp);
+ ::decode(omap_digest, bp);
+ ::decode(data_digest_present, bp);
+ ::decode(data_digest, bp);
+ DECODE_FINISH(bp);
+}
+
+inconsistent_obj_wrapper::inconsistent_obj_wrapper(const hobject_t& hoid)
+ : inconsistent_obj_t{librados::object_id_t{hoid.oid.name,
+ hoid.nspace,
+ hoid.get_key(), hoid.snap}}
+{}
+
+void inconsistent_obj_wrapper::add_shard(const pg_shard_t& pgs,
+ const shard_info_wrapper& shard)
+{
+ errors |= shard.errors;
+ shards[pgs.osd] = shard;
+}
+
+void
+inconsistent_obj_wrapper::set_auth_missing(const hobject_t& hoid,
+ const map<pg_shard_t, ScrubMap*>& maps)
+{
+ errors |= (err_t::SHARD_MISSING |
+ err_t::SHARD_READ_ERR |
+ err_t::OMAP_DIGEST_MISMATCH |
+ err_t::DATA_DIGEST_MISMATCH |
+ err_t::ATTR_MISMATCH);
+ for (auto pg_map : maps) {
+ auto oid_object = pg_map.second->objects.find(hoid);
+ shard_info_wrapper shard;
+ if (oid_object == pg_map.second->objects.end()) {
+ shard.set_missing();
+ } else {
+ shard.set_object(oid_object->second);
+ }
+ shards[pg_map.first.osd] = shard;
+ }
+}
+
+namespace librados {
+ static void encode(const shard_info_t& shard, bufferlist& bl)
+ {
+ reinterpret_cast<const shard_info_wrapper&>(shard).encode(bl);
+ }
+}
+
+void inconsistent_obj_wrapper::encode(bufferlist& bl) const
+{
+ ENCODE_START(1, 1, bl);
+ ::encode(errors, bl);
+ ::encode(object, bl);
+ ::encode(shards, bl);
+ ENCODE_FINISH(bl);
+}
+
+void inconsistent_obj_wrapper::decode(bufferlist::iterator& bp)
+{
+ DECODE_START(1, bp);
+ ::decode(errors, bp);
+ ::decode(object, bp);
+ ::decode(shards, bp);
+ DECODE_FINISH(bp);
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_SCRUB_TYPES_H
+#define CEPH_SCRUB_TYPES_H
+
+#include "include/rados/rados_types.hpp"
+#include "common/hobject.h"
+#include "osd/osd_types.h"
+
+
+// wrappers around scrub types to offer the necessary bits other than
+// the minimal set that the lirados requires
+struct object_id_wrapper : public librados::object_id_t {
+ object_id_wrapper(const hobject_t& hoid)
+ : object_id_t{hoid.oid.name, hoid.nspace, hoid.get_key(), hoid.snap}
+ {}
+ void encode(bufferlist& bl) const;
+ void decode(bufferlist::iterator& bl);
+};
+
+WRITE_CLASS_ENCODER(object_id_wrapper)
+
+inline void decode(librados::object_id_t& obj, bufferlist::iterator& bp) {
+ reinterpret_cast<object_id_wrapper&>(obj).decode(bp);
+}
+
+struct shard_info_wrapper : public librados::shard_info_t {
+public:
+ shard_info_wrapper() = default;
+ shard_info_wrapper(const ScrubMap::object& object) {
+ set_object(object);
+ }
+ void set_object(const ScrubMap::object& object);
+ void set_missing() {
+ errors |= err_t::SHARD_MISSING;
+ }
+ void set_omap_digest_mismatch() {
+ errors |= err_t::OMAP_DIGEST_MISMATCH;
+ }
+ void set_omap_digest_mismatch_oi() {
+ errors |= err_t::OMAP_DIGEST_MISMATCH_OI;
+ }
+ void set_data_digest_mismatch() {
+ errors |= err_t::DATA_DIGEST_MISMATCH;
+ }
+ void set_data_digest_mismatch_oi() {
+ errors |= err_t::DATA_DIGEST_MISMATCH_OI;
+ }
+ void set_size_mismatch() {
+ errors |= err_t::SIZE_MISMATCH;
+ }
+ void set_attr_missing() {
+ errors |= err_t::ATTR_MISMATCH;
+ }
+ void set_attr_mismatch() {
+ errors |= err_t::ATTR_MISMATCH;
+ }
+ void set_attr_unexpected() {
+ errors |= err_t::ATTR_MISMATCH;
+ }
+ void encode(bufferlist& bl) const;
+ void decode(bufferlist::iterator& bp);
+};
+
+WRITE_CLASS_ENCODER(shard_info_wrapper)
+
+namespace librados {
+ inline void decode(librados::shard_info_t& shard,
+ bufferlist::iterator& bp) {
+ reinterpret_cast<shard_info_wrapper&>(shard).decode(bp);
+ }
+}
+
+struct inconsistent_obj_wrapper : librados::inconsistent_obj_t {
+ inconsistent_obj_wrapper(const hobject_t& hoid);
+
+ void add_shard(const pg_shard_t& pgs, const shard_info_wrapper& shard);
+ void set_auth_missing(const hobject_t& hoid,
+ const map<pg_shard_t, ScrubMap*>& map);
+ void encode(bufferlist& bl) const;
+ void decode(bufferlist::iterator& bp);
+};
+
+WRITE_CLASS_ENCODER(inconsistent_obj_wrapper)
+
+inline void decode(librados::inconsistent_obj_t& obj,
+ bufferlist::iterator& bp) {
+ reinterpret_cast<inconsistent_obj_wrapper&>(obj).decode(bp);
+}
+
+#endif
osd/ClassHandler.cc \
osd/OpRequest.cc \
osd/SnapMapper.cc \
+ osd/ScrubStore.cc \
objclass/class_api.cc
libosd_a_CXXFLAGS = ${AM_CXXFLAGS}
osd/ECMsgTypes.h \
osd/ECTransaction.h \
osd/Watch.h \
+ osd/ScrubStore.h \
osd/osd_types.h
endif # WITH_OSD
#include "common/config.h"
#include "OSD.h"
#include "OpRequest.h"
+#include "ScrubStore.h"
#include "common/Timer.h"
agent_clear();
}
+PG::Scrubber::Scrubber()
+ : reserved(false), reserve_failed(false),
+ epoch_start(0),
+ active(false), queue_snap_trim(false),
+ waiting_on(0), shallow_errors(0), deep_errors(0), fixed(0),
+ must_scrub(false), must_deep_scrub(false), must_repair(false),
+ auto_repair(false),
+ num_digest_updates_pending(0),
+ state(INACTIVE),
+ deep(false),
+ seed(0)
+{}
+
+PG::Scrubber::~Scrubber() {}
+
/**
* find_best_info
*
scrubber.reserved_peers.clear();
}
+ {
+ ObjectStore::Transaction t;
+ scrubber.store.reset(Scrub::Store::create(osd->store, &t,
+ info.pgid, coll));
+ osd->store->queue_transaction(osr.get(), std::move(t), nullptr);
+ }
+
// Don't include temporary objects when scrubbing
scrubber.start = info.pgid.pgid.get_hobj_start();
scrubber.state = PG::Scrubber::NEW_CHUNK;
missing_digest,
scrubber.shallow_errors,
scrubber.deep_errors,
+ scrubber.store.get(),
info.pgid, acting,
ss);
dout(2) << ss.str() << dendl;
// ok, do the pg-type specific scrubbing
_scrub(authmap, missing_digest);
+ if (!state_test(PG_STATE_REPAIR) && !scrubber.store->empty()) {
+ dout(10) << __func__ << ": updating scrub object" << dendl;
+ ObjectStore::Transaction t;
+ scrubber.store->flush(&t);
+ osd->store->queue_transaction(osr.get(), std::move(t), nullptr);
+ }
}
bool PG::scrub_process_inconsistent()
class PG;
+namespace Scrub {
+ class Store;
+}
+
void intrusive_ptr_add_ref(PG *pg);
void intrusive_ptr_release(PG *pg);
// -- scrub --
struct Scrubber {
- Scrubber() :
- reserved(false), reserve_failed(false),
- epoch_start(0),
- active(false), queue_snap_trim(false),
- waiting_on(0), shallow_errors(0), deep_errors(0), fixed(0),
- must_scrub(false), must_deep_scrub(false), must_repair(false),
- auto_repair(false),
- num_digest_updates_pending(0),
- state(INACTIVE),
- deep(false),
- seed(0)
- {
- }
+ Scrubber();
+ ~Scrubber();
// metadata
set<pg_shard_t> reserved_peers;
FINISH,
} state;
+ std::unique_ptr<Scrub::Store> store;
// deep scrub
bool deep;
uint32_t seed;
num_digest_updates_pending = 0;
}
+ void create_results(const hobject_t& obj);
} scrubber;
bool scrub_after_recovery;
#include "common/errno.h"
+#include "common/scrub_types.h"
#include "ReplicatedBackend.h"
+#include "ScrubStore.h"
#include "ECBackend.h"
#include "PGBackend.h"
#include "OSD.h"
const object_info_t& auth_oi,
bool okseed,
const ScrubMap::object &candidate,
+ shard_info_wrapper &result,
ostream &errorstream)
{
enum scrub_error_type error = CLEAN;
<< (known ? "known" : "best guess")
<< " data_digest 0x" << auth.digest << std::dec
<< " from auth shard " << auth_shard;
+ result.set_data_digest_mismatch();
}
}
if (auth.omap_digest_present && candidate.omap_digest_present) {
<< (known ? "known" : "best guess")
<< " omap_digest 0x" << auth.omap_digest << std::dec
<< " from auth shard " << auth_shard;
+ result.set_omap_digest_mismatch();
}
}
if (!candidate.stat_error && auth.size != candidate.size) {
<< " != "
<< (known ? "known" : "best guess")
<< " size " << auth.size;
+ result.set_size_mismatch();
}
for (map<string,bufferptr>::const_iterator i = auth.attrs.begin();
i != auth.attrs.end();
if (error != DEEP_ERROR)
error = SHALLOW_ERROR;
errorstream << "missing attr " << i->first;
+ result.set_attr_missing();
} else if (candidate.attrs.find(i->first)->second.cmp(i->second)) {
if (error != CLEAN)
errorstream << ", ";
if (error != DEEP_ERROR)
error = SHALLOW_ERROR;
errorstream << "attr value mismatch " << i->first;
+ result.set_attr_mismatch();
}
}
for (map<string,bufferptr>::const_iterator i = candidate.attrs.begin();
if (error != DEEP_ERROR)
error = SHALLOW_ERROR;
errorstream << "extra attr " << i->first;
+ result.set_attr_unexpected();
}
}
return error;
map<hobject_t, list<pg_shard_t>, hobject_t::BitwiseComparator> &authoritative,
map<hobject_t, pair<uint32_t,uint32_t>, hobject_t::BitwiseComparator> &missing_digest,
int &shallow_errors, int &deep_errors,
+ Scrub::Store *store,
const spg_t& pgid,
const vector<int> &acting,
ostream &errorstream)
object_info_t auth_oi;
map<pg_shard_t, ScrubMap *>::const_iterator auth =
be_select_auth_object(*k, maps, okseed, &auth_oi);
+ inconsistent_obj_wrapper object_error{*k};
+
list<pg_shard_t> auth_list;
if (auth == maps.end()) {
+ object_error.set_auth_missing(*k, maps);
++shallow_errors;
errorstream << pgid.pgid << " soid " << *k
<< ": failed to pick suitable auth object\n";
for (j = maps.begin(); j != maps.end(); ++j) {
if (j == auth)
continue;
+ shard_info_wrapper shard_info;
if (j->second->objects.count(*k)) {
+ shard_info.set_object(j->second->objects[*k]);
// Compare
stringstream ss;
enum scrub_error_type error =
auth_oi,
okseed,
j->second->objects[*k],
+ shard_info,
ss);
if (error != CLEAN) {
clean = false;
++shallow_errors;
errorstream << pgid << " shard " << j->first << " missing " << *k
<< "\n";
+ shard_info.set_missing();
}
+ object_error.add_shard(j->first, shard_info);
}
if (!cur_missing.empty()) {
missing[*k] = cur_missing;
}
if (!cur_inconsistent.empty() || !cur_missing.empty()) {
authoritative[*k] = auth_list;
+ shard_info_wrapper auth_shard{auth_object};
+ object_error.add_shard(auth->first, auth_shard);
}
if (okseed &&
update = MAYBE;
}
+ shard_info_wrapper auth_shard{auth_object};
// recorded digest != actual digest?
if (auth_oi.is_data_digest() && auth_object.digest_present &&
auth_oi.data_digest != auth_object.digest) {
+ auth_shard.set_data_digest_mismatch_oi();
++deep_errors;
errorstream << pgid << " recorded data digest 0x"
<< std::hex << auth_oi.data_digest << " != on disk 0x"
}
if (auth_oi.is_omap_digest() && auth_object.omap_digest_present &&
auth_oi.omap_digest != auth_object.omap_digest) {
+ auth_shard.set_omap_digest_mismatch_oi();
++deep_errors;
errorstream << pgid << " recorded omap digest 0x"
<< std::hex << auth_oi.omap_digest << " != on disk 0x"
if (repair)
update = FORCE;
}
+ object_error.add_shard(auth->first, auth_shard);
if (update != NO) {
utime_t age = now - auth_oi.local_mtime;
}
}
}
+ if (object_error.errors) {
+ store->add_object_error(k->pool, object_error);
+ }
}
}
#include "common/LogClient.h"
#include <string>
+namespace Scrub {
+ class Store;
+}
+struct shard_info_wrapper;
+
/**
* PGBackend
*
const object_info_t& auth_oi,
bool okseed,
const ScrubMap::object &candidate,
+ shard_info_wrapper& shard_error,
ostream &errorstream);
map<pg_shard_t, ScrubMap *>::const_iterator be_select_auth_object(
const hobject_t &obj,
map<hobject_t, list<pg_shard_t>, hobject_t::BitwiseComparator> &authoritative,
map<hobject_t, pair<uint32_t,uint32_t>, hobject_t::BitwiseComparator> &missing_digest,
int &shallow_errors, int &deep_errors,
+ Scrub::Store *store,
const spg_t& pgid,
const vector<int> &acting,
ostream &errorstream);
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "ScrubStore.h"
+#include "osd_types.h"
+#include "common/scrub_types.h"
+#include "include/rados/rados_types.hpp"
+
+namespace {
+hobject_t make_scrub_object(const spg_t& pgid)
+{
+ ostringstream ss;
+ ss << "scrub_" << pgid;
+ return pgid.make_temp_object(ss.str());
+}
+
+string first_object_key(int64_t pool)
+{
+ auto hoid = hobject_t(object_t(),
+ "",
+ 0,
+ 0x00000000,
+ pool,
+ "");
+ hoid.build_hash_cache();
+ return "SCRUB_OBJ_" + hoid.to_str();
+}
+
+// the object_key should be unique across pools
+string to_object_key(int64_t pool, const librados::object_id_t& oid)
+{
+ auto hoid = hobject_t(object_t(oid.name),
+ oid.locator, // key
+ oid.snap,
+ 0, // hash
+ pool,
+ oid.nspace);
+ hoid.build_hash_cache();
+ return "SCRUB_OBJ_" + hoid.to_str();
+}
+
+string last_object_key(int64_t pool)
+{
+ auto hoid = hobject_t(object_t(),
+ "",
+ 0,
+ 0xffffffff,
+ pool,
+ "");
+ hoid.build_hash_cache();
+ return "SCRUB_OBJ_" + hoid.to_str();
+}
+}
+
+namespace Scrub {
+
+Store*
+Store::create(ObjectStore* store,
+ ObjectStore::Transaction* t,
+ const spg_t& pgid,
+ const coll_t& coll)
+{
+ assert(store);
+ assert(t);
+ hobject_t oid = make_scrub_object(pgid);
+ coll_t temp_coll = coll.get_temp();
+ t->touch(temp_coll, ghobject_t{oid});
+ return new Store{temp_coll, oid, store};
+}
+
+Store::Store(const coll_t& coll, const hobject_t& oid, ObjectStore* store)
+ : coll(coll),
+ hoid(oid),
+ driver(store, coll, hoid),
+ backend(&driver)
+{}
+
+Store::~Store()
+{
+ assert(results.empty());
+}
+
+void Store::add_object_error(int64_t pool, const inconsistent_obj_wrapper& e)
+{
+ bufferlist bl;
+ e.encode(bl);
+ results[to_object_key(pool, e.object)] = bl;
+}
+
+bool Store::empty() const
+{
+ return results.empty();
+}
+
+void Store::flush(ObjectStore::Transaction* t)
+{
+ OSDriver::OSTransaction txn = driver.get_transaction(t);
+ backend.set_keys(results, &txn);
+ results.clear();
+}
+
+void Store::cleanup(ObjectStore::Transaction* t)
+{
+ t->remove(coll, hoid);
+}
+
+std::vector<bufferlist>
+Store::get_object_errors(ObjectStore* store,
+ int64_t pool,
+ const librados::object_id_t& start,
+ uint64_t max_return)
+{
+ const string begin = (start.name.empty() ?
+ first_object_key(pool) : to_object_key(pool, start));
+ const string end = last_object_key(pool);
+ return get_errors(store, begin, end, max_return);
+}
+
+std::vector<bufferlist>
+Store::get_errors(ObjectStore* store,
+ const string& begin,
+ const string& end,
+ uint64_t max_return)
+{
+ vector<bufferlist> errors;
+ auto next = std::make_pair(begin, bufferlist{});
+ while (max_return && !backend.get_next(next.first, &next)) {
+ if (next.first >= end)
+ break;
+ errors.push_back(next.second);
+ max_return--;
+ }
+ return errors;
+}
+} // namespace Scrub
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_SCRUB_RESULT_H
+#define CEPH_SCRUB_RESULT_H
+
+#include "SnapMapper.h" // for OSDriver
+#include "common/map_cacher.hpp"
+
+namespace librados {
+ struct object_id_t;
+}
+
+struct inconsistent_obj_wrapper;
+
+namespace Scrub {
+
+class Store {
+public:
+ ~Store();
+ static Store* create(ObjectStore* store,
+ ObjectStore::Transaction* t,
+ const spg_t& pgid,
+ const coll_t& coll);
+ void add_object_error(int64_t pool, const inconsistent_obj_wrapper& e);
+ bool empty() const;
+ void flush(ObjectStore::Transaction *);
+ void cleanup(ObjectStore::Transaction *);
+ std::vector<bufferlist> get_object_errors(ObjectStore* store,
+ int64_t pool,
+ const librados::object_id_t& start,
+ uint64_t max_return);
+private:
+ Store(const coll_t& coll, const hobject_t& oid, ObjectStore* store);
+ std::vector<bufferlist> get_errors(ObjectStore* store,
+ const string& start, const string& end,
+ uint64_t max_return);
+private:
+ const coll_t coll;
+ const ghobject_t hoid;
+ // a temp object holding mappings from seq-id to inconsistencies found in
+ // scrubbing
+ OSDriver driver;
+ MapCacher::MapCacher<std::string, bufferlist> backend;
+ map<string, bufferlist> results;
+};
+}
+
+#endif // CEPH_SCRUB_RESULT_H