Exclude timestamp from prefix extractor (#7668)

Summary:
Timestamp should not be included in prefix extractor, as we discussed here: https://github.com/facebook/rocksdb/pull/7589#discussion_r511068586

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

Test Plan: added unittest

Reviewed By: riversand963

Differential Revision: D24966265

Pulled By: jay-zhuang

fbshipit-source-id: 0dae618c333d4b7942a40d556535a1795e060aea
main
Jay Zhuang 4 years ago committed by Facebook GitHub Bot
parent b937be3779
commit 9e1640403a
  1. 1
      HISTORY.md
  2. 19
      db/db_iter.cc
  3. 221
      db/db_with_timestamp_basic_test.cc
  4. 42
      db/memtable.cc
  5. 8
      table/block_based/block_based_filter_block.cc
  6. 2
      table/block_based/block_based_filter_block.h
  7. 2
      table/block_based/block_based_table_iterator.h
  8. 15
      table/block_based/block_based_table_reader.cc
  9. 7
      table/block_based/filter_block.h
  10. 32
      table/block_based/full_filter_block.cc
  11. 2
      table/block_based/full_filter_block.h

@ -8,6 +8,7 @@
### Bug Fixes
* Fixed the logic of populating native data structure for `read_amp_bytes_per_bit` during OPTIONS file parsing on big-endian architecture. Without this fix, original code introduced in PR7659, when running on big-endian machine, can mistakenly store read_amp_bytes_per_bit (an uint32) in little endian format. Future access to `read_amp_bytes_per_bit` will give wrong values. Little endian architecture is not affected.
* Fixed prefix extractor with timestamp issues.
## 6.15.0 (11/13/2020)
### Bug Fixes

@ -221,25 +221,28 @@ bool DBIter::FindNextUserEntryInternal(bool skipping_saved_key,
is_key_seqnum_zero_ = false;
return false;
}
Slice user_key_without_ts =
StripTimestampFromUserKey(ikey_.user_key, timestamp_size_);
is_key_seqnum_zero_ = (ikey_.sequence == 0);
assert(iterate_upper_bound_ == nullptr ||
iter_.UpperBoundCheckResult() != IterBoundCheck::kInbound ||
user_comparator_.CompareWithoutTimestamp(
ikey_.user_key, /*a_has_ts=*/true, *iterate_upper_bound_,
user_key_without_ts, /*a_has_ts=*/false, *iterate_upper_bound_,
/*b_has_ts=*/false) < 0);
if (iterate_upper_bound_ != nullptr &&
iter_.UpperBoundCheckResult() != IterBoundCheck::kInbound &&
user_comparator_.CompareWithoutTimestamp(
ikey_.user_key, /*a_has_ts=*/true, *iterate_upper_bound_,
user_key_without_ts, /*a_has_ts=*/false, *iterate_upper_bound_,
/*b_has_ts=*/false) >= 0) {
break;
}
assert(prefix == nullptr || prefix_extractor_ != nullptr);
if (prefix != nullptr &&
prefix_extractor_->Transform(ikey_.user_key).compare(*prefix) != 0) {
prefix_extractor_->Transform(user_key_without_ts).compare(*prefix) !=
0) {
assert(prefix_same_as_start_);
break;
}
@ -704,7 +707,9 @@ void DBIter::PrevInternal(const Slice* prefix) {
assert(prefix == nullptr || prefix_extractor_ != nullptr);
if (prefix != nullptr &&
prefix_extractor_->Transform(saved_key_.GetUserKey())
prefix_extractor_
->Transform(StripTimestampFromUserKey(saved_key_.GetUserKey(),
timestamp_size_))
.compare(*prefix) != 0) {
assert(prefix_same_as_start_);
// Current key does not have the same prefix as start
@ -1399,7 +1404,8 @@ void DBIter::SeekToFirst() {
}
if (valid_ && prefix_same_as_start_) {
assert(prefix_extractor_ != nullptr);
prefix_.SetUserKey(prefix_extractor_->Transform(saved_key_.GetUserKey()));
prefix_.SetUserKey(prefix_extractor_->Transform(
StripTimestampFromUserKey(saved_key_.GetUserKey(), timestamp_size_)));
}
}
@ -1450,7 +1456,8 @@ void DBIter::SeekToLast() {
}
if (valid_ && prefix_same_as_start_) {
assert(prefix_extractor_ != nullptr);
prefix_.SetUserKey(prefix_extractor_->Transform(saved_key_.GetUserKey()));
prefix_.SetUserKey(prefix_extractor_->Transform(
StripTimestampFromUserKey(saved_key_.GetUserKey(), timestamp_size_)));
}
}

@ -32,6 +32,22 @@ class DBBasicTestWithTimestampBase : public DBTestBase {
return ret;
}
static std::string KeyWithPrefix(std::string prefix, uint64_t k) {
std::string ret;
PutFixed64(&ret, k);
std::reverse(ret.begin(), ret.end());
return prefix + ret;
}
static std::vector<Slice> ConvertStrToSlice(
std::vector<std::string>& strings) {
std::vector<Slice> ret;
for (const auto& s : strings) {
ret.emplace_back(s);
}
return ret;
}
class TestComparator : public Comparator {
private:
const Comparator* cmp_without_ts_;
@ -249,6 +265,8 @@ TEST_F(DBBasicTestWithTimestamp, SeekWithPrefixLessThanKey) {
options.env = env_;
options.create_if_missing = true;
options.prefix_extractor.reset(NewFixedPrefixTransform(3));
options.memtable_whole_key_filtering = true;
options.memtable_prefix_bloom_size_ratio = 0.1;
BlockBasedTableOptions bbto;
bbto.filter_policy.reset(NewBloomFilterPolicy(10, false));
bbto.cache_index_and_filter_blocks = true;
@ -285,6 +303,10 @@ TEST_F(DBBasicTestWithTimestamp, SeekWithPrefixLessThanKey) {
iter->Seek("foo");
ASSERT_TRUE(iter->Valid());
ASSERT_OK(iter->status());
iter->Next();
ASSERT_TRUE(iter->Valid());
ASSERT_OK(iter->status());
iter->Seek("bbb");
ASSERT_FALSE(iter->Valid());
ASSERT_OK(iter->status());
@ -298,6 +320,8 @@ TEST_F(DBBasicTestWithTimestamp, SeekWithPrefixLargerThanKey) {
options.env = env_;
options.create_if_missing = true;
options.prefix_extractor.reset(NewFixedPrefixTransform(20));
options.memtable_whole_key_filtering = true;
options.memtable_prefix_bloom_size_ratio = 0.1;
BlockBasedTableOptions bbto;
bbto.filter_policy.reset(NewBloomFilterPolicy(10, false));
bbto.cache_index_and_filter_blocks = true;
@ -329,6 +353,61 @@ TEST_F(DBBasicTestWithTimestamp, SeekWithPrefixLargerThanKey) {
std::string read_ts = Timestamp(2, 0);
ts = read_ts;
read_opts.timestamp = &ts;
{
std::unique_ptr<Iterator> iter(db_->NewIterator(read_opts));
// Make sure the prefix extractor doesn't include timestamp, otherwise it
// may return invalid result.
iter->Seek("foo");
ASSERT_TRUE(iter->Valid());
ASSERT_OK(iter->status());
iter->Next();
ASSERT_TRUE(iter->Valid());
ASSERT_OK(iter->status());
}
Close();
}
TEST_F(DBBasicTestWithTimestamp, SeekWithBound) {
Options options = CurrentOptions();
options.env = env_;
options.create_if_missing = true;
options.prefix_extractor.reset(NewFixedPrefixTransform(2));
BlockBasedTableOptions bbto;
bbto.filter_policy.reset(NewBloomFilterPolicy(10, false));
bbto.cache_index_and_filter_blocks = true;
bbto.whole_key_filtering = true;
options.table_factory.reset(NewBlockBasedTableFactory(bbto));
const size_t kTimestampSize = Timestamp(0, 0).size();
TestComparator test_cmp(kTimestampSize);
options.comparator = &test_cmp;
DestroyAndReopen(options);
WriteOptions write_opts;
std::string ts_str = Timestamp(1, 0);
Slice ts = ts_str;
write_opts.timestamp = &ts;
ASSERT_OK(db_->Put(write_opts, "foo1", "bar"));
ASSERT_OK(Flush());
ASSERT_OK(db_->Put(write_opts, "foo2", "bar"));
ASSERT_OK(Flush());
// Move sst file to next level
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
ASSERT_OK(db_->Put(write_opts, "foo3", "bar"));
ASSERT_OK(Flush());
ReadOptions read_opts;
std::string read_ts = Timestamp(2, 0);
ts = read_ts;
read_opts.timestamp = &ts;
std::string up_bound = "foo5";
Slice up_bound_slice = up_bound;
read_opts.iterate_upper_bound = &up_bound_slice;
read_opts.auto_prefix_mode = true;
{
std::unique_ptr<Iterator> iter(db_->NewIterator(read_opts));
// Make sure the prefix extractor doesn't include timestamp, otherwise it
@ -611,7 +690,7 @@ TEST_F(DBBasicTestWithTimestamp, MultiGetWithFastLocalBloom) {
ASSERT_OK(db_->Put(write_opts, "foo", "bar"));
Flush();
ASSERT_OK(Flush());
// Read with MultiGet
ReadOptions read_opts;
@ -652,7 +731,7 @@ TEST_F(DBBasicTestWithTimestamp, MultiGetWithPrefix) {
ASSERT_OK(db_->Put(write_opts, "foo", "bar"));
Flush();
ASSERT_OK(Flush());
// Read with MultiGet
ReadOptions read_opts;
@ -740,7 +819,7 @@ TEST_F(DBBasicTestWithTimestamp, MultiGetRangeFiltering) {
Slice key_slice = key;
Slice value_slice = value;
ASSERT_OK(db_->Put(write_opts, key_slice, value_slice));
Flush();
ASSERT_OK(Flush());
}
// Make num_levels to 2 to do key range filtering of sst files
@ -748,7 +827,7 @@ TEST_F(DBBasicTestWithTimestamp, MultiGetRangeFiltering) {
ASSERT_OK(db_->Put(write_opts, "foo", "bar"));
Flush();
ASSERT_OK(Flush());
// Read with MultiGet
ts_str = Timestamp(2, 0);
@ -791,7 +870,7 @@ TEST_F(DBBasicTestWithTimestamp, MultiGetPrefixFilter) {
ASSERT_OK(db_->Put(write_opts, "foo", "bar"));
Flush();
ASSERT_OK(Flush());
// Read with MultiGet
ts_str = Timestamp(2, 0);
ts = ts_str;
@ -909,6 +988,131 @@ TEST_F(DBBasicTestWithTimestamp, CompactDeletionWithTimestampMarkerToBottom) {
Close();
}
class DBBasicTestWithTimestampFilterPrefixSettings
: public DBBasicTestWithTimestampBase,
public testing::WithParamInterface<
std::tuple<std::shared_ptr<const FilterPolicy>, bool, bool,
std::shared_ptr<const SliceTransform>, bool, double>> {
public:
DBBasicTestWithTimestampFilterPrefixSettings()
: DBBasicTestWithTimestampBase(
"db_basic_test_with_timestamp_filter_prefix") {}
};
TEST_P(DBBasicTestWithTimestampFilterPrefixSettings, GetAndMultiGet) {
Options options = CurrentOptions();
options.env = env_;
options.create_if_missing = true;
BlockBasedTableOptions bbto;
bbto.filter_policy = std::get<0>(GetParam());
bbto.whole_key_filtering = std::get<1>(GetParam());
bbto.cache_index_and_filter_blocks = std::get<2>(GetParam());
options.table_factory.reset(NewBlockBasedTableFactory(bbto));
options.prefix_extractor = std::get<3>(GetParam());
options.memtable_whole_key_filtering = std::get<4>(GetParam());
options.memtable_prefix_bloom_size_ratio = std::get<5>(GetParam());
const size_t kTimestampSize = Timestamp(0, 0).size();
TestComparator test_cmp(kTimestampSize);
options.comparator = &test_cmp;
DestroyAndReopen(options);
const int kMaxKey = 1000;
// Write any value
WriteOptions write_opts;
std::string ts_str = Timestamp(1, 0);
Slice ts = ts_str;
write_opts.timestamp = &ts;
int idx = 0;
for (; idx < kMaxKey / 4; idx++) {
ASSERT_OK(db_->Put(write_opts, Key1(idx), "bar"));
ASSERT_OK(db_->Put(write_opts, KeyWithPrefix("foo", idx), "bar"));
}
ASSERT_OK(Flush());
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
for (; idx < kMaxKey / 2; idx++) {
ASSERT_OK(db_->Put(write_opts, Key1(idx), "bar"));
ASSERT_OK(db_->Put(write_opts, KeyWithPrefix("foo", idx), "bar"));
}
ASSERT_OK(Flush());
for (; idx < kMaxKey; idx++) {
ASSERT_OK(db_->Put(write_opts, Key1(idx), "bar"));
ASSERT_OK(db_->Put(write_opts, KeyWithPrefix("foo", idx), "bar"));
}
// Read with MultiGet
ReadOptions read_opts;
read_opts.timestamp = &ts;
ReadOptions read_opts_total_order;
read_opts_total_order.timestamp = &ts;
read_opts_total_order.total_order_seek = true;
for (idx = 0; idx < kMaxKey; idx++) {
size_t batch_size = 4;
std::vector<std::string> keys_str(batch_size);
std::vector<PinnableSlice> values(batch_size);
std::vector<Status> statuses(batch_size);
ColumnFamilyHandle* cfh = db_->DefaultColumnFamily();
keys_str[0] = Key1(idx);
keys_str[1] = KeyWithPrefix("foo", idx);
keys_str[2] = Key1(kMaxKey + idx);
keys_str[3] = KeyWithPrefix("foo", kMaxKey + idx);
auto keys = ConvertStrToSlice(keys_str);
db_->MultiGet(read_opts, cfh, batch_size, keys.data(), values.data(),
statuses.data());
for (int i = 0; i < 2; i++) {
ASSERT_OK(statuses[i]);
}
for (int i = 2; i < 4; i++) {
ASSERT_TRUE(statuses[i].IsNotFound());
}
for (int i = 0; i < 2; i++) {
std::string value;
ASSERT_OK(db_->Get(read_opts, keys[i], &value));
std::unique_ptr<Iterator> it1(db_->NewIterator(read_opts));
ASSERT_NE(nullptr, it1);
ASSERT_OK(it1->status());
// TODO(zjay) Fix seek with prefix
// it1->Seek(keys[i]);
// ASSERT_TRUE(it1->Valid());
}
for (int i = 2; i < 4; i++) {
std::string value;
Status s = db_->Get(read_opts, keys[i], &value);
ASSERT_TRUE(s.IsNotFound());
}
}
Close();
}
INSTANTIATE_TEST_CASE_P(
Timestamp, DBBasicTestWithTimestampFilterPrefixSettings,
::testing::Combine(
::testing::Values(
std::shared_ptr<const FilterPolicy>(nullptr),
std::shared_ptr<const FilterPolicy>(NewBloomFilterPolicy(10, true)),
std::shared_ptr<const FilterPolicy>(NewBloomFilterPolicy(10,
false))),
::testing::Bool(), ::testing::Bool(),
::testing::Values(
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(1)),
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(4)),
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(7)),
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(8))),
::testing::Bool(), ::testing::Values(0, 0.1)));
class DataVisibilityTest : public DBBasicTestWithTimestampBase {
public:
DataVisibilityTest() : DBBasicTestWithTimestampBase("data_visibility_test") {
@ -1372,7 +1576,7 @@ TEST_F(DataVisibilityTest, MultiGetWithTimestamp) {
VerifyDefaultCF(snap0);
VerifyDefaultCF(snap1);
Flush();
ASSERT_OK(Flush());
const Snapshot* snap2 = db_->GetSnapshot();
PutTestData(2);
@ -1458,7 +1662,7 @@ TEST_F(DataVisibilityTest, MultiGetCrossCF) {
VerifyDefaultCF(snap0);
VerifyDefaultCF(snap1);
Flush();
ASSERT_OK(Flush());
const Snapshot* snap2 = db_->GetSnapshot();
PutTestData(2);
@ -1839,6 +2043,8 @@ TEST_F(DBBasicTestWithTimestamp, BatchWriteAndMultiGet) {
options.create_if_missing = true;
options.env = env_;
options.memtable_factory.reset(new SpecialSkipListFactory(kNumKeysPerFile));
options.memtable_prefix_bloom_size_ratio = 0.1;
options.memtable_whole_key_filtering = true;
size_t ts_sz = Timestamp(0, 0).size();
TestComparator test_cmp(ts_sz);
@ -2093,6 +2299,7 @@ INSTANTIATE_TEST_CASE_P(
Timestamp, DBBasicTestWithTimestampPrefixSeek,
::testing::Combine(
::testing::Values(
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(1)),
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(4)),
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(7)),
std::shared_ptr<const SliceTransform>(NewFixedPrefixTransform(8))),

@ -328,9 +328,11 @@ class MemTableIterator : public InternalIterator {
PERF_COUNTER_ADD(seek_on_memtable_count, 1);
if (bloom_) {
// iterator should only use prefix bloom filter
Slice user_k(ExtractUserKey(k));
if (prefix_extractor_->InDomain(user_k) &&
!bloom_->MayContain(prefix_extractor_->Transform(user_k))) {
auto ts_sz = comparator_.comparator.user_comparator()->timestamp_size();
Slice user_k_without_ts(ExtractUserKeyAndStripTimestamp(k, ts_sz));
if (prefix_extractor_->InDomain(user_k_without_ts) &&
!bloom_->MayContain(
prefix_extractor_->Transform(user_k_without_ts))) {
PERF_COUNTER_ADD(bloom_memtable_miss_count, 1);
valid_ = false;
return;
@ -345,9 +347,11 @@ class MemTableIterator : public InternalIterator {
PERF_TIMER_GUARD(seek_on_memtable_time);
PERF_COUNTER_ADD(seek_on_memtable_count, 1);
if (bloom_) {
Slice user_k(ExtractUserKey(k));
if (prefix_extractor_->InDomain(user_k) &&
!bloom_->MayContain(prefix_extractor_->Transform(user_k))) {
auto ts_sz = comparator_.comparator.user_comparator()->timestamp_size();
Slice user_k_without_ts(ExtractUserKeyAndStripTimestamp(k, ts_sz));
if (prefix_extractor_->InDomain(user_k_without_ts) &&
!bloom_->MayContain(
prefix_extractor_->Transform(user_k_without_ts))) {
PERF_COUNTER_ADD(bloom_memtable_miss_count, 1);
valid_ = false;
return;
@ -511,6 +515,7 @@ Status MemTable::Add(SequenceNumber s, ValueType type,
memcpy(p, value.data(), val_size);
assert((unsigned)(p + val_size - buf) == (unsigned)encoded_len);
size_t ts_sz = GetInternalKeyComparator().user_comparator()->timestamp_size();
Slice key_without_ts = StripTimestampFromUserKey(key, ts_sz);
if (!allow_concurrent) {
// Extract prefix for insert with hint.
@ -540,12 +545,11 @@ Status MemTable::Add(SequenceNumber s, ValueType type,
}
if (bloom_filter_ && prefix_extractor_ &&
prefix_extractor_->InDomain(key)) {
bloom_filter_->Add(
prefix_extractor_->Transform(StripTimestampFromUserKey(key, ts_sz)));
prefix_extractor_->InDomain(key_without_ts)) {
bloom_filter_->Add(prefix_extractor_->Transform(key_without_ts));
}
if (bloom_filter_ && moptions_.memtable_whole_key_filtering) {
bloom_filter_->Add(StripTimestampFromUserKey(key, ts_sz));
bloom_filter_->Add(key_without_ts);
}
// The first sequence number inserted into the memtable
@ -577,11 +581,12 @@ Status MemTable::Add(SequenceNumber s, ValueType type,
}
if (bloom_filter_ && prefix_extractor_ &&
prefix_extractor_->InDomain(key)) {
bloom_filter_->AddConcurrently(prefix_extractor_->Transform(key));
prefix_extractor_->InDomain(key_without_ts)) {
bloom_filter_->AddConcurrently(
prefix_extractor_->Transform(key_without_ts));
}
if (bloom_filter_ && moptions_.memtable_whole_key_filtering) {
bloom_filter_->AddConcurrently(StripTimestampFromUserKey(key, ts_sz));
bloom_filter_->AddConcurrently(key_without_ts);
}
// atomically update first_seqno_ and earliest_seqno_.
@ -821,22 +826,21 @@ bool MemTable::Get(const LookupKey& key, std::string* value,
range_del_iter->MaxCoveringTombstoneSeqnum(key.user_key()));
}
Slice user_key = key.user_key();
bool found_final_value = false;
bool merge_in_progress = s->IsMergeInProgress();
bool may_contain = true;
size_t ts_sz = GetInternalKeyComparator().user_comparator()->timestamp_size();
Slice user_key_without_ts = StripTimestampFromUserKey(key.user_key(), ts_sz);
if (bloom_filter_) {
// when both memtable_whole_key_filtering and prefix_extractor_ are set,
// only do whole key filtering for Get() to save CPU
if (moptions_.memtable_whole_key_filtering) {
may_contain =
bloom_filter_->MayContain(StripTimestampFromUserKey(user_key, ts_sz));
may_contain = bloom_filter_->MayContain(user_key_without_ts);
} else {
assert(prefix_extractor_);
may_contain =
!prefix_extractor_->InDomain(user_key) ||
bloom_filter_->MayContain(prefix_extractor_->Transform(user_key));
may_contain = !prefix_extractor_->InDomain(user_key_without_ts) ||
bloom_filter_->MayContain(
prefix_extractor_->Transform(user_key_without_ts));
}
}

@ -80,13 +80,13 @@ void BlockBasedFilterBlockBuilder::StartBlock(uint64_t block_offset) {
}
}
void BlockBasedFilterBlockBuilder::Add(const Slice& key) {
if (prefix_extractor_ && prefix_extractor_->InDomain(key)) {
AddPrefix(key);
void BlockBasedFilterBlockBuilder::Add(const Slice& key_without_ts) {
if (prefix_extractor_ && prefix_extractor_->InDomain(key_without_ts)) {
AddPrefix(key_without_ts);
}
if (whole_key_filtering_) {
AddKey(key);
AddKey(key_without_ts);
}
}

@ -44,7 +44,7 @@ class BlockBasedFilterBlockBuilder : public FilterBlockBuilder {
virtual bool IsBlockBased() override { return true; }
virtual void StartBlock(uint64_t block_offset) override;
virtual void Add(const Slice& key) override;
virtual void Add(const Slice& key_without_ts) override;
virtual size_t NumAdded() const override { return num_added_; }
virtual Slice Finish(const BlockHandle& tmp, Status* status) override;
using FilterBlockBuilder::Finish;

@ -230,7 +230,7 @@ class BlockBasedTableIterator : public InternalIteratorBase<Slice> {
bool CheckPrefixMayMatch(const Slice& ikey, IterDirection direction) {
if (need_upper_bound_check_ && direction == IterDirection::kBackward) {
// Upper bound check isn't sufficnet for backward direction to
// Upper bound check isn't sufficient for backward direction to
// guarantee the same result as total order, so disable prefix
// check.
return true;

@ -2064,8 +2064,10 @@ bool BlockBasedTable::PrefixMayMatch(
} else {
prefix_extractor = rep_->table_prefix_extractor.get();
}
auto user_key = ExtractUserKey(internal_key);
if (!prefix_extractor->InDomain(user_key)) {
auto ts_sz = rep_->internal_comparator.user_comparator()->timestamp_size();
auto user_key_without_ts =
ExtractUserKeyAndStripTimestamp(internal_key, ts_sz);
if (!prefix_extractor->InDomain(user_key_without_ts)) {
return true;
}
@ -2080,15 +2082,16 @@ bool BlockBasedTable::PrefixMayMatch(
if (!filter->IsBlockBased()) {
const Slice* const const_ikey_ptr = &internal_key;
may_match = filter->RangeMayExist(
read_options.iterate_upper_bound, user_key, prefix_extractor,
rep_->internal_comparator.user_comparator(), const_ikey_ptr,
&filter_checked, need_upper_bound_check, no_io, lookup_context);
read_options.iterate_upper_bound, user_key_without_ts,
prefix_extractor, rep_->internal_comparator.user_comparator(),
const_ikey_ptr, &filter_checked, need_upper_bound_check, no_io,
lookup_context);
} else {
// if prefix_extractor changed for block based filter, skip filter
if (need_upper_bound_check) {
return true;
}
auto prefix = prefix_extractor->Transform(user_key);
auto prefix = prefix_extractor->Transform(user_key_without_ts);
InternalKey internal_key_prefix(prefix, kMaxSequenceNumber, kTypeValue);
auto internal_prefix = internal_key_prefix.Encode();

@ -60,7 +60,8 @@ class FilterBlockBuilder {
virtual bool IsBlockBased() = 0; // If is blockbased filter
virtual void StartBlock(uint64_t block_offset) = 0; // Start new block filter
virtual void Add(const Slice& key) = 0; // Add a key to current filter
virtual void Add(
const Slice& key_without_ts) = 0; // Add a key to current filter
virtual size_t NumAdded() const = 0; // Number of keys added
Slice Finish() { // Generate Filter
const BlockHandle empty_handle;
@ -158,7 +159,7 @@ class FilterBlockReader {
}
virtual bool RangeMayExist(const Slice* /*iterate_upper_bound*/,
const Slice& user_key,
const Slice& user_key_without_ts,
const SliceTransform* prefix_extractor,
const Comparator* /*comparator*/,
const Slice* const const_ikey_ptr,
@ -169,7 +170,7 @@ class FilterBlockReader {
return true;
}
*filter_checked = true;
Slice prefix = prefix_extractor->Transform(user_key);
Slice prefix = prefix_extractor->Transform(user_key_without_ts);
return PrefixMayMatch(prefix, prefix_extractor, kNotValid, no_io,
const_ikey_ptr, /* get_context */ nullptr,
lookup_context);

@ -27,26 +27,29 @@ FullFilterBlockBuilder::FullFilterBlockBuilder(
filter_bits_builder_.reset(filter_bits_builder);
}
void FullFilterBlockBuilder::Add(const Slice& key) {
const bool add_prefix = prefix_extractor_ && prefix_extractor_->InDomain(key);
void FullFilterBlockBuilder::Add(const Slice& key_without_ts) {
const bool add_prefix =
prefix_extractor_ && prefix_extractor_->InDomain(key_without_ts);
if (whole_key_filtering_) {
if (!add_prefix) {
AddKey(key);
AddKey(key_without_ts);
} else {
// if both whole_key and prefix are added to bloom then we will have whole
// key and prefix addition being interleaved and thus cannot rely on the
// bits builder to properly detect the duplicates by comparing with the
// last item.
// key_without_ts and prefix addition being interleaved and thus cannot
// rely on the bits builder to properly detect the duplicates by comparing
// with the last item.
Slice last_whole_key = Slice(last_whole_key_str_);
if (!last_whole_key_recorded_ || last_whole_key.compare(key) != 0) {
AddKey(key);
if (!last_whole_key_recorded_ ||
last_whole_key.compare(key_without_ts) != 0) {
AddKey(key_without_ts);
last_whole_key_recorded_ = true;
last_whole_key_str_.assign(key.data(), key.size());
last_whole_key_str_.assign(key_without_ts.data(),
key_without_ts.size());
}
}
}
if (add_prefix) {
AddPrefix(key);
AddPrefix(key_without_ts);
}
}
@ -283,16 +286,16 @@ size_t FullFilterBlockReader::ApproximateMemoryUsage() const {
}
bool FullFilterBlockReader::RangeMayExist(
const Slice* iterate_upper_bound, const Slice& user_key,
const Slice* iterate_upper_bound, const Slice& user_key_without_ts,
const SliceTransform* prefix_extractor, const Comparator* comparator,
const Slice* const const_ikey_ptr, bool* filter_checked,
bool need_upper_bound_check, bool no_io,
BlockCacheLookupContext* lookup_context) {
if (!prefix_extractor || !prefix_extractor->InDomain(user_key)) {
if (!prefix_extractor || !prefix_extractor->InDomain(user_key_without_ts)) {
*filter_checked = false;
return true;
}
Slice prefix = prefix_extractor->Transform(user_key);
Slice prefix = prefix_extractor->Transform(user_key_without_ts);
if (need_upper_bound_check &&
!IsFilterCompatible(iterate_upper_bound, prefix, comparator)) {
*filter_checked = false;
@ -318,7 +321,8 @@ bool FullFilterBlockReader::IsFilterCompatible(
}
Slice upper_bound_xform = prefix_extractor->Transform(*iterate_upper_bound);
// first check if user_key and upper_bound all share the same prefix
if (!comparator->Equal(prefix, upper_bound_xform)) {
if (comparator->CompareWithoutTimestamp(prefix, false, upper_bound_xform,
false) != 0) {
// second check if user_key's prefix is the immediate predecessor of
// upper_bound and have the same length. If so, we know for sure all
// keys in the range [user_key, upper_bound) share the same prefix.

@ -50,7 +50,7 @@ class FullFilterBlockBuilder : public FilterBlockBuilder {
virtual bool IsBlockBased() override { return false; }
virtual void StartBlock(uint64_t /*block_offset*/) override {}
virtual void Add(const Slice& key) override;
virtual void Add(const Slice& key_without_ts) override;
virtual size_t NumAdded() const override { return num_added_; }
virtual Slice Finish(const BlockHandle& tmp, Status* status) override;
using FilterBlockBuilder::Finish;

Loading…
Cancel
Save