From f81ea75df75d9bfed572426f8fbd255e378d1651 Mon Sep 17 00:00:00 2001 From: Peter Dillinger Date: Thu, 23 Jun 2022 11:00:27 -0700 Subject: [PATCH] Don't count no prefix as Bloom hit (#10244) Summary: When a key is "out of domain" for the prefix_extractor (no prefix assigned) then the Bloom filter is not queried. PerfContext was counting this as a Bloom "hit" while Statistics doesn't count this as a prefix Bloom checked. I think it's more accurate to call it neither hit nor miss, so changing the counting to make it PerfContext coounting more like Statistics. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10244 Test Plan: tests updates and expanded (Get and MultiGet). Iterator test coverage of the change will come in next PR Reviewed By: bjlemaire Differential Revision: D37371297 Pulled By: pdillinger fbshipit-source-id: fed132fba6a92b2314ab898d449fce2d1586c157 --- db/db_basic_test.cc | 38 ++++++++++++++++++---------------- db/memtable.cc | 50 +++++++++++++++++++++++---------------------- 2 files changed, 46 insertions(+), 42 deletions(-) diff --git a/db/db_basic_test.cc b/db/db_basic_test.cc index 55fff5d7b..8e71a49c6 100644 --- a/db/db_basic_test.cc +++ b/db/db_basic_test.cc @@ -2467,35 +2467,37 @@ TEST_P(MultiGetPrefixExtractorTest, Batched) { SetPerfLevel(kEnableCount); get_perf_context()->Reset(); - // First key is not in the prefix_extractor domain ASSERT_OK(Put("k", "v0")); ASSERT_OK(Put("kk1", "v1")); ASSERT_OK(Put("kk2", "v2")); ASSERT_OK(Put("kk3", "v3")); ASSERT_OK(Put("kk4", "v4")); - std::vector mem_keys( + std::vector keys( {"k", "kk1", "kk2", "kk3", "kk4", "rofl", "lmho"}); - std::vector inmem_values; - inmem_values = MultiGet(mem_keys, nullptr); - ASSERT_EQ(inmem_values[0], "v0"); - ASSERT_EQ(inmem_values[1], "v1"); - ASSERT_EQ(inmem_values[2], "v2"); - ASSERT_EQ(inmem_values[3], "v3"); - ASSERT_EQ(inmem_values[4], "v4"); + std::vector expected( + {"v0", "v1", "v2", "v3", "v4", "NOT_FOUND", "NOT_FOUND"}); + std::vector values; + values = MultiGet(keys, nullptr); + ASSERT_EQ(values, expected); + // One key ("k") is not queried against the filter because it is outside + // the prefix_extractor domain, leaving 6 keys with queried prefixes. ASSERT_EQ(get_perf_context()->bloom_memtable_miss_count, 2); - ASSERT_EQ(get_perf_context()->bloom_memtable_hit_count, 5); + ASSERT_EQ(get_perf_context()->bloom_memtable_hit_count, 4); ASSERT_OK(Flush()); - std::vector keys({"k", "kk1", "kk2", "kk3", "kk4"}); - std::vector values; get_perf_context()->Reset(); values = MultiGet(keys, nullptr); - ASSERT_EQ(values[0], "v0"); - ASSERT_EQ(values[1], "v1"); - ASSERT_EQ(values[2], "v2"); - ASSERT_EQ(values[3], "v3"); - ASSERT_EQ(values[4], "v4"); - // Filter hits for 4 in-domain keys + ASSERT_EQ(values, expected); + ASSERT_EQ(get_perf_context()->bloom_sst_miss_count, 2); + ASSERT_EQ(get_perf_context()->bloom_sst_hit_count, 4); + + // Also check Get stat + get_perf_context()->Reset(); + for (size_t i = 0; i < keys.size(); ++i) { + values[i] = Get(keys[i]); + } + ASSERT_EQ(values, expected); + ASSERT_EQ(get_perf_context()->bloom_sst_miss_count, 2); ASSERT_EQ(get_perf_context()->bloom_sst_hit_count, 4); } diff --git a/db/memtable.cc b/db/memtable.cc index 2f86d0758..f97bb292e 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -334,14 +334,15 @@ class MemTableIterator : public InternalIterator { // iterator should only use prefix bloom filter 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; - } else { - PERF_COUNTER_ADD(bloom_memtable_hit_count, 1); + if (prefix_extractor_->InDomain(user_k_without_ts)) { + if (!bloom_->MayContain( + prefix_extractor_->Transform(user_k_without_ts))) { + PERF_COUNTER_ADD(bloom_memtable_miss_count, 1); + valid_ = false; + return; + } else { + PERF_COUNTER_ADD(bloom_memtable_hit_count, 1); + } } } iter_->Seek(k, nullptr); @@ -353,14 +354,15 @@ class MemTableIterator : public InternalIterator { if (bloom_) { 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; - } else { - PERF_COUNTER_ADD(bloom_memtable_hit_count, 1); + if (prefix_extractor_->InDomain(user_k_without_ts)) { + if (!bloom_->MayContain( + prefix_extractor_->Transform(user_k_without_ts))) { + PERF_COUNTER_ADD(bloom_memtable_miss_count, 1); + valid_ = false; + return; + } else { + PERF_COUNTER_ADD(bloom_memtable_hit_count, 1); + } } } iter_->Seek(k, nullptr); @@ -893,16 +895,20 @@ bool MemTable::Get(const LookupKey& key, std::string* value, bool may_contain = true; size_t ts_sz = GetInternalKeyComparator().user_comparator()->timestamp_size(); Slice user_key_without_ts = StripTimestampFromUserKey(key.user_key(), ts_sz); + bool bloom_checked = false; 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(user_key_without_ts); + bloom_checked = true; } else { assert(prefix_extractor_); - may_contain = !prefix_extractor_->InDomain(user_key_without_ts) || - bloom_filter_->MayContain( - prefix_extractor_->Transform(user_key_without_ts)); + if (prefix_extractor_->InDomain(user_key_without_ts)) { + may_contain = bloom_filter_->MayContain( + prefix_extractor_->Transform(user_key_without_ts)); + bloom_checked = true; + } } } @@ -911,7 +917,7 @@ bool MemTable::Get(const LookupKey& key, std::string* value, PERF_COUNTER_ADD(bloom_memtable_miss_count, 1); *seq = kMaxSequenceNumber; } else { - if (bloom_filter_) { + if (bloom_checked) { PERF_COUNTER_ADD(bloom_memtable_hit_count, 1); } GetFromTable(key, *max_covering_tombstone_seq, do_merge, callback, @@ -988,10 +994,6 @@ void MemTable::MultiGet(const ReadOptions& read_options, MultiGetRange* range, bloom_keys[num_keys] = prefix_extractor_->Transform(iter->ukey_without_ts); range_indexes[num_keys++] = iter.index(); - } else { - // TODO: consider not counting these as Bloom hits to more closely - // match bloom_sst_hit_count - PERF_COUNTER_ADD(bloom_memtable_hit_count, 1); } } bloom_filter_->MayContain(num_keys, &bloom_keys[0], &may_match[0]);