Option to decouple index and filter partitions (#12939)

Summary:
Partitioned metadata blocks were introduced back in 2017 to deal more gracefully with large DBs where RAM is relatively scarce and some data might be much colder than other data. The feature allows metadata blocks to compete for memory in the block cache against data blocks while alleviating tail latencies and thrash conditions that can arise with large metadata blocks (sometimes megabytes each) that can arise with large SST files. In general, the cost to partitioned metadata is more CPU in accesses (especially for filters where more binary search is needed before hashing can be used) and a bit more memory fragmentation and related overheads.

However the feature has always had a subtle limitation with a subtle effect on performance: index partitions and filter partitions must be cut at the same time, regardless of which wins the space race (hahaha) to metadata_block_size. Commonly filters will be a few times larger than indexes, so index partitions will be under-sized compared to filter (and data) blocks. While this does affect fragmentation and related overheads a bit, I suspect the bigger impact on performance is in the block cache. The coupling of the partition cuts would be defensible if the binary search done to find the filter block was used (on filter hit) to short-circuit binary search to an index partition, but that optimization has not been developed.

Consider two metadata blocks, an under-sized one and a normal-sized one, covering proportional sections of the key space with the same density of read queries. The under-sized one will be more prone to eviction from block cache because it is used less often. This is unfair because of its despite its proportionally smaller cost of keeping in block cache, and most of the cost of a miss to re-load it (random IO) is not proportional to the size (similar latency etc. up to ~32KB).

 ## This change

Adds a new table option decouple_partitioned_filters allows filter blocks and index blocks to be cut independently. To make this work, the partitioned filter block builder needs to know about the previous key, to generate an appropriate separator for the partition index. In most cases, BlockBasedTableBuilder already has easy access to the previous key to provide to the filter block builder.

This change includes refactoring to pass that previous key to the filter builder when available, with the filter building caching the previous key itself when unavailable, such as during compression dictionary training and some unit tests. Access to the previous key eliminates the need to track the previous prefix, which results in a small SST construction CPU win in prefix filtering cases, regardless of coupling, and possibly a small regression for some non-prefix cases, regardless of coupling, but still overall improvement especially with https://github.com/facebook/rocksdb/issues/12931.

Suggested follow-up:
* Update confusing use of "last key" to refer to "previous key"
* Expand unit test coverage with parallel compression and dictionary training
* Consider an option or enhancement to alleviate under-sized metadata blocks "at the end" of an SST file due to no coordination or awareness of when files are cut.

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

Test Plan:
unit tests updated. Also did some unit test runs with "hard wired" usage of parallel compression and dictionary training code paths to ensure they were working. Also ran blackbox_crash_test for a while with the new feature.

 ## SST write performance (CPU)

Using the same testing setup as in https://github.com/facebook/rocksdb/issues/12931 but with -decouple_partitioned_filters=1 in the "after" configuration, which benchmarking shows makes almost no difference in terms of SST write CPU. "After" vs. "before" this PR
```
-partition_index_and_filters=0 -prefix_size=0 -whole_key_filtering=1
923691 vs. 924851 (-0.13%)
-partition_index_and_filters=0 -prefix_size=8 -whole_key_filtering=0
921398 vs. 922973 (-0.17%)
-partition_index_and_filters=0 -prefix_size=8 -whole_key_filtering=1
902259 vs. 908756 (-0.71%)
-partition_index_and_filters=1 -prefix_size=8 -whole_key_filtering=0
917932 vs. 916901 (+0.60%)
-partition_index_and_filters=1 -prefix_size=8 -whole_key_filtering=0
912755 vs. 907298 (+0.60%)
-partition_index_and_filters=1 -prefix_size=8 -whole_key_filtering=1
899754 vs. 892433 (+0.82%)
```
I think this is a pretty good trade, especially in attracting more movement toward partitioned configurations.

 ## Read performance

Let's see how decoupling affects read performance across various degrees of memory constraint. To simplify LSM structure, we're using FIFO compaction. Since decoupling will overall increase metadata block size, we control for this somewhat with an extra "before" configuration with larger metadata block size setting (8k instead of 4k). Basic setup:

```
(for CS in 0300 1200; do TEST_TMPDIR=/dev/shm/rocksdb1 ./db_bench -benchmarks=fillrandom,flush,readrandom,block_cache_entry_stats -num=5000000 -duration=30 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=10 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters=1 -statistics=1 -cache_size=${CS}000000 -metadata_block_size=4096 -decouple_partitioned_filters=1 2>&1 | tee results-$CS; done)
```

And read ops/s results:

```CSV
Cache size MB,After/decoupled/4k,Before/4k,Before/8k
3,15593,15158,12826
6,16295,16693,14134
10,20427,20813,18459
20,27035,26836,27384
30,33250,31810,33846
60,35518,32585,35329
100,36612,31805,35292
300,35780,31492,35481
1000,34145,31551,35411
1100,35219,31380,34302
1200,35060,31037,34322
```

If you graph this with log scale on the X axis (internal link: https://pxl.cl/5qKRc), you see that the decoupled/4k configuration is essentially the best of both the before/4k and before/8k configurations: handles really tight memory closer to the old 4k configuration and handles generous memory closer to the old 8k configuration.

Reviewed By: jowlyzhang

Differential Revision: D61376772

Pulled By: pdillinger

fbshipit-source-id: fc2af2aee44290e2d9620f79651a30640799e01f
This commit is contained in:
Peter Dillinger 2024-08-16 15:34:31 -07:00 committed by Facebook GitHub Bot
parent 75a1230ce8
commit 4d3518951a
20 changed files with 275 additions and 60 deletions

View File

@ -48,6 +48,7 @@ const std::string kAutoRibbon = RibbonFilterPolicy::kClassName();
enum class FilterPartitioning {
kUnpartitionedFilter,
kCoupledPartitionedFilter,
kDecoupledPartitionedFilter,
};
template <typename T>
@ -597,7 +598,8 @@ INSTANTIATE_TEST_CASE_P(
DBBloomFilterTestWithPartitioningParam,
DBBloomFilterTestWithPartitioningParam,
::testing::Values(FilterPartitioning::kUnpartitionedFilter,
FilterPartitioning::kCoupledPartitionedFilter));
FilterPartitioning::kCoupledPartitionedFilter,
FilterPartitioning::kDecoupledPartitionedFilter));
TEST_P(DBBloomFilterTestWithFormatParams, BloomFilter) {
do {
@ -607,6 +609,10 @@ TEST_P(DBBloomFilterTestWithFormatParams, BloomFilter) {
// ChangeCompactOptions() only changes compaction style, which does not
// trigger reset of table_factory
BlockBasedTableOptions table_options;
// When partitioned filters are coupled to index blocks, they tend to get
// extra fractional bits per key when rounding up to the next cache line
// size. Here we correct for that to get similar effective bits per key.
bits_per_key_ = table_options.decouple_partitioned_filters ? 10.5 : 10;
SetInTableOptions(&table_options);
table_options.no_block_cache = true;
table_options.optimize_filters_for_memory = false;
@ -614,7 +620,12 @@ TEST_P(DBBloomFilterTestWithFormatParams, BloomFilter) {
// value delta encoding challenged more with index interval > 1
table_options.index_block_restart_interval = 8;
}
table_options.metadata_block_size = 32;
// This test is rather sensitive to the actual filter partition block size,
// and keeping that consistent between coupled and uncoupled requires a
// different metadata block size for this example (where it controls index
// block size).
table_options.metadata_block_size =
table_options.decouple_partitioned_filters ? 320 : 32;
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
CreateAndReopenWithCF({"pikachu"}, options);
@ -876,6 +887,9 @@ INSTANTIATE_TEST_CASE_P(
std::make_tuple(kAutoBloom,
FilterPartitioning::kCoupledPartitionedFilter,
test::kDefaultFormatVersion),
std::make_tuple(kAutoBloom,
FilterPartitioning::kDecoupledPartitionedFilter,
test::kDefaultFormatVersion),
std::make_tuple(kAutoBloom, FilterPartitioning::kUnpartitionedFilter,
test::kDefaultFormatVersion),
std::make_tuple(kAutoRibbon, FilterPartitioning::kUnpartitionedFilter,
@ -887,6 +901,9 @@ INSTANTIATE_TEST_CASE_P(
std::make_tuple(kAutoBloom,
FilterPartitioning::kCoupledPartitionedFilter,
test::kDefaultFormatVersion),
std::make_tuple(kAutoBloom,
FilterPartitioning::kDecoupledPartitionedFilter,
test::kDefaultFormatVersion),
std::make_tuple(kAutoBloom, FilterPartitioning::kUnpartitionedFilter,
test::kDefaultFormatVersion),
std::make_tuple(kAutoRibbon, FilterPartitioning::kUnpartitionedFilter,
@ -898,6 +915,9 @@ INSTANTIATE_TEST_CASE_P(
std::make_tuple(kAutoBloom,
FilterPartitioning::kCoupledPartitionedFilter,
kLatestFormatVersion),
std::make_tuple(kAutoBloom,
FilterPartitioning::kDecoupledPartitionedFilter,
kLatestFormatVersion),
std::make_tuple(kAutoBloom, FilterPartitioning::kUnpartitionedFilter,
kLatestFormatVersion),
std::make_tuple(kAutoRibbon, FilterPartitioning::kUnpartitionedFilter,
@ -1149,6 +1169,9 @@ INSTANTIATE_TEST_CASE_P(
std::make_tuple(CacheEntryRoleOptions::Decision::kEnabled,
kFastLocalBloom,
FilterPartitioning::kCoupledPartitionedFilter, true),
std::make_tuple(CacheEntryRoleOptions::Decision::kEnabled,
kFastLocalBloom,
FilterPartitioning::kDecoupledPartitionedFilter, true),
std::make_tuple(CacheEntryRoleOptions::Decision::kEnabled,
kStandard128Ribbon,
@ -1162,6 +1185,9 @@ INSTANTIATE_TEST_CASE_P(
std::make_tuple(CacheEntryRoleOptions::Decision::kEnabled,
kStandard128Ribbon,
FilterPartitioning::kCoupledPartitionedFilter, true),
std::make_tuple(CacheEntryRoleOptions::Decision::kEnabled,
kStandard128Ribbon,
FilterPartitioning::kDecoupledPartitionedFilter, true),
std::make_tuple(CacheEntryRoleOptions::Decision::kEnabled, kLegacyBloom,
FilterPartitioning::kUnpartitionedFilter, false)));
@ -1540,10 +1566,14 @@ INSTANTIATE_TEST_CASE_P(
FilterPartitioning::kUnpartitionedFilter),
std::make_tuple(true, kFastLocalBloom,
FilterPartitioning::kCoupledPartitionedFilter),
std::make_tuple(true, kFastLocalBloom,
FilterPartitioning::kDecoupledPartitionedFilter),
std::make_tuple(true, kStandard128Ribbon,
FilterPartitioning::kUnpartitionedFilter),
std::make_tuple(true, kStandard128Ribbon,
FilterPartitioning::kCoupledPartitionedFilter)));
FilterPartitioning::kCoupledPartitionedFilter),
std::make_tuple(true, kStandard128Ribbon,
FilterPartitioning::kDecoupledPartitionedFilter)));
TEST_P(DBFilterConstructionCorruptionTestWithParam, DetectCorruption) {
Options options = CurrentOptions();
@ -2531,10 +2561,14 @@ INSTANTIATE_TEST_CASE_P(
std::make_tuple(kLegacyBloom, FilterPartitioning::kUnpartitionedFilter),
std::make_tuple(kLegacyBloom,
FilterPartitioning::kCoupledPartitionedFilter),
std::make_tuple(kLegacyBloom,
FilterPartitioning::kDecoupledPartitionedFilter),
std::make_tuple(kFastLocalBloom,
FilterPartitioning::kUnpartitionedFilter),
std::make_tuple(kFastLocalBloom,
FilterPartitioning::kCoupledPartitionedFilter),
std::make_tuple(kFastLocalBloom,
FilterPartitioning::kDecoupledPartitionedFilter),
std::make_tuple(kPlainTable,
FilterPartitioning::kUnpartitionedFilter)));

View File

@ -366,6 +366,11 @@ Options DBTestBase::GetOptions(
table_options.block_cache = NewLRUCache(/* too small */ 1);
}
// Test anticipated new default as much as reasonably possible (and remove
// this code when obsolete)
assert(!table_options.decouple_partitioned_filters);
table_options.decouple_partitioned_filters = true;
bool can_allow_mmap = IsMemoryMappedAccessSupported();
switch (option_config) {
case kHashSkipList:

View File

@ -213,7 +213,6 @@ TEST_P(DBTestUniversalCompaction, UniversalCompactionTrigger) {
options.num_levels = num_levels_;
options.write_buffer_size = 105 << 10; // 105KB
options.arena_block_size = 4 << 10;
options.target_file_size_base = 32 << 10; // 32KB
// trigger compaction if there are >= 4 files
options.level0_file_num_compaction_trigger = 4;
KeepFilterFactory* filter = new KeepFilterFactory(true);

View File

@ -330,17 +330,16 @@ inline Slice ExtractUserKey(const Slice& internal_key) {
// output : <user_provided_key>
inline Slice ExtractUserKeyAndStripTimestamp(const Slice& internal_key,
size_t ts_sz) {
Slice ret = internal_key;
ret.remove_suffix(kNumInternalBytes + ts_sz);
return ret;
assert(internal_key.size() >= kNumInternalBytes + ts_sz);
return Slice(internal_key.data(),
internal_key.size() - (kNumInternalBytes + ts_sz));
}
// input [user key]: <user_provided_key | ts>
// output: <user_provided_key>
inline Slice StripTimestampFromUserKey(const Slice& user_key, size_t ts_sz) {
Slice ret = user_key;
ret.remove_suffix(ts_sz);
return ret;
assert(user_key.size() >= ts_sz);
return Slice(user_key.data(), user_key.size() - ts_sz);
}
// input [user key]: <user_provided_key | ts>

View File

@ -150,6 +150,7 @@ DECLARE_bool(charge_filter_construction);
DECLARE_bool(charge_table_reader);
DECLARE_bool(charge_file_metadata);
DECLARE_bool(charge_blob_cache);
DECLARE_bool(decouple_partitioned_filters);
DECLARE_int32(top_level_index_pinning);
DECLARE_int32(partition_pinning);
DECLARE_int32(unpartitioned_pinning);

View File

@ -380,6 +380,11 @@ DEFINE_bool(charge_blob_cache, false,
"CacheEntryRoleOptions::charged of "
"kBlobCache");
DEFINE_bool(
decouple_partitioned_filters,
ROCKSDB_NAMESPACE::BlockBasedTableOptions().decouple_partitioned_filters,
"Decouple filter partitioning from index partitioning.");
DEFINE_int32(
top_level_index_pinning,
static_cast<int32_t>(ROCKSDB_NAMESPACE::PinningTier::kFallback),

View File

@ -3863,6 +3863,8 @@ void InitializeOptionsFromFlags(
const std::shared_ptr<const FilterPolicy>& filter_policy,
Options& options) {
BlockBasedTableOptions block_based_options;
block_based_options.decouple_partitioned_filters =
FLAGS_decouple_partitioned_filters;
block_based_options.block_cache = cache;
block_based_options.cache_index_and_filter_blocks =
FLAGS_cache_index_and_filter_blocks;

View File

@ -291,15 +291,11 @@ struct BlockBasedTableOptions {
// Same as block_restart_interval but used for the index block.
int index_block_restart_interval = 1;
// Block size for partitioned metadata. Currently applied to indexes when
// kTwoLevelIndexSearch is used and to filters when partition_filters is used.
// Note: Since in the current implementation the filters and index partitions
// are aligned, an index/filter block is created when either index or filter
// block size reaches the specified limit.
// Note: this limit is currently applied to only index blocks; a filter
// partition is cut right after an index block is cut
// TODO(myabandeh): remove the note above when filter partitions are cut
// separately
// Target block size for partitioned metadata. Currently applied to indexes
// when kTwoLevelIndexSearch is used and to filters when partition_filters is
// used. When decouple_partitioned_filters=false (original behavior), there is
// much more deviation from this target size. See the comment on
// decouple_partitioned_filters.
uint64_t metadata_block_size = 4096;
// `cache_usage_options` allows users to specify the default
@ -398,6 +394,23 @@ struct BlockBasedTableOptions {
// block cache even when cache_index_and_filter_blocks=false.
bool partition_filters = false;
// When both partitioned indexes and partitioned filters are enabled,
// this enables independent partitioning boundaries between the two. Most
// notably, this enables these metadata blocks to hit their target size much
// more accurately, as there is often a disparity between index sizes and
// filter sizes. This should reduce fragmentation and metadata overheads in
// the block cache, as well as treat blocks more fairly for cache eviction
// purposes.
//
// There are no SST format compatibility issues with this option. (All
// versions of RocksDB able to read partitioned filters are able to read
// decoupled partitioned filters.)
//
// decouple_partitioned_filters = false is the original behavior, because of
// limitations in the initial implementation, and the new behavior
// decouple_partitioned_filters = true is expected to become the new default.
bool decouple_partitioned_filters = false;
// Option to generate Bloom/Ribbon filters that minimize memory
// internal fragmentation.
//
@ -679,6 +692,11 @@ struct BlockBasedTablePropertyNames {
static const std::string kWholeKeyFiltering;
// value is "1" for true and "0" for false.
static const std::string kPrefixFiltering;
// Set to "1" when partitioned filters are decoupled from partitioned indexes.
// This metadata is recorded in case a read-time optimization for coupled
// filter+index partitioning is ever developed; that optimization/assumption
// would be disabled when this is set.
static const std::string kDecoupledPartitionedFilters;
};
// Create default block based table factory.

View File

@ -188,6 +188,7 @@ TEST_F(OptionsSettableTest, BlockBasedTableOptionsAllFieldsSettable) {
"block_size_deviation=8;block_restart_interval=4; "
"metadata_block_size=1024;"
"partition_filters=false;"
"decouple_partitioned_filters=true;"
"optimize_filters_for_memory=true;"
"use_delta_encoding=true;"
"index_block_restart_interval=4;"

View File

@ -96,7 +96,8 @@ FilterBlockBuilder* CreateFilterBlockBuilder(
mopt.prefix_extractor.get(), table_opt.whole_key_filtering,
filter_bits_builder, table_opt.index_block_restart_interval,
use_delta_encoding_for_index_values, p_index_builder, partition_size,
ts_sz, persist_user_defined_timestamps);
ts_sz, persist_user_defined_timestamps,
table_opt.decouple_partitioned_filters);
} else {
return new FullFilterBlockBuilder(mopt.prefix_extractor.get(),
table_opt.whole_key_filtering,
@ -213,10 +214,11 @@ class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector
public:
explicit BlockBasedTablePropertiesCollector(
BlockBasedTableOptions::IndexType index_type, bool whole_key_filtering,
bool prefix_filtering)
bool prefix_filtering, bool decoupled_partitioned_filters)
: index_type_(index_type),
whole_key_filtering_(whole_key_filtering),
prefix_filtering_(prefix_filtering) {}
prefix_filtering_(prefix_filtering),
decoupled_partitioned_filters_(decoupled_partitioned_filters) {}
Status InternalAdd(const Slice& /*key*/, const Slice& /*value*/,
uint64_t /*file_size*/) override {
@ -240,6 +242,11 @@ class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector
whole_key_filtering_ ? kPropTrue : kPropFalse});
properties->insert({BlockBasedTablePropertyNames::kPrefixFiltering,
prefix_filtering_ ? kPropTrue : kPropFalse});
if (decoupled_partitioned_filters_) {
properties->insert(
{BlockBasedTablePropertyNames::kDecoupledPartitionedFilters,
kPropTrue});
}
return Status::OK();
}
@ -257,6 +264,7 @@ class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector
BlockBasedTableOptions::IndexType index_type_;
bool whole_key_filtering_;
bool prefix_filtering_;
bool decoupled_partitioned_filters_;
};
struct BlockBasedTableBuilder::Rep {
@ -594,7 +602,8 @@ struct BlockBasedTableBuilder::Rep {
table_properties_collectors.emplace_back(
new BlockBasedTablePropertiesCollector(
table_options.index_type, table_options.whole_key_filtering,
prefix_extractor != nullptr));
prefix_extractor != nullptr,
table_options.decouple_partitioned_filters));
if (ts_sz > 0 && persist_user_defined_timestamps) {
table_properties_collectors.emplace_back(
new TimestampTablePropertiesCollector(
@ -654,6 +663,7 @@ struct BlockBasedTableBuilder::Rep {
};
struct BlockBasedTableBuilder::ParallelCompressionRep {
// TODO: consider replacing with autovector or similar
// Keys is a wrapper of vector of strings avoiding
// releasing string memories during vector clear()
// in order to save memory allocation overhead
@ -1064,8 +1074,11 @@ void BlockBasedTableBuilder::Add(const Slice& ikey, const Slice& value) {
r->pc_rep->curr_block_keys->PushBack(ikey);
} else {
if (r->filter_builder != nullptr) {
r->filter_builder->Add(
ExtractUserKeyAndStripTimestamp(ikey, r->ts_sz));
r->filter_builder->AddWithPrevKey(
ExtractUserKeyAndStripTimestamp(ikey, r->ts_sz),
r->last_ikey.empty()
? Slice{}
: ExtractUserKeyAndStripTimestamp(r->last_ikey, r->ts_sz));
}
}
}
@ -1454,6 +1467,8 @@ void BlockBasedTableBuilder::BGWorkWriteMaybeCompressedBlock() {
Rep* r = rep_;
ParallelCompressionRep::BlockRepSlot* slot = nullptr;
ParallelCompressionRep::BlockRep* block_rep = nullptr;
// Starts empty; see FilterBlockBuilder::AddWithPrevKey
std::string prev_block_last_key_no_ts;
while (r->pc_rep->write_queue.pop(slot)) {
assert(slot != nullptr);
slot->Take(block_rep);
@ -1467,13 +1482,20 @@ void BlockBasedTableBuilder::BGWorkWriteMaybeCompressedBlock() {
continue;
}
Slice prev_key_no_ts = prev_block_last_key_no_ts;
for (size_t i = 0; i < block_rep->keys->Size(); i++) {
auto& key = (*block_rep->keys)[i];
if (r->filter_builder != nullptr) {
r->filter_builder->Add(ExtractUserKeyAndStripTimestamp(key, r->ts_sz));
Slice key_no_ts = ExtractUserKeyAndStripTimestamp(key, r->ts_sz);
r->filter_builder->AddWithPrevKey(key_no_ts, prev_key_no_ts);
prev_key_no_ts = key_no_ts;
}
r->index_builder->OnKeyAdded(key);
}
if (r->filter_builder != nullptr) {
prev_block_last_key_no_ts.assign(prev_key_no_ts.data(),
prev_key_no_ts.size());
}
r->pc_rep->file_size_estimator.SetCurrBlockUncompSize(
block_rep->data->size());
@ -1565,6 +1587,13 @@ void BlockBasedTableBuilder::WriteFilterBlock(
// No filter block needed
return;
}
if (!rep_->last_ikey.empty()) {
// We might have been using AddWithPrevKey, so need PrevKeyBeforeFinish
// to be safe. And because we are re-synchronized after buffered/parallel
// operation, rep_->last_ikey is accurate.
rep_->filter_builder->PrevKeyBeforeFinish(
ExtractUserKeyAndStripTimestamp(rep_->last_ikey, rep_->ts_sz));
}
BlockHandle filter_block_handle;
bool is_partitioned_filter = rep_->table_options.partition_filters;
if (ok()) {
@ -1979,6 +2008,10 @@ void BlockBasedTableBuilder::EnterUnbuffered() {
for (; iter->Valid(); iter->Next()) {
Slice key = iter->key();
if (r->filter_builder != nullptr) {
// NOTE: AddWithPrevKey here would only save key copying if prev is
// pinned (iter->IsKeyPinned()), which is probably rare with delta
// encoding. OK to go from Add() here to AddWithPrevKey() in
// unbuffered operation.
r->filter_builder->Add(
ExtractUserKeyAndStripTimestamp(key, r->ts_sz));
}
@ -1992,6 +2025,7 @@ void BlockBasedTableBuilder::EnterUnbuffered() {
Slice* first_key_in_next_block_ptr = &first_key_in_next_block;
iter->SeekToLast();
assert(iter->Valid());
r->index_builder->AddIndexEntry(
iter->key(), first_key_in_next_block_ptr, r->pending_handle,
&r->index_separator_scratch);

View File

@ -304,6 +304,10 @@ static std::unordered_map<std::string, OptionTypeInfo>
{offsetof(struct BlockBasedTableOptions, partition_filters),
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"decouple_partitioned_filters",
{offsetof(struct BlockBasedTableOptions, decouple_partitioned_filters),
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"optimize_filters_for_memory",
{offsetof(struct BlockBasedTableOptions, optimize_filters_for_memory),
OptionType::kBoolean, OptionVerificationType::kNormal,
@ -971,6 +975,8 @@ const std::string BlockBasedTablePropertyNames::kWholeKeyFiltering =
"rocksdb.block.based.table.whole.key.filtering";
const std::string BlockBasedTablePropertyNames::kPrefixFiltering =
"rocksdb.block.based.table.prefix.filtering";
const std::string BlockBasedTablePropertyNames::kDecoupledPartitionedFilters =
"rocksdb.block.based.table.decoupled.partitioned.filters";
const std::string kHashIndexPrefixesBlock = "rocksdb.hashindex.prefixes";
const std::string kHashIndexPrefixesMetadataBlock =
"rocksdb.hashindex.metadata";

View File

@ -52,12 +52,27 @@ class FilterBlockBuilder {
virtual ~FilterBlockBuilder() {}
virtual void Add(
const Slice& key_without_ts) = 0; // Add a key to current filter
// Add a key to current filter.
virtual void Add(const Slice& key_without_ts) = 0;
// A potentially more efficient version of Add(), though you cannot go back
// to Add() after using AddWithPrevKey() on a FilterBlockBuilder.
// prev_key_without_ts should be the empty Slice for the first key added
// (regardless of comparator; e.g. for bootstrapping delta encoding).
// More detail: The previous key is used when filters are key-range
// partitioned, and the PartitionedFilterBlockBuilder doesn't need to buffer
// the previous key when it is provided by calling this function.
virtual void AddWithPrevKey(const Slice& key_without_ts,
const Slice& /*prev_key_without_ts*/) = 0;
virtual bool IsEmpty() const = 0; // Empty == none added
// For reporting stats on how many entries the builder considered unique
virtual size_t EstimateEntriesAdded() = 0;
// When using AddWithPrevKey, this must be called before Finish(). (May also
// be called without AddWithPrevKey, but prev_key_without_ts must be
// accurate regardless.)
virtual void PrevKeyBeforeFinish(const Slice& /*prev_key_without_ts*/) {}
// Generate a filter block. Returns OK if finished, or Incomplete if more
// filters are needed (partitioned filter). In the latter case, subsequent
// calls require the BlockHandle of the most recently generated and written

View File

@ -30,6 +30,11 @@ size_t FullFilterBlockBuilder::EstimateEntriesAdded() {
return filter_bits_builder_->EstimateEntriesAdded();
}
void FullFilterBlockBuilder::AddWithPrevKey(
const Slice& key_without_ts, const Slice& /*prev_key_without_ts*/) {
FullFilterBlockBuilder::Add(key_without_ts);
}
void FullFilterBlockBuilder::Add(const Slice& key_without_ts) {
if (prefix_extractor_ && prefix_extractor_->InDomain(key_without_ts)) {
Slice prefix = prefix_extractor_->Transform(key_without_ts);

View File

@ -50,6 +50,9 @@ class FullFilterBlockBuilder : public FilterBlockBuilder {
~FullFilterBlockBuilder() {}
void Add(const Slice& key_without_ts) override;
void AddWithPrevKey(const Slice& key_without_ts,
const Slice& prev_key_without_ts) override;
bool IsEmpty() const override {
return filter_bits_builder_->EstimateEntriesAdded() == 0;
}

View File

@ -27,10 +27,13 @@ PartitionedFilterBlockBuilder::PartitionedFilterBlockBuilder(
const bool use_value_delta_encoding,
PartitionedIndexBuilder* const p_index_builder,
const uint32_t partition_size, size_t ts_sz,
const bool persist_user_defined_timestamps)
const bool persist_user_defined_timestamps,
bool decouple_from_index_partitions)
: FullFilterBlockBuilder(_prefix_extractor, whole_key_filtering,
filter_bits_builder),
p_index_builder_(p_index_builder),
ts_sz_(ts_sz),
decouple_from_index_partitions_(decouple_from_index_partitions),
index_on_filter_block_builder_(
index_block_restart_interval, true /*use_delta_encoding*/,
use_value_delta_encoding,
@ -66,6 +69,11 @@ PartitionedFilterBlockBuilder::PartitionedFilterBlockBuilder(
}
}
}
if (keys_per_partition_ > 1 && prefix_extractor()) {
// Correct for adding next prefix in CutAFilterBlock *after* checking
// against this threshold
keys_per_partition_--;
}
}
PartitionedFilterBlockBuilder::~PartitionedFilterBlockBuilder() {
@ -73,18 +81,26 @@ PartitionedFilterBlockBuilder::~PartitionedFilterBlockBuilder() {
}
bool PartitionedFilterBlockBuilder::DecideCutAFilterBlock() {
size_t added = filter_bits_builder_->EstimateEntriesAdded();
if (decouple_from_index_partitions_) {
// NOTE: Can't just use ==, because estimated might be incremented by more
// than one. +1 is for adding next_prefix below.
if (filter_bits_builder_->EstimateEntriesAdded() + 1 >= keys_per_partition_) {
// Currently only index builder is in charge of cutting a partition. We keep
// requesting until it is granted.
// than one.
return added >= keys_per_partition_;
} else {
// NOTE: Can't just use ==, because estimated might be incremented by more
// than one.
if (added >= keys_per_partition_) {
// Currently only index builder is in charge of cutting a partition. We
// keep requesting until it is granted.
p_index_builder_->RequestPartitionCut();
}
return p_index_builder_->ShouldCutFilterBlock();
}
}
void PartitionedFilterBlockBuilder::CutAFilterBlock(const Slice* next_key,
const Slice* next_prefix) {
const Slice* next_prefix,
const Slice& prev_key) {
// When there is a next partition, add the prefix of the first key in the
// next partition before closing this one out. This is needed to support
// prefix Seek, because there could exist a key k where
@ -115,46 +131,77 @@ void PartitionedFilterBlockBuilder::CutAFilterBlock(const Slice* next_key,
if (filter_construction_status.ok()) {
filter_construction_status = filter_bits_builder_->MaybePostVerify(filter);
}
filters_.push_back(
{p_index_builder_->GetPartitionKey(), std::move(filter_data), filter});
std::string ikey;
if (decouple_from_index_partitions_) {
if (ts_sz_ > 0) {
AppendKeyWithMinTimestamp(&ikey, prev_key, ts_sz_);
} else {
ikey = prev_key.ToString();
}
AppendInternalKeyFooter(&ikey, /*seqno*/ 0, ValueType::kTypeDeletion);
} else {
ikey = p_index_builder_->GetPartitionKey();
}
filters_.push_back({std::move(ikey), std::move(filter_data), filter});
partitioned_filters_construction_status_.UpdateIfOk(
filter_construction_status);
// If we are building another filter partition, the last prefix in the
// previous partition should be added to support prefix SeekForPrev.
// (Analogous to above fix for prefix Seek.)
if (next_key && last_key_in_domain_) {
if (next_key && prefix_extractor() &&
prefix_extractor()->InDomain(prev_key)) {
// NOTE: At the beginning of building filter bits, we don't need a special
// case for treating prefix as an "alt" entry.
// See DBBloomFilterTest.FilterBitsBuilderDedup
filter_bits_builder_->AddKey(last_prefix_str_);
filter_bits_builder_->AddKey(prefix_extractor()->Transform(prev_key));
}
}
void PartitionedFilterBlockBuilder::Add(const Slice& key_without_ts) {
assert(!DEBUG_add_with_prev_key_called_);
AddImpl(key_without_ts, prev_key_without_ts_);
prev_key_without_ts_.assign(key_without_ts.data(), key_without_ts.size());
}
void PartitionedFilterBlockBuilder::AddWithPrevKey(
const Slice& key_without_ts, const Slice& prev_key_without_ts) {
#ifndef NDEBUG
if (!DEBUG_add_with_prev_key_called_) {
assert(prev_key_without_ts.compare(prev_key_without_ts_) == 0);
DEBUG_add_with_prev_key_called_ = true;
} else {
assert(prev_key_without_ts.compare(DEBUG_prev_key_without_ts_) == 0);
}
DEBUG_prev_key_without_ts_.assign(key_without_ts.data(),
key_without_ts.size());
#endif
AddImpl(key_without_ts, prev_key_without_ts);
}
void PartitionedFilterBlockBuilder::AddImpl(const Slice& key_without_ts,
const Slice& prev_key_without_ts) {
// When filter partitioning is coupled to index partitioning, we need to
// check for cutting a block even if we aren't adding anything this time.
bool cut = DecideCutAFilterBlock();
if (prefix_extractor() && prefix_extractor()->InDomain(key_without_ts)) {
Slice prefix = prefix_extractor()->Transform(key_without_ts);
if (cut) {
CutAFilterBlock(&key_without_ts, &prefix);
CutAFilterBlock(&key_without_ts, &prefix, prev_key_without_ts);
}
if (whole_key_filtering()) {
filter_bits_builder_->AddKeyAndAlt(key_without_ts, prefix);
} else {
filter_bits_builder_->AddKey(prefix);
}
last_key_in_domain_ = true;
last_prefix_str_.assign(prefix.data(), prefix.size());
} else {
if (cut) {
CutAFilterBlock(&key_without_ts, nullptr);
CutAFilterBlock(&key_without_ts, nullptr /*no prefix*/,
prev_key_without_ts);
}
if (whole_key_filtering()) {
filter_bits_builder_->AddKey(key_without_ts);
}
last_key_in_domain_ = false;
}
}
@ -162,6 +209,17 @@ size_t PartitionedFilterBlockBuilder::EstimateEntriesAdded() {
return total_added_in_built_ + filter_bits_builder_->EstimateEntriesAdded();
}
void PartitionedFilterBlockBuilder::PrevKeyBeforeFinish(
const Slice& prev_key_without_ts) {
assert(prev_key_without_ts.compare(DEBUG_add_with_prev_key_called_
? DEBUG_prev_key_without_ts_
: prev_key_without_ts_) == 0);
if (filter_bits_builder_->EstimateEntriesAdded() > 0) {
CutAFilterBlock(nullptr /*no next key*/, nullptr /*no next prefix*/,
prev_key_without_ts);
}
}
Status PartitionedFilterBlockBuilder::Finish(
const BlockHandle& last_partition_block_handle, Slice* filter,
std::unique_ptr<const char[]>* filter_owner) {
@ -192,8 +250,11 @@ Status PartitionedFilterBlockBuilder::Finish(
} else {
assert(last_partition_block_handle == BlockHandle{});
if (filter_bits_builder_->EstimateEntriesAdded() > 0) {
CutAFilterBlock(nullptr, nullptr);
// PrevKeyBeforeFinish was not called
assert(!DEBUG_add_with_prev_key_called_);
CutAFilterBlock(nullptr, nullptr, prev_key_without_ts_);
}
// Nothing uncommitted
assert(filter_bits_builder_->EstimateEntriesAdded() == 0);
}

View File

@ -32,17 +32,22 @@ class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
const bool use_value_delta_encoding,
PartitionedIndexBuilder* const p_index_builder,
const uint32_t partition_size, size_t ts_sz,
const bool persist_user_defined_timestamps);
const bool persist_user_defined_timestamps,
bool decouple_from_index_partitions);
virtual ~PartitionedFilterBlockBuilder();
void Add(const Slice& key_without_ts) override;
void AddWithPrevKey(const Slice& key_without_ts,
const Slice& prev_key_without_ts) override;
bool IsEmpty() const override {
return filter_bits_builder_->EstimateEntriesAdded() == 0 &&
filters_.empty();
}
size_t EstimateEntriesAdded() override;
void PrevKeyBeforeFinish(const Slice& prev_key_without_ts) override;
Status Finish(const BlockHandle& last_partition_block_handle, Slice* filter,
std::unique_ptr<const char[]>* filter_owner = nullptr) override;
@ -65,7 +70,10 @@ class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
private: // fns
// Whether to cut a filter block before the next key
bool DecideCutAFilterBlock();
void CutAFilterBlock(const Slice* next_key, const Slice* next_prefix);
void CutAFilterBlock(const Slice* next_key, const Slice* next_prefix,
const Slice& prev_key);
void AddImpl(const Slice& key_without_ts, const Slice& prev_key_without_ts);
private: // data
// Currently we keep the same number of partitions for filters and indexes.
@ -73,6 +81,8 @@ class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
// optimizations did not realize we can use different number of partitions and
// eliminate p_index_builder_
PartitionedIndexBuilder* const p_index_builder_;
const size_t ts_sz_;
const bool decouple_from_index_partitions_;
// Filter data
struct FilterEntry {
@ -88,17 +98,22 @@ class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
// in all the filters we have fully built
uint64_t total_added_in_built_ = 0;
// Tracking state about previous prefix, to solve issue with prefix Seeks
// at partition boundaries.
bool last_key_in_domain_ = false;
std::string last_prefix_str_;
// Set to the first non-okay status if any of the filter
// partitions experiences construction error.
// If partitioned_filters_construction_status_ is non-okay,
// then the whole partitioned filters should not be used.
Status partitioned_filters_construction_status_;
// For Add without prev key
std::string prev_key_without_ts_;
#ifndef NDEBUG
// For verifying accurate previous keys are provided by the caller, so that
// release code can be fast
bool DEBUG_add_with_prev_key_called_ = false;
std::string DEBUG_prev_key_without_ts_;
#endif // NDEBUG
// ===== State for Finish() =====
// top-level index builder on internal keys

View File

@ -58,7 +58,7 @@ class MyPartitionedFilterBlockReader : public PartitionedFilterBlockReader {
class PartitionedFilterBlockTest
: public testing::Test,
virtual public ::testing::WithParamInterface<
std::tuple<uint32_t, test::UserDefinedTimestampTestMode>> {
std::tuple<uint32_t, test::UserDefinedTimestampTestMode, bool>> {
public:
Options options_;
ImmutableOptions ioptions_;
@ -70,6 +70,7 @@ class PartitionedFilterBlockTest
int bits_per_key_;
size_t ts_sz_;
bool user_defined_timestamps_persisted_;
bool decouple_partitioned_filters;
PartitionedFilterBlockTest() : bits_per_key_(10) {
auto udt_test_mode = std::get<1>(GetParam());
@ -85,6 +86,8 @@ class PartitionedFilterBlockTest
NewBloomFilterPolicy(bits_per_key_, false));
table_options_.format_version = std::get<0>(GetParam());
table_options_.index_block_restart_interval = 3;
table_options_.decouple_partitioned_filters = decouple_partitioned_filters =
std::get<2>(GetParam());
}
~PartitionedFilterBlockTest() override = default;
@ -160,7 +163,7 @@ class PartitionedFilterBlockTest
FilterBuildingContext(table_options_)),
table_options_.index_block_restart_interval, !kValueDeltaEncoded,
p_index_builder, partition_size, ts_sz_,
user_defined_timestamps_persisted_);
user_defined_timestamps_persisted_, decouple_partitioned_filters);
}
PartitionedFilterBlockReader* NewReader(
@ -343,10 +346,10 @@ class PartitionedFilterBlockTest
// Format versions potentially intersting to partitioning
INSTANTIATE_TEST_CASE_P(
FormatVersions, PartitionedFilterBlockTest,
testing::Combine(testing::ValuesIn(std::set<uint32_t>{
2, 3, 4, 5, test::kDefaultFormatVersion,
kLatestFormatVersion}),
testing::ValuesIn(test::GetUDTTestModes())));
testing::Combine(
testing::ValuesIn(std::set<uint32_t>{
2, 3, 4, 5, test::kDefaultFormatVersion, kLatestFormatVersion}),
testing::ValuesIn(test::GetUDTTestModes()), testing::Bool()));
TEST_P(PartitionedFilterBlockTest, EmptyBuilder) {
std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());

View File

@ -641,6 +641,11 @@ DEFINE_bool(use_cache_jemalloc_no_dump_allocator, false,
DEFINE_bool(use_cache_memkind_kmem_allocator, false,
"Use memkind kmem allocator for block/blob cache.");
DEFINE_bool(
decouple_partitioned_filters,
ROCKSDB_NAMESPACE::BlockBasedTableOptions().decouple_partitioned_filters,
"Decouple filter partitioning from index partitioning.");
DEFINE_bool(partition_index_and_filters, false,
"Partition index and filter blocks.");
@ -4362,6 +4367,8 @@ class Benchmark {
} else {
block_based_options.index_type = BlockBasedTableOptions::kBinarySearch;
}
block_based_options.decouple_partitioned_filters =
FLAGS_decouple_partitioned_filters;
if (FLAGS_partition_index_and_filters || FLAGS_partition_index) {
if (FLAGS_index_with_first_key) {
fprintf(stderr,

View File

@ -75,6 +75,7 @@ default_params = {
"compaction_pri": random.randint(0, 4),
"key_may_exist_one_in": lambda: random.choice([100, 100000]),
"data_block_index_type": lambda: random.choice([0, 1]),
"decouple_partitioned_filters": lambda: random.choice([0, 1, 1]),
"delpercent": 4,
"delrangepercent": 1,
"destroy_db_initially": 0,

View File

@ -0,0 +1 @@
* New option `BlockBasedTableOptions::decouple_partitioned_filters` should improve efficiency in serving read queries because filter and index partitions can consistently target the configured `metadata_block_size`. This option is currently opt-in.