rocksdb/table/block_based/full_filter_block.cc
Hui Xiao 920386f2b7 Detect (new) Bloom/Ribbon Filter construction corruption (#9342)
Summary:
Note: rebase on and merge after https://github.com/facebook/rocksdb/pull/9349, https://github.com/facebook/rocksdb/pull/9345, (optional) https://github.com/facebook/rocksdb/pull/9393
**Context:**
(Quoted from pdillinger) Layers of information during new Bloom/Ribbon Filter construction in building block-based tables includes the following:
a) set of keys to add to filter
b) set of hashes to add to filter (64-bit hash applied to each key)
c) set of Bloom indices to set in filter, with duplicates
d) set of Bloom indices to set in filter, deduplicated
e) final filter and its checksum

This PR aims to detect corruption (e.g, unexpected hardware/software corruption on data structures residing in the memory for a long time) from b) to e) and leave a) as future works for application level.
- b)'s corruption is detected by verifying the xor checksum of the hash entries calculated as the entries accumulate before being added to the filter. (i.e, `XXPH3FilterBitsBuilder::MaybeVerifyHashEntriesChecksum()`)
- c) - e)'s corruption is detected by verifying the hash entries indeed exists in the constructed filter by re-querying these hash entries in the filter (i.e, `FilterBitsBuilder::MaybePostVerify()`) after computing the block checksum (except for PartitionFilter, which is done right after each `FilterBitsBuilder::Finish` for impl simplicity - see code comment for more). For this stage of detection, we assume hash entries are not corrupted after checking on b) since the time interval from b) to c) is relatively short IMO.

Option to enable this feature of detection is `BlockBasedTableOptions::detect_filter_construct_corruption` which is false by default.

