Basic MultiGet support for partitioned filters (#6757)
Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
This commit is contained in:
parent
a7f0b27b39
commit
bae6f58696
@ -25,8 +25,10 @@
|
||||
### Bug Fixes
|
||||
* Fix a bug when making options.bottommost_compression, options.compression_opts and options.bottommost_compression_opts dynamically changeable: the modified values are not written to option files or returned back to users when being queried.
|
||||
* Fix a bug where index key comparisons were unaccounted in `PerfContext::user_key_comparison_count` for lookups in files written with `format_version >= 3`.
|
||||
* Fix many bloom.filter statistics not being updated in batch MultiGet.
|
||||
|
||||
### Performance Improvements
|
||||
* Improve performance of batch MultiGet with partitioned filters, by sharing block cache lookups to applicable filter blocks.
|
||||
* Reduced memory copies when fetching and uncompressing compressed blocks from sst files.
|
||||
|
||||
## 6.9.0 (03/29/2020)
|
||||
|
@ -1030,6 +1030,204 @@ TEST_F(DBBloomFilterTest, MemtablePrefixBloomOutOfDomain) {
|
||||
ASSERT_EQ(kKey, iter->key());
|
||||
}
|
||||
|
||||
class DBBloomFilterTestVaryPrefixAndFormatVer
|
||||
: public DBTestBase,
|
||||
public testing::WithParamInterface<std::tuple<bool, uint32_t>> {
|
||||
protected:
|
||||
bool use_prefix_;
|
||||
uint32_t format_version_;
|
||||
|
||||
public:
|
||||
DBBloomFilterTestVaryPrefixAndFormatVer()
|
||||
: DBTestBase("/db_bloom_filter_tests") {}
|
||||
|
||||
~DBBloomFilterTestVaryPrefixAndFormatVer() override {}
|
||||
|
||||
void SetUp() override {
|
||||
use_prefix_ = std::get<0>(GetParam());
|
||||
format_version_ = std::get<1>(GetParam());
|
||||
}
|
||||
|
||||
static std::string UKey(uint32_t i) { return Key(static_cast<int>(i)); }
|
||||
};
|
||||
|
||||
TEST_P(DBBloomFilterTestVaryPrefixAndFormatVer, PartitionedMultiGet) {
|
||||
Options options = CurrentOptions();
|
||||
if (use_prefix_) {
|
||||
// Entire key from UKey()
|
||||
options.prefix_extractor.reset(NewCappedPrefixTransform(9));
|
||||
}
|
||||
options.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
|
||||
BlockBasedTableOptions bbto;
|
||||
bbto.filter_policy.reset(NewBloomFilterPolicy(20));
|
||||
bbto.partition_filters = true;
|
||||
bbto.index_type = BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
|
||||
bbto.whole_key_filtering = !use_prefix_;
|
||||
bbto.metadata_block_size = 128;
|
||||
options.table_factory.reset(NewBlockBasedTableFactory(bbto));
|
||||
DestroyAndReopen(options);
|
||||
ReadOptions ropts;
|
||||
|
||||
constexpr uint32_t N = 10000;
|
||||
// Add N/2 evens
|
||||
for (uint32_t i = 0; i < N; i += 2) {
|
||||
ASSERT_OK(Put(UKey(i), UKey(i)));
|
||||
}
|
||||
ASSERT_OK(Flush());
|
||||
ASSERT_EQ(TotalTableFiles(), 1);
|
||||
|
||||
constexpr uint32_t Q = 29;
|
||||
// MultiGet In
|
||||
std::array<std::string, Q> keys;
|
||||
std::array<Slice, Q> key_slices;
|
||||
std::array<ColumnFamilyHandle*, Q> column_families;
|
||||
// MultiGet Out
|
||||
std::array<Status, Q> statuses;
|
||||
std::array<PinnableSlice, Q> values;
|
||||
|
||||
TestGetAndResetTickerCount(options, BLOCK_CACHE_FILTER_HIT);
|
||||
TestGetAndResetTickerCount(options, BLOCK_CACHE_FILTER_MISS);
|
||||
TestGetAndResetTickerCount(options, BLOOM_FILTER_PREFIX_USEFUL);
|
||||
TestGetAndResetTickerCount(options, BLOOM_FILTER_USEFUL);
|
||||
TestGetAndResetTickerCount(options, BLOOM_FILTER_PREFIX_CHECKED);
|
||||
TestGetAndResetTickerCount(options, BLOOM_FILTER_FULL_POSITIVE);
|
||||
TestGetAndResetTickerCount(options, BLOOM_FILTER_FULL_TRUE_POSITIVE);
|
||||
|
||||
// Check that initial clump of keys only loads one partition filter from
|
||||
// block cache.
|
||||
// And that spread out keys load many partition filters.
|
||||
// In both cases, mix present vs. not present keys.
|
||||
for (uint32_t stride : {uint32_t{1}, (N / Q) | 1}) {
|
||||
for (uint32_t i = 0; i < Q; ++i) {
|
||||
keys[i] = UKey(i * stride);
|
||||
key_slices[i] = Slice(keys[i]);
|
||||
column_families[i] = db_->DefaultColumnFamily();
|
||||
statuses[i] = Status();
|
||||
values[i] = PinnableSlice();
|
||||
}
|
||||
|
||||
db_->MultiGet(ropts, Q, &column_families[0], &key_slices[0], &values[0],
|
||||
/*timestamps=*/nullptr, &statuses[0], true);
|
||||
|
||||
// Confirm correct status results
|
||||
uint32_t number_not_found = 0;
|
||||
for (uint32_t i = 0; i < Q; ++i) {
|
||||
if ((i * stride % 2) == 0) {
|
||||
ASSERT_OK(statuses[i]);
|
||||
} else {
|
||||
ASSERT_TRUE(statuses[i].IsNotFound());
|
||||
++number_not_found;
|
||||
}
|
||||
}
|
||||
|
||||
// Confirm correct Bloom stats (no FPs)
|
||||
uint64_t filter_useful = TestGetAndResetTickerCount(
|
||||
options,
|
||||
use_prefix_ ? BLOOM_FILTER_PREFIX_USEFUL : BLOOM_FILTER_USEFUL);
|
||||
uint64_t filter_checked =
|
||||
TestGetAndResetTickerCount(options, use_prefix_
|
||||
? BLOOM_FILTER_PREFIX_CHECKED
|
||||
: BLOOM_FILTER_FULL_POSITIVE) +
|
||||
(use_prefix_ ? 0 : filter_useful);
|
||||
EXPECT_EQ(filter_useful, number_not_found);
|
||||
EXPECT_EQ(filter_checked, Q);
|
||||
if (!use_prefix_) {
|
||||
EXPECT_EQ(
|
||||
TestGetAndResetTickerCount(options, BLOOM_FILTER_FULL_TRUE_POSITIVE),
|
||||
Q - number_not_found);
|
||||
}
|
||||
|
||||
// Confirm no duplicate loading same filter partition
|
||||
uint64_t filter_accesses =
|
||||
TestGetAndResetTickerCount(options, BLOCK_CACHE_FILTER_HIT) +
|
||||
TestGetAndResetTickerCount(options, BLOCK_CACHE_FILTER_MISS);
|
||||
if (stride == 1) {
|
||||
EXPECT_EQ(filter_accesses, 1);
|
||||
} else {
|
||||
// for large stride
|
||||
EXPECT_GE(filter_accesses, Q / 2 + 1);
|
||||
}
|
||||
}
|
||||
|
||||
// Check that a clump of keys (present and not) works when spanning
|
||||
// two partitions
|
||||
int found_spanning = 0;
|
||||
for (uint32_t start = 0; start < N / 2;) {
|
||||
for (uint32_t i = 0; i < Q; ++i) {
|
||||
keys[i] = UKey(start + i);
|
||||
key_slices[i] = Slice(keys[i]);
|
||||
column_families[i] = db_->DefaultColumnFamily();
|
||||
statuses[i] = Status();
|
||||
values[i] = PinnableSlice();
|
||||
}
|
||||
|
||||
db_->MultiGet(ropts, Q, &column_families[0], &key_slices[0], &values[0],
|
||||
/*timestamps=*/nullptr, &statuses[0], true);
|
||||
|
||||
// Confirm correct status results
|
||||
uint32_t number_not_found = 0;
|
||||
for (uint32_t i = 0; i < Q; ++i) {
|
||||
if (((start + i) % 2) == 0) {
|
||||
ASSERT_OK(statuses[i]);
|
||||
} else {
|
||||
ASSERT_TRUE(statuses[i].IsNotFound());
|
||||
++number_not_found;
|
||||
}
|
||||
}
|
||||
|
||||
// Confirm correct Bloom stats (might see some FPs)
|
||||
uint64_t filter_useful = TestGetAndResetTickerCount(
|
||||
options,
|
||||
use_prefix_ ? BLOOM_FILTER_PREFIX_USEFUL : BLOOM_FILTER_USEFUL);
|
||||
uint64_t filter_checked =
|
||||
TestGetAndResetTickerCount(options, use_prefix_
|
||||
? BLOOM_FILTER_PREFIX_CHECKED
|
||||
: BLOOM_FILTER_FULL_POSITIVE) +
|
||||
(use_prefix_ ? 0 : filter_useful);
|
||||
EXPECT_GE(filter_useful, number_not_found - 2); // possible FP
|
||||
EXPECT_EQ(filter_checked, Q);
|
||||
if (!use_prefix_) {
|
||||
EXPECT_EQ(
|
||||
TestGetAndResetTickerCount(options, BLOOM_FILTER_FULL_TRUE_POSITIVE),
|
||||
Q - number_not_found);
|
||||
}
|
||||
|
||||
// Confirm no duplicate loading of same filter partition
|
||||
uint64_t filter_accesses =
|
||||
TestGetAndResetTickerCount(options, BLOCK_CACHE_FILTER_HIT) +
|
||||
TestGetAndResetTickerCount(options, BLOCK_CACHE_FILTER_MISS);
|
||||
if (filter_accesses == 2) {
|
||||
// Spanned across partitions.
|
||||
++found_spanning;
|
||||
if (found_spanning >= 2) {
|
||||
break;
|
||||
} else {
|
||||
// Ensure that at least once we have at least one present and
|
||||
// one non-present key on both sides of partition boundary.
|
||||
start += 2;
|
||||
}
|
||||
} else {
|
||||
EXPECT_EQ(filter_accesses, 1);
|
||||
// See explanation at "start += 2"
|
||||
start += Q - 4;
|
||||
}
|
||||
}
|
||||
EXPECT_TRUE(found_spanning >= 2);
|
||||
}
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(DBBloomFilterTestVaryPrefixAndFormatVer,
|
||||
DBBloomFilterTestVaryPrefixAndFormatVer,
|
||||
::testing::Values(
|
||||
// (use_prefix, format_version)
|
||||
std::make_tuple(false, 2),
|
||||
std::make_tuple(false, 3),
|
||||
std::make_tuple(false, 4),
|
||||
std::make_tuple(false, 5),
|
||||
std::make_tuple(true, 2),
|
||||
std::make_tuple(true, 3),
|
||||
std::make_tuple(true, 4),
|
||||
std::make_tuple(true, 5)));
|
||||
|
||||
#ifndef ROCKSDB_LITE
|
||||
namespace {
|
||||
namespace BFP2 {
|
||||
|
@ -2147,14 +2147,37 @@ void BlockBasedTable::FullFilterKeysMayMatch(
|
||||
if (filter == nullptr || filter->IsBlockBased()) {
|
||||
return;
|
||||
}
|
||||
uint64_t before_keys = range->KeysLeft();
|
||||
assert(before_keys > 0); // Caller should ensure
|
||||
if (rep_->whole_key_filtering) {
|
||||
filter->KeysMayMatch(range, prefix_extractor, kNotValid, no_io,
|
||||
lookup_context);
|
||||
uint64_t after_keys = range->KeysLeft();
|
||||
if (after_keys) {
|
||||
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_FULL_POSITIVE,
|
||||
after_keys);
|
||||
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, after_keys,
|
||||
rep_->level);
|
||||
}
|
||||
uint64_t filtered_keys = before_keys - after_keys;
|
||||
if (filtered_keys) {
|
||||
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL, filtered_keys);
|
||||
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, filtered_keys,
|
||||
rep_->level);
|
||||
}
|
||||
} else if (!read_options.total_order_seek && prefix_extractor &&
|
||||
rep_->table_properties->prefix_extractor_name.compare(
|
||||
prefix_extractor->Name()) == 0) {
|
||||
filter->PrefixesMayMatch(range, prefix_extractor, kNotValid, false,
|
||||
lookup_context);
|
||||
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_PREFIX_CHECKED,
|
||||
before_keys);
|
||||
uint64_t after_keys = range->KeysLeft();
|
||||
uint64_t filtered_keys = before_keys - after_keys;
|
||||
if (filtered_keys) {
|
||||
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_PREFIX_USEFUL,
|
||||
filtered_keys);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -2342,6 +2365,12 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
|
||||
const MultiGetRange* mget_range,
|
||||
const SliceTransform* prefix_extractor,
|
||||
bool skip_filters) {
|
||||
if (mget_range->empty()) {
|
||||
// Caller should ensure non-empty (performance bug)
|
||||
assert(false);
|
||||
return; // Nothing to do
|
||||
}
|
||||
|
||||
FilterBlockReader* const filter =
|
||||
!skip_filters ? rep_->filter.get() : nullptr;
|
||||
MultiGetRange sst_file_range(*mget_range, mget_range->begin(),
|
||||
@ -2351,7 +2380,7 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
|
||||
// If full filter not useful, Then go into each block
|
||||
const bool no_io = read_options.read_tier == kBlockCacheTier;
|
||||
uint64_t tracing_mget_id = BlockCacheTraceHelper::kReservedGetId;
|
||||
if (!sst_file_range.empty() && sst_file_range.begin()->get_context) {
|
||||
if (sst_file_range.begin()->get_context) {
|
||||
tracing_mget_id = sst_file_range.begin()->get_context->get_tracing_get_id();
|
||||
}
|
||||
BlockCacheLookupContext lookup_context{
|
||||
@ -2360,7 +2389,7 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
|
||||
FullFilterKeysMayMatch(read_options, filter, &sst_file_range, no_io,
|
||||
prefix_extractor, &lookup_context);
|
||||
|
||||
if (skip_filters || !sst_file_range.empty()) {
|
||||
if (!sst_file_range.empty()) {
|
||||
IndexBlockIter iiter_on_stack;
|
||||
// if prefix_extractor found in block differs from options, disable
|
||||
// BlockPrefixIndex. Only do this check when index_type is kHashSearch.
|
||||
|
@ -192,7 +192,6 @@ void FullFilterBlockReader::KeysMayMatch(
|
||||
uint64_t block_offset, const bool no_io,
|
||||
BlockCacheLookupContext* lookup_context) {
|
||||
#ifdef NDEBUG
|
||||
(void)range;
|
||||
(void)block_offset;
|
||||
#endif
|
||||
assert(block_offset == kNotValid);
|
||||
@ -209,7 +208,6 @@ void FullFilterBlockReader::PrefixesMayMatch(
|
||||
uint64_t block_offset, const bool no_io,
|
||||
BlockCacheLookupContext* lookup_context) {
|
||||
#ifdef NDEBUG
|
||||
(void)range;
|
||||
(void)block_offset;
|
||||
#endif
|
||||
assert(block_offset == kNotValid);
|
||||
|
@ -172,13 +172,23 @@ bool PartitionedFilterBlockReader::KeyMayMatch(
|
||||
&FullFilterBlockReader::KeyMayMatch);
|
||||
}
|
||||
|
||||
void PartitionedFilterBlockReader::KeysMayMatch(
|
||||
MultiGetRange* range, const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, const bool no_io,
|
||||
BlockCacheLookupContext* lookup_context) {
|
||||
assert(block_offset == kNotValid);
|
||||
if (!whole_key_filtering()) {
|
||||
return; // Any/all may match
|
||||
}
|
||||
|
||||
MayMatch(range, prefix_extractor, block_offset, no_io, lookup_context,
|
||||
&FullFilterBlockReader::KeysMayMatch);
|
||||
}
|
||||
|
||||
bool PartitionedFilterBlockReader::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(const_ikey_ptr != nullptr);
|
||||
assert(block_offset == kNotValid);
|
||||
if (!table_prefix_extractor() && !prefix_extractor) {
|
||||
@ -190,6 +200,19 @@ bool PartitionedFilterBlockReader::PrefixMayMatch(
|
||||
&FullFilterBlockReader::PrefixMayMatch);
|
||||
}
|
||||
|
||||
void PartitionedFilterBlockReader::PrefixesMayMatch(
|
||||
MultiGetRange* range, const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, const bool no_io,
|
||||
BlockCacheLookupContext* lookup_context) {
|
||||
assert(block_offset == kNotValid);
|
||||
if (!table_prefix_extractor() && !prefix_extractor) {
|
||||
return; // Any/all may match
|
||||
}
|
||||
|
||||
MayMatch(range, prefix_extractor, block_offset, no_io, lookup_context,
|
||||
&FullFilterBlockReader::PrefixesMayMatch);
|
||||
}
|
||||
|
||||
BlockHandle PartitionedFilterBlockReader::GetFilterPartitionHandle(
|
||||
const CachableEntry<Block>& filter_block, const Slice& entry) const {
|
||||
IndexBlockIter iter;
|
||||
@ -285,6 +308,79 @@ bool PartitionedFilterBlockReader::MayMatch(
|
||||
lookup_context);
|
||||
}
|
||||
|
||||
void PartitionedFilterBlockReader::MayMatch(
|
||||
MultiGetRange* range, const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, bool no_io, BlockCacheLookupContext* lookup_context,
|
||||
FilterManyFunction filter_function) const {
|
||||
CachableEntry<Block> filter_block;
|
||||
Status s = GetOrReadFilterBlock(no_io, range->begin()->get_context,
|
||||
lookup_context, &filter_block);
|
||||
if (UNLIKELY(!s.ok())) {
|
||||
IGNORE_STATUS_IF_ERROR(s);
|
||||
return; // Any/all may match
|
||||
}
|
||||
|
||||
if (UNLIKELY(filter_block.GetValue()->size() == 0)) {
|
||||
return; // Any/all may match
|
||||
}
|
||||
|
||||
auto start_iter_same_handle = range->begin();
|
||||
BlockHandle prev_filter_handle = BlockHandle::NullBlockHandle();
|
||||
|
||||
// For all keys mapping to same partition (must be adjacent in sorted order)
|
||||
// share block cache lookup and use full filter multiget on the partition
|
||||
// filter.
|
||||
for (auto iter = start_iter_same_handle; iter != range->end(); ++iter) {
|
||||
// TODO: re-use one top-level index iterator
|
||||
BlockHandle this_filter_handle =
|
||||
GetFilterPartitionHandle(filter_block, iter->ikey);
|
||||
if (!prev_filter_handle.IsNull() &&
|
||||
this_filter_handle != prev_filter_handle) {
|
||||
MultiGetRange subrange(*range, start_iter_same_handle, iter);
|
||||
MayMatchPartition(&subrange, prefix_extractor, block_offset,
|
||||
prev_filter_handle, no_io, lookup_context,
|
||||
filter_function);
|
||||
range->AddSkipsFrom(subrange);
|
||||
start_iter_same_handle = iter;
|
||||
}
|
||||
if (UNLIKELY(this_filter_handle.size() == 0)) { // key is out of range
|
||||
// Not reachable with current behavior of GetFilterPartitionHandle
|
||||
assert(false);
|
||||
range->SkipKey(iter);
|
||||
prev_filter_handle = BlockHandle::NullBlockHandle();
|
||||
} else {
|
||||
prev_filter_handle = this_filter_handle;
|
||||
}
|
||||
}
|
||||
if (!prev_filter_handle.IsNull()) {
|
||||
MultiGetRange subrange(*range, start_iter_same_handle, range->end());
|
||||
MayMatchPartition(&subrange, prefix_extractor, block_offset,
|
||||
prev_filter_handle, no_io, lookup_context,
|
||||
filter_function);
|
||||
range->AddSkipsFrom(subrange);
|
||||
}
|
||||
}
|
||||
|
||||
void PartitionedFilterBlockReader::MayMatchPartition(
|
||||
MultiGetRange* range, const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, BlockHandle filter_handle, bool no_io,
|
||||
BlockCacheLookupContext* lookup_context,
|
||||
FilterManyFunction filter_function) const {
|
||||
CachableEntry<ParsedFullFilterBlock> filter_partition_block;
|
||||
Status s = GetFilterPartitionBlock(
|
||||
nullptr /* prefetch_buffer */, filter_handle, no_io,
|
||||
range->begin()->get_context, lookup_context, &filter_partition_block);
|
||||
if (UNLIKELY(!s.ok())) {
|
||||
IGNORE_STATUS_IF_ERROR(s);
|
||||
return; // Any/all may match
|
||||
}
|
||||
|
||||
FullFilterBlockReader filter_partition(table(),
|
||||
std::move(filter_partition_block));
|
||||
(filter_partition.*filter_function)(range, prefix_extractor, block_offset,
|
||||
no_io, lookup_context);
|
||||
}
|
||||
|
||||
size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const {
|
||||
size_t usage = ApproximateFilterBlockMemoryUsage();
|
||||
#ifdef ROCKSDB_MALLOC_USABLE_SIZE
|
||||
|
@ -80,12 +80,21 @@ class PartitionedFilterBlockReader : public FilterBlockReaderCommon<Block> {
|
||||
uint64_t block_offset, const bool no_io,
|
||||
const Slice* const const_ikey_ptr, GetContext* get_context,
|
||||
BlockCacheLookupContext* lookup_context) override;
|
||||
void KeysMayMatch(MultiGetRange* range,
|
||||
const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, const bool no_io,
|
||||
BlockCacheLookupContext* lookup_context) override;
|
||||
|
||||
bool 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) override;
|
||||
void PrefixesMayMatch(MultiGetRange* range,
|
||||
const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, const bool no_io,
|
||||
BlockCacheLookupContext* lookup_context) override;
|
||||
|
||||
size_t ApproximateMemoryUsage() const override;
|
||||
|
||||
@ -108,6 +117,19 @@ class PartitionedFilterBlockReader : public FilterBlockReaderCommon<Block> {
|
||||
GetContext* get_context,
|
||||
BlockCacheLookupContext* lookup_context,
|
||||
FilterFunction filter_function) const;
|
||||
using FilterManyFunction = void (FullFilterBlockReader::*)(
|
||||
MultiGetRange* range, const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, const bool no_io,
|
||||
BlockCacheLookupContext* lookup_context);
|
||||
void MayMatch(MultiGetRange* range, const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, bool no_io,
|
||||
BlockCacheLookupContext* lookup_context,
|
||||
FilterManyFunction filter_function) const;
|
||||
void MayMatchPartition(MultiGetRange* range,
|
||||
const SliceTransform* prefix_extractor,
|
||||
uint64_t block_offset, BlockHandle filter_handle,
|
||||
bool no_io, BlockCacheLookupContext* lookup_context,
|
||||
FilterManyFunction filter_function) const;
|
||||
void CacheDependencies(bool pin) override;
|
||||
|
||||
const InternalKeyComparator* internal_comparator() const;
|
||||
|
@ -38,6 +38,8 @@ const int kMagicNumberLengthByte = 8;
|
||||
// block or a meta block.
|
||||
class BlockHandle {
|
||||
public:
|
||||
// Creates a block handle with special values indicating "uninitialized,"
|
||||
// distinct from the "null" block handle.
|
||||
BlockHandle();
|
||||
BlockHandle(uint64_t offset, uint64_t size);
|
||||
|
||||
@ -65,6 +67,13 @@ class BlockHandle {
|
||||
// Maximum encoding length of a BlockHandle
|
||||
enum { kMaxEncodedLength = 10 + 10 };
|
||||
|
||||
inline bool operator==(const BlockHandle& rhs) const {
|
||||
return offset_ == rhs.offset_ && size_ == rhs.size_;
|
||||
}
|
||||
inline bool operator!=(const BlockHandle& rhs) const {
|
||||
return !(*this == rhs);
|
||||
}
|
||||
|
||||
private:
|
||||
uint64_t offset_;
|
||||
uint64_t size_;
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include "rocksdb/statistics.h"
|
||||
#include "rocksdb/types.h"
|
||||
#include "util/autovector.h"
|
||||
#include "util/math.h"
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
class GetContext;
|
||||
@ -157,7 +158,7 @@ class MultiGetContext {
|
||||
Iterator(const Range* range, size_t idx)
|
||||
: range_(range), ctx_(range->ctx_), index_(idx) {
|
||||
while (index_ < range_->end_ &&
|
||||
(1ull << index_) &
|
||||
(uint64_t{1} << index_) &
|
||||
(range_->ctx_->value_mask_ | range_->skip_mask_))
|
||||
index_++;
|
||||
}
|
||||
@ -167,7 +168,7 @@ class MultiGetContext {
|
||||
|
||||
Iterator& operator++() {
|
||||
while (++index_ < range_->end_ &&
|
||||
(1ull << index_) &
|
||||
(uint64_t{1} << index_) &
|
||||
(range_->ctx_->value_mask_ | range_->skip_mask_))
|
||||
;
|
||||
return *this;
|
||||
@ -209,6 +210,8 @@ class MultiGetContext {
|
||||
start_ = first.index_;
|
||||
end_ = last.index_;
|
||||
skip_mask_ = mget_range.skip_mask_;
|
||||
assert(start_ < 64);
|
||||
assert(end_ < 64);
|
||||
}
|
||||
|
||||
Range() = default;
|
||||
@ -217,31 +220,27 @@ class MultiGetContext {
|
||||
|
||||
Iterator end() const { return Iterator(this, end_); }
|
||||
|
||||
bool empty() {
|
||||
return (((1ull << end_) - 1) & ~((1ull << start_) - 1) &
|
||||
~(ctx_->value_mask_ | skip_mask_)) == 0;
|
||||
}
|
||||
bool empty() const { return RemainingMask() == 0; }
|
||||
|
||||
void SkipKey(const Iterator& iter) { skip_mask_ |= 1ull << iter.index_; }
|
||||
void SkipKey(const Iterator& iter) {
|
||||
skip_mask_ |= uint64_t{1} << iter.index_;
|
||||
}
|
||||
|
||||
// Update the value_mask_ in MultiGetContext so its
|
||||
// immediately reflected in all the Range Iterators
|
||||
void MarkKeyDone(Iterator& iter) {
|
||||
ctx_->value_mask_ |= (1ull << iter.index_);
|
||||
ctx_->value_mask_ |= (uint64_t{1} << iter.index_);
|
||||
}
|
||||
|
||||
bool CheckKeyDone(Iterator& iter) {
|
||||
return ctx_->value_mask_ & (1ull << iter.index_);
|
||||
bool CheckKeyDone(Iterator& iter) const {
|
||||
return ctx_->value_mask_ & (uint64_t{1} << iter.index_);
|
||||
}
|
||||
|
||||
uint64_t KeysLeft() {
|
||||
uint64_t new_val = skip_mask_ | ctx_->value_mask_;
|
||||
uint64_t count = 0;
|
||||
while (new_val) {
|
||||
new_val = new_val & (new_val - 1);
|
||||
count++;
|
||||
}
|
||||
return end_ - count;
|
||||
uint64_t KeysLeft() const { return BitsSetToOne(RemainingMask()); }
|
||||
|
||||
void AddSkipsFrom(const Range& other) {
|
||||
assert(ctx_ == other.ctx_);
|
||||
skip_mask_ |= other.skip_mask_;
|
||||
}
|
||||
|
||||
private:
|
||||
@ -252,7 +251,14 @@ class MultiGetContext {
|
||||
uint64_t skip_mask_;
|
||||
|
||||
Range(MultiGetContext* ctx, size_t num_keys)
|
||||
: ctx_(ctx), start_(0), end_(num_keys), skip_mask_(0) {}
|
||||
: ctx_(ctx), start_(0), end_(num_keys), skip_mask_(0) {
|
||||
assert(num_keys < 64);
|
||||
}
|
||||
|
||||
uint64_t RemainingMask() const {
|
||||
return (((uint64_t{1} << end_) - 1) & ~((uint64_t{1} << start_) - 1) &
|
||||
~(ctx_->value_mask_ | skip_mask_));
|
||||
}
|
||||
};
|
||||
|
||||
// Return the initial range that encompasses all the keys in the batch
|
||||
|
38
util/math.h
Normal file
38
util/math.h
Normal file
@ -0,0 +1,38 @@
|
||||
// Copyright (c) Facebook, Inc. and its affiliates. 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).
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <assert.h>
|
||||
#include <stdint.h>
|
||||
#ifdef _MSC_VER
|
||||
#include <intrin.h>
|
||||
#endif
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
|
||||
template <typename T>
|
||||
inline int BitsSetToOne(T v) {
|
||||
static_assert(std::is_integral<T>::value, "non-integral type");
|
||||
#ifdef _MSC_VER
|
||||
static_assert(sizeof(T) <= sizeof(uint64_t), "type too big");
|
||||
if (sizeof(T) > sizeof(uint32_t)) {
|
||||
return static_cast<int>(__popcnt64(static_cast<uint64_t>(v)));
|
||||
} else {
|
||||
return static_cast<int>(__popcnt(static_cast<uint32_t>(v)));
|
||||
}
|
||||
#else
|
||||
static_assert(sizeof(T) <= sizeof(unsigned long long), "type too big");
|
||||
if (sizeof(T) > sizeof(unsigned long)) {
|
||||
return __builtin_popcountll(static_cast<unsigned long long>(v));
|
||||
} else if (sizeof(T) > sizeof(unsigned int)) {
|
||||
return __builtin_popcountl(static_cast<unsigned long>(v));
|
||||
} else {
|
||||
return __builtin_popcount(static_cast<unsigned int>(v));
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
Loading…
Reference in New Issue
Block a user