Redesign block cache pinning API (#7520)

Summary:
The old flag-based APIs (`BlockBasedTableOptions::pin_l0_filter_and_index_blocks_in_cache` and `BlockBasedTableOptions::pin_top_level_index_and_filter`) were insufficient for our needs. For example, it was impossible to pin only unpartitioned meta-blocks, which could prevent block cache contention when turning on dictionary compression or during a migration to partitioned indexes/filters. It was also impossible to pin all meta-blocks in memory while having predictable memory usage via block cache. If we had continued adding flags to address these scenarios, they would have had significant overlap causing confusion. Instead, this PR deprecates the flags and starts a new API with non-overlapping options.

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

Test Plan:
- new unit test
- added new options to stress/crash test and ran for a while: `$ python tools/db_crashtest.py blackbox --simple --max_key=1000000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 --interval=10 -value_size_mult=33 -column_families=1 -reopen=0`

Reviewed By: pdillinger

Differential Revision: D24200034

Pulled By: ajkr

fbshipit-source-id: 3fa7cfc71e7960f7a867511dd6ae5834dd73b13e
main
Andrew Kryczka 4 years ago committed by Facebook GitHub Bot
parent 12b78e40bd
commit 75d3b6fdf0
  1. 8
      HISTORY.md
  2. 157
      db/db_block_cache_test.cc
  3. 3
      db_stress_tool/db_stress_common.h
  4. 18
      db_stress_tool/db_stress_gflags.cc
  5. 6
      db_stress_tool/db_stress_test_base.cc
  6. 87
      include/rocksdb/table.h
  7. 3
      options/options_settable_test.cc
  8. 31
      table/block_based/block_based_table_factory.cc
  9. 85
      table/block_based/block_based_table_reader.cc
  10. 2
      table/block_based/partitioned_index_reader.cc
  11. 5
      table/iterator_wrapper.h
  12. 4
      table/two_level_iterator.cc
  13. 3
      tools/db_crashtest.py

@ -1,4 +1,12 @@
# Rocksdb Change Log
## Unreleased
### Bug Fixes
* Fixed a bug in the following combination of features: indexes with user keys (`format_version >= 3`), indexes are partitioned (`index_type == kTwoLevelIndexSearch`), and some index partitions are pinned in memory (`BlockBasedTableOptions::pin_l0_filter_and_index_blocks_in_cache`). The bug could cause keys to be truncated when read from the index leading to wrong read results or other unexpected behavior.
* Fixed a bug when indexes are partitioned (`index_type == kTwoLevelIndexSearch`), some index partitions are pinned in memory (`BlockBasedTableOptions::pin_l0_filter_and_index_blocks_in_cache`), and partitions reads could be mixed between block cache and directly from the file (e.g., with `enable_index_compression == 1` and `mmap_read == 1`, partitions that were stored uncompressed due to poor compression ratio would be read directly from the file via mmap, while partitions that were stored compressed would be read from block cache). The bug could cause index partitions to be mistakenly considered empty during reads leading to wrong read results.
### Public API Change
* Deprecate `BlockBasedTableOptions::pin_l0_filter_and_index_blocks_in_cache` and `BlockBasedTableOptions::pin_top_level_index_and_filter`. These options still take effect until users migrate to the replacement APIs in `BlockBasedTableOptions::metadata_cache_options`. Migration guidance can be found in the API comments on the deprecated options.
## 6.14 (10/09/2020)
### Bug fixes
* Fixed a bug after a `CompactRange()` with `CompactRangeOptions::change_level` set fails due to a conflict in the level change step, which caused all subsequent calls to `CompactRange()` with `CompactRangeOptions::change_level` set to incorrectly fail with a `Status::NotSupported("another thread is refitting")` error.

@ -888,6 +888,163 @@ TEST_F(DBBlockCacheTest, CacheCompressionDict) {
#endif // ROCKSDB_LITE
class DBBlockCachePinningTest
: public DBTestBase,
public testing::WithParamInterface<
std::tuple<bool, PinningTier, PinningTier, PinningTier>> {
public:
DBBlockCachePinningTest()
: DBTestBase("/db_block_cache_test", /*env_do_fsync=*/false) {}
void SetUp() override {
partition_index_and_filters_ = std::get<0>(GetParam());
top_level_index_pinning_ = std::get<1>(GetParam());
partition_pinning_ = std::get<2>(GetParam());
unpartitioned_pinning_ = std::get<3>(GetParam());
}
bool partition_index_and_filters_;
PinningTier top_level_index_pinning_;
PinningTier partition_pinning_;
PinningTier unpartitioned_pinning_;
};
TEST_P(DBBlockCachePinningTest, TwoLevelDB) {
// Creates one file in L0 and one file in L1. Both files have enough data that
// their index and filter blocks are partitioned. The L1 file will also have
// a compression dictionary (those are trained only during compaction), which
// must be unpartitioned.
const int kKeySize = 32;
const int kBlockSize = 128;
const int kNumBlocksPerFile = 128;
const int kNumKeysPerFile = kBlockSize * kNumBlocksPerFile / kKeySize;
Options options = CurrentOptions();
// `kNoCompression` makes the unit test more portable. But it relies on the
// current behavior of persisting/accessing dictionary even when there's no
// (de)compression happening, which seems fairly likely to change over time.
options.compression = kNoCompression;
options.compression_opts.max_dict_bytes = 4 << 10;
options.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
BlockBasedTableOptions table_options;
table_options.block_cache = NewLRUCache(1 << 20 /* capacity */);
table_options.block_size = kBlockSize;
table_options.metadata_block_size = kBlockSize;
table_options.cache_index_and_filter_blocks = true;
table_options.metadata_cache_options.top_level_index_pinning =
top_level_index_pinning_;
table_options.metadata_cache_options.partition_pinning = partition_pinning_;
table_options.metadata_cache_options.unpartitioned_pinning =
unpartitioned_pinning_;
table_options.filter_policy.reset(
NewBloomFilterPolicy(10 /* bits_per_key */));
if (partition_index_and_filters_) {
table_options.index_type =
BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
table_options.partition_filters = true;
}
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
Reopen(options);
Random rnd(301);
for (int i = 0; i < 2; ++i) {
for (int j = 0; j < kNumKeysPerFile; ++j) {
ASSERT_OK(Put(Key(i * kNumKeysPerFile + j), rnd.RandomString(kKeySize)));
}
ASSERT_OK(Flush());
if (i == 0) {
// Prevent trivial move so file will be rewritten with dictionary and
// reopened with L1's pinning settings.
CompactRangeOptions cro;
cro.bottommost_level_compaction = BottommostLevelCompaction::kForce;
ASSERT_OK(db_->CompactRange(cro, nullptr, nullptr));
}
}
// Clear all unpinned blocks so unpinned blocks will show up as cache misses
// when reading a key from a file.
table_options.block_cache->EraseUnRefEntries();
// Get base cache values
uint64_t filter_misses = TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS);
uint64_t index_misses = TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS);
uint64_t compression_dict_misses =
TestGetTickerCount(options, BLOCK_CACHE_COMPRESSION_DICT_MISS);
// Read a key from the L0 file
Get(Key(kNumKeysPerFile));
uint64_t expected_filter_misses = filter_misses;
uint64_t expected_index_misses = index_misses;
uint64_t expected_compression_dict_misses = compression_dict_misses;
if (partition_index_and_filters_) {
if (top_level_index_pinning_ == PinningTier::kNone) {
++expected_filter_misses;
++expected_index_misses;
}
if (partition_pinning_ == PinningTier::kNone) {
++expected_filter_misses;
++expected_index_misses;
}
} else {
if (unpartitioned_pinning_ == PinningTier::kNone) {
++expected_filter_misses;
++expected_index_misses;
}
}
ASSERT_EQ(expected_filter_misses,
TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
ASSERT_EQ(expected_index_misses,
TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS));
ASSERT_EQ(expected_compression_dict_misses,
TestGetTickerCount(options, BLOCK_CACHE_COMPRESSION_DICT_MISS));
// Clear all unpinned blocks so unpinned blocks will show up as cache misses
// when reading a key from a file.
table_options.block_cache->EraseUnRefEntries();
// Read a key from the L1 file
Get(Key(0));
if (partition_index_and_filters_) {
if (top_level_index_pinning_ == PinningTier::kNone ||
top_level_index_pinning_ == PinningTier::kFlushedAndSimilar) {
++expected_filter_misses;
++expected_index_misses;
}
if (partition_pinning_ == PinningTier::kNone ||
partition_pinning_ == PinningTier::kFlushedAndSimilar) {
++expected_filter_misses;
++expected_index_misses;
}
} else {
if (unpartitioned_pinning_ == PinningTier::kNone ||
unpartitioned_pinning_ == PinningTier::kFlushedAndSimilar) {
++expected_filter_misses;
++expected_index_misses;
}
}
if (unpartitioned_pinning_ == PinningTier::kNone ||
unpartitioned_pinning_ == PinningTier::kFlushedAndSimilar) {
++expected_compression_dict_misses;
}
ASSERT_EQ(expected_filter_misses,
TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
ASSERT_EQ(expected_index_misses,
TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS));
ASSERT_EQ(expected_compression_dict_misses,
TestGetTickerCount(options, BLOCK_CACHE_COMPRESSION_DICT_MISS));
}
INSTANTIATE_TEST_CASE_P(
DBBlockCachePinningTest, DBBlockCachePinningTest,
::testing::Combine(
::testing::Bool(),
::testing::Values(PinningTier::kNone, PinningTier::kFlushedAndSimilar,
PinningTier::kAll),
::testing::Values(PinningTier::kNone, PinningTier::kFlushedAndSimilar,
PinningTier::kAll),
::testing::Values(PinningTier::kNone, PinningTier::kFlushedAndSimilar,
PinningTier::kAll)));
} // namespace ROCKSDB_NAMESPACE
int main(int argc, char** argv) {

@ -132,6 +132,9 @@ DECLARE_int32(set_options_one_in);
DECLARE_int32(set_in_place_one_in);
DECLARE_int64(cache_size);
DECLARE_bool(cache_index_and_filter_blocks);
DECLARE_int32(top_level_index_pinning);
DECLARE_int32(partition_pinning);
DECLARE_int32(unpartitioned_pinning);
DECLARE_bool(use_clock_cache);
DECLARE_uint64(subcompactions);
DECLARE_uint64(periodic_compaction_seconds);

@ -287,6 +287,24 @@ DEFINE_int64(cache_size, 2LL * KB * KB * KB,
DEFINE_bool(cache_index_and_filter_blocks, false,
"True if indexes/filters should be cached in block cache.");
DEFINE_int32(
top_level_index_pinning,
static_cast<int32_t>(ROCKSDB_NAMESPACE::PinningTier::kFallback),
"Type of pinning for top-level indexes into metadata partitions (see "
"`enum PinningTier` in table.h)");
DEFINE_int32(
partition_pinning,
static_cast<int32_t>(ROCKSDB_NAMESPACE::PinningTier::kFallback),
"Type of pinning for metadata partitions (see `enum PinningTier` in "
"table.h)");
DEFINE_int32(
unpartitioned_pinning,
static_cast<int32_t>(ROCKSDB_NAMESPACE::PinningTier::kFallback),
"Type of pinning for unpartitioned metadata blocks (see `enum PinningTier` "
"in table.h)");
DEFINE_bool(use_clock_cache, false,
"Replace default LRU block cache with clock cache.");

@ -1964,6 +1964,12 @@ void StressTest::Open() {
block_based_options.block_cache = cache_;
block_based_options.cache_index_and_filter_blocks =
FLAGS_cache_index_and_filter_blocks;
block_based_options.metadata_cache_options.top_level_index_pinning =
static_cast<PinningTier>(FLAGS_top_level_index_pinning);
block_based_options.metadata_cache_options.partition_pinning =
static_cast<PinningTier>(FLAGS_partition_pinning);
block_based_options.metadata_cache_options.unpartitioned_pinning =
static_cast<PinningTier>(FLAGS_unpartitioned_pinning);
block_based_options.block_cache_compressed = compressed_cache_;
block_based_options.checksum = checksum_type_e;
block_based_options.block_size = FLAGS_block_size;

@ -51,6 +51,55 @@ enum ChecksumType : char {
kxxHash64 = 0x3,
};
// `PinningTier` is used to specify which tier of block-based tables should
// be affected by a block cache pinning setting (see
// `MetadataCacheOptions` below).
enum class PinningTier {
// For compatibility, this value specifies to fallback to the behavior
// indicated by the deprecated options,
// `pin_l0_filter_and_index_blocks_in_cache` and
// `pin_top_level_index_and_filter`.
kFallback,
// This tier contains no block-based tables.
kNone,
// This tier contains block-based tables that may have originated from a
// memtable flush. In particular, it includes tables from L0 that are smaller
// than 1.5 times the current `write_buffer_size`. Note these criteria imply
// it can include intra-L0 compaction outputs and ingested files, as long as
// they are not abnormally large compared to flushed files in L0.
kFlushedAndSimilar,
// This tier contains all block-based tables.
kAll,
};
// `MetadataCacheOptions` contains members indicating the desired caching
// behavior for the different categories of metadata blocks.
struct MetadataCacheOptions {
// The tier of block-based tables whose top-level index into metadata
// partitions will be pinned. Currently indexes and filters may be
// partitioned.
//
// Note `cache_index_and_filter_blocks` must be true for this option to have
// any effect. Otherwise any top-level index into metadata partitions would be
// held in table reader memory, outside the block cache.
PinningTier top_level_index_pinning = PinningTier::kFallback;
// The tier of block-based tables whose metadata partitions will be pinned.
// Currently indexes and filters may be partitioned.
PinningTier partition_pinning = PinningTier::kFallback;
// The tier of block-based tables whose unpartitioned metadata blocks will be
// pinned.
//
// Note `cache_index_and_filter_blocks` must be true for this option to have
// any effect. Otherwise the unpartitioned meta-blocks would be held in table
// reader memory, outside the block cache.
PinningTier unpartitioned_pinning = PinningTier::kFallback;
};
// For advanced user only
struct BlockBasedTableOptions {
static const char* kName() { return "BlockTableOptions"; };
@ -79,12 +128,44 @@ struct BlockBasedTableOptions {
// than data blocks.
bool cache_index_and_filter_blocks_with_high_priority = true;
// DEPRECATED: This option will be removed in a future version. For now, this
// option still takes effect by updating each of the following variables that
// has the default value, `PinningTier::kFallback`:
//
// - `MetadataCacheOptions::partition_pinning`
// - `MetadataCacheOptions::unpartitioned_pinning`
//
// The updated value is chosen as follows:
//
// - `pin_l0_filter_and_index_blocks_in_cache == false` ->
// `PinningTier::kNone`
// - `pin_l0_filter_and_index_blocks_in_cache == true` ->
// `PinningTier::kFlushedAndSimilar`
//
// To migrate away from this flag, explicitly configure
// `MetadataCacheOptions` as described above.
//
// if cache_index_and_filter_blocks is true and the below is true, then
// filter and index blocks are stored in the cache, but a reference is
// held in the "table reader" object so the blocks are pinned and only
// evicted from cache when the table reader is freed.
bool pin_l0_filter_and_index_blocks_in_cache = false;
// DEPRECATED: This option will be removed in a future version. For now, this
// option still takes effect by updating
// `MetadataCacheOptions::top_level_index_pinning` when it has the
// default value, `PinningTier::kFallback`.
//
// The updated value is chosen as follows:
//
// - `pin_top_level_index_and_filter == false` ->
// `PinningTier::kNone`
// - `pin_top_level_index_and_filter == true` ->
// `PinningTier::kAll`
//
// To migrate away from this flag, explicitly configure
// `MetadataCacheOptions` as described above.
//
// If cache_index_and_filter_blocks is true and the below is true, then
// the top-level index of partitioned filter and index blocks are stored in
// the cache, but a reference is held in the "table reader" object so the
@ -92,6 +173,12 @@ struct BlockBasedTableOptions {
// freed. This is not limited to l0 in LSM tree.
bool pin_top_level_index_and_filter = true;
// The desired block cache pinning behavior for the different categories of
// metadata blocks. While pinning can reduce block cache contention, users
// must take care not to pin excessive amounts of data, which risks
// overflowing block cache.
MetadataCacheOptions metadata_cache_options;
// The index type that will be used for this table.
enum IndexType : char {
// A space efficient index block that is optimized for

@ -158,6 +158,9 @@ TEST_F(OptionsSettableTest, BlockBasedTableOptionsAllFieldsSettable) {
*bbto,
"cache_index_and_filter_blocks=1;"
"cache_index_and_filter_blocks_with_high_priority=true;"
"metadata_cache_options={top_level_index_pinning=kFallback;"
"partition_pinning=kAll;"
"unpartitioned_pinning=kFlushedAndSimilar;};"
"pin_l0_filter_and_index_blocks_in_cache=1;"
"pin_top_level_index_and_filter=1;"
"index_type=kHashSearch;"

@ -160,6 +160,16 @@ size_t TailPrefetchStats::GetSuggestedPrefetchSize() {
}
#ifndef ROCKSDB_LITE
const std::string kOptNameMetadataCacheOpts = "metadata_cache_options";
static std::unordered_map<std::string, PinningTier>
pinning_tier_type_string_map = {
{"kFallback", PinningTier::kFallback},
{"kNone", PinningTier::kNone},
{"kFlushedAndSimilar", PinningTier::kFlushedAndSimilar},
{"kAll", PinningTier::kAll}};
static std::unordered_map<std::string, BlockBasedTableOptions::IndexType>
block_base_table_index_type_string_map = {
{"kBinarySearch", BlockBasedTableOptions::IndexType::kBinarySearch},
@ -187,6 +197,22 @@ static std::unordered_map<std::string,
{"kShortenSeparatorsAndSuccessor",
BlockBasedTableOptions::IndexShorteningMode::
kShortenSeparatorsAndSuccessor}};
static std::unordered_map<std::string, OptionTypeInfo>
metadata_cache_options_type_info = {
{"top_level_index_pinning",
OptionTypeInfo::Enum<PinningTier>(
offsetof(struct MetadataCacheOptions, top_level_index_pinning),
&pinning_tier_type_string_map)},
{"partition_pinning",
OptionTypeInfo::Enum<PinningTier>(
offsetof(struct MetadataCacheOptions, partition_pinning),
&pinning_tier_type_string_map)},
{"unpartitioned_pinning",
OptionTypeInfo::Enum<PinningTier>(
offsetof(struct MetadataCacheOptions, unpartitioned_pinning),
&pinning_tier_type_string_map)}};
#endif // ROCKSDB_LITE
static std::unordered_map<std::string, OptionTypeInfo>
@ -348,6 +374,11 @@ static std::unordered_map<std::string, OptionTypeInfo>
pin_top_level_index_and_filter),
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{kOptNameMetadataCacheOpts,
OptionTypeInfo::Struct(
kOptNameMetadataCacheOpts, &metadata_cache_options_type_info,
offsetof(struct BlockBasedTableOptions, metadata_cache_options),
OptionVerificationType::kNormal, OptionTypeFlags::kNone)},
{"block_cache",
{offsetof(struct BlockBasedTableOptions, block_cache),
OptionType::kUnknown, OptionVerificationType::kNormal,

@ -974,6 +974,9 @@ Status BlockBasedTable::PrefetchIndexAndFilterBlocks(
}
}
}
// Partition filters cannot be enabled without partition indexes
assert(rep_->filter_type != Rep::FilterType::kPartitionedFilter ||
rep_->index_type == BlockBasedTableOptions::kTwoLevelIndexSearch);
// Find compression dictionary handle
bool found_compression_dict = false;
@ -987,20 +990,53 @@ Status BlockBasedTable::PrefetchIndexAndFilterBlocks(
const bool use_cache = table_options.cache_index_and_filter_blocks;
// pin both index and filters, down to all partitions.
const bool pin_all =
rep_->table_options.pin_l0_filter_and_index_blocks_in_cache &&
const bool maybe_flushed =
level == 0 && file_size <= max_file_size_for_l0_meta_pin;
std::function<bool(PinningTier, PinningTier)> is_pinned =
[maybe_flushed, &is_pinned](PinningTier pinning_tier,
PinningTier fallback_pinning_tier) {
// Fallback to fallback would lead to infinite recursion. Disallow it.
assert(fallback_pinning_tier != PinningTier::kFallback);
switch (pinning_tier) {
case PinningTier::kFallback:
return is_pinned(fallback_pinning_tier,
PinningTier::kNone /* fallback_pinning_tier */);
case PinningTier::kNone:
return false;
case PinningTier::kFlushedAndSimilar:
return maybe_flushed;
case PinningTier::kAll:
return true;
};
// In GCC, this is needed to suppress `control reaches end of non-void
// function [-Werror=return-type]`.
assert(false);
return false;
};
const bool pin_top_level_index = is_pinned(
table_options.metadata_cache_options.top_level_index_pinning,
table_options.pin_top_level_index_and_filter ? PinningTier::kAll
: PinningTier::kNone);
const bool pin_partition =
is_pinned(table_options.metadata_cache_options.partition_pinning,
table_options.pin_l0_filter_and_index_blocks_in_cache
? PinningTier::kFlushedAndSimilar
: PinningTier::kNone);
const bool pin_unpartitioned =
is_pinned(table_options.metadata_cache_options.unpartitioned_pinning,
table_options.pin_l0_filter_and_index_blocks_in_cache
? PinningTier::kFlushedAndSimilar
: PinningTier::kNone);
// prefetch the first level of index
const bool prefetch_index =
prefetch_all ||
(table_options.pin_top_level_index_and_filter &&
index_type == BlockBasedTableOptions::kTwoLevelIndexSearch);
// pin the first level of index
const bool pin_index =
pin_all || (table_options.pin_top_level_index_and_filter &&
index_type == BlockBasedTableOptions::kTwoLevelIndexSearch);
index_type == BlockBasedTableOptions::kTwoLevelIndexSearch
? pin_top_level_index
: pin_unpartitioned;
// prefetch the first level of index
const bool prefetch_index = prefetch_all || pin_index;
std::unique_ptr<IndexReader> index_reader;
s = new_table->CreateIndexReader(ro, prefetch_buffer, meta_iter, use_cache,
@ -1015,24 +1051,20 @@ Status BlockBasedTable::PrefetchIndexAndFilterBlocks(
// The partitions of partitioned index are always stored in cache. They
// are hence follow the configuration for pin and prefetch regardless of
// the value of cache_index_and_filter_blocks
if (prefetch_all) {
s = rep_->index_reader->CacheDependencies(ro, pin_all);
if (prefetch_all || pin_partition) {
s = rep_->index_reader->CacheDependencies(ro, pin_partition);
}
if (!s.ok()) {
return s;
}
// prefetch the first level of filter
const bool prefetch_filter =
prefetch_all ||
(table_options.pin_top_level_index_and_filter &&
rep_->filter_type == Rep::FilterType::kPartitionedFilter);
// Partition fitlers cannot be enabled without partition indexes
assert(!prefetch_filter || prefetch_index);
// pin the first level of filter
const bool pin_filter =
pin_all || (table_options.pin_top_level_index_and_filter &&
rep_->filter_type == Rep::FilterType::kPartitionedFilter);
rep_->filter_type == Rep::FilterType::kPartitionedFilter
? pin_top_level_index
: pin_unpartitioned;
// prefetch the first level of filter
const bool prefetch_filter = prefetch_all || pin_filter;
if (rep_->filter_policy) {
auto filter = new_table->CreateFilterBlockReader(
@ -1040,8 +1072,8 @@ Status BlockBasedTable::PrefetchIndexAndFilterBlocks(
lookup_context);
if (filter) {
// Refer to the comment above about paritioned indexes always being cached
if (prefetch_all) {
filter->CacheDependencies(ro, pin_all);
if (prefetch_all || pin_partition) {
filter->CacheDependencies(ro, pin_partition);
}
rep_->filter = std::move(filter);
@ -1050,9 +1082,9 @@ Status BlockBasedTable::PrefetchIndexAndFilterBlocks(
if (!rep_->compression_dict_handle.IsNull()) {
std::unique_ptr<UncompressionDictReader> uncompression_dict_reader;
s = UncompressionDictReader::Create(this, ro, prefetch_buffer, use_cache,
prefetch_all, pin_all, lookup_context,
&uncompression_dict_reader);
s = UncompressionDictReader::Create(
this, ro, prefetch_buffer, use_cache, prefetch_all || pin_unpartitioned,
pin_unpartitioned, lookup_context, &uncompression_dict_reader);
if (!s.ok()) {
return s;
}
@ -1990,6 +2022,7 @@ BlockBasedTable::PartitionedIndexIteratorState::NewSecondaryIterator(
rep->index_value_is_full);
}
// Create an empty iterator
// TODO(ajkr): this is not the right way to handle an unpinned partition.
return new IndexBlockIter();
}

@ -173,7 +173,7 @@ Status PartitionIndexReader::CacheDependencies(const ReadOptions& ro,
return s;
}
if (block.GetValue() != nullptr) {
if (block.IsCached()) {
if (block.IsCached() || block.GetOwnValue()) {
if (pin) {
partition_map_[handle.offset()] = std::move(block);
}

@ -149,6 +149,11 @@ class IteratorWrapperBase {
return result_.value_prepared;
}
Slice user_key() const {
assert(Valid());
return iter_->user_key();
}
private:
void Update() {
valid_ = iter_->Valid();

@ -43,6 +43,10 @@ class TwoLevelIndexIterator : public InternalIteratorBase<IndexValue> {
assert(Valid());
return second_level_iter_.key();
}
Slice user_key() const override {
assert(Valid());
return second_level_iter_.user_key();
}
IndexValue value() const override {
assert(Valid());
return second_level_iter_.value();

@ -80,6 +80,7 @@ default_params = {
"open_files": lambda : random.choice([-1, -1, 100, 500000]),
"optimize_filters_for_memory": lambda: random.randint(0, 1),
"partition_filters": lambda: random.randint(0, 1),
"partition_pinning": lambda: random.randint(0, 3),
"pause_background_one_in": 1000000,
"prefixpercent": 5,
"progress_reports": 0,
@ -93,6 +94,8 @@ default_params = {
"subcompactions": lambda: random.randint(1, 4),
"target_file_size_base": 2097152,
"target_file_size_multiplier": 2,
"top_level_index_pinning": lambda: random.randint(0, 3),
"unpartitioned_pinning": lambda: random.randint(0, 3),
"use_direct_reads": lambda: random.randint(0, 1),
"use_direct_io_for_flush_and_compaction": lambda: random.randint(0, 1),
"mock_direct_io": False,

Loading…
Cancel
Save