#include <sys/types.h>
#include <sys/stat.h>
#include <fcntl.h>
+#include <algorithm>
#include <boost/container/flat_set.hpp>
-#include "boost/algorithm/string.hpp"
+#include <boost/algorithm/string.hpp>
#include "include/cpp-btree/btree_set.h"
const string PREFIX_DEFERRED = "L"; // id -> deferred_transaction_t
const string PREFIX_ALLOC = "B"; // u64 offset -> u64 length (freelist)
const string PREFIX_ALLOC_BITMAP = "b";// (see BitmapFreelistManager)
-const string PREFIX_SHARED_BLOB = "X"; // u64 offset -> shared_blob_t
+const string PREFIX_SHARED_BLOB = "X"; // u64 SB id -> shared_blob_t
#ifdef HAVE_LIBZBD
const string PREFIX_ZONED_FM_META = "Z"; // (see ZonedFreelistManager)
return 0;
}
-
int BlueStore::_fsck_check_extents(
- const coll_t& cid,
- const ghobject_t& oid,
+ std::string_view ctx_descr,
const PExtentVector& extents,
bool compressed,
mempool_dynamic_bitset &used_blocks,
store_statfs_t& expected_statfs,
FSCKDepth depth)
{
- dout(30) << __func__ << " oid " << oid << " extents " << extents << dendl;
+ dout(30) << __func__ << " " << ctx_descr << ", extents " << extents << dendl;
int errors = 0;
for (auto e : extents) {
if (!e.is_valid())
}
if (depth != FSCK_SHALLOW) {
bool already = false;
- //dout(1) << __func__ << "::NCB::FSCK<" << e.offset << "," << e.length << ">" << dendl;
apply_for_bitset_range(
e.offset, e.length, granularity, used_blocks,
[&](uint64_t pos, mempool_dynamic_bitset &bs) {
pos * min_alloc_size, min_alloc_size, !already);
}
if (!already) {
- derr << __func__ << "::fsck error: " << oid << " extent " << e
+ derr << __func__ << "::fsck error: " << ctx_descr << ", extent " << e
<< " or a subset is already allocated (misreferenced)" << dendl;
++errors;
already = true;
else
bs.set(pos);
});
- if (repairer) {
- repairer->set_space_used(e.offset, e.length, cid, oid);
- }
if (e.end() > bdev->get_size()) {
- derr << "fsck error: " << oid << " extent " << e
+ derr << "fsck error: " << ctx_descr << ", extent " << e
<< " past end of block device" << dendl;
++errors;
}
}
}
+void BlueStore::_fsck_repair_shared_blobs(
+ BlueStoreRepairer& repairer,
+ shared_blob_2hash_tracker_t& sb_ref_counts,
+ sb_info_space_efficient_map_t& sb_info)
+{
+ auto sb_ref_mismatches = sb_ref_counts.count_non_zero();
+ dout(1) << __func__ << " repairing shared_blobs, ref mismatch estimate: "
+ << sb_ref_mismatches << dendl;
+ if (!sb_ref_mismatches) // not expected to succeed, just in case
+ return;
+
+ mempool::bluestore_fsck::map<uint64_t, bluestore_extent_ref_map_t> ref_maps;
+ auto it = db->get_iterator(PREFIX_OBJ, KeyValueDB::ITERATOR_NOCACHE);
+ if (it) {
+ CollectionRef c;
+ spg_t pgid;
+ for (it->lower_bound(string()); it->valid(); it->next()) {
+ dout(30) << __func__ << " key "
+ << pretty_binary_string(it->key()) << dendl;
+ if (is_extent_shard_key(it->key())) {
+ continue;
+ }
+
+ ghobject_t oid;
+ int r = get_key_object(it->key(), &oid);
+ if (r < 0) {
+ continue;
+ }
+
+ if (!c ||
+ oid.shard_id != pgid.shard ||
+ oid.hobj.get_logical_pool() != (int64_t)pgid.pool() ||
+ !c->contains(oid)) {
+ c = nullptr;
+ for (auto& p : coll_map) {
+ if (p.second->contains(oid)) {
+ c = p.second;
+ break;
+ }
+ }
+ if (!c) {
+ continue;
+ }
+ }
+ dout(20) << __func__ << " inspecting shared blob refs for col:" << c->cid
+ << " obj:" << oid << dendl;
+
+ OnodeRef o;
+ o.reset(Onode::decode(c, oid, it->key(), it->value()));
+ o->extent_map.fault_range(db, 0, OBJECT_MAX_SIZE);
+
+ _dump_onode<30>(cct, *o);
+
+ mempool::bluestore_fsck::set<BlobRef> passed_sbs;
+ for (auto& e : o->extent_map.extent_map) {
+ auto& b = e.blob->get_blob();
+ if (b.is_shared() && passed_sbs.count(e.blob) == 0) {
+ auto sbid = e.blob->shared_blob->get_sbid();
+ passed_sbs.emplace(e.blob);
+ bluestore_extent_ref_map_t ref_map_candidate;
+ bool to_be_updated = false;
+ auto it = ref_maps.lower_bound(sbid);
+ bool add_existing = it != ref_maps.end() && it->first == sbid;
+ for (auto& p : b.get_extents()) {
+ if (p.is_valid()) {
+ if (add_existing) {
+ it->second.get(p.offset, p.length);
+ } else {
+ ref_map_candidate.get(p.offset, p.length);
+ if (!to_be_updated) {
+ to_be_updated =
+ !sb_ref_counts.test_all_zero_range(sbid, p.offset, p.length);
+ }
+ }
+ }
+ }
+ if (to_be_updated) {
+ ceph_assert(!ref_map_candidate.empty());
+ ceph_assert(!add_existing);
+ ref_maps.emplace_hint(it, sbid, std::move(ref_map_candidate));
+ }
+ dout(20) << __func__ << " inspected shared blob refs for col:" << c->cid
+ << " obj:" << oid << " sbid 0x " << std::hex << sbid << std::dec
+ << " existed " << add_existing
+ << " to be updated " << to_be_updated
+ << dendl;
+ } // if b.shared ....
+ } // for ... extent_map
+ } // for ... it->valid
+ } //if (it(PREFIX_OBJ))
+
+ // update shared blob records
+ auto ref_it = ref_maps.begin();
+ while (ref_it != ref_maps.end()) {
+ size_t cnt = 0;
+ const size_t max_transactions = 4096;
+ KeyValueDB::Transaction txn = db->get_transaction();
+ for (cnt = 0;
+ cnt < max_transactions && ref_it != ref_maps.end();
+ ref_it++) {
+ auto sbid = ref_it->first;
+ dout(20) << __func__ << " repaired shared_blob 0x"
+ << std::hex << sbid << std::dec
+ << ref_it->second << dendl;
+ repairer.fix_shared_blob(txn, sbid, &ref_it->second, 0);
+ cnt++;
+ }
+ if (cnt) {
+ db->submit_transaction_sync(txn);
+ cnt = 0;
+ }
+ }
+ // remove stray shared blob records
+ size_t cnt = 0;
+ const size_t max_transactions = 4096;
+ KeyValueDB::Transaction txn = db->get_transaction();
+ sb_info.foreach_stray([&](const sb_info_t& sbi) {
+ auto sbid = sbi.get_sbid();
+ dout(20) << __func__ << " removing stray shared_blob 0x"
+ << std::hex << sbid << std::dec
+ << dendl;
+ repairer.fix_shared_blob(txn, sbid, nullptr, 0);
+ cnt++;
+ if (cnt >= max_transactions) {}
+ db->submit_transaction_sync(txn);
+ txn = db->get_transaction();
+ cnt = 0;
+ });
+ if (cnt > 0) {
+ db->submit_transaction_sync(txn);
+ }
+
+ // amount of repairs to report to be equal to previously
+ // determined error estimation, not the actual number of updated shared blobs
+ repairer.inc_repaired(sb_ref_mismatches);
+}
+
BlueStore::OnodeRef BlueStore::fsck_check_objects_shallow(
BlueStore::FSCKDepth depth,
int64_t pool_id,
auto used_blocks = ctx.used_blocks;
auto sb_info_lock = ctx.sb_info_lock;
auto& sb_info = ctx.sb_info;
+ auto& sb_ref_counts = ctx.sb_ref_counts;
auto repairer = ctx.repairer;
store_statfs_t* res_statfs = (per_pool_stat_collection || repairer) ?
res_statfs->data_compressed_original +=
i.first->get_referenced_bytes();
}
+ if (depth != FSCK_SHALLOW && repairer) {
+ for (auto e : blob.get_extents()) {
+ if (!e.is_valid())
+ continue;
+ repairer->set_space_used(e.offset, e.length, c->cid, oid);
+ }
+ }
if (blob.is_shared()) {
if (i.first->shared_blob->get_sbid() > blobid_max) {
derr << "fsck error: " << oid << " blob " << blob
<< " sbid " << i.first->shared_blob->get_sbid() << " > blobid_max "
<< blobid_max << dendl;
++errors;
- }
- else if (i.first->shared_blob->get_sbid() == 0) {
+ } else if (i.first->shared_blob->get_sbid() == 0) {
derr << "fsck error: " << oid << " blob " << blob
<< " marked as shared but has uninitialized sbid"
<< dendl;
if (sb_info_lock) {
sb_info_lock->lock();
}
- sb_info_t& sbi = sb_info[i.first->shared_blob->get_sbid()];
- ceph_assert(sbi.cid == coll_t() || sbi.cid == c->cid);
- ceph_assert(sbi.pool_id == INT64_MIN ||
+ auto sbid = i.first->shared_blob->get_sbid();
+ sb_info_t& sbi = sb_info.add_or_adopt(i.first->shared_blob->get_sbid());
+ ceph_assert(sbi.pool_id == sb_info_t::INVALID_POOL_ID ||
sbi.pool_id == oid.hobj.get_logical_pool());
- sbi.cid = c->cid;
sbi.pool_id = oid.hobj.get_logical_pool();
- sbi.sb = i.first->shared_blob;
- sbi.oids.push_back(oid);
- sbi.compressed = blob.is_compressed();
+ bool compressed = blob.is_compressed();
for (auto e : blob.get_extents()) {
if (e.is_valid()) {
- sbi.ref_map.get(e.offset, e.length);
+ if (compressed) {
+ ceph_assert(sbi.allocated_chunks <= 0);
+ sbi.allocated_chunks -= (e.length >> min_alloc_size_order);
+ } else {
+ ceph_assert(sbi.allocated_chunks >= 0);
+ sbi.allocated_chunks += (e.length >> min_alloc_size_order);
+ }
+ sb_ref_counts.inc_range(sbid, e.offset, e.length, 1);
}
}
if (sb_info_lock) {
}
} else if (depth != FSCK_SHALLOW) {
ceph_assert(used_blocks);
- errors += _fsck_check_extents(c->cid, oid, blob.get_extents(),
+ string ctx_descr = " oid " + stringify(oid);
+ errors += _fsck_check_extents(ctx_descr,
+ blob.get_extents(),
blob.is_compressed(),
*used_blocks,
fm->get_alloc_size(),
- repairer,
+ repairer,
*res_statfs,
depth);
} else {
BlueStore* store = nullptr;
ceph::mutex* sb_info_lock = nullptr;
- BlueStore::sb_info_map_t* sb_info = nullptr;
+ sb_info_space_efficient_map_t* sb_info = nullptr;
+ shared_blob_2hash_tracker_t* sb_ref_counts = nullptr;
BlueStoreRepairer* repairer = nullptr;
Batch* batches = nullptr;
size_t _batchCount,
BlueStore* _store,
ceph::mutex* _sb_info_lock,
- BlueStore::sb_info_map_t& _sb_info,
+ sb_info_space_efficient_map_t& _sb_info,
+ shared_blob_2hash_tracker_t& _sb_ref_counts,
BlueStoreRepairer* _repairer) :
WorkQueue_(n, ceph::timespan::zero(), ceph::timespan::zero()),
batchCount(_batchCount),
store(_store),
sb_info_lock(_sb_info_lock),
sb_info(&_sb_info),
+ sb_ref_counts(&_sb_ref_counts),
repairer(_repairer)
{
batches = new Batch[batchCount];
nullptr,
sb_info_lock,
*sb_info,
+ *sb_ref_counts,
batch->expected_store_statfs,
batch->expected_pool_statfs,
repairer);
auto& errors = ctx.errors;
auto sb_info_lock = ctx.sb_info_lock;
auto& sb_info = ctx.sb_info;
+ auto& sb_ref_counts = ctx.sb_ref_counts;
auto repairer = ctx.repairer;
uint64_t_btree_t used_nids;
this,
sb_info_lock,
sb_info,
+ sb_ref_counts,
repairer));
ShallowFSCKThreadPool thread_pool(cct, "ShallowFSCKThreadPool", "ShallowFSCK", thread_count);
store_statfs_t expected_store_statfs, actual_statfs;
per_pool_statfs expected_pool_statfs;
- sb_info_map_t sb_info;
+ sb_info_space_efficient_map_t sb_info;
+ shared_blob_2hash_tracker_t sb_ref_counts(
+ cct->_conf->bluestore_fsck_shared_blob_tracker_size,
+ min_alloc_size);
+ size_t sb_ref_mismatches = 0;
/// map of oid -> (first_)offset for each zone
std::vector<std::unordered_map<ghobject_t, uint64_t>> zone_refs; // FIXME: this may be a lot of RAM!
}
#endif
+ dout(1) << __func__ << " checking shared_blobs (phase 1)" << dendl;
+ it = db->get_iterator(PREFIX_SHARED_BLOB, KeyValueDB::ITERATOR_NOCACHE);
+ if (it) {
+ for (it->lower_bound(string()); it->valid(); it->next()) {
+ string key = it->key();
+ uint64_t sbid;
+ if (get_key_shared_blob(key, &sbid)) {
+ // this gonna to be handled at the second stage
+ continue;
+ }
+ bluestore_shared_blob_t shared_blob(sbid);
+ bufferlist bl = it->value();
+ auto blp = bl.cbegin();
+ try {
+ decode(shared_blob, blp);
+ }
+ catch (ceph::buffer::error& e) {
+ // this gonna to be handled at the second stage
+ continue;
+ }
+ dout(20) << __func__ << " " << shared_blob << dendl;
+ auto& sbi = sb_info.add_maybe_stray(sbid);
+
+ // primarily to silent the 'unused' warning
+ ceph_assert(sbi.pool_id == sb_info_t::INVALID_POOL_ID);
+
+ for (auto& r : shared_blob.ref_map.ref_map) {
+ sb_ref_counts.inc_range(
+ sbid,
+ r.first,
+ r.second.length,
+ -r.second.refs);
+ }
+ }
+ } // if (it) //checking shared_blobs (phase1)
+
// walk PREFIX_OBJ
{
dout(1) << __func__ << " walking object keyspace" << dendl;
// there is no multithreading in this case
depth == FSCK_SHALLOW ? &sb_info_lock : nullptr,
sb_info,
+ sb_ref_counts,
expected_store_statfs,
expected_pool_statfs,
repair ? &repairer : nullptr);
}
#endif
- dout(1) << __func__ << " checking shared_blobs" << dendl;
+ sb_ref_mismatches = sb_ref_counts.count_non_zero();
+ if (sb_ref_mismatches != 0) {
+ derr << "fsck error: shared blob references aren't matching, at least "
+ << sb_ref_mismatches << " found" << dendl;
+ errors += sb_ref_mismatches;
+ }
+
+ if (depth != FSCK_SHALLOW && repair && sb_ref_mismatches) {
+ _fsck_repair_shared_blobs(repairer, sb_ref_counts, sb_info);
+ }
+ dout(1) << __func__ << " checking shared_blobs (phase 2)" << dendl;
it = db->get_iterator(PREFIX_SHARED_BLOB, KeyValueDB::ITERATOR_NOCACHE);
if (it) {
// FIXME minor: perhaps simplify for shallow mode?
// fill global if not overriden below
auto expected_statfs = &expected_store_statfs;
-
for (it->lower_bound(string()); it->valid(); it->next()) {
string key = it->key();
uint64_t sbid;
if (get_key_shared_blob(key, &sbid)) {
derr << "fsck error: bad key '" << key
- << "' in shared blob namespace" << dendl;
+ << "' in shared blob namespace" << dendl;
if (repair) {
repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
}
}
auto p = sb_info.find(sbid);
if (p == sb_info.end()) {
- derr << "fsck error: found stray shared blob data for sbid 0x"
- << std::hex << sbid << std::dec << dendl;
- if (repair) {
- repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
+ if (sb_ref_mismatches > 0) {
+ // highly likely this has been already reported before, ignoring...
+ dout(5) << __func__ << " found duplicate(?) stray shared blob data for sbid 0x"
+ << std::hex << sbid << std::dec << dendl;
+ } else {
+ derr<< "fsck error: found stray shared blob data for sbid 0x"
+ << std::hex << sbid << std::dec << dendl;
+ ++errors;
+ if (repair) {
+ repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
+ }
}
- ++errors;
} else {
++num_shared_blobs;
- sb_info_t& sbi = p->second;
+ sb_info_t& sbi = *p;
bluestore_shared_blob_t shared_blob(sbid);
bufferlist bl = it->value();
auto blp = bl.cbegin();
try {
- decode(shared_blob, blp);
- } catch (ceph::buffer::error& e) {
- ++errors;
- // Force update and don't report as missing
- sbi.updated = sbi.passed = true;
-
- derr << "fsck error: failed to decode Shared Blob"
- << pretty_binary_string(it->key()) << dendl;
- if (repair) {
- dout(20) << __func__ << " undecodable Shared Blob, key:'"
- << pretty_binary_string(it->key())
- << "', removing" << dendl;
- repairer.remove_key(db, PREFIX_SHARED_BLOB, it->key());
- }
- continue;
- }
- dout(20) << __func__ << " " << *sbi.sb << " " << shared_blob << dendl;
- if (shared_blob.ref_map != sbi.ref_map) {
- derr << "fsck error: shared blob 0x" << std::hex << sbid
- << std::dec << " ref_map " << shared_blob.ref_map
- << " != expected " << sbi.ref_map << dendl;
- sbi.updated = true; // will update later in repair mode only!
+ decode(shared_blob, blp);
+ }
+ catch (ceph::buffer::error& e) {
++errors;
+
+ derr << "fsck error: failed to decode Shared Blob"
+ << pretty_binary_string(key) << dendl;
+ if (repair) {
+ dout(20) << __func__ << " undecodable Shared Blob, key:'"
+ << pretty_binary_string(key)
+ << "', removing" << dendl;
+ repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
+ }
+ continue;
}
+ dout(20) << __func__ << " " << shared_blob << dendl;
PExtentVector extents;
- for (auto &r : shared_blob.ref_map.ref_map) {
+ for (auto& r : shared_blob.ref_map.ref_map) {
extents.emplace_back(bluestore_pextent_t(r.first, r.second.length));
}
- if (per_pool_stat_collection || repair) {
+ if (sbi.pool_id != sb_info_t::INVALID_POOL_ID &&
+ (per_pool_stat_collection || repair)) {
expected_statfs = &expected_pool_statfs[sbi.pool_id];
}
- errors += _fsck_check_extents(sbi.cid,
- sbi.oids.front(),
- extents,
- sbi.compressed,
- used_blocks,
- fm->get_alloc_size(),
- repair ? &repairer : nullptr,
- *expected_statfs,
- depth);
- sbi.passed = true;
+ std::stringstream ss;
+ ss << "sbid 0x" << std::hex << sbid << std::dec;
+ errors += _fsck_check_extents(ss.str(),
+ extents,
+ sbi.allocated_chunks < 0,
+ used_blocks,
+ fm->get_alloc_size(),
+ repair ? &repairer : nullptr,
+ *expected_statfs,
+ depth);
}
}
- } // if (it)
+ } // if (it) /* checking shared_blobs (phase 2)*/
if (repair && repairer.preprocess_misreference(db)) {
if (b->get_blob().is_shared()) {
b->dirty_blob().clear_flag(bluestore_blob_t::FLAG_SHARED);
- auto sb_it = sb_info.find(b->shared_blob->get_sbid());
+ auto sbid = b->shared_blob->get_sbid();
+ auto sb_it = sb_info.find(sbid);
ceph_assert(sb_it != sb_info.end());
- sb_info_t& sbi = sb_it->second;
-
- for (auto& r : sbi.ref_map.ref_map) {
- expected_statfs->allocated -= r.second.length;
- if (sbi.compressed) {
- // NB: it's crucial to use compressed flag from sb_info_t
- // as we originally used that value while accumulating
- // expected_statfs
- expected_statfs->data_compressed_allocated -= r.second.length;
- }
+ sb_info_t& sbi = *sb_it;
+
+ if (sbi.allocated_chunks < 0) {
+ // NB: it's crucial to use compressed_allocated_chunks from sb_info_t
+ // as we originally used that value while accumulating
+ // expected_statfs
+ expected_statfs->allocated -= uint64_t(-sbi.allocated_chunks) << min_alloc_size_order;
+ expected_statfs->data_compressed_allocated -=
+ uint64_t(-sbi.allocated_chunks) << min_alloc_size_order;
+ } else {
+ expected_statfs->allocated -= uint64_t(sbi.allocated_chunks) << min_alloc_size_order;
}
- sbi.updated = sbi.passed = true;
- sbi.ref_map.clear();
-
+ sbi.allocated_chunks = 0;
+ repairer.fix_shared_blob(txn, sbid, nullptr, 0);
+
// relying on blob's pextents to decide what to release.
for (auto& p : pext_to_release) {
to_release.union_insert(p.offset, p.length);
to_release.clear();
} // if (it) {
} //if (repair && repairer.preprocess_misreference()) {
-
- if (depth != FSCK_SHALLOW) {
- for (auto &p : sb_info) {
- sb_info_t& sbi = p.second;
- if (!sbi.passed) {
- derr << "fsck error: missing " << *sbi.sb << dendl;
- ++errors;
- }
- if (repair && (!sbi.passed || sbi.updated)) {
- auto sbid = p.first;
- if (sbi.ref_map.empty()) {
- ceph_assert(sbi.passed);
- dout(20) << __func__ << " " << *sbi.sb
- << " is empty, removing" << dendl;
- repairer.fix_shared_blob(db, sbid, nullptr);
- } else {
- bufferlist bl;
- bluestore_shared_blob_t persistent(sbid, std::move(sbi.ref_map));
- encode(persistent, bl);
- dout(20) << __func__ << " " << *sbi.sb
- << " is " << bl.length() << " bytes, updating"
- << dendl;
-
- repairer.fix_shared_blob(db, sbid, &bl);
- // we need to account for shared blob pextents at both
- // stats and used blocks to avoid related errors.
- PExtentVector extents;
- for (auto& r : persistent.ref_map.ref_map) {
- extents.emplace_back(bluestore_pextent_t(r.first, r.second.length));
- }
- auto* expected_statfs = &expected_pool_statfs[sbi.pool_id];
- int errors = _fsck_check_extents(sbi.cid,
- ghobject_t(), // doesn't matter
- extents,
- sbi.compressed,
- used_blocks,
- fm->get_alloc_size(),
- nullptr,
- *expected_statfs,
- depth);
- if (errors) {
- derr << __func__ << " " << errors
- << " unexpected error(s) after missed shared blob repair,"
- << " perhaps worth one more repair attempt"
- << dendl;
- }
- }
- }
- }
- }
sb_info.clear();
+ sb_ref_counts.reset();
// check global stats only if fscking (not repairing) w/o per-pool stats
if (!per_pool_stat_collection &&
db->submit_transaction_sync(txn);
};
+void BlueStore::inject_stray_shared_blob_key(uint64_t sbid)
+{
+ KeyValueDB::Transaction txn;
+ txn = db->get_transaction();
+
+ dout(5) << __func__ << " " << sbid << dendl;
+
+ string key;
+ get_shared_blob_key(sbid, &key);
+ bluestore_shared_blob_t persistent(sbid);
+ persistent.ref_map.get(0xdead0000, 0x1000);
+ bufferlist bl;
+ encode(persistent, bl);
+ dout(20) << __func__ << " sbid " << sbid
+ << " takes " << bl.length() << " bytes, updating"
+ << dendl;
+
+ txn->set(PREFIX_SHARED_BLOB, key, bl);
+ db->submit_transaction_sync(txn);
+};
+
void BlueStore::inject_leaked(uint64_t len)
{
}
bool BlueStoreRepairer::fix_shared_blob(
- KeyValueDB *db,
+ KeyValueDB::Transaction txn,
uint64_t sbid,
- const bufferlist* bl)
+ bluestore_extent_ref_map_t* ref_map,
+ size_t repaired)
{
- std::lock_guard l(lock); // possibly redundant
- KeyValueDB::Transaction txn;
- if (fix_misreferences_txn) { // reuse this txn
- txn = fix_misreferences_txn;
- } else {
- if (!fix_shared_blob_txn) {
- fix_shared_blob_txn = db->get_transaction();
- }
- txn = fix_shared_blob_txn;
- }
string key;
get_shared_blob_key(sbid, &key);
-
- ++to_repair_cnt;
- if (bl) {
- txn->set(PREFIX_SHARED_BLOB, key, *bl);
+ if (ref_map) {
+ bluestore_shared_blob_t persistent(sbid, std::move(*ref_map));
+ bufferlist bl;
+ encode(persistent, bl);
+ txn->set(PREFIX_SHARED_BLOB, key, bl);
} else {
txn->rmkey(PREFIX_SHARED_BLOB, key);
}
+ to_repair_cnt += repaired;
return true;
}
#define CEPH_OSD_BLUESTORE_BLUESTORE_TYPES_H
#include <ostream>
-#include <bitset>
#include <type_traits>
+#include <vector>
+#include <array>
#include "include/mempool.h"
#include "include/types.h"
#include "include/interval_set.h"
#include "compressor/Compressor.h"
#include "common/Checksummer.h"
#include "include/mempool.h"
+#include "include/ceph_hash.h"
namespace ceph {
class Formatter;
return !(l == r);
}
-/// blob_use_tracker: a set of per-alloc unit ref counters to track blob usage
+/// blob_use_tracker: a set of per-alloc unit ref buckets to track blob usage
struct bluestore_blob_use_tracker_t {
// N.B.: There is no need to minimize au_size/num_au
// as much as possible (e.g. have just a single byte for au_size) since:
};
WRITE_CLASS_DENC(bluestore_compression_header_t)
+template <template <typename> typename V, class COUNTER_TYPE = int32_t>
+class ref_counter_2hash_tracker_t {
+ size_t num_non_zero = 0;
+ size_t num_buckets = 0;
+ V<COUNTER_TYPE> buckets1;
+ V<COUNTER_TYPE> buckets2;
+
+public:
+ ref_counter_2hash_tracker_t(uint64_t mem_cap) {
+ num_buckets = mem_cap / sizeof(COUNTER_TYPE) / 2;
+ ceph_assert(num_buckets);
+ buckets1.resize(num_buckets);
+ buckets2.resize(num_buckets);
+ reset();
+ }
+
+ size_t get_num_buckets() const {
+ return num_buckets;
+ }
+
+ void inc(const char* hash_val, size_t hash_val_len, int n) {
+ auto h = ceph_str_hash_rjenkins((const char*)hash_val, hash_val_len) %
+ num_buckets;
+ if (buckets1[h] == 0 && n) {
+ ++num_non_zero;
+ } else if (buckets1[h] == -n) {
+ --num_non_zero;
+ }
+ buckets1[h] += n;
+ h = ceph_str_hash_linux((const char*)hash_val, hash_val_len) % num_buckets;
+ if (buckets2[h] == 0 && n) {
+ ++num_non_zero;
+ } else if (buckets2[h] == -n) {
+ --num_non_zero;
+ }
+ buckets2[h] += n;
+ }
+
+ bool test_hash_conflict(
+ const char* hash_val1,
+ const char* hash_val2,
+ size_t hash_val_len) const {
+
+ auto h1 = ceph_str_hash_rjenkins((const char*)hash_val1, hash_val_len);
+ auto h2 = ceph_str_hash_rjenkins((const char*)hash_val2, hash_val_len);
+ auto h3 = ceph_str_hash_linux((const char*)hash_val1, hash_val_len);
+ auto h4 = ceph_str_hash_linux((const char*)hash_val2, hash_val_len);
+ return ((h1 % num_buckets) == (h2 % num_buckets)) &&
+ ((h3 % num_buckets) == (h4 % num_buckets));
+ }
+
+ bool test_all_zero(const char* hash_val, size_t hash_val_len) const {
+ auto h = ceph_str_hash_rjenkins((const char*)hash_val, hash_val_len);
+ if (buckets1[h % num_buckets] != 0) {
+ return false;
+ }
+ h = ceph_str_hash_linux((const char*)hash_val, hash_val_len);
+ return buckets2[h % num_buckets] == 0;
+ }
+
+ // returns number of mismatching buckets
+ size_t count_non_zero() const {
+ return num_non_zero;
+ }
+ void reset() {
+ for (size_t i = 0; i < num_buckets; i++) {
+ buckets1[i] = 0;
+ buckets2[i] = 0;
+ }
+ num_non_zero = 0;
+ }
+};
+
+class shared_blob_2hash_tracker_t
+ : public ref_counter_2hash_tracker_t<mempool::bluestore_fsck::vector> {
+
+ static const size_t hash_input_len = 3;
+ typedef std::array<uint64_t, hash_input_len> hash_input_t;
+ inline hash_input_t build_hash_input(uint64_t sbid, uint64_t offset) const;
+ static size_t get_hash_input_size() {
+ return hash_input_len * sizeof(hash_input_t::value_type);
+ }
+
+ size_t au_void_bits = 0;
+
+public:
+ shared_blob_2hash_tracker_t(uint64_t mem_cap, size_t alloc_unit)
+ : ref_counter_2hash_tracker_t(mem_cap) {
+ ceph_assert(!!alloc_unit);
+ ceph_assert(isp2(alloc_unit));
+ au_void_bits = ctz(alloc_unit);
+ }
+ void inc(uint64_t sbid, uint64_t offset, int n);
+ void inc_range(uint64_t sbid, uint64_t offset, uint32_t len, int n);
+
+ bool test_hash_conflict(
+ uint64_t sbid,
+ uint64_t offset,
+ uint64_t sbid2,
+ uint64_t offset2) const;
+ bool test_all_zero(
+ uint64_t sbid,
+ uint64_t offset) const;
+ bool test_all_zero_range(
+ uint64_t sbid,
+ uint64_t offset,
+ uint32_t len) const;
+};
+
+class sb_info_t {
+ // subzero value indicates (potentially) stray blob,
+ // i.e. blob that has got no real references from onodes
+ int64_t sbid = 0;
+
+public:
+ enum {
+ INVALID_POOL_ID = INT64_MIN
+ };
+
+ int64_t pool_id = INVALID_POOL_ID;
+ // subzero value indicates compressed_allocated as well
+ int32_t allocated_chunks = 0;
+
+ sb_info_t(int64_t _sbid = 0) : sbid(_sbid)
+ {
+ }
+ bool operator< (const sb_info_t& other) const {
+ return std::abs(sbid) < std::abs(other.sbid);
+ }
+ bool operator< (const uint64_t& other_sbid) const {
+ return uint64_t(std::abs(sbid)) < other_sbid;
+ }
+ bool is_stray() const {
+ return sbid < 0;
+ }
+ uint64_t get_sbid() const {
+ return uint64_t(std::abs(sbid));
+ }
+ void adopt() {
+ sbid = std::abs(sbid);
+ }
+} __attribute__((packed));
+
+// Space-efficient container to keep a set of sb_info structures
+// given that the majority of entries are appended in a proper id-sorted
+// order. Hence one can keep them in a regular vector and apply binary search
+// whenever specific entry to be found.
+// For the rare occasions when out-of-order append takes place - an auxilliary
+// regular map is used.
+struct sb_info_space_efficient_map_t {
+ // large array sorted by the user
+ mempool::bluestore_fsck::vector<sb_info_t> items;
+ // small additional set of items we maintain sorting ourselves
+ // this would never keep an entry with id > items.back().id
+ mempool::bluestore_fsck::vector<sb_info_t> aux_items;
+
+ sb_info_t& add_maybe_stray(uint64_t sbid) {
+ return _add(-int64_t(sbid));
+ }
+ sb_info_t& add_or_adopt(uint64_t sbid) {
+ auto& r = _add(sbid);
+ r.adopt();
+ return r;
+ }
+ auto find(uint64_t id) {
+ if (items.size() != 0) {
+ auto it = std::lower_bound(
+ items.begin(),
+ items.end() - 1,
+ id,
+ [](const sb_info_t& a, const uint64_t& b) {
+ return a < b;
+ });
+ if (it->get_sbid() == id) {
+ return it;
+ }
+ if (aux_items.size() != 0) {
+ auto it = std::lower_bound(
+ aux_items.begin(),
+ aux_items.end() - 1,
+ id,
+ [](const sb_info_t& a, const uint64_t& b) {
+ return a < b;
+ });
+ if (it->get_sbid() == id) {
+ return it;
+ }
+ }
+ }
+ return items.end();
+ }
+ // enumerates strays, order isn't guaranteed.
+ void foreach_stray(std::function<void(const sb_info_t&)> cb) {
+ for (auto& sbi : items) {
+ if (sbi.is_stray()) {
+ cb(sbi);
+ }
+ }
+ for (auto& sbi : aux_items) {
+ if (sbi.is_stray()) {
+ cb(sbi);
+ }
+ }
+ }
+ auto end() {
+ return items.end();
+ }
+
+ void shrink() {
+ items.shrink_to_fit();
+ aux_items.shrink_to_fit();
+ }
+ void clear() {
+ items.clear();
+ aux_items.clear();
+ shrink();
+ }
+private:
+ sb_info_t& _add(int64_t id) {
+ if (items.size() == 0 || uint64_t(std::abs(id)) > items.back().get_sbid()) {
+ return items.emplace_back(id);
+ }
+ auto it = find(uint64_t(std::abs(id)));
+ if (it != items.end()) {
+ return *it;
+ }
+ if (aux_items.size() == 0 || uint64_t(std::abs(id)) > aux_items.back().get_sbid()) {
+ return aux_items.emplace_back(id);
+ }
+ // do sorted insertion, may be expensive!
+ it = std::upper_bound(
+ aux_items.begin(),
+ aux_items.end() - 1,
+ uint64_t(std::abs(id)),
+ [](const uint64_t& a, const sb_info_t& b) {
+ return a < b.get_sbid();
+ });
+ return *aux_items.emplace(it, id);
+ }
+};
#endif
#include "common/ceph_argparse.h"
#include "global/global_init.h"
#include "global/global_context.h"
+#include "perfglue/heap_profiler.h"
#include <sstream>
P(bluestore_onode_t);
P(bluestore_blob_t);
P(PExtentVector);
+ P(ghobject_t);
P(bluestore_shared_blob_t);
P(bluestore_extent_ref_map_t);
P(bluestore_extent_ref_map_t::record_t);
P(bufferlist);
P(bufferptr);
P(range_seg_t);
+ P(sb_info_t);
cout << "map<uint64_t,uint64_t>\t" << sizeof(map<uint64_t,uint64_t>) << std::endl;
cout << "map<char,char>\t" << sizeof(map<char,char>) << std::endl;
}
+void dump_mempools()
+{
+ ostringstream ostr;
+ Formatter* f = Formatter::create("json-pretty", "json-pretty", "json-pretty");
+ ostr << "Mempools: ";
+ f->open_object_section("mempools");
+ mempool::dump(f);
+ f->close_section();
+ f->flush(ostr);
+ delete f;
+ cout << ostr.str() << std::endl;
+}
+
+TEST(sb_info_space_efficient_map_t, basic) {
+ sb_info_space_efficient_map_t sb_info;
+ const size_t num_shared = 1000;
+ for (size_t i = 0; i < num_shared; i += 2) {
+ auto& sbi = sb_info.add_maybe_stray(i);
+ sbi.pool_id = i;
+ }
+ ASSERT_TRUE(sb_info.find(0) != sb_info.end());
+ ASSERT_TRUE(sb_info.find(1) == sb_info.end());
+ ASSERT_TRUE(sb_info.find(2) != sb_info.end());
+ ASSERT_TRUE(sb_info.find(4)->pool_id == 4);
+ ASSERT_TRUE(sb_info.find(num_shared) == sb_info.end());
+
+ // ordered insertion
+ sb_info.add_or_adopt(num_shared).pool_id = num_shared;
+ ASSERT_TRUE(sb_info.find(num_shared) != sb_info.end());
+ ASSERT_TRUE(sb_info.find(num_shared)->pool_id == num_shared);
+
+ // out of order insertion
+ sb_info.add_or_adopt(1).pool_id = 1;
+ ASSERT_TRUE(sb_info.find(1) != sb_info.end());
+ ASSERT_TRUE(sb_info.find(1)->pool_id == 1);
+
+ // ordered insertion
+ sb_info.add_maybe_stray(num_shared + 1).pool_id = num_shared + 1;
+ ASSERT_TRUE(sb_info.find(num_shared + 1) != sb_info.end());
+ ASSERT_TRUE(sb_info.find(num_shared + 1)->pool_id == num_shared + 1);
+
+ // out of order insertion
+ sb_info.add_maybe_stray(105).pool_id = 105;
+ ASSERT_TRUE(sb_info.find(105) != sb_info.end());
+ ASSERT_TRUE(sb_info.find(105)->pool_id == 105);
+}
+
+TEST(sb_info_space_efficient_map_t, size) {
+ const size_t num_shared = 10000000;
+ sb_info_space_efficient_map_t sb_info;
+
+ BlueStore store(g_ceph_context, "", 4096);
+ BlueStore::OnodeCacheShard* oc = BlueStore::OnodeCacheShard::create(
+ g_ceph_context, "lru", NULL);
+ BlueStore::BufferCacheShard* bc = BlueStore::BufferCacheShard::create(
+ g_ceph_context, "lru", NULL);
+
+ auto coll = ceph::make_ref<BlueStore::Collection>(&store, oc, bc, coll_t());
+
+ for (size_t i = 0; i < num_shared; i++) {
+ auto& sbi = sb_info.add_or_adopt(i);
+ // primarily to silent the 'unused' warning
+ ceph_assert(sbi.pool_id == sb_info_t::INVALID_POOL_ID);
+ }
+ dump_mempools();
+}
+
TEST(bluestore_extent_ref_map_t, add)
{
bluestore_extent_ref_map_t m;
}
}
+TEST(shared_blob_2hash_tracker_t, basic_test)
+{
+ shared_blob_2hash_tracker_t t1(1024 * 1024, 4096);
+
+ ASSERT_TRUE(t1.count_non_zero() == 0);
+
+ t1.inc(0, 0, 1);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(0, 0, -1);
+ ASSERT_TRUE(t1.count_non_zero() == 0);
+
+ t1.inc(3, 0x1000, 2);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(3, 0x1000, -1);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(3, 0x1000, -1);
+ ASSERT_TRUE(t1.count_non_zero() == 0);
+
+ t1.inc(2, 0x2000, 5);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(18, 0x2000, -5);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(18, 0x2000, 1);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(2, 0x2000, -1);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(18, 0x2000, 4);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(2, 0x2000, -4);
+ ASSERT_TRUE(t1.count_non_zero() == 0);
+
+ t1.inc(3, 0x3000, 2);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(4, 0x3000, -1);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(4, 0x3000, -1);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(3, 0x3000, -2);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(4, 0x3000, 1);
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+ t1.inc(4, 0x3000, 1);
+ ASSERT_TRUE(t1.count_non_zero() == 0);
+
+ t1.inc(5, 0x1000, 1);
+ t1.inc(5, 0x2000, 3);
+ t1.inc(5, 0x3000, 2);
+ t1.inc(5, 0x8000, 1);
+
+ ASSERT_TRUE(t1.count_non_zero() != 0);
+
+ ASSERT_TRUE(!t1.test_all_zero(5,0x1000));
+ ASSERT_TRUE(!t1.test_all_zero(5, 0x2000));
+ ASSERT_TRUE(!t1.test_all_zero(5, 0x3000));
+ ASSERT_TRUE(t1.test_all_zero(5, 0x4000));
+ ASSERT_TRUE(!t1.test_all_zero(5, 0x8000));
+
+ ASSERT_TRUE(t1.test_all_zero_range(5, 0, 0x1000));
+ ASSERT_TRUE(t1.test_all_zero_range(5, 0x500, 0x500));
+ ASSERT_TRUE(!t1.test_all_zero_range(5, 0x500, 0x1500));
+ ASSERT_TRUE(!t1.test_all_zero_range(5, 0x1500, 0x3200));
+ ASSERT_TRUE(t1.test_all_zero_range(5, 0x4500, 0x1500));
+ ASSERT_TRUE(t1.test_all_zero_range(5, 0x4500, 0x3b00));
+ ASSERT_TRUE(!t1.test_all_zero_range(5, 0, 0x9000));
+}
int main(int argc, char **argv) {
auto args = argv_to_vec(argc, argv);
auto cct = global_init(NULL, args, CEPH_ENTITY_TYPE_CLIENT,