diff --git a/HISTORY.md b/HISTORY.md index 6bcc48efd..fbd993af4 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -17,6 +17,9 @@ * Add experimental `PerfContext` counters `iter_{next|prev|seek}_count` for db iterator, each counting the times of corresponding API being called. * Allow runtime changes to whether `WriteBufferManager` allows stall or not by calling `SetAllowStall()` +### Bug Fixes +* In block cache tracing, fixed some cases of bad hit/miss information (and more) with MultiGet. + ## 8.1.0 (03/18/2023) ### Behavior changes * Compaction output file cutting logic now considers range tombstone start keys. For example, SST partitioner now may receive ParitionRequest for range tombstone start keys. diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index 687b56c3c..76aa9cec6 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -1563,75 +1563,111 @@ BlockBasedTable::MaybeReadBlockAndLoadToCache( } } - // Fill lookup_context. + // TODO: optimize so that lookup_context != nullptr implies the others if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() && lookup_context) { - size_t usage = 0; - uint64_t nkeys = 0; - if (out_parsed_block->GetValue()) { - // Approximate the number of keys in the block using restarts. - // FIXME: Should this only apply to data blocks? - nkeys = rep_->table_options.block_restart_interval * - GetBlockNumRestarts(*out_parsed_block->GetValue()); - usage = out_parsed_block->GetValue()->ApproximateMemoryUsage(); - } - TraceType trace_block_type = TraceType::kTraceMax; - switch (TBlocklike::kBlockType) { - case BlockType::kData: - trace_block_type = TraceType::kBlockTraceDataBlock; - break; - case BlockType::kFilter: - case BlockType::kFilterPartitionIndex: - trace_block_type = TraceType::kBlockTraceFilterBlock; - break; - case BlockType::kCompressionDictionary: - trace_block_type = TraceType::kBlockTraceUncompressionDictBlock; - break; - case BlockType::kRangeDeletion: - trace_block_type = TraceType::kBlockTraceRangeDeletionBlock; - break; - case BlockType::kIndex: - trace_block_type = TraceType::kBlockTraceIndexBlock; - break; - default: - // This cannot happen. - assert(false); - break; - } - bool no_insert = no_io || !ro.fill_cache; - if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock( - trace_block_type, lookup_context->caller)) { - // Defer logging the access to Get() and MultiGet() to trace additional - // information, e.g., referenced_key_exist_in_block. - - // Make a copy of the block key here since it will be logged later. - lookup_context->FillLookupContext( - is_cache_hit, no_insert, trace_block_type, - /*block_size=*/usage, /*block_key=*/key.ToString(), nkeys); - } else { - // Avoid making copy of block_key and cf_name when constructing the access - // record. - BlockCacheTraceRecord access_record( - rep_->ioptions.clock->NowMicros(), - /*block_key=*/"", trace_block_type, - /*block_size=*/usage, rep_->cf_id_for_tracing(), - /*cf_name=*/"", rep_->level_for_tracing(), - rep_->sst_number_for_tracing(), lookup_context->caller, is_cache_hit, - no_insert, lookup_context->get_id, - lookup_context->get_from_user_specified_snapshot, - /*referenced_key=*/""); - // TODO: Should handle this error? - block_cache_tracer_ - ->WriteBlockAccess(access_record, key, rep_->cf_name_for_tracing(), - lookup_context->referenced_key) - .PermitUncheckedError(); - } + SaveLookupContextOrTraceRecord( + key, is_cache_hit, ro, out_parsed_block->GetValue(), lookup_context); } assert(s.ok() || out_parsed_block->GetValue() == nullptr); return s; } +template +WithBlocklikeCheck +BlockBasedTable::SaveLookupContextOrTraceRecord( + const Slice& block_key, bool is_cache_hit, const ReadOptions& ro, + const TBlocklike* parsed_block_value, + BlockCacheLookupContext* lookup_context) const { + assert(lookup_context); + size_t usage = 0; + uint64_t nkeys = 0; + if (parsed_block_value) { + // Approximate the number of keys in the block using restarts. + int interval = rep_->table_options.block_restart_interval; + nkeys = interval * GetBlockNumRestarts(*parsed_block_value); + // On average, the last restart should be just over half utilized. + // Specifically, 1..N should be N/2 + 0.5. For example, 7 -> 4, 8 -> 4.5. + // Use the get_id to alternate between rounding up vs. down. + if (nkeys > 0) { + bool rounding = static_cast(lookup_context->get_id) & 1; + nkeys -= (interval - rounding) / 2; + } + usage = parsed_block_value->ApproximateMemoryUsage(); + } + TraceType trace_block_type = TraceType::kTraceMax; + switch (TBlocklike::kBlockType) { + case BlockType::kData: + trace_block_type = TraceType::kBlockTraceDataBlock; + break; + case BlockType::kFilter: + case BlockType::kFilterPartitionIndex: + trace_block_type = TraceType::kBlockTraceFilterBlock; + break; + case BlockType::kCompressionDictionary: + trace_block_type = TraceType::kBlockTraceUncompressionDictBlock; + break; + case BlockType::kRangeDeletion: + trace_block_type = TraceType::kBlockTraceRangeDeletionBlock; + break; + case BlockType::kIndex: + trace_block_type = TraceType::kBlockTraceIndexBlock; + break; + default: + // This cannot happen. + assert(false); + break; + } + const bool no_io = ro.read_tier == kBlockCacheTier; + bool no_insert = no_io || !ro.fill_cache; + if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock( + trace_block_type, lookup_context->caller)) { + // Make a copy of the block key here since it will be logged later. + lookup_context->FillLookupContext(is_cache_hit, no_insert, trace_block_type, + /*block_size=*/usage, + block_key.ToString(), nkeys); + + // Defer logging the access to Get() and MultiGet() to trace additional + // information, e.g., referenced_key + } else { + // Avoid making copy of block_key if it doesn't need to be saved in + // BlockCacheLookupContext + lookup_context->FillLookupContext(is_cache_hit, no_insert, trace_block_type, + /*block_size=*/usage, + /*block_key=*/{}, nkeys); + + // Fill in default values for irrelevant/unknown fields + FinishTraceRecord(*lookup_context, block_key, + lookup_context->referenced_key, + /*does_referenced_key_exist*/ false, + /*referenced_data_size*/ 0); + } +} + +void BlockBasedTable::FinishTraceRecord( + const BlockCacheLookupContext& lookup_context, const Slice& block_key, + const Slice& referenced_key, bool does_referenced_key_exist, + uint64_t referenced_data_size) const { + // Avoid making copy of referenced_key if it doesn't need to be saved in + // BlockCacheLookupContext + BlockCacheTraceRecord access_record( + rep_->ioptions.clock->NowMicros(), + /*block_key=*/"", lookup_context.block_type, lookup_context.block_size, + rep_->cf_id_for_tracing(), + /*cf_name=*/"", rep_->level_for_tracing(), rep_->sst_number_for_tracing(), + lookup_context.caller, lookup_context.is_cache_hit, + lookup_context.no_insert, lookup_context.get_id, + lookup_context.get_from_user_specified_snapshot, + /*referenced_key=*/"", referenced_data_size, + lookup_context.num_keys_in_block, does_referenced_key_exist); + // TODO: Should handle status here? + block_cache_tracer_ + ->WriteBlockAccess(access_record, block_key, rep_->cf_name_for_tracing(), + referenced_key) + .PermitUncheckedError(); +} + template WithBlocklikeCheck BlockBasedTable::RetrieveBlock( FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, @@ -2142,25 +2178,9 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key, } else { referenced_key = key; } - BlockCacheTraceRecord access_record( - rep_->ioptions.clock->NowMicros(), - /*block_key=*/"", lookup_data_block_context.block_type, - lookup_data_block_context.block_size, rep_->cf_id_for_tracing(), - /*cf_name=*/"", rep_->level_for_tracing(), - rep_->sst_number_for_tracing(), lookup_data_block_context.caller, - lookup_data_block_context.is_cache_hit, - lookup_data_block_context.no_insert, - lookup_data_block_context.get_id, - lookup_data_block_context.get_from_user_specified_snapshot, - /*referenced_key=*/"", referenced_data_size, - lookup_data_block_context.num_keys_in_block, - does_referenced_key_exist); - // TODO: Should handle status here? - block_cache_tracer_ - ->WriteBlockAccess(access_record, - lookup_data_block_context.block_key, - rep_->cf_name_for_tracing(), referenced_key) - .PermitUncheckedError(); + FinishTraceRecord(lookup_data_block_context, + lookup_data_block_context.block_key, referenced_key, + does_referenced_key_exist, referenced_data_size); } if (done) { diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index 3df5cb6cb..2108416f1 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -351,11 +351,22 @@ class BlockBasedTable : public TableReader { BlockCacheLookupContext* lookup_context, bool for_compaction, bool use_cache, bool async_read) const; + template + WithBlocklikeCheck SaveLookupContextOrTraceRecord( + const Slice& block_key, bool is_cache_hit, const ReadOptions& ro, + const TBlocklike* parsed_block_value, + BlockCacheLookupContext* lookup_context) const; + + void FinishTraceRecord(const BlockCacheLookupContext& lookup_context, + const Slice& block_key, const Slice& referenced_key, + bool does_referenced_key_exist, + uint64_t referenced_data_size) const; + DECLARE_SYNC_AND_ASYNC_CONST( void, RetrieveMultipleBlocks, const ReadOptions& options, const MultiGetRange* batch, const autovector* handles, - Status* statuses, CachableEntry* results, char* scratch, + Status* statuses, CachableEntry* results, char* scratch, const UncompressionDict& uncompression_dict); // Get the iterator from the index reader. diff --git a/table/block_based/block_based_table_reader_sync_and_async.h b/table/block_based/block_based_table_reader_sync_and_async.h index c5e1e44fe..5efb279f4 100644 --- a/table/block_based/block_based_table_reader_sync_and_async.h +++ b/table/block_based/block_based_table_reader_sync_and_async.h @@ -32,7 +32,7 @@ namespace ROCKSDB_NAMESPACE { DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) (const ReadOptions& options, const MultiGetRange* batch, const autovector* handles, - Status* statuses, CachableEntry* results, char* scratch, + Status* statuses, CachableEntry* results, char* scratch, const UncompressionDict& uncompression_dict) const { RandomAccessFileReader* file = rep_->file.get(); const Footer& footer = rep_->footer; @@ -44,17 +44,16 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) size_t idx_in_batch = 0; for (auto mget_iter = batch->begin(); mget_iter != batch->end(); ++mget_iter, ++idx_in_batch) { - BlockCacheLookupContext lookup_data_block_context( - TableReaderCaller::kUserMultiGet); const BlockHandle& handle = (*handles)[idx_in_batch]; if (handle.IsNull()) { continue; } + // XXX: use_cache=true means double cache query? statuses[idx_in_batch] = RetrieveBlock(nullptr, options, handle, uncompression_dict, &results[idx_in_batch].As(), - mget_iter->get_context, &lookup_data_block_context, + mget_iter->get_context, /* lookup_context */ nullptr, /* for_compaction */ false, /* use_cache */ true, /* async_read */ false); } @@ -259,17 +258,15 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) if (s.ok()) { if (options.fill_cache) { - BlockCacheLookupContext lookup_data_block_context( - TableReaderCaller::kUserMultiGet); - CachableEntry* block_entry = &results[idx_in_batch]; + CachableEntry* block_entry = &results[idx_in_batch]; // MaybeReadBlockAndLoadToCache will insert into the block caches if // necessary. Since we're passing the serialized block contents, it // will avoid looking up the block cache s = MaybeReadBlockAndLoadToCache( nullptr, options, handle, uncompression_dict, - /*for_compaction=*/false, &block_entry->As(), - mget_iter->get_context, &lookup_data_block_context, - &serialized_block, /*async_read=*/false); + /*for_compaction=*/false, block_entry, mget_iter->get_context, + /*lookup_context=*/nullptr, &serialized_block, + /*async_read=*/false); // block_entry value could be null if no block cache is present, i.e // BlockBasedTableOptions::no_block_cache is true and no compressed @@ -301,7 +298,7 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) contents = std::move(serialized_block); } if (s.ok()) { - results[idx_in_batch].SetOwnedValue(std::make_unique( + results[idx_in_batch].SetOwnedValue(std::make_unique( std::move(contents), read_amp_bytes_per_bit, ioptions.stats)); } } @@ -331,11 +328,13 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) if (sst_file_range.begin()->get_context) { tracing_mget_id = sst_file_range.begin()->get_context->get_tracing_get_id(); } - BlockCacheLookupContext lookup_context{ + // TODO: need more than one lookup_context here to track individual filter + // and index partition hits and misses. + BlockCacheLookupContext metadata_lookup_context{ TableReaderCaller::kUserMultiGet, tracing_mget_id, /*_get_from_user_specified_snapshot=*/read_options.snapshot != nullptr}; FullFilterKeysMayMatch(filter, &sst_file_range, no_io, prefix_extractor, - &lookup_context, read_options); + &metadata_lookup_context, read_options); if (!sst_file_range.empty()) { IndexBlockIter iiter_on_stack; @@ -345,9 +344,9 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) if (rep_->index_type == BlockBasedTableOptions::kHashSearch) { need_upper_bound_check = PrefixExtractorChanged(prefix_extractor); } - auto iiter = - NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack, - sst_file_range.begin()->get_context, &lookup_context); + auto iiter = NewIndexIterator( + read_options, need_upper_bound_check, &iiter_on_stack, + sst_file_range.begin()->get_context, &metadata_lookup_context); std::unique_ptr> iiter_unique_ptr; if (iiter != &iiter_on_stack) { iiter_unique_ptr.reset(iiter); @@ -355,11 +354,22 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) uint64_t prev_offset = std::numeric_limits::max(); autovector block_handles; - std::array, MultiGetContext::MAX_BATCH_SIZE> results; + std::array, MultiGetContext::MAX_BATCH_SIZE> + results; std::array statuses; + // Empty data_lookup_contexts means "unused," when block cache tracing is + // disabled. (Limited options as element type is not default contructible.) + std::vector data_lookup_contexts; MultiGetContext::Mask reused_mask = 0; char stack_buf[kMultiGetReadStackBufSize]; std::unique_ptr block_buf; + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) { + // Awkward because BlockCacheLookupContext is not CopyAssignable + data_lookup_contexts.reserve(MultiGetContext::MAX_BATCH_SIZE); + for (size_t i = 0; i < MultiGetContext::MAX_BATCH_SIZE; ++i) { + data_lookup_contexts.push_back(metadata_lookup_context); + } + } { MultiGetRange data_block_range(sst_file_range, sst_file_range.begin(), sst_file_range.end()); @@ -413,7 +423,7 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) ->GetOrReadUncompressionDictionary( nullptr /* prefetch_buffer */, no_io, read_options.verify_checksums, get_context, - &lookup_context, &uncompression_dict); + &metadata_lookup_context, &uncompression_dict); uncompression_dict_inited = true; } @@ -442,9 +452,6 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) // Lookup the cache for the given data block referenced by an index // iterator value (i.e BlockHandle). If it exists in the cache, // initialize block to the contents of the data block. - // TODO? - // BlockCacheLookupContext lookup_data_block_context( - // TableReaderCaller::kUserMultiGet); // An async version of MaybeReadBlockAndLoadToCache / // GetDataBlockFromCache @@ -492,6 +499,11 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) total_len += BlockSizeWithTrailer(block_handles[i]); UpdateCacheMissMetrics(BlockType::kData, get_context); } + if (!data_lookup_contexts.empty()) { + // Populate cache key before it's discarded + data_lookup_contexts[i].block_key = + async_handles[lookup_idx].key.ToString(); + } ++lookup_idx; } } @@ -547,24 +559,26 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) bool first_block = true; do { DataBlockIter* biter = nullptr; + uint64_t referenced_data_size = 0; + Block_kData* parsed_block_value = nullptr; bool reusing_prev_block; bool later_reused; - uint64_t referenced_data_size = 0; bool does_referenced_key_exist = false; - BlockCacheLookupContext lookup_data_block_context( - TableReaderCaller::kUserMultiGet, tracing_mget_id, - /*_get_from_user_specified_snapshot=*/read_options.snapshot != - nullptr); + bool handle_present = false; + BlockCacheLookupContext* lookup_data_block_context = + data_lookup_contexts.empty() ? nullptr + : &data_lookup_contexts[idx_in_batch]; if (first_block) { - if (!block_handles[idx_in_batch].IsNull() || - !results[idx_in_batch].IsEmpty()) { + handle_present = !block_handles[idx_in_batch].IsNull(); + parsed_block_value = results[idx_in_batch].GetValue(); + if (handle_present || parsed_block_value) { first_biter.Invalidate(Status::OK()); NewDataBlockIterator( - read_options, results[idx_in_batch], &first_biter, + read_options, results[idx_in_batch].As(), &first_biter, statuses[idx_in_batch]); reusing_prev_block = false; } else { - // If handler is null and result is empty, then the status is never + // If handle is null and result is empty, then the status is never // set, which should be the initial value: ok(). assert(statuses[idx_in_batch].ok()); reusing_prev_block = true; @@ -589,7 +603,7 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) Status tmp_s; NewDataBlockIterator( read_options, iiter->value().handle, &next_biter, - BlockType::kData, get_context, &lookup_data_block_context, + BlockType::kData, get_context, lookup_data_block_context, /* prefetch_buffer= */ nullptr, /* for_compaction = */ false, /*async_read = */ false, tmp_s); biter = &next_biter; @@ -684,35 +698,23 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) // Write the block cache access. // XXX: There appear to be 'break' statements above that bypass this // writing of the block cache trace record - if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() && - !reusing_prev_block) { - // Avoid making copy of block_key, cf_name, and referenced_key when - // constructing the access record. + if (lookup_data_block_context && !reusing_prev_block && first_block) { Slice referenced_key; if (does_referenced_key_exist) { referenced_key = biter->key(); } else { referenced_key = key; } - BlockCacheTraceRecord access_record( - rep_->ioptions.clock->NowMicros(), - /*_block_key=*/"", lookup_data_block_context.block_type, - lookup_data_block_context.block_size, rep_->cf_id_for_tracing(), - /*_cf_name=*/"", rep_->level_for_tracing(), - rep_->sst_number_for_tracing(), lookup_data_block_context.caller, - lookup_data_block_context.is_cache_hit, - lookup_data_block_context.no_insert, - lookup_data_block_context.get_id, - lookup_data_block_context.get_from_user_specified_snapshot, - /*_referenced_key=*/"", referenced_data_size, - lookup_data_block_context.num_keys_in_block, - does_referenced_key_exist); - // TODO: Should handle status here? - block_cache_tracer_ - ->WriteBlockAccess(access_record, - lookup_data_block_context.block_key, - rep_->cf_name_for_tracing(), referenced_key) - .PermitUncheckedError(); + + // block_key is self-assigned here (previously assigned from + // cache_keys / async_handles, now out of scope) + SaveLookupContextOrTraceRecord(lookup_data_block_context->block_key, + /*is_cache_hit=*/!handle_present, + read_options, parsed_block_value, + lookup_data_block_context); + FinishTraceRecord( + *lookup_data_block_context, lookup_data_block_context->block_key, + referenced_key, does_referenced_key_exist, referenced_data_size); } s = biter->status(); if (done) { diff --git a/table/table_test.cc b/table/table_test.cc index df9e508f5..e2ce3139e 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -1147,19 +1147,16 @@ class BlockBasedTableTest std::move(trace_writer)); ASSERT_NE(block_cache_trace_writer, nullptr); // Always return Status::OK(). - assert(c->block_cache_tracer_ - .StartTrace(trace_opt, std::move(block_cache_trace_writer)) - .ok()); + ASSERT_OK(c->block_cache_tracer_.StartTrace( + trace_opt, std::move(block_cache_trace_writer))); { - std::string user_key = "k01"; - InternalKey internal_key(user_key, 0, kTypeValue); + InternalKey internal_key(auto_add_key1, 0, kTypeValue); std::string encoded_key = internal_key.Encode().ToString(); c->Add(encoded_key, kDummyValue); } { - std::string user_key = "k02"; - InternalKey internal_key(user_key, 0, kTypeValue); + InternalKey internal_key(auto_add_key2, 0, kTypeValue); std::string encoded_key = internal_key.Encode().ToString(); c->Add(encoded_key, kDummyValue); } @@ -1180,6 +1177,7 @@ class BlockBasedTableTest EXPECT_OK(reader.ReadHeader(&header)); uint32_t index = 0; while (s.ok()) { + SCOPED_TRACE("expected_records[" + std::to_string(index) + "]"); BlockCacheTraceRecord access; s = reader.ReadAccess(&access); if (!s.ok()) { @@ -1192,22 +1190,33 @@ class BlockBasedTableTest EXPECT_EQ(access.caller, expected_records[index].caller); EXPECT_EQ(access.no_insert, expected_records[index].no_insert); EXPECT_EQ(access.is_cache_hit, expected_records[index].is_cache_hit); - // Get - if (access.caller == TableReaderCaller::kUserGet) { + EXPECT_EQ(access.get_id, expected_records[index].get_id); + // The well-populated cases + if (access.caller == TableReaderCaller::kUserGet || + (access.caller == TableReaderCaller::kUserMultiGet && + access.block_type == TraceType::kBlockTraceDataBlock)) { EXPECT_EQ(access.referenced_key, expected_records[index].referenced_key); - EXPECT_EQ(access.get_id, expected_records[index].get_id); EXPECT_EQ(access.get_from_user_specified_snapshot, expected_records[index].get_from_user_specified_snapshot); if (access.block_type == TraceType::kBlockTraceDataBlock) { EXPECT_GT(access.referenced_data_size, 0); EXPECT_GT(access.num_keys_in_block, 0); + if (access.caller == TableReaderCaller::kUserMultiGet) { + // Test num_keys_in_block estimate, assuming default restart + // interval of 16 and just one interval. + // Rounding depends on get_id. + if (access.get_id & 1) { + EXPECT_EQ(access.num_keys_in_block, 9); + } else { + EXPECT_EQ(access.num_keys_in_block, 8); + } + } EXPECT_EQ(access.referenced_key_exist_in_block, expected_records[index].referenced_key_exist_in_block); } } else { EXPECT_EQ(access.referenced_key, ""); - EXPECT_EQ(access.get_id, 0); EXPECT_FALSE(access.get_from_user_specified_snapshot); EXPECT_EQ(access.referenced_data_size, 0); EXPECT_EQ(access.num_keys_in_block, 0); @@ -1223,6 +1232,8 @@ class BlockBasedTableTest protected: uint64_t IndexUncompressedHelper(bool indexCompress); + const std::string auto_add_key1 = "aak01"; + const std::string auto_add_key2 = "aak02"; private: uint32_t format_; @@ -3068,15 +3079,14 @@ TEST_P(BlockBasedTableTest, TracingGetTest) { MutableCFOptions moptions(options); c.Finish(options, ioptions, moptions, table_options, GetPlainInternalComparator(options.comparator), &keys, &kvmap); - std::string user_key = "k01"; - InternalKey internal_key(user_key, 0, kTypeValue); + InternalKey internal_key(auto_add_key1, 0, kTypeValue); std::string encoded_key = internal_key.Encode().ToString(); for (uint32_t i = 1; i <= 2; i++) { PinnableSlice value; - GetContext get_context(options.comparator, nullptr, nullptr, nullptr, - GetContext::kNotFound, user_key, &value, nullptr, - nullptr, nullptr, true, nullptr, nullptr, nullptr, - nullptr, nullptr, nullptr, /*tracing_get_id=*/i); + GetContext get_context( + options.comparator, nullptr, nullptr, nullptr, GetContext::kNotFound, + auto_add_key1, &value, nullptr, nullptr, nullptr, true, nullptr, + nullptr, nullptr, nullptr, nullptr, nullptr, /*tracing_get_id=*/i); get_perf_context()->Reset(); ASSERT_OK(c.GetTableReader()->Get(ReadOptions(), encoded_key, &get_context, moptions.prefix_extractor.get())); @@ -3126,6 +3136,189 @@ TEST_P(BlockBasedTableTest, TracingGetTest) { c.ResetTableReader(); } +struct HitMissCountingCache : public CacheWrapper { + using CacheWrapper::CacheWrapper; + const char* Name() const override { return "HitMissCountingCache"; } + + uint64_t hit_count_ = 0; + uint64_t miss_count_ = 0; + + void Reset() { + hit_count_ = 0; + miss_count_ = 0; + } + + Handle* Lookup(const Slice& key, const CacheItemHelper* helper, + CreateContext* create_context, + Priority priority = Priority::LOW, + Statistics* stats = nullptr) override { + // ASSUMES no blocking async lookups + Handle* h = target_->Lookup(key, helper, create_context, priority, stats); + if (h) { + hit_count_++; + } else { + miss_count_++; + } + return h; + } + + void StartAsyncLookup(AsyncLookupHandle& async_handle) override { + target_->StartAsyncLookup(async_handle); + // If not pending, caller might not call WaitAll, so have to account here. + if (!async_handle.IsPending()) { + if (async_handle.Result()) { + hit_count_++; + } else { + miss_count_++; + } + } + } + + void WaitAll(AsyncLookupHandle* async_handles, size_t count) override { + // If !pending, then we already accounted for it in StartAsyncLookup. + // Assume the pending status does not change asynchronously (since + // StartAsyncLookup) and remember which still need accounting. + std::vector needs_accounting; + for (size_t i = 0; i < count; ++i) { + if (async_handles[i].IsPending()) { + needs_accounting.push_back(async_handles + i); + } + } + target_->WaitAll(async_handles, count); + for (auto ah : needs_accounting) { + if (ah->Result()) { + hit_count_++; + } else { + miss_count_++; + } + } + } + + void VerifyExpectedHitMissCounts( + const std::vector& expected_records) { + uint64_t expected_hits = 0; + uint64_t expected_misses = 0; + for (const auto& r : expected_records) { + if (r.is_cache_hit) { + expected_hits++; + } else { + expected_misses++; + } + } + EXPECT_EQ(expected_hits, hit_count_); + EXPECT_EQ(expected_misses, miss_count_); + Reset(); + } +}; + +TEST_P(BlockBasedTableTest, TracingMultiGetTest) { + TableConstructor c(BytewiseComparator()); + Options options; + BlockBasedTableOptions table_options = GetBlockBasedTableOptions(); + options.create_if_missing = true; + auto cache = + std::make_shared(NewLRUCache(1024 * 1024, 0)); + table_options.block_cache = cache; + table_options.cache_index_and_filter_blocks = true; + table_options.filter_policy.reset(NewBloomFilterPolicy(10)); + // Put auto_add_key1 and auto_add_key2 in the same data block + table_options.block_size = kDummyValue.size() * 2 + 100; + options.table_factory.reset(new BlockBasedTableFactory(table_options)); + SetupTracingTest(&c); + std::vector keys; + stl_wrappers::KVMap kvmap; + ImmutableOptions ioptions(options); + MutableCFOptions moptions(options); + c.Finish(options, ioptions, moptions, table_options, + GetPlainInternalComparator(options.comparator), &keys, &kvmap); + + std::vector expected_records; + + for (bool first_pass : {true, false}) { + uint64_t get_id_offset = first_pass ? 2 : 5; + ReadOptions ro; + std::array ukeys{{auto_add_key1, auto_add_key2}}; + std::array values; + std::vector get_contexts; + get_contexts.emplace_back( + options.comparator, nullptr, nullptr, nullptr, GetContext::kNotFound, + ukeys[0], &values[0], nullptr, nullptr, nullptr, true, nullptr, nullptr, + nullptr, nullptr, nullptr, nullptr, get_id_offset); + get_contexts.emplace_back( + options.comparator, nullptr, nullptr, nullptr, GetContext::kNotFound, + ukeys[1], &values[1], nullptr, nullptr, nullptr, true, nullptr, nullptr, + nullptr, nullptr, nullptr, nullptr, get_id_offset + 1); + std::array encoded_keys; + encoded_keys[0] = InternalKey(ukeys[0], 0, kTypeValue).Encode().ToString(); + encoded_keys[1] = InternalKey(ukeys[1], 0, kTypeValue).Encode().ToString(); + std::array statuses; + autovector key_context; + key_context.emplace_back(/*ColumnFamilyHandle omitted*/ nullptr, ukeys[0], + &values[0], + /*PinnableWideColumns omitted*/ nullptr, + /*timestamp omitted*/ nullptr, &statuses[0]); + key_context[0].ukey_without_ts = ukeys[0]; + key_context[0].ikey = encoded_keys[0]; + key_context[0].get_context = &get_contexts[0]; + key_context.emplace_back(/*ColumnFamilyHandle omitted*/ nullptr, ukeys[1], + &values[1], + /*PinnableWideColumns omitted*/ nullptr, + /*timestamp omitted*/ nullptr, &statuses[1]); + key_context[1].ukey_without_ts = ukeys[1]; + key_context[1].ikey = encoded_keys[1]; + key_context[1].get_context = &get_contexts[1]; + autovector sorted_keys; + sorted_keys.push_back(&key_context[0]); + sorted_keys.push_back(&key_context[1]); + MultiGetContext m_context( + &sorted_keys, 0, sorted_keys.size(), /*SequenceNumber*/ 42, ro, + options.env->GetFileSystem().get(), options.statistics.get()); + MultiGetRange range = m_context.GetMultiGetRange(); + + get_perf_context()->Reset(); + c.GetTableReader()->MultiGet(ro, &range, /*prefix_extractor*/ nullptr); + + // Verify read op result + for (uint32_t i = 0; i <= 1; i++) { + ASSERT_OK(statuses[i]); + ASSERT_EQ(get_contexts[i].State(), GetContext::kFound); + ASSERT_EQ(values[i].ToString(), kDummyValue); + } + + // Verify traces. + BlockCacheTraceRecord record; + if (first_pass) { + // The first two records should be prefetching index and filter blocks. + record.get_id = 0; + record.block_type = TraceType::kBlockTraceIndexBlock; + record.caller = TableReaderCaller::kPrefetch; + record.is_cache_hit = false; + record.no_insert = false; + expected_records.push_back(record); + record.block_type = TraceType::kBlockTraceFilterBlock; + expected_records.push_back(record); + } + // Then we should have three records for one index, one filter, and one data + // block access. (The two keys share a data block.) + record.get_id = get_id_offset; + record.block_type = TraceType::kBlockTraceFilterBlock; + record.caller = TableReaderCaller::kUserMultiGet; + record.get_from_user_specified_snapshot = false; + record.referenced_key = encoded_keys[0]; + record.referenced_key_exist_in_block = true; + record.is_cache_hit = true; + expected_records.push_back(record); + record.block_type = TraceType::kBlockTraceIndexBlock; + expected_records.push_back(record); + record.is_cache_hit = !first_pass; + record.block_type = TraceType::kBlockTraceDataBlock; + expected_records.push_back(record); + } + VerifyBlockAccessTrace(&c, expected_records); + cache->VerifyExpectedHitMissCounts(expected_records); + c.ResetTableReader(); +} + TEST_P(BlockBasedTableTest, TracingApproximateOffsetOfTest) { TableConstructor c(BytewiseComparator()); Options options; @@ -3143,8 +3336,7 @@ TEST_P(BlockBasedTableTest, TracingApproximateOffsetOfTest) { c.Finish(options, ioptions, moptions, table_options, GetPlainInternalComparator(options.comparator), &keys, &kvmap); for (uint32_t i = 1; i <= 2; i++) { - std::string user_key = "k01"; - InternalKey internal_key(user_key, 0, kTypeValue); + InternalKey internal_key(auto_add_key1, 0, kTypeValue); std::string encoded_key = internal_key.Encode().ToString(); c.GetTableReader()->ApproximateOffsetOf( encoded_key, TableReaderCaller::kUserApproximateSize);