Avoid per-key upper bound check in BlockBasedTableIterator (#5142)

Summary:
This is second attempt for #5101. Original commit message:
`BlockBasedTableIterator` avoid reading next block on `Next()` if it detects the iterator will be out of bound, by checking against index key. The optimization was added in #2239, and by the time it only check the bound per block. It seems later change make it a per-key check, which introduce unnecessary key comparisons.

This patch come with two fixes:

Fix 1: To optimize checking for bounds, we need comparing the bounds with index key as well. However BlockBasedTableIterator doesn't know whether its index iterator is internally using user keys or internal keys. The patch fixes that by extending InternalIterator with a user_key() function that is overridden by In IndexBlockIter.

Fix 2: In #5101 we return `IsOutOfBound()=true` when block index key is out of bound. But the index key can be larger than smallest key of the next file on the level. That file can be within upper bound and should not be filtered out.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5142

Differential Revision: D14907113

Pulled By: siying

fbshipit-source-id: ac95775c5b4e7b700f76ab43e39f45402c98fbfb
This commit is contained in:
yiwu-arbug 2019-04-16 11:32:03 -07:00 committed by Facebook Github Bot
parent 71a82a0abe
commit f1239d5f10
11 changed files with 220 additions and 95 deletions

View File

@ -4,6 +4,7 @@
### New Features
* When reading from option file/string/map, customized comparators and/or merge operators can be filled according to object registry.
* Introduce Periodic Compaction for Level style compaction. Files are re-compacted periodically and put in the same level.
* Improve range scan performance by avoiding per-key upper bound check in BlockBasedTableIterator.
### Public API Change
* Change the behavior of OptimizeForPointLookup(): move away from hash-based block-based-table index, and use whole key memtable filtering.
@ -15,6 +16,7 @@
* Fix a race condition between WritePrepared::Get and ::Put with duplicate keys.
* Fix crash when memtable prefix bloom is enabled and read/write a key out of domain of prefix extractor.
## 6.1.0 (3/27/2019)
### New Features
* Introduce two more stats levels, kExceptHistogramOrTimers and kExceptTimers.

View File

@ -15,6 +15,7 @@
#include "port/stack_trace.h"
#include "rocksdb/iostats_context.h"
#include "rocksdb/perf_context.h"
#include "table/flush_block_policy.h"
namespace rocksdb {
@ -59,35 +60,6 @@ class DBIteratorTest : public DBTestBase,
std::vector<std::unique_ptr<DummyReadCallback>> read_callbacks_;
};
class FlushBlockEveryKeyPolicy : public FlushBlockPolicy {
public:
bool Update(const Slice& /*key*/, const Slice& /*value*/) override {
if (!start_) {
start_ = true;
return false;
}
return true;
}
private:
bool start_ = false;
};
class FlushBlockEveryKeyPolicyFactory : public FlushBlockPolicyFactory {
public:
explicit FlushBlockEveryKeyPolicyFactory() {}
const char* Name() const override {
return "FlushBlockEveryKeyPolicyFactory";
}
FlushBlockPolicy* NewFlushBlockPolicy(
const BlockBasedTableOptions& /*table_options*/,
const BlockBuilder& /*data_block_builder*/) const override {
return new FlushBlockEveryKeyPolicy;
}
};
TEST_P(DBIteratorTest, IteratorProperty) {
// The test needs to be changed if kPersistedTier is supported in iterator.
Options options = CurrentOptions();
@ -1034,48 +1006,48 @@ TEST_P(DBIteratorTest, DBIteratorBoundMultiSeek) {
#endif
TEST_P(DBIteratorTest, DBIteratorBoundOptimizationTest) {
int upper_bound_hits = 0;
Options options = CurrentOptions();
rocksdb::SyncPoint::GetInstance()->SetCallBack(
"BlockBasedTable::BlockEntryIteratorState::KeyReachedUpperBound",
[&upper_bound_hits](void* arg) {
assert(arg != nullptr);
upper_bound_hits += (*static_cast<bool*>(arg) ? 1 : 0);
});
rocksdb::SyncPoint::GetInstance()->EnableProcessing();
options.env = env_;
options.create_if_missing = true;
options.prefix_extractor = nullptr;
BlockBasedTableOptions table_options;
table_options.flush_block_policy_factory =
std::make_shared<FlushBlockEveryKeyPolicyFactory>();
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
for (auto format_version : {2, 3, 4}) {
int upper_bound_hits = 0;
Options options = CurrentOptions();
rocksdb::SyncPoint::GetInstance()->SetCallBack(
"BlockBasedTableIterator:out_of_bound",
[&upper_bound_hits](void*) { upper_bound_hits++; });
rocksdb::SyncPoint::GetInstance()->EnableProcessing();
options.env = env_;
options.create_if_missing = true;
options.prefix_extractor = nullptr;
BlockBasedTableOptions table_options;
table_options.format_version = format_version;
table_options.flush_block_policy_factory =
std::make_shared<FlushBlockEveryKeyPolicyFactory>();
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
DestroyAndReopen(options);
ASSERT_OK(Put("foo1", "bar1"));
ASSERT_OK(Put("foo2", "bar2"));
ASSERT_OK(Put("foo4", "bar4"));
ASSERT_OK(Flush());
DestroyAndReopen(options);
ASSERT_OK(Put("foo1", "bar1"));
ASSERT_OK(Put("foo2", "bar2"));
ASSERT_OK(Put("foo4", "bar4"));
ASSERT_OK(Flush());
Slice ub("foo3");
ReadOptions ro;
ro.iterate_upper_bound = &ub;
Slice ub("foo3");
ReadOptions ro;
ro.iterate_upper_bound = &ub;
std::unique_ptr<Iterator> iter(NewIterator(ro));
std::unique_ptr<Iterator> iter(NewIterator(ro));
iter->Seek("foo");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ(iter->key().compare(Slice("foo1")), 0);
ASSERT_EQ(upper_bound_hits, 0);
iter->Seek("foo");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ(iter->key().compare(Slice("foo1")), 0);
ASSERT_EQ(upper_bound_hits, 0);
iter->Next();
ASSERT_TRUE(iter->Valid());
ASSERT_EQ(iter->key().compare(Slice("foo2")), 0);
ASSERT_EQ(upper_bound_hits, 0);
iter->Next();
ASSERT_TRUE(iter->Valid());
ASSERT_EQ(iter->key().compare(Slice("foo2")), 0);
ASSERT_EQ(upper_bound_hits, 0);
iter->Next();
ASSERT_FALSE(iter->Valid());
ASSERT_EQ(upper_bound_hits, 1);
iter->Next();
ASSERT_FALSE(iter->Valid());
ASSERT_EQ(upper_bound_hits, 1);
}
}
// TODO(3.13): fix the issue of Seek() + Prev() which might not necessary
// return the biggest key which is smaller than the seek key.

View File

@ -381,9 +381,9 @@ void ForwardIterator::SeekInternal(const Slice& internal_key,
}
}
Slice user_key;
Slice target_user_key;
if (!seek_to_first) {
user_key = ExtractUserKey(internal_key);
target_user_key = ExtractUserKey(internal_key);
}
const VersionStorageInfo* vstorage = sv_->current->storage_info();
const std::vector<FileMetaData*>& l0 = vstorage->LevelFiles(0);
@ -396,8 +396,8 @@ void ForwardIterator::SeekInternal(const Slice& internal_key,
} else {
// If the target key passes over the larget key, we are sure Next()
// won't go over this file.
if (user_comparator_->Compare(user_key,
l0[i]->largest.user_key()) > 0) {
if (user_comparator_->Compare(target_user_key,
l0[i]->largest.user_key()) > 0) {
if (read_options_.iterate_upper_bound != nullptr) {
has_iter_trimmed_for_upper_bound_ = true;
DeleteIterator(l0_iters_[i]);

View File

@ -407,9 +407,10 @@ bool FragmentedRangeTombstoneIterator::Valid() const {
}
SequenceNumber FragmentedRangeTombstoneIterator::MaxCoveringTombstoneSeqnum(
const Slice& user_key) {
SeekToCoveringTombstone(user_key);
return ValidPos() && ucmp_->Compare(start_key(), user_key) <= 0 ? seq() : 0;
const Slice& target_user_key) {
SeekToCoveringTombstone(target_user_key);
return ValidPos() && ucmp_->Compare(start_key(), target_user_key) <= 0 ? seq()
: 0;
}
std::map<SequenceNumber, std::unique_ptr<FragmentedRangeTombstoneIterator>>

View File

@ -286,9 +286,10 @@ void DataBlockIter::Seek(const Slice& target) {
// with a smaller [ type | seqno ] (i.e. a larger seqno, or the same seqno
// but larger type).
bool DataBlockIter::SeekForGetImpl(const Slice& target) {
Slice user_key = ExtractUserKey(target);
Slice target_user_key = ExtractUserKey(target);
uint32_t map_offset = restarts_ + num_restarts_ * sizeof(uint32_t);
uint8_t entry = data_block_hash_index_->Lookup(data_, map_offset, user_key);
uint8_t entry =
data_block_hash_index_->Lookup(data_, map_offset, target_user_key);
if (entry == kCollision) {
// HashSeek not effective, falling back
@ -360,7 +361,7 @@ bool DataBlockIter::SeekForGetImpl(const Slice& target) {
return true;
}
if (user_comparator_->Compare(key_.GetUserKey(), user_key) != 0) {
if (user_comparator_->Compare(key_.GetUserKey(), target_user_key) != 0) {
// the key is not in this block and cannot be at the next block either.
return false;
}

View File

@ -496,6 +496,13 @@ class IndexBlockIter final : public BlockIter<BlockHandle> {
value_delta_encoded_ = !value_is_full;
}
Slice user_key() const override {
if (key_includes_seq_) {
return ExtractUserKey(key());
}
return key();
}
virtual BlockHandle value() const override {
assert(Valid());
if (value_delta_encoded_) {

View File

@ -2347,6 +2347,7 @@ void BlockBasedTableIterator<TBlockIter, TValue>::Seek(const Slice& target) {
block_iter_.Seek(target);
FindKeyForward();
CheckOutOfBound();
assert(
!block_iter_.Valid() ||
(key_includes_seq_ && icomp_.Compare(target, block_iter_.key()) <= 0) ||
@ -2410,6 +2411,7 @@ void BlockBasedTableIterator<TBlockIter, TValue>::SeekToFirst() {
InitDataBlock();
block_iter_.SeekToFirst();
FindKeyForward();
CheckOutOfBound();
}
template <class TBlockIter, typename TValue>
@ -2492,19 +2494,33 @@ void BlockBasedTableIterator<TBlockIter, TValue>::InitDataBlock() {
template <class TBlockIter, typename TValue>
void BlockBasedTableIterator<TBlockIter, TValue>::FindKeyForward() {
assert(!is_out_of_bound_);
// TODO the while loop inherits from two-level-iterator. We don't know
// whether a block can be empty so it can be replaced by an "if".
while (!block_iter_.Valid()) {
if (!block_iter_.status().ok()) {
return;
}
// Whether next data block is out of upper bound, if there is one.
bool next_block_is_out_of_bound = false;
if (read_options_.iterate_upper_bound != nullptr &&
block_iter_points_to_real_block_) {
next_block_is_out_of_bound =
(user_comparator_.Compare(*read_options_.iterate_upper_bound,
index_iter_->user_key()) <= 0);
}
ResetDataIter();
// We used to check the current index key for upperbound.
// It will only save a data reading for a small percentage of use cases,
// so for code simplicity, we removed it. We can add it back if there is a
// significnat performance regression.
index_iter_->Next();
if (next_block_is_out_of_bound) {
// The next block is out of bound. No need to read it.
TEST_SYNC_POINT_CALLBACK("BlockBasedTableIterator:out_of_bound", nullptr);
// We need to make sure this is not the last data block before setting
// is_out_of_bound_, since the index key for the last data block can be
// larger than smallest key of the next file on the same level.
if (index_iter_->Valid()) {
is_out_of_bound_ = true;
}
return;
}
if (index_iter_->Valid()) {
InitDataBlock();
@ -2513,25 +2529,10 @@ void BlockBasedTableIterator<TBlockIter, TValue>::FindKeyForward() {
return;
}
}
// Check upper bound on the current key
bool reached_upper_bound =
(read_options_.iterate_upper_bound != nullptr &&
block_iter_points_to_real_block_ && block_iter_.Valid() &&
user_comparator_.Compare(ExtractUserKey(block_iter_.key()),
*read_options_.iterate_upper_bound) >= 0);
TEST_SYNC_POINT_CALLBACK(
"BlockBasedTable::BlockEntryIteratorState::KeyReachedUpperBound",
&reached_upper_bound);
if (reached_upper_bound) {
is_out_of_bound_ = true;
return;
}
}
template <class TBlockIter, typename TValue>
void BlockBasedTableIterator<TBlockIter, TValue>::FindKeyBackward() {
assert(!is_out_of_bound_);
while (!block_iter_.Valid()) {
if (!block_iter_.status().ok()) {
return;
@ -2552,6 +2553,15 @@ void BlockBasedTableIterator<TBlockIter, TValue>::FindKeyBackward() {
// code simplicity.
}
template <class TBlockIter, typename TValue>
void BlockBasedTableIterator<TBlockIter, TValue>::CheckOutOfBound() {
if (read_options_.iterate_upper_bound != nullptr &&
block_iter_points_to_real_block_ && block_iter_.Valid()) {
is_out_of_bound_ = user_comparator_.Compare(
*read_options_.iterate_upper_bound, user_key()) <= 0;
}
}
InternalIterator* BlockBasedTable::NewIterator(
const ReadOptions& read_options, const SliceTransform* prefix_extractor,
Arena* arena, bool skip_filters, bool for_compaction) {

View File

@ -621,6 +621,10 @@ class BlockBasedTableIterator : public InternalIteratorBase<TValue> {
assert(Valid());
return block_iter_.key();
}
Slice user_key() const override {
assert(Valid());
return block_iter_.user_key();
}
TValue value() const override {
assert(Valid());
return block_iter_.value();
@ -635,6 +639,7 @@ class BlockBasedTableIterator : public InternalIteratorBase<TValue> {
}
}
// Whether iterator invalidated for being out of bound.
bool IsOutOfBound() override { return is_out_of_bound_; }
void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
@ -685,6 +690,7 @@ class BlockBasedTableIterator : public InternalIteratorBase<TValue> {
void InitDataBlock();
void FindKeyForward();
void FindKeyBackward();
void CheckOutOfBound();
private:
BlockBasedTable* table_;

View File

@ -0,0 +1,41 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#include "rocksdb/flush_block_policy.h"
namespace rocksdb {
// FlushBlockEveryKeyPolicy currently used only in tests.
class FlushBlockEveryKeyPolicy : public FlushBlockPolicy {
public:
bool Update(const Slice& /*key*/, const Slice& /*value*/) override {
if (!start_) {
start_ = true;
return false;
}
return true;
}
private:
bool start_ = false;
};
class FlushBlockEveryKeyPolicyFactory : public FlushBlockPolicyFactory {
public:
explicit FlushBlockEveryKeyPolicyFactory() {}
const char* Name() const override {
return "FlushBlockEveryKeyPolicyFactory";
}
FlushBlockPolicy* NewFlushBlockPolicy(
const BlockBasedTableOptions& /*table_options*/,
const BlockBuilder& /*data_block_builder*/) const override {
return new FlushBlockEveryKeyPolicy;
}
};
} // namespace rocksdb

View File

@ -7,6 +7,7 @@
#pragma once
#include <string>
#include "db/dbformat.h"
#include "rocksdb/comparator.h"
#include "rocksdb/iterator.h"
#include "rocksdb/status.h"
@ -64,6 +65,10 @@ class InternalIteratorBase : public Cleanable {
// REQUIRES: Valid()
virtual Slice key() const = 0;
// Return user key for the current entry.
// REQUIRES: Valid()
virtual Slice user_key() const { return ExtractUserKey(key()); }
// Return the value for the current entry. The underlying storage for
// the returned slice is valid only until the next modification of
// the iterator.

View File

@ -38,6 +38,7 @@
#include "table/block_based_table_reader.h"
#include "table/block_builder.h"
#include "table/block_fetcher.h"
#include "table/flush_block_policy.h"
#include "table/format.h"
#include "table/get_context.h"
#include "table/internal_iterator.h"
@ -276,6 +277,7 @@ class KeyConvertingIterator : public InternalIterator {
void SeekToLast() override { iter_->SeekToLast(); }
void Next() override { iter_->Next(); }
void Prev() override { iter_->Prev(); }
bool IsOutOfBound() override { return iter_->IsOutOfBound(); }
Slice key() const override {
assert(Valid());
@ -3871,6 +3873,84 @@ TEST_P(BlockBasedTableTest, DataBlockHashIndex) {
}
}
// BlockBasedTableIterator should invalidate itself and return
// OutOfBound()=true immediately after Seek(), to allow LevelIterator
// filter out corresponding level.
TEST_P(BlockBasedTableTest, OutOfBoundOnSeek) {
TableConstructor c(BytewiseComparator(), true /*convert_to_internal_key*/);
c.Add("foo", "v1");
std::vector<std::string> keys;
stl_wrappers::KVMap kvmap;
Options options;
BlockBasedTableOptions table_opt(GetBlockBasedTableOptions());
options.table_factory.reset(NewBlockBasedTableFactory(table_opt));
const ImmutableCFOptions ioptions(options);
const MutableCFOptions moptions(options);
c.Finish(options, ioptions, moptions, table_opt,
GetPlainInternalComparator(BytewiseComparator()), &keys, &kvmap);
auto* reader = c.GetTableReader();
ReadOptions read_opt;
std::string upper_bound = "bar";
Slice upper_bound_slice(upper_bound);
read_opt.iterate_upper_bound = &upper_bound_slice;
std::unique_ptr<InternalIterator> iter;
iter.reset(new KeyConvertingIterator(
reader->NewIterator(read_opt, nullptr /*prefix_extractor*/)));
iter->SeekToFirst();
ASSERT_FALSE(iter->Valid());
ASSERT_TRUE(iter->IsOutOfBound());
iter.reset(new KeyConvertingIterator(
reader->NewIterator(read_opt, nullptr /*prefix_extractor*/)));
iter->Seek("foo");
ASSERT_FALSE(iter->Valid());
ASSERT_TRUE(iter->IsOutOfBound());
}
// BlockBasedTableIterator should invalidate itself and return
// OutOfBound()=true after Next(), if it finds current index key is no smaller
// than upper bound, unless it is pointing to the last data block.
TEST_P(BlockBasedTableTest, OutOfBoundOnNext) {
TableConstructor c(BytewiseComparator(), true /*convert_to_internal_key*/);
c.Add("bar", "v");
c.Add("foo", "v");
std::vector<std::string> keys;
stl_wrappers::KVMap kvmap;
Options options;
BlockBasedTableOptions table_opt(GetBlockBasedTableOptions());
table_opt.flush_block_policy_factory =
std::make_shared<FlushBlockEveryKeyPolicyFactory>();
options.table_factory.reset(NewBlockBasedTableFactory(table_opt));
const ImmutableCFOptions ioptions(options);
const MutableCFOptions moptions(options);
c.Finish(options, ioptions, moptions, table_opt,
GetPlainInternalComparator(BytewiseComparator()), &keys, &kvmap);
auto* reader = c.GetTableReader();
ReadOptions read_opt;
std::string ub1 = "bar_after";
Slice ub_slice1(ub1);
read_opt.iterate_upper_bound = &ub_slice1;
std::unique_ptr<InternalIterator> iter;
iter.reset(new KeyConvertingIterator(
reader->NewIterator(read_opt, nullptr /*prefix_extractor*/)));
iter->Seek("bar");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ("bar", iter->key());
iter->Next();
ASSERT_FALSE(iter->Valid());
ASSERT_TRUE(iter->IsOutOfBound());
std::string ub2 = "foo_after";
Slice ub_slice2(ub2);
read_opt.iterate_upper_bound = &ub_slice2;
iter.reset(new KeyConvertingIterator(
reader->NewIterator(read_opt, nullptr /*prefix_extractor*/)));
iter->Seek("foo");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ("foo", iter->key());
iter->Next();
ASSERT_FALSE(iter->Valid());
ASSERT_FALSE(iter->IsOutOfBound());
}
} // namespace rocksdb
int main(int argc, char** argv) {