**Summary:**
- Implemented new functions `XXPH3FilterBitsBuilder::MaybeVerifyHashEntriesChecksum()` and `FilterBitsBuilder::MaybePostVerify()`
- Ensured hash entries, final filter and banding and their [cache reservation ](https://github.com/facebook/rocksdb/issues/9073) are released properly despite corruption
   - See [Filter.construction.artifacts.release.point.pdf ](https://github.com/facebook/rocksdb/files/7923487/Design.Filter.construction.artifacts.release.point.pdf) for high-level design
   -  Bundled and refactored hash entries's related artifact in XXPH3FilterBitsBuilder into `HashEntriesInfo` for better control on lifetime of these artifact during `SwapEntires`, `ResetEntries`
- Ensured RocksDB block-based table builder calls `FilterBitsBuilder::MaybePostVerify()` after constructing the filter by `FilterBitsBuilder::Finish()`
- When encountering such filter construction corruption, stop writing the filter content to files and mark such a block-based table building non-ok by storing the corruption status in the builder.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/9342

Test Plan:
- Added new unit test `DBFilterConstructionCorruptionTestWithParam.DetectCorruption`
- Included this new feature in `DBFilterConstructionReserveMemoryTestWithParam.ReserveMemory` as this feature heavily touch ReserveMemory's impl
   - For fallback case, I run `./filter_bench -impl=3 -detect_filter_construct_corruption=true -reserve_table_builder_memory=true -strict_capacity_limit=true  -quick -runs 10 | grep 'Build avg'` to make sure nothing break.
- Added to `filter_bench`: increased filter construction time by **30%**, mostly by `MaybePostVerify()`
   -  FastLocalBloom
       - Before change: `./filter_bench -impl=2 -quick -runs 10 | grep 'Build avg'`: **28.86643s**
       - After change:
          -  `./filter_bench -impl=2 -detect_filter_construct_corruption=false -quick -runs 10 | grep 'Build avg'` (expect a tiny increase due to MaybePostVerify is always called regardless): **27.6644s (-4% perf improvement might be due to now we don't drop bloom hash entry in `AddAllEntries` along iteration but in bulk later, same with the bypassing-MaybePostVerify case below)**
          - `./filter_bench -impl=2 -detect_filter_construct_corruption=true -quick -runs 10 | grep 'Build avg'` (expect acceptable increase): **34.41159s (+20%)**
          - `./filter_bench -impl=2 -detect_filter_construct_corruption=true -quick -runs 10 | grep 'Build avg'` (by-passing MaybePostVerify, expect minor increase): **27.13431s (-6%)**
    -  Standard128Ribbon
       - Before change: `./filter_bench -impl=3 -quick -runs 10 | grep 'Build avg'`: **122.5384s**
       - After change:
          - `./filter_bench -impl=3 -detect_filter_construct_corruption=false -quick -runs 10 | grep 'Build avg'` (expect a tiny increase due to MaybePostVerify is always called regardless - verified by removing MaybePostVerify under this case and found only +-1ns difference): **124.3588s (+2%)**
          - `./filter_bench -impl=3 -detect_filter_construct_corruption=true -quick -runs 10 | grep 'Build avg'`(expect acceptable increase): **159.4946s (+30%)**
          - `./filter_bench -impl=3 -detect_filter_construct_corruption=true -quick -runs 10 | grep 'Build avg'`(by-passing MaybePostVerify, expect minor increase) : **125.258s (+2%)**
- Added to `db_stress`: `make crash_test`, `./db_stress --detect_filter_construct_corruption=true`
- Manually smoke-tested: manually corrupted the filter construction in some db level tests with basic PUT and background flush. As expected, the error did get returned to users in subsequent PUT and Flush status.

Reviewed By: pdillinger

Differential Revision: D33746928

Pulled By: hx235

fbshipit-source-id: cb056426be5a7debc1cd16f23bc250f36a08ca57
2022-02-01 17:42:35 -08:00

366 lines
12 KiB
C++

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#include "table/block_based/full_filter_block.h"
#include <array>
#include "monitoring/perf_context_imp.h"
#include "port/malloc.h"
#include "port/port.h"
#include "rocksdb/filter_policy.h"
#include "table/block_based/block_based_table_reader.h"
#include "util/coding.h"
namespace ROCKSDB_NAMESPACE {
FullFilterBlockBuilder::FullFilterBlockBuilder(
const SliceTransform* _prefix_extractor, bool whole_key_filtering,
FilterBitsBuilder* filter_bits_builder)
: prefix_extractor_(_prefix_extractor),
whole_key_filtering_(whole_key_filtering),
last_whole_key_recorded_(false),
last_prefix_recorded_(false),
last_key_in_domain_(false),
any_added_(false) {
assert(filter_bits_builder != nullptr);
filter_bits_builder_.reset(filter_bits_builder);
}
size_t FullFilterBlockBuilder::EstimateEntriesAdded() {
return filter_bits_builder_->EstimateEntriesAdded();
}
void FullFilterBlockBuilder::Add(const Slice& key_without_ts) {
const bool add_prefix =
prefix_extractor_ && prefix_extractor_->InDomain(key_without_ts);
if (!last_prefix_recorded_ && last_key_in_domain_) {
// We can reach here when a new filter partition starts in partitioned
// filter. The last prefix in the previous partition should be added if
// necessary regardless of key_without_ts, to support prefix SeekForPrev.
AddKey(last_prefix_str_);
last_prefix_recorded_ = true;
}
if (whole_key_filtering_) {
if (!add_prefix) {
AddKey(key_without_ts);
} else {
// if both whole_key and prefix are added to bloom then we will have whole
// key_without_ts and prefix addition being interleaved and thus cannot
// rely on the bits builder to properly detect the duplicates by comparing
// with the last item.
Slice last_whole_key = Slice(last_whole_key_str_);
if (!last_whole_key_recorded_ ||
last_whole_key.compare(key_without_ts) != 0) {
AddKey(key_without_ts);
last_whole_key_recorded_ = true;
last_whole_key_str_.assign(key_without_ts.data(),
key_without_ts.size());
}
}
}
if (add_prefix) {
last_key_in_domain_ = true;
AddPrefix(key_without_ts);
} else {
last_key_in_domain_ = false;
}
}
// Add key to filter if needed
inline void FullFilterBlockBuilder::AddKey(const Slice& key) {
filter_bits_builder_->AddKey(key);
any_added_ = true;
}
// Add prefix to filter if needed
void FullFilterBlockBuilder::AddPrefix(const Slice& key) {
assert(prefix_extractor_ && prefix_extractor_->InDomain(key));
Slice prefix = prefix_extractor_->Transform(key);
if (whole_key_filtering_) {
// if both whole_key and prefix are added to bloom then we will have whole
// key and prefix addition being interleaved and thus cannot rely on the
// bits builder to properly detect the duplicates by comparing with the last
// item.
Slice last_prefix = Slice(last_prefix_str_);
if (!last_prefix_recorded_ || last_prefix.compare(prefix) != 0) {
AddKey(prefix);
last_prefix_recorded_ = true;
last_prefix_str_.assign(prefix.data(), prefix.size());
}
} else {
AddKey(prefix);
}
}
void FullFilterBlockBuilder::Reset() {
last_whole_key_recorded_ = false;
last_prefix_recorded_ = false;
}
Slice FullFilterBlockBuilder::Finish(
const BlockHandle& /*tmp*/, Status* status,
std::unique_ptr<const char[]>* filter_data) {
Reset();
// In this impl we ignore BlockHandle
*status = Status::OK();
if (any_added_) {
any_added_ = false;
Slice filter_content = filter_bits_builder_->Finish(
filter_data ? filter_data : &filter_data_, status);
return filter_content;
}
return Slice();
}
FullFilterBlockReader::FullFilterBlockReader(
const BlockBasedTable* t,
CachableEntry<ParsedFullFilterBlock>&& filter_block)
: FilterBlockReaderCommon(t, std::move(filter_block)) {
const SliceTransform* const prefix_extractor = table_prefix_extractor();
if (prefix_extractor) {
full_length_enabled_ =
prefix_extractor->FullLengthEnabled(&prefix_extractor_full_length_);
}
}
bool FullFilterBlockReader::KeyMayMatch(
const Slice& key, const SliceTransform* /*prefix_extractor*/,
uint64_t block_offset, const bool no_io,
const Slice* const /*const_ikey_ptr*/, GetContext* get_context,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)block_offset;
#endif
assert(block_offset == kNotValid);
if (!whole_key_filtering()) {
return true;
}
return MayMatch(key, no_io, get_context, lookup_context);
}
std::unique_ptr<FilterBlockReader> FullFilterBlockReader::Create(
const BlockBasedTable* table, const ReadOptions& ro,
FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
bool pin, BlockCacheLookupContext* lookup_context) {
assert(table);
assert(table->get_rep());
assert(!pin || prefetch);
CachableEntry<ParsedFullFilterBlock> filter_block;
if (prefetch || !use_cache) {
const Status s = ReadFilterBlock(table, prefetch_buffer, ro, use_cache,
nullptr /* get_context */, lookup_context,
&filter_block);
if (!s.ok()) {
IGNORE_STATUS_IF_ERROR(s);
return std::unique_ptr<FilterBlockReader>();
}
if (use_cache && !pin) {
filter_block.Reset();
}
}
return std::unique_ptr<FilterBlockReader>(
new FullFilterBlockReader(table, std::move(filter_block)));
}
bool FullFilterBlockReader::PrefixMayMatch(
const Slice& prefix, const SliceTransform* /* prefix_extractor */,
uint64_t block_offset, const bool no_io,
const Slice* const /*const_ikey_ptr*/, GetContext* get_context,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)block_offset;
#endif
assert(block_offset == kNotValid);
return MayMatch(prefix, no_io, get_context, lookup_context);
}
bool FullFilterBlockReader::MayMatch(
const Slice& entry, bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context) const {
CachableEntry<ParsedFullFilterBlock> filter_block;
const Status s =
GetOrReadFilterBlock(no_io, get_context, lookup_context, &filter_block);
if (!s.ok()) {
IGNORE_STATUS_IF_ERROR(s);
return true;
}
assert(filter_block.GetValue());
FilterBitsReader* const filter_bits_reader =
filter_block.GetValue()->filter_bits_reader();
if (filter_bits_reader) {
if (filter_bits_reader->MayMatch(entry)) {
PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
return true;
} else {
PERF_COUNTER_ADD(bloom_sst_miss_count, 1);
return false;
}
}
return true; // remain the same with block_based filter
}
void FullFilterBlockReader::KeysMayMatch(
MultiGetRange* range, const SliceTransform* /*prefix_extractor*/,
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)block_offset;
#endif
assert(block_offset == kNotValid);
if (!whole_key_filtering()) {
// Simply return. Don't skip any key - consider all keys as likely to be
// present
return;
}
MayMatch(range, no_io, nullptr, lookup_context);
}
void FullFilterBlockReader::PrefixesMayMatch(
MultiGetRange* range, const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)block_offset;
#endif
assert(block_offset == kNotValid);
MayMatch(range, no_io, prefix_extractor, lookup_context);
}
void FullFilterBlockReader::MayMatch(
MultiGetRange* range, bool no_io, const SliceTransform* prefix_extractor,
BlockCacheLookupContext* lookup_context) const {
CachableEntry<ParsedFullFilterBlock> filter_block;
const Status s = GetOrReadFilterBlock(no_io, range->begin()->get_context,
lookup_context, &filter_block);
if (!s.ok()) {
IGNORE_STATUS_IF_ERROR(s);
return;
}
assert(filter_block.GetValue());
FilterBitsReader* const filter_bits_reader =
filter_block.GetValue()->filter_bits_reader();
if (!filter_bits_reader) {
return;
}
// We need to use an array instead of autovector for may_match since
// &may_match[0] doesn't work for autovector<bool> (compiler error). So
// declare both keys and may_match as arrays, which is also slightly less
// expensive compared to autovector
std::array<Slice*, MultiGetContext::MAX_BATCH_SIZE> keys;
std::array<bool, MultiGetContext::MAX_BATCH_SIZE> may_match = {{true}};
autovector<Slice, MultiGetContext::MAX_BATCH_SIZE> prefixes;
int num_keys = 0;
MultiGetRange filter_range(*range, range->begin(), range->end());
for (auto iter = filter_range.begin(); iter != filter_range.end(); ++iter) {
if (!prefix_extractor) {
keys[num_keys++] = &iter->ukey_without_ts;
} else if (prefix_extractor->InDomain(iter->ukey_without_ts)) {
prefixes.emplace_back(prefix_extractor->Transform(iter->ukey_without_ts));
keys[num_keys++] = &prefixes.back();
} else {
filter_range.SkipKey(iter);
}
}
filter_bits_reader->MayMatch(num_keys, &keys[0], &may_match[0]);
int i = 0;
for (auto iter = filter_range.begin(); iter != filter_range.end(); ++iter) {
if (!may_match[i]) {
// Update original MultiGet range to skip this key. The filter_range
// was temporarily used just to skip keys not in prefix_extractor domain
range->SkipKey(iter);
PERF_COUNTER_ADD(bloom_sst_miss_count, 1);
} else {
// PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
PerfContext* perf_ctx = get_perf_context();
perf_ctx->bloom_sst_hit_count++;
}
++i;
}
}
size_t FullFilterBlockReader::ApproximateMemoryUsage() const {
size_t usage = ApproximateFilterBlockMemoryUsage();
#ifdef ROCKSDB_MALLOC_USABLE_SIZE
usage += malloc_usable_size(const_cast<FullFilterBlockReader*>(this));
#else
usage += sizeof(*this);
#endif // ROCKSDB_MALLOC_USABLE_SIZE
return usage;
}
bool FullFilterBlockReader::RangeMayExist(
const Slice* iterate_upper_bound, const Slice& user_key_without_ts,
const SliceTransform* prefix_extractor, const Comparator* comparator,
const Slice* const const_ikey_ptr, bool* filter_checked,
bool need_upper_bound_check, bool no_io,
BlockCacheLookupContext* lookup_context) {
if (!prefix_extractor || !prefix_extractor->InDomain(user_key_without_ts)) {
*filter_checked = false;
return true;
}
Slice prefix = prefix_extractor->Transform(user_key_without_ts);
if (need_upper_bound_check &&
!IsFilterCompatible(iterate_upper_bound, prefix, comparator)) {
*filter_checked = false;
return true;
} else {
*filter_checked = true;
return PrefixMayMatch(prefix, prefix_extractor, kNotValid, no_io,
const_ikey_ptr, /* get_context */ nullptr,
lookup_context);
}
}
bool FullFilterBlockReader::IsFilterCompatible(
const Slice* iterate_upper_bound, const Slice& prefix,
const Comparator* comparator) const {
// Try to reuse the bloom filter in the SST table if prefix_extractor in
// mutable_cf_options has changed. If range [user_key, upper_bound) all
// share the same prefix then we may still be able to use the bloom filter.
const SliceTransform* const prefix_extractor = table_prefix_extractor();
if (iterate_upper_bound != nullptr && prefix_extractor) {
if (!prefix_extractor->InDomain(*iterate_upper_bound)) {
return false;
}
Slice upper_bound_xform = prefix_extractor->Transform(*iterate_upper_bound);
// first check if user_key and upper_bound all share the same prefix
if (comparator->CompareWithoutTimestamp(prefix, false, upper_bound_xform,
false) != 0) {
// second check if user_key's prefix is the immediate predecessor of
// upper_bound and have the same length. If so, we know for sure all
// keys in the range [user_key, upper_bound) share the same prefix.
// Also need to make sure upper_bound are full length to ensure
// correctness
if (!full_length_enabled_ ||
iterate_upper_bound->size() != prefix_extractor_full_length_ ||
!comparator->IsSameLengthImmediateSuccessor(prefix,
*iterate_upper_bound)) {
return false;
}
}
return true;
} else {
return false;
}
}
} // namespace ROCKSDB_NAMESPACE