Add more callers for table reader. (#5454)

Summary:
This PR adds more callers for table readers. These information are only used for block cache analysis so that we can know which caller accesses a block.
1. It renames the BlockCacheLookupCaller to TableReaderCaller as passing the caller from upstream requires changes to table_reader.h and TableReaderCaller is a more appropriate name.
2. It adds more table reader callers in table/table_reader_caller.h, e.g., kCompactionRefill, kExternalSSTIngestion, and kBuildTable.

This PR is long as it requires modification of interfaces in table_reader.h, e.g., NewIterator.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5454

Test Plan: make clean && COMPILE_WITH_ASAN=1 make check -j32.

Differential Revision: D15819451

Pulled By: HaoyuHuang

fbshipit-source-id: b6caa704c8fb96ddd15b9a934b7e7ea87f88092d
main
haoyuhuang 5 years ago committed by Facebook Github Bot
parent 0b0cb6f1a2
commit 705b8eecb4
  1. 5
      db/builder.cc
  2. 12
      db/compaction/compaction_job.cc
  3. 2
      db/convenience.cc
  4. 2
      db/db_impl/db_impl.cc
  5. 6
      db/external_sst_file_ingestion_job.cc
  6. 29
      db/forward_iterator.cc
  7. 6
      db/repair.cc
  8. 6
      db/table_cache.cc
  9. 10
      db/table_cache.h
  10. 100
      db/version_set.cc
  11. 7
      db/version_set.h
  12. 48
      table/block_based/block_based_table_reader.cc
  13. 44
      table/block_based/block_based_table_reader.h
  14. 2
      table/block_based/partitioned_filter_block.cc
  15. 2
      table/cuckoo/cuckoo_table_reader.cc
  16. 8
      table/cuckoo/cuckoo_table_reader.h
  17. 9
      table/cuckoo/cuckoo_table_reader_test.cc
  18. 3
      table/mock_table.cc
  19. 9
      table/mock_table.h
  20. 4
      table/plain/plain_table_reader.cc
  21. 8
      table/plain/plain_table_reader.h
  22. 7
      table/sst_file_reader.cc
  23. 16
      table/table_reader.h
  24. 4
      table/table_reader_bench.cc
  25. 39
      table/table_reader_caller.h
  26. 73
      table/table_test.cc
  27. 38
      tools/block_cache_trace_analyzer.cc
  28. 2
      tools/block_cache_trace_analyzer.h
  29. 20
      tools/block_cache_trace_analyzer_test.cc
  30. 9
      tools/sst_dump_tool.cc
  31. 10
      trace_replay/block_cache_tracer.cc
  32. 24
      trace_replay/block_cache_tracer.h
  33. 16
      trace_replay/block_cache_tracer_test.cc

@ -221,8 +221,9 @@ Status BuildTable(
mutable_cf_options.prefix_extractor.get(), nullptr,
(internal_stats == nullptr) ? nullptr
: internal_stats->GetFileReadHist(0),
false /* for_compaction */, nullptr /* arena */,
false /* skip_filter */, level));
TableReaderCaller::kFlush, /*arena=*/nullptr,
/*skip_filter=*/false, level, /*smallest_compaction_key=*/nullptr,
/*largest_compaction_key*/ nullptr));
s = it->status();
if (s.ok() && paranoid_file_checks) {
for (it->SeekToFirst(); it->Valid(); it->Next()) {

@ -521,7 +521,7 @@ void CompactionJob::GenSubcompactionBoundaries() {
// mutex to reduce contention
db_mutex_->Unlock();
uint64_t size = versions_->ApproximateSize(v, a, b, start_lvl, out_lvl + 1,
/*for_compaction*/ true);
TableReaderCaller::kCompaction);
db_mutex_->Lock();
ranges.emplace_back(a, b, size);
sum += size;
@ -646,12 +646,14 @@ Status CompactionJob::Run() {
// to cache it here for further user reads
InternalIterator* iter = cfd->table_cache()->NewIterator(
ReadOptions(), env_options_, cfd->internal_comparator(),
*files_meta[file_idx], nullptr /* range_del_agg */,
prefix_extractor, nullptr,
*files_meta[file_idx], /*range_del_agg=*/nullptr, prefix_extractor,
/*table_reader_ptr=*/nullptr,
cfd->internal_stats()->GetFileReadHist(
compact_->compaction->output_level()),
false, nullptr /* arena */, false /* skip_filters */,
compact_->compaction->output_level());
TableReaderCaller::kCompactionRefill, /*arena=*/nullptr,
/*skip_filters=*/false, compact_->compaction->output_level(),
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr);
auto s = iter->status();
if (s.ok() && paranoid_file_checks_) {

@ -59,7 +59,7 @@ Status VerifySstFileChecksum(const Options& options,
if (!s.ok()) {
return s;
}
s = table_reader->VerifyChecksum();
s = table_reader->VerifyChecksum(TableReaderCaller::kUserVerifyChecksum);
return s;
}

@ -2771,7 +2771,7 @@ void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
if (include_flags & DB::SizeApproximationFlags::INCLUDE_FILES) {
sizes[i] += versions_->ApproximateSize(
v, k1.Encode(), k2.Encode(), /*start_level=*/0, /*end_level=*/-1,
/*for_compaction=*/false);
TableReaderCaller::kUserApproximateSize);
}
if (include_flags & DB::SizeApproximationFlags::INCLUDE_MEMTABLES) {
sizes[i] += sv->mem->ApproximateStats(k1.Encode(), k2.Encode()).size;

@ -308,7 +308,8 @@ Status ExternalSstFileIngestionJob::GetIngestedFileInfo(
}
if (ingestion_options_.verify_checksums_before_ingest) {
status = table_reader->VerifyChecksum();
status =
table_reader->VerifyChecksum(TableReaderCaller::kExternalSSTIngestion);
}
if (!status.ok()) {
return status;
@ -368,7 +369,8 @@ Status ExternalSstFileIngestionJob::GetIngestedFileInfo(
// updating the block cache.
ro.fill_cache = false;
std::unique_ptr<InternalIterator> iter(table_reader->NewIterator(
ro, sv->mutable_cf_options.prefix_extractor.get()));
ro, sv->mutable_cf_options.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kExternalSSTIngestion));
std::unique_ptr<InternalIterator> range_del_iter(
table_reader->NewRangeTombstoneIterator(ro));

@ -79,7 +79,11 @@ class ForwardLevelIterator : public InternalIterator {
read_options_, *(cfd_->soptions()), cfd_->internal_comparator(),
*files_[file_index_],
read_options_.ignore_range_deletions ? nullptr : &range_del_agg,
prefix_extractor_, nullptr /* table_reader_ptr */, nullptr, false);
prefix_extractor_, /*table_reader_ptr=*/nullptr,
/*file_read_hist=*/nullptr, TableReaderCaller::kUserIterator,
/*arena=*/nullptr, /*skip_filters=*/false, /*level=*/-1,
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr);
file_iter_->SetPinnedItersMgr(pinned_iters_mgr_);
valid_ = false;
if (!range_del_agg.IsEmpty()) {
@ -642,7 +646,12 @@ void ForwardIterator::RebuildIterators(bool refresh_sv) {
l0_iters_.push_back(cfd_->table_cache()->NewIterator(
read_options_, *cfd_->soptions(), cfd_->internal_comparator(), *l0,
read_options_.ignore_range_deletions ? nullptr : &range_del_agg,
sv_->mutable_cf_options.prefix_extractor.get()));
sv_->mutable_cf_options.prefix_extractor.get(),
/*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr,
TableReaderCaller::kUserIterator, /*arena=*/nullptr,
/*skip_filters=*/false, /*level=*/-1,
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr));
}
BuildLevelIterators(vstorage);
current_ = nullptr;
@ -714,7 +723,12 @@ void ForwardIterator::RenewIterators() {
read_options_, *cfd_->soptions(), cfd_->internal_comparator(),
*l0_files_new[inew],
read_options_.ignore_range_deletions ? nullptr : &range_del_agg,
svnew->mutable_cf_options.prefix_extractor.get()));
svnew->mutable_cf_options.prefix_extractor.get(),
/*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr,
TableReaderCaller::kUserIterator, /*arena=*/nullptr,
/*skip_filters=*/false, /*level=*/-1,
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr));
}
for (auto* f : l0_iters_) {
@ -772,8 +786,13 @@ void ForwardIterator::ResetIncompleteIterators() {
DeleteIterator(l0_iters_[i]);
l0_iters_[i] = cfd_->table_cache()->NewIterator(
read_options_, *cfd_->soptions(), cfd_->internal_comparator(),
*l0_files[i], nullptr /* range_del_agg */,
sv_->mutable_cf_options.prefix_extractor.get());
*l0_files[i], /*range_del_agg=*/nullptr,
sv_->mutable_cf_options.prefix_extractor.get(),
/*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr,
TableReaderCaller::kUserIterator, /*arena=*/nullptr,
/*skip_filters=*/false, /*level=*/-1,
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr);
l0_iters_[i]->SetPinnedItersMgr(pinned_iters_mgr_);
}

@ -520,7 +520,11 @@ class Repairer {
InternalIterator* iter = table_cache_->NewIterator(
ropts, env_options_, cfd->internal_comparator(), t->meta,
nullptr /* range_del_agg */,
cfd->GetLatestMutableCFOptions()->prefix_extractor.get());
cfd->GetLatestMutableCFOptions()->prefix_extractor.get(),
/*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr,
TableReaderCaller::kRepair, /*arena=*/nullptr, /*skip_filters=*/false,
/*level=*/-1, /*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr);
bool empty = true;
ParsedInternalKey parsed;
t->min_sequence = 0;

@ -176,7 +176,7 @@ InternalIterator* TableCache::NewIterator(
const InternalKeyComparator& icomparator, const FileMetaData& file_meta,
RangeDelAggregator* range_del_agg, const SliceTransform* prefix_extractor,
TableReader** table_reader_ptr, HistogramImpl* file_read_hist,
bool for_compaction, Arena* arena, bool skip_filters, int level,
TableReaderCaller caller, Arena* arena, bool skip_filters, int level,
const InternalKey* smallest_compaction_key,
const InternalKey* largest_compaction_key) {
PERF_TIMER_GUARD(new_table_iterator_nanos);
@ -187,7 +187,7 @@ InternalIterator* TableCache::NewIterator(
if (table_reader_ptr != nullptr) {
*table_reader_ptr = nullptr;
}
bool for_compaction = caller == TableReaderCaller::kCompaction;
auto& fd = file_meta.fd;
table_reader = fd.table_reader;
if (table_reader == nullptr) {
@ -206,7 +206,7 @@ InternalIterator* TableCache::NewIterator(
result = NewEmptyInternalIterator<Slice>(arena);
} else {
result = table_reader->NewIterator(options, prefix_extractor, arena,
skip_filters, for_compaction,
skip_filters, caller,
env_options.compaction_readahead_size);
}
if (handle != nullptr) {

@ -70,12 +70,10 @@ class TableCache {
const ReadOptions& options, const EnvOptions& toptions,
const InternalKeyComparator& internal_comparator,
const FileMetaData& file_meta, RangeDelAggregator* range_del_agg,
const SliceTransform* prefix_extractor = nullptr,
TableReader** table_reader_ptr = nullptr,
HistogramImpl* file_read_hist = nullptr, bool for_compaction = false,
Arena* arena = nullptr, bool skip_filters = false, int level = -1,
const InternalKey* smallest_compaction_key = nullptr,
const InternalKey* largest_compaction_key = nullptr);
const SliceTransform* prefix_extractor, TableReader** table_reader_ptr,
HistogramImpl* file_read_hist, TableReaderCaller caller, Arena* arena,
bool skip_filters, int level, const InternalKey* smallest_compaction_key,
const InternalKey* largest_compaction_key);
// If a seek to internal key "k" in specified file finds an entry,
// call get_context->SaveValue() repeatedly until

@ -850,14 +850,15 @@ namespace {
class LevelIterator final : public InternalIterator {
public:
LevelIterator(
TableCache* table_cache, const ReadOptions& read_options,
const EnvOptions& env_options, const InternalKeyComparator& icomparator,
const LevelFilesBrief* flevel, const SliceTransform* prefix_extractor,
bool should_sample, HistogramImpl* file_read_hist, bool for_compaction,
bool skip_filters, int level, RangeDelAggregator* range_del_agg,
const std::vector<AtomicCompactionUnitBoundary>* compaction_boundaries =
nullptr)
LevelIterator(TableCache* table_cache, const ReadOptions& read_options,
const EnvOptions& env_options,
const InternalKeyComparator& icomparator,
const LevelFilesBrief* flevel,
const SliceTransform* prefix_extractor, bool should_sample,
HistogramImpl* file_read_hist, TableReaderCaller caller,
bool skip_filters, int level, RangeDelAggregator* range_del_agg,
const std::vector<AtomicCompactionUnitBoundary>*
compaction_boundaries = nullptr)
: InternalIterator(false),
table_cache_(table_cache),
read_options_(read_options),
@ -868,7 +869,7 @@ class LevelIterator final : public InternalIterator {
prefix_extractor_(prefix_extractor),
file_read_hist_(file_read_hist),
should_sample_(should_sample),
for_compaction_(for_compaction),
caller_(caller),
skip_filters_(skip_filters),
file_index_(flevel_->num_files),
level_(level),
@ -957,9 +958,9 @@ class LevelIterator final : public InternalIterator {
return table_cache_->NewIterator(
read_options_, env_options_, icomparator_, *file_meta.file_metadata,
range_del_agg_, prefix_extractor_,
nullptr /* don't need reference to table */,
file_read_hist_, for_compaction_, nullptr /* arena */, skip_filters_,
level_, smallest_compaction_key, largest_compaction_key);
nullptr /* don't need reference to table */, file_read_hist_, caller_,
/*arena=*/nullptr, skip_filters_, level_, smallest_compaction_key,
largest_compaction_key);
}
TableCache* table_cache_;
@ -973,7 +974,7 @@ class LevelIterator final : public InternalIterator {
HistogramImpl* file_read_hist_;
bool should_sample_;
bool for_compaction_;
TableReaderCaller caller_;
bool skip_filters_;
size_t file_index_;
int level_;
@ -1442,10 +1443,14 @@ void Version::AddIteratorsForLevel(const ReadOptions& read_options,
for (size_t i = 0; i < storage_info_.LevelFilesBrief(0).num_files; i++) {
const auto& file = storage_info_.LevelFilesBrief(0).files[i];
merge_iter_builder->AddIterator(cfd_->table_cache()->NewIterator(
read_options, soptions, cfd_->internal_comparator(), *file.file_metadata,
range_del_agg, mutable_cf_options_.prefix_extractor.get(), nullptr,
cfd_->internal_stats()->GetFileReadHist(0), false, arena,
false /* skip_filters */, 0 /* level */));
read_options, soptions, cfd_->internal_comparator(),
*file.file_metadata, range_del_agg,
mutable_cf_options_.prefix_extractor.get(), nullptr,
cfd_->internal_stats()->GetFileReadHist(0),
TableReaderCaller::kUserIterator, arena,
/*skip_filters=*/false, /*level=*/0,
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr));
}
if (should_sample) {
// Count ones for every L0 files. This is done per iterator creation
@ -1466,8 +1471,8 @@ void Version::AddIteratorsForLevel(const ReadOptions& read_options,
cfd_->internal_comparator(), &storage_info_.LevelFilesBrief(level),
mutable_cf_options_.prefix_extractor.get(), should_sample_file_read(),
cfd_->internal_stats()->GetFileReadHist(level),
false /* for_compaction */, IsFilterSkipped(level), level,
range_del_agg));
TableReaderCaller::kUserIterator, IsFilterSkipped(level), level,
range_del_agg, /*largest_compaction_key=*/nullptr));
}
}
@ -1496,10 +1501,14 @@ Status Version::OverlapWithLevelIterator(const ReadOptions& read_options,
continue;
}
ScopedArenaIterator iter(cfd_->table_cache()->NewIterator(
read_options, env_options, cfd_->internal_comparator(), *file->file_metadata,
&range_del_agg, mutable_cf_options_.prefix_extractor.get(), nullptr,
cfd_->internal_stats()->GetFileReadHist(0), false, &arena,
false /* skip_filters */, 0 /* level */));
read_options, env_options, cfd_->internal_comparator(),
*file->file_metadata, &range_del_agg,
mutable_cf_options_.prefix_extractor.get(), nullptr,
cfd_->internal_stats()->GetFileReadHist(0),
TableReaderCaller::kUserIterator, &arena,
/*skip_filters=*/false, /*level=*/0,
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr));
status = OverlapWithIterator(
ucmp, smallest_user_key, largest_user_key, iter.get(), overlap);
if (!status.ok() || *overlap) {
@ -1513,7 +1522,7 @@ Status Version::OverlapWithLevelIterator(const ReadOptions& read_options,
cfd_->internal_comparator(), &storage_info_.LevelFilesBrief(level),
mutable_cf_options_.prefix_extractor.get(), should_sample_file_read(),
cfd_->internal_stats()->GetFileReadHist(level),
false /* for_compaction */, IsFilterSkipped(level), level,
TableReaderCaller::kUserIterator, IsFilterSkipped(level), level,
&range_del_agg));
status = OverlapWithIterator(
ucmp, smallest_user_key, largest_user_key, iter.get(), overlap);
@ -4823,7 +4832,7 @@ Status VersionSet::WriteSnapshot(log::Writer* log) {
// maintain state of where they first appear in the files.
uint64_t VersionSet::ApproximateSize(Version* v, const Slice& start,
const Slice& end, int start_level,
int end_level, bool for_compaction) {
int end_level, TableReaderCaller caller) {
// pre-condition
assert(v->cfd_->internal_comparator().Compare(start, end) <= 0);
@ -4844,7 +4853,7 @@ uint64_t VersionSet::ApproximateSize(Version* v, const Slice& start,
if (!level) {
// level 0 data is sorted order, handle the use case explicitly
size += ApproximateSizeLevel0(v, files_brief, start, end, for_compaction);
size += ApproximateSizeLevel0(v, files_brief, start, end, caller);
continue;
}
@ -4861,7 +4870,7 @@ uint64_t VersionSet::ApproximateSize(Version* v, const Slice& start,
// inferred from the sorted order
for (uint64_t i = idx_start; i < files_brief.num_files; i++) {
uint64_t val;
val = ApproximateSize(v, files_brief.files[i], end, for_compaction);
val = ApproximateSize(v, files_brief.files[i], end, caller);
if (!val) {
// the files after this will not have the range
break;
@ -4872,7 +4881,7 @@ uint64_t VersionSet::ApproximateSize(Version* v, const Slice& start,
if (i == idx_start) {
// subtract the bytes needed to be scanned to get to the starting
// key
val = ApproximateSize(v, files_brief.files[i], start, for_compaction);
val = ApproximateSize(v, files_brief.files[i], start, caller);
assert(size >= val);
size -= val;
}
@ -4886,15 +4895,15 @@ uint64_t VersionSet::ApproximateSizeLevel0(Version* v,
const LevelFilesBrief& files_brief,
const Slice& key_start,
const Slice& key_end,
bool for_compaction) {
TableReaderCaller caller) {
// level 0 files are not in sorted order, we need to iterate through
// the list to compute the total bytes that require scanning
uint64_t size = 0;
for (size_t i = 0; i < files_brief.num_files; i++) {
const uint64_t start =
ApproximateSize(v, files_brief.files[i], key_start, for_compaction);
ApproximateSize(v, files_brief.files[i], key_start, caller);
const uint64_t end =
ApproximateSize(v, files_brief.files[i], key_end, for_compaction);
ApproximateSize(v, files_brief.files[i], key_end, caller);
assert(end >= start);
size += end - start;
}
@ -4902,7 +4911,8 @@ uint64_t VersionSet::ApproximateSizeLevel0(Version* v,
}
uint64_t VersionSet::ApproximateSize(Version* v, const FdWithKeyRange& f,
const Slice& key, bool for_compaction) {
const Slice& key,
TableReaderCaller caller) {
// pre-condition
assert(v);
@ -4920,9 +4930,13 @@ uint64_t VersionSet::ApproximateSize(Version* v, const FdWithKeyRange& f,
InternalIterator* iter = v->cfd_->table_cache()->NewIterator(
ReadOptions(), v->env_options_, v->cfd_->internal_comparator(),
*f.file_metadata, nullptr /* range_del_agg */,
v->GetMutableCFOptions().prefix_extractor.get(), &table_reader_ptr);
v->GetMutableCFOptions().prefix_extractor.get(), &table_reader_ptr,
/*file_read_hist=*/nullptr, caller,
/*arena=*/nullptr, /*skip_filters=*/false, /*level=*/-1,
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr);
if (table_reader_ptr != nullptr) {
result = table_reader_ptr->ApproximateOffsetOf(key, for_compaction);
result = table_reader_ptr->ApproximateOffsetOf(key, caller);
}
delete iter;
}
@ -5001,10 +5015,12 @@ InternalIterator* VersionSet::MakeInputIterator(
read_options, env_options_compactions, cfd->internal_comparator(),
*flevel->files[i].file_metadata, range_del_agg,
c->mutable_cf_options()->prefix_extractor.get(),
nullptr /* table_reader_ptr */,
nullptr /* no per level latency histogram */,
true /* for_compaction */, nullptr /* arena */,
false /* skip_filters */, static_cast<int>(which) /* level */);
/*table_reader_ptr=*/nullptr,
/*file_read_hist=*/nullptr, TableReaderCaller::kCompaction,
/*arena=*/nullptr,
/*skip_filters=*/false, /*level=*/static_cast<int>(which),
/*smallest_compaction_key=*/nullptr,
/*largest_compaction_key=*/nullptr);
}
} else {
// Create concatenating iterator for the files from this level
@ -5012,10 +5028,10 @@ InternalIterator* VersionSet::MakeInputIterator(
cfd->table_cache(), read_options, env_options_compactions,
cfd->internal_comparator(), c->input_levels(which),
c->mutable_cf_options()->prefix_extractor.get(),
false /* should_sample */,
nullptr /* no per level latency histogram */,
true /* for_compaction */, false /* skip_filters */,
static_cast<int>(which) /* level */, range_del_agg,
/*should_sample=*/false,
/*no per level latency histogram=*/nullptr,
TableReaderCaller::kCompaction, /*skip_filters=*/false,
/*level=*/static_cast<int>(which), range_del_agg,
c->boundaries(which));
}
}

@ -984,7 +984,8 @@ class VersionSet {
// in levels [start_level, end_level). If end_level == 0 it will search
// through all non-empty levels
uint64_t ApproximateSize(Version* v, const Slice& start, const Slice& end,
int start_level, int end_level, bool for_compaction);
int start_level, int end_level,
TableReaderCaller caller);
// Return the size of the current manifest file
uint64_t manifest_file_size() const { return manifest_file_size_; }
@ -1035,10 +1036,10 @@ class VersionSet {
// ApproximateSize helper
uint64_t ApproximateSizeLevel0(Version* v, const LevelFilesBrief& files_brief,
const Slice& start, const Slice& end,
bool for_compaction);
TableReaderCaller caller);
uint64_t ApproximateSize(Version* v, const FdWithKeyRange& f,
const Slice& key, bool for_compaction);
const Slice& key, TableReaderCaller caller);
// Save current contents to *log
Status WriteSnapshot(log::Writer* log);

@ -349,7 +349,9 @@ class PartitionIndexReader : public BlockBasedTable::IndexReaderCommon {
nullptr, kNullStats, true, index_key_includes_seq(),
index_value_is_full()),
false, true, /* prefix_extractor */ nullptr, BlockType::kIndex,
index_key_includes_seq(), index_value_is_full());
index_key_includes_seq(), index_value_is_full(),
lookup_context ? lookup_context->caller
: TableReaderCaller::kUncategorized);
}
assert(it != nullptr);
@ -365,7 +367,7 @@ class PartitionIndexReader : public BlockBasedTable::IndexReaderCommon {
void CacheDependencies(bool pin) override {
// Before read partitions, prefetch them to avoid lots of IOs
BlockCacheLookupContext lookup_context{BlockCacheLookupCaller::kPrefetch};
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
auto rep = table()->rep_;
IndexBlockIter biter;
BlockHandle handle;
@ -1075,7 +1077,7 @@ Status BlockBasedTable::Open(
// Better not mutate rep_ after the creation. eg. internal_prefix_transform
// raw pointer will be used to create HashIndexReader, whose reset may
// access a dangling pointer.
BlockCacheLookupContext lookup_context{BlockCacheLookupCaller::kPrefetch};
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
Rep* rep = new BlockBasedTable::Rep(ioptions, env_options, table_options,
internal_comparator, skip_filters, level,
immortal_table);
@ -2681,7 +2683,7 @@ void BlockBasedTableIterator<TBlockIter, TValue>::InitDataBlock() {
// Enabled after 2 sequential IOs when ReadOptions.readahead_size == 0.
// Explicit user requested readahead:
// Enabled from the very first IO when ReadOptions.readahead_size is set.
if (!for_compaction_) {
if (lookup_context_.caller != TableReaderCaller::kCompaction) {
if (read_options_.readahead_size == 0) {
// Implicit auto readahead
num_file_reads_++;
@ -2728,7 +2730,8 @@ void BlockBasedTableIterator<TBlockIter, TValue>::InitDataBlock() {
read_options_, data_block_handle, &block_iter_, block_type_,
key_includes_seq_, index_key_is_full_,
/*get_context=*/nullptr, &lookup_context_, s, prefetch_buffer_.get(),
for_compaction_);
/*for_compaction=*/lookup_context_.caller ==
TableReaderCaller::kCompaction);
block_iter_points_to_real_block_ = true;
}
}
@ -2814,11 +2817,8 @@ void BlockBasedTableIterator<TBlockIter, TValue>::CheckOutOfBound() {
InternalIterator* BlockBasedTable::NewIterator(
const ReadOptions& read_options, const SliceTransform* prefix_extractor,
Arena* arena, bool skip_filters, bool for_compaction,
size_t compaction_readahead_size) {
BlockCacheLookupContext lookup_context{
for_compaction ? BlockCacheLookupCaller::kCompaction
: BlockCacheLookupCaller::kUserIterator};
Arena* arena, bool skip_filters, TableReaderCaller caller, size_t compaction_readahead_size) {
BlockCacheLookupContext lookup_context{caller};
bool need_upper_bound_check =
PrefixExtractorChanged(rep_->table_properties.get(), prefix_extractor);
if (arena == nullptr) {
@ -2832,7 +2832,7 @@ InternalIterator* BlockBasedTable::NewIterator(
!skip_filters && !read_options.total_order_seek &&
prefix_extractor != nullptr,
need_upper_bound_check, prefix_extractor, BlockType::kData,
true /*key_includes_seq*/, true /*index_key_is_full*/, for_compaction,
/*key_includes_seq=*/true, /*index_key_is_full=*/true, caller,
compaction_readahead_size);
} else {
auto* mem =
@ -2845,8 +2845,7 @@ InternalIterator* BlockBasedTable::NewIterator(
!skip_filters && !read_options.total_order_seek &&
prefix_extractor != nullptr,
need_upper_bound_check, prefix_extractor, BlockType::kData,
true /*key_includes_seq*/, true /*index_key_is_full*/, for_compaction,
compaction_readahead_size);
/*key_includes_seq=*/true, /*index_key_is_full=*/true, caller, compaction_readahead_size);
}
}
@ -2933,7 +2932,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
CachableEntry<FilterBlockReader> filter_entry;
bool may_match;
FilterBlockReader* filter = nullptr;
BlockCacheLookupContext lookup_context{BlockCacheLookupCaller::kUserGet};
BlockCacheLookupContext lookup_context{TableReaderCaller::kUserGet};
{
if (!skip_filters) {
filter_entry = GetFilter(prefix_extractor, /*prefetch_buffer=*/nullptr,
@ -2989,7 +2988,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
break;
} else {
BlockCacheLookupContext lookup_data_block_context{
BlockCacheLookupCaller::kUserGet};
TableReaderCaller::kUserGet};
bool does_referenced_key_exist = false;
DataBlockIter biter;
uint64_t referenced_data_size = 0;
@ -3084,7 +3083,7 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
const MultiGetRange* mget_range,
const SliceTransform* prefix_extractor,
bool skip_filters) {
BlockCacheLookupContext lookup_context{BlockCacheLookupCaller::kUserMGet};
BlockCacheLookupContext lookup_context{TableReaderCaller::kUserMultiGet};
const bool no_io = read_options.read_tier == kBlockCacheTier;
CachableEntry<FilterBlockReader> filter_entry;
FilterBlockReader* filter = nullptr;
@ -3135,7 +3134,7 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
uint64_t referenced_data_size = 0;
bool does_referenced_key_exist = false;
BlockCacheLookupContext lookup_data_block_context(
BlockCacheLookupCaller::kUserMGet);
TableReaderCaller::kUserMultiGet);
if (iiter->value().offset() != offset) {
offset = iiter->value().offset();
biter.Invalidate(Status::OK());
@ -3244,7 +3243,7 @@ Status BlockBasedTable::Prefetch(const Slice* const begin,
if (begin && end && comparator.Compare(*begin, *end) > 0) {
return Status::InvalidArgument(*begin, *end);
}
BlockCacheLookupContext lookup_context{BlockCacheLookupCaller::kPrefetch};
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
IndexBlockIter iiter_on_stack;
auto iiter = NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false,
&iiter_on_stack, /*get_context=*/nullptr,
@ -3299,9 +3298,7 @@ Status BlockBasedTable::Prefetch(const Slice* const begin,
return Status::OK();
}
Status BlockBasedTable::VerifyChecksum() {
// TODO(haoyu): This function is called by external sst ingestion and the
// verify checksum public API. We don't log its block cache accesses for now.
Status BlockBasedTable::VerifyChecksum(TableReaderCaller caller) {
Status s;
// Check Meta blocks
std::unique_ptr<Block> meta;
@ -3317,9 +3314,10 @@ Status BlockBasedTable::VerifyChecksum() {
}
// Check Data blocks
IndexBlockIter iiter_on_stack;
BlockCacheLookupContext context{caller};
InternalIteratorBase<BlockHandle>* iiter = NewIndexIterator(
ReadOptions(), /*need_upper_bound_check=*/false, &iiter_on_stack,
/*get_context=*/nullptr, /*lookup_contex=*/nullptr);
/*get_context=*/nullptr, &context);
std::unique_ptr<InternalIteratorBase<BlockHandle>> iiter_unique_ptr;
if (iiter != &iiter_on_stack) {
iiter_unique_ptr =
@ -3536,10 +3534,8 @@ Status BlockBasedTable::CreateIndexReader(
}
uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key,
bool for_compaction) {
BlockCacheLookupContext context(
for_compaction ? BlockCacheLookupCaller::kCompaction
: BlockCacheLookupCaller::kUserApproximateSize);
TableReaderCaller caller) {
BlockCacheLookupContext context(caller);
std::unique_ptr<InternalIteratorBase<BlockHandle>> index_iter(
NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false,
/*input_iter=*/nullptr, /*get_context=*/nullptr,

@ -123,18 +123,13 @@ class BlockBasedTable : public TableReader {
// The result of NewIterator() is initially invalid (caller must
// call one of the Seek methods on the iterator before using it).
// @param skip_filters Disables loading/accessing the filter block
// compaction_readahead_size: its value will only be used if for_compaction =
// true
InternalIterator* NewIterator(
const ReadOptions&, const SliceTransform* prefix_extractor,
Arena* arena = nullptr, bool skip_filters = false,
// TODO(haoyu) 1. External SST ingestion sets for_compaction as false. 2.
// Compaction also sets it to false when paranoid_file_checks is true,
// i.e., it will populate the block cache with blocks in the new SST
// files. We treat those as a user is calling iterator for now. We should
// differentiate the callers.
bool for_compaction = false,
size_t compaction_readahead_size = 0) override;
// compaction_readahead_size: its value will only be used if caller =
// kCompaction.
InternalIterator* NewIterator(const ReadOptions&,
const SliceTransform* prefix_extractor,
Arena* arena, bool skip_filters,
TableReaderCaller caller,
size_t compaction_readahead_size = 0) override;
FragmentedRangeTombstoneIterator* NewRangeTombstoneIterator(
const ReadOptions& read_options) override;
@ -160,7 +155,8 @@ class BlockBasedTable : public TableReader {
// bytes, and so includes effects like compression of the underlying data.
// E.g., the approximate offset of the last key in the table will
// be close to the file length.
uint64_t ApproximateOffsetOf(const Slice& key, bool for_compaction) override;
uint64_t ApproximateOffsetOf(const Slice& key,
TableReaderCaller caller) override;
bool TEST_BlockInCache(const BlockHandle& handle) const;
@ -180,7 +176,7 @@ class BlockBasedTable : public TableReader {
Status DumpTable(WritableFile* out_file,
const SliceTransform* prefix_extractor = nullptr) override;
Status VerifyChecksum() override;
Status VerifyChecksum(TableReaderCaller caller) override;
void Close() override;
@ -609,9 +605,8 @@ class BlockBasedTableIterator : public InternalIteratorBase<TValue> {
InternalIteratorBase<BlockHandle>* index_iter,
bool check_filter, bool need_upper_bound_check,
const SliceTransform* prefix_extractor,
BlockType block_type, bool key_includes_seq = true,
bool index_key_is_full = true,
bool for_compaction = false,
BlockType block_type, bool key_includes_seq,
bool index_key_is_full, TableReaderCaller caller,
size_t compaction_readahead_size = 0)
: InternalIteratorBase<TValue>(false),
table_(table),
@ -627,11 +622,8 @@ class BlockBasedTableIterator : public InternalIteratorBase<TValue> {
block_type_(block_type),
key_includes_seq_(key_includes_seq),
index_key_is_full_(index_key_is_full),
for_compaction_(for_compaction),
compaction_readahead_size_(compaction_readahead_size),
lookup_context_(for_compaction
? BlockCacheLookupCaller::kCompaction
: BlockCacheLookupCaller::kUserIterator) {}
lookup_context_(caller),
compaction_readahead_size_(compaction_readahead_size) {}
~BlockBasedTableIterator() { delete index_iter_; }
@ -740,13 +732,11 @@ class BlockBasedTableIterator : public InternalIteratorBase<TValue> {
// If the keys in the blocks over which we iterate include 8 byte sequence
bool key_includes_seq_;
bool index_key_is_full_;
// If this iterator is created for compaction
bool for_compaction_;
// Readahead size used in compaction, its value is used only if
// for_compaction_ = true
size_t compaction_readahead_size_;
BlockHandle prev_index_value_;
BlockCacheLookupContext lookup_context_;
// Readahead size used in compaction, its value is used only if
// lookup_context_.caller = kCompaction.
size_t compaction_readahead_size_;
// All the below fields control iterator readahead
static const size_t kInitAutoReadaheadSize = 8 * 1024;

@ -277,7 +277,7 @@ size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const {
void PartitionedFilterBlockReader::CacheDependencies(
bool pin, const SliceTransform* prefix_extractor) {
// Before read partitions, prefetch them to avoid lots of IOs
BlockCacheLookupContext lookup_context{BlockCacheLookupCaller::kPrefetch};
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
IndexBlockIter biter;
Statistics* kNullStats = nullptr;
idx_on_fltr_blk_->NewIterator<IndexBlockIter>(

@ -377,7 +377,7 @@ Slice CuckooTableIterator::value() const {
InternalIterator* CuckooTableReader::NewIterator(
const ReadOptions& /*read_options*/,
const SliceTransform* /* prefix_extractor */, Arena* arena,
bool /*skip_filters*/, bool /*for_compaction*/,
bool /*skip_filters*/, TableReaderCaller /*caller*/,
size_t /*compaction_readahead_size*/) {
if (!status().ok()) {
return NewErrorInternalIterator<Slice>(

@ -50,10 +50,8 @@ class CuckooTableReader: public TableReader {
// true
InternalIterator* NewIterator(const ReadOptions&,
const SliceTransform* prefix_extractor,
Arena* arena = nullptr,
bool skip_filters = false,
bool for_compaction = false,
size_t compaction_readahead_size = 0) override;
Arena* arena, bool skip_filters,
TableReaderCaller caller, size_t compaction_readahead_size = 0) override;
void Prepare(const Slice& target) override;
// Report an approximation of how much memory has been used.
@ -61,7 +59,7 @@ class CuckooTableReader: public TableReader {
// Following methods are not implemented for Cuckoo Table Reader
uint64_t ApproximateOffsetOf(const Slice& /*key*/,
bool /*for_compaction*/ = false) override {
TableReaderCaller /*caller*/) override {
return 0;
}
void SetupForCompaction() override {}

@ -146,8 +146,9 @@ class CuckooReaderTest : public testing::Test {
CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucomp,
GetSliceHash);
ASSERT_OK(reader.status());
InternalIterator* it =
reader.NewIterator(ReadOptions(), nullptr, nullptr, false);
InternalIterator* it = reader.NewIterator(
ReadOptions(), /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized);
ASSERT_OK(it->status());
ASSERT_TRUE(!it->Valid());
it->SeekToFirst();
@ -186,7 +187,9 @@ class CuckooReaderTest : public testing::Test {
delete it;
Arena arena;
it = reader.NewIterator(ReadOptions(), nullptr, &arena);
it = reader.NewIterator(ReadOptions(), /*prefix_extractor=*/nullptr, &arena,
/*skip_filters=*/false,
TableReaderCaller::kUncategorized);
ASSERT_OK(it->status());
ASSERT_TRUE(!it->Valid());
it->Seek(keys[num_items/2]);

@ -34,8 +34,7 @@ stl_wrappers::KVMap MakeMockFile(
InternalIterator* MockTableReader::NewIterator(
const ReadOptions&, const SliceTransform* /* prefix_extractor */,
Arena* /*arena*/, bool /*skip_filters*/, bool /*for_compaction*/,
size_t /*compaction_readahead_size*/) {
Arena* /*arena*/, bool /*skip_filters*/, TableReaderCaller /*caller*/, size_t /*compaction_readahead_size*/) {
return new MockTableIterator(table_);
}

@ -42,17 +42,16 @@ class MockTableReader : public TableReader {
InternalIterator* NewIterator(const ReadOptions&,
const SliceTransform* prefix_extractor,
Arena* arena = nullptr,
bool skip_filters = false,
bool for_compaction = false,
size_t compaction_readahead_size = 0) override;
Arena* arena, bool skip_filters,
TableReaderCaller caller,
size_t compaction_readahead_size = 0) override;
Status Get(const ReadOptions& readOptions, const Slice& key,
GetContext* get_context, const SliceTransform* prefix_extractor,
bool skip_filters = false) override;
uint64_t ApproximateOffsetOf(const Slice& /*key*/,
bool /*for_compaction*/ = false) override {
TableReaderCaller /*caller*/) override {
return 0;
}

@ -196,7 +196,7 @@ void PlainTableReader::SetupForCompaction() {
InternalIterator* PlainTableReader::NewIterator(
const ReadOptions& options, const SliceTransform* /* prefix_extractor */,
Arena* arena, bool /*skip_filters*/, bool /*for_compaction*/,
Arena* arena, bool /*skip_filters*/, TableReaderCaller /*caller*/,
size_t /*compaction_readahead_size*/) {
bool use_prefix_seek = !IsTotalOrderMode() && !options.total_order_seek;
if (arena == nullptr) {
@ -616,7 +616,7 @@ Status PlainTableReader::Get(const ReadOptions& /*ro*/, const Slice& target,
}
uint64_t PlainTableReader::ApproximateOffsetOf(const Slice& /*key*/,
bool /*for_compaction*/) {
TableReaderCaller /*caller*/) {
return 0;
}

@ -82,10 +82,8 @@ class PlainTableReader: public TableReader {
// true
InternalIterator* NewIterator(const ReadOptions&,
const SliceTransform* prefix_extractor,
Arena* arena = nullptr,
bool skip_filters = false,
bool for_compaction = false,
size_t compaction_readahead_size = 0) override;
Arena* arena, bool skip_filters,
TableReaderCaller caller, size_t compaction_readahead_size = 0) override;
void Prepare(const Slice& target) override;
@ -94,7 +92,7 @@ class PlainTableReader: public TableReader {
bool skip_filters = false) override;
uint64_t ApproximateOffsetOf(const Slice& key,
bool for_compaction = false) override;
TableReaderCaller caller) override;
uint32_t GetIndexSize() const { return index_.GetIndexSize(); }
void SetupForCompaction() override;

@ -65,8 +65,9 @@ Iterator* SstFileReader::NewIterator(const ReadOptions& options) {
auto sequence = options.snapshot != nullptr
? options.snapshot->GetSequenceNumber()
: kMaxSequenceNumber;
auto internal_iter =
r->table_reader->NewIterator(options, r->moptions.prefix_extractor.get());
auto internal_iter = r->table_reader->NewIterator(
options, r->moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kSSTFileReader);
return NewDBIterator(r->options.env, options, r->ioptions, r->moptions,
r->ioptions.user_comparator, internal_iter, sequence,
r->moptions.max_sequential_skip_in_iterations,
@ -79,7 +80,7 @@ std::shared_ptr<const TableProperties> SstFileReader::GetTableProperties()
}
Status SstFileReader::VerifyChecksum() {
return rep_->table_reader->VerifyChecksum();
return rep_->table_reader->VerifyChecksum(TableReaderCaller::kSSTFileReader);
}
} // namespace rocksdb

@ -14,6 +14,7 @@
#include "table/get_context.h"
#include "table/internal_iterator.h"
#include "table/multiget_context.h"
#include "table/table_reader_caller.h"
namespace rocksdb {
@ -44,12 +45,11 @@ class TableReader {
// all the states but those allocated in arena.
// skip_filters: disables checking the bloom filters even if they exist. This
// option is effective only for block-based table format.
// compaction_readahead_size: its value will only be used if for_compaction =
// true
virtual InternalIterator* NewIterator(
const ReadOptions&, const SliceTransform* prefix_extractor,
Arena* arena = nullptr, bool skip_filters = false,
bool for_compaction = false, size_t compaction_readahead_size = 0) = 0;
// compaction_readahead_size: its value will only be used if caller = kCompaction
virtual InternalIterator* NewIterator(const ReadOptions&,
const SliceTransform* prefix_extractor,
Arena* arena, bool skip_filters,
TableReaderCaller caller, size_t compaction_readahead_size = 0) = 0;
virtual FragmentedRangeTombstoneIterator* NewRangeTombstoneIterator(
const ReadOptions& /*read_options*/) {
@ -63,7 +63,7 @@ class TableReader {
// E.g., the approximate offset of the last key in the table will
// be close to the file length.
virtual uint64_t ApproximateOffsetOf(const Slice& key,
bool for_compaction = false) = 0;
TableReaderCaller caller) = 0;
// Set up the table for Compaction. Might change some parameters with
// posix_fadvise
@ -122,7 +122,7 @@ class TableReader {
}
// check whether there is corruption in this db file
virtual Status VerifyChecksum() {
virtual Status VerifyChecksum(TableReaderCaller /*caller*/) {
return Status::NotSupported("VerifyChecksum() not supported");
}

@ -198,7 +198,9 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
Iterator* iter = nullptr;
InternalIterator* iiter = nullptr;
if (!through_db) {
iiter = table_reader->NewIterator(read_options, nullptr);
iiter = table_reader->NewIterator(
read_options, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized);
} else {
iter = db->NewIterator(read_options);
}

@ -0,0 +1,39 @@
// 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).
#pragma once
namespace rocksdb {
// A list of callers for a table reader. It is used to trace the caller that
// accesses on a block. This is only used for block cache tracing and analysis.
// A user may use kUncategorized if the caller is not interesting for analysis
// or the table reader is called in the test environment, e.g., unit test, table
// reader benchmark, etc.
enum TableReaderCaller : char {
kUserGet = 1,
kUserMultiGet = 2,
kUserIterator = 3,
kUserApproximateSize = 4,
kUserVerifyChecksum = 5,
kSSTDumpTool = 6,
kExternalSSTIngestion = 7,
kRepair = 8,
kPrefetch = 9,
kCompaction = 10,
// A compaction job may refill the block cache with blocks in the new SST
// files if paranoid_file_checks is true.
kCompactionRefill = 11,
// After building a table, it may load all its blocks into the block cache if
// paranoid_file_checks is true.
kFlush = 12,
// sst_file_reader.
kSSTFileReader = 13,
// A list of callers that are either not interesting for analysis or are
// calling from a test environment, e.g., unit test, benchmark, etc.
kUncategorized = 14,
// All callers should be added before kMaxBlockCacheLookupCaller.
kMaxBlockCacheLookupCaller
};
} // namespace rocksdb

@ -370,7 +370,9 @@ class TableConstructor: public Constructor {
InternalIterator* NewIterator(
const SliceTransform* prefix_extractor) const override {
ReadOptions ro;
InternalIterator* iter = table_reader_->NewIterator(ro, prefix_extractor);
InternalIterator* iter = table_reader_->NewIterator(
ro, prefix_extractor, /*arena=*/nullptr, /*skip_filters=*/false,
TableReaderCaller::kUncategorized);
if (convert_to_internal_key_) {
return new KeyConvertingIterator(iter);
} else {
@ -382,9 +384,11 @@ class TableConstructor: public Constructor {
if (convert_to_internal_key_) {
InternalKey ikey(key, kMaxSequenceNumber, kTypeValue);
const Slice skey = ikey.Encode();
return table_reader_->ApproximateOffsetOf(skey);
return table_reader_->ApproximateOffsetOf(
skey, TableReaderCaller::kUncategorized);
}
return table_reader_->ApproximateOffsetOf(key);
return table_reader_->ApproximateOffsetOf(
key, TableReaderCaller::kUncategorized);
}
virtual Status Reopen(const ImmutableCFOptions& ioptions,
@ -1538,8 +1542,9 @@ TEST_P(BlockBasedTableTest, TotalOrderSeekOnHashIndex) {
auto* reader = c.GetTableReader();
ReadOptions ro;
ro.total_order_seek = true;
std::unique_ptr<InternalIterator> iter(
reader->NewIterator(ro, moptions.prefix_extractor.get()));
std::unique_ptr<InternalIterator> iter(reader->NewIterator(
ro, moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
iter->Seek(InternalKey("b", 0, kTypeValue).Encode());
ASSERT_OK(iter->status());
@ -1597,8 +1602,9 @@ TEST_P(BlockBasedTableTest, NoopTransformSeek) {
for (int i = 0; i < 2; ++i) {
ReadOptions ro;
ro.total_order_seek = (i == 0);
std::unique_ptr<InternalIterator> iter(
reader->NewIterator(ro, moptions.prefix_extractor.get()));
std::unique_ptr<InternalIterator> iter(reader->NewIterator(
ro, moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
iter->Seek(key.Encode());
ASSERT_OK(iter->status());
@ -1635,8 +1641,9 @@ TEST_P(BlockBasedTableTest, SkipPrefixBloomFilter) {
const MutableCFOptions new_moptions(options);
c.Reopen(new_ioptions, new_moptions);
auto reader = c.GetTableReader();
std::unique_ptr<InternalIterator> db_iter(
reader->NewIterator(ReadOptions(), new_moptions.prefix_extractor.get()));
std::unique_ptr<InternalIterator> db_iter(reader->NewIterator(
ReadOptions(), new_moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
// Test point lookup
// only one kv
@ -1702,8 +1709,9 @@ void TableTest::IndexTest(BlockBasedTableOptions table_options) {
ASSERT_EQ(5u, props->num_data_blocks);
// TODO(Zhongyi): update test to use MutableCFOptions
std::unique_ptr<InternalIterator> index_iter(
reader->NewIterator(ReadOptions(), moptions.prefix_extractor.get()));
std::unique_ptr<InternalIterator> index_iter(reader->NewIterator(
ReadOptions(), moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
// -- Find keys do not exist, but have common prefix.
std::vector<std::string> prefixes = {"001", "003", "005", "007", "009"};
@ -1819,8 +1827,9 @@ TEST_P(BlockBasedTableTest, IndexSeekOptimizationIncomplete) {
auto reader = c.GetTableReader();
ReadOptions ropt;
ropt.read_tier = ReadTier::kBlockCacheTier;
std::unique_ptr<InternalIterator> iter(
reader->NewIterator(ropt, /* prefix_extractor */ nullptr));
std::unique_ptr<InternalIterator> iter(reader->NewIterator(
ropt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
auto ikey = [](Slice user_key) {
return InternalKey(user_key, 0, kTypeValue).Encode().ToString();
@ -3136,8 +3145,9 @@ TEST_P(IndexBlockRestartIntervalTest, IndexBlockRestartInterval) {
&kvmap);
auto reader = c.GetTableReader();
std::unique_ptr<InternalIterator> db_iter(
reader->NewIterator(ReadOptions(), moptions.prefix_extractor.get()));
std::unique_ptr<InternalIterator> db_iter(reader->NewIterator(
ReadOptions(), moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
// Test point lookup
for (auto& kv : kvmap) {
@ -3329,8 +3339,9 @@ TEST_P(BlockBasedTableTest, DISABLED_TableWithGlobalSeqno) {
EnvOptions(), ikc),
std::move(file_reader), ss_rw.contents().size(), &table_reader);
return table_reader->NewIterator(ReadOptions(),
moptions.prefix_extractor.get());
return table_reader->NewIterator(
ReadOptions(), moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized);
};
GetVersionAndGlobalSeqno();
@ -3501,7 +3512,8 @@ TEST_P(BlockBasedTableTest, BlockAlignTest) {
std::move(file_reader), ss_rw.contents().size(), &table_reader));
std::unique_ptr<InternalIterator> db_iter(table_reader->NewIterator(
ReadOptions(), moptions2.prefix_extractor.get()));
ReadOptions(), moptions2.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
int expected_key = 1;
for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
@ -3795,8 +3807,9 @@ TEST_P(BlockBasedTableTest, DataBlockHashIndex) {
auto reader = c.GetTableReader();
std::unique_ptr<InternalIterator> seek_iter;
seek_iter.reset(
reader->NewIterator(ReadOptions(), moptions.prefix_extractor.get()));
seek_iter.reset(reader->NewIterator(
ReadOptions(), moptions.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized));
for (int i = 0; i < 2; ++i) {
ReadOptions ro;
// for every kv, we seek using two method: Get() and Seek()
@ -3877,13 +3890,15 @@ TEST_P(BlockBasedTableTest, OutOfBoundOnSeek) {
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.reset(new KeyConvertingIterator(reader->NewIterator(
read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized)));
iter->SeekToFirst();
ASSERT_FALSE(iter->Valid());
ASSERT_TRUE(iter->IsOutOfBound());
iter.reset(new KeyConvertingIterator(
reader->NewIterator(read_opt, nullptr /*prefix_extractor*/)));
iter.reset(new KeyConvertingIterator(reader->NewIterator(
read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized)));
iter->Seek("foo");
ASSERT_FALSE(iter->Valid());
ASSERT_TRUE(iter->IsOutOfBound());
@ -3913,8 +3928,9 @@ TEST_P(BlockBasedTableTest, OutOfBoundOnNext) {
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.reset(new KeyConvertingIterator(reader->NewIterator(
read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized)));
iter->Seek("bar");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ("bar", iter->key());
@ -3924,8 +3940,9 @@ TEST_P(BlockBasedTableTest, OutOfBoundOnNext) {
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.reset(new KeyConvertingIterator(reader->NewIterator(
read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kUncategorized)));
iter->Seek("foo");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ("foo", iter->key());

@ -67,18 +67,36 @@ std::string block_type_to_string(TraceType type) {
return "InvalidType";
}
std::string caller_to_string(BlockCacheLookupCaller caller) {
std::string caller_to_string(TableReaderCaller caller) {
switch (caller) {
case kUserGet:
return "Get";
case kUserMGet:
case kUserMultiGet:
return "MultiGet";
case kUserIterator:
return "Iterator";
case kUserApproximateSize:
return "ApproximateSize";
case kUserVerifyChecksum:
return "VerifyChecksum";
case kSSTDumpTool:
return "SSTDumpTool";
case kExternalSSTIngestion:
return "ExternalSSTIngestion";
case kRepair:
return "Repair";
case kPrefetch:
return "Prefetch";
case kCompaction:
return "Compaction";
case kCompactionRefill:
return "CompactionRefill";
case kFlush:
return "Flush";
case kSSTFileReader:
return "SSTFileReader";
case kUncategorized:
return "Uncategorized";
default:
break;
}
@ -450,10 +468,10 @@ void BlockCacheTraceAnalyzer::PrintStatsSummary() const {
uint64_t total_num_blocks = 0;
uint64_t total_num_accesses = 0;
std::map<TraceType, uint64_t> bt_num_blocks_map;
std::map<BlockCacheLookupCaller, uint64_t> caller_num_access_map;
std::map<BlockCacheLookupCaller, std::map<TraceType, uint64_t>>
std::map<TableReaderCaller, uint64_t> caller_num_access_map;
std::map<TableReaderCaller, std::map<TraceType, uint64_t>>
caller_bt_num_access_map;
std::map<BlockCacheLookupCaller, std::map<uint32_t, uint64_t>>
std::map<TableReaderCaller, std::map<uint32_t, uint64_t>>
caller_level_num_access_map;
for (auto const& cf_aggregates : cf_aggregates_map_) {
// Stats per column family.
@ -462,12 +480,12 @@ void BlockCacheTraceAnalyzer::PrintStatsSummary() const {
uint64_t cf_num_blocks = 0;
std::map<TraceType, uint64_t> cf_bt_blocks;
uint64_t cf_num_accesses = 0;
std::map<BlockCacheLookupCaller, uint64_t> cf_caller_num_accesses_map;
std::map<BlockCacheLookupCaller, std::map<uint64_t, uint64_t>>
std::map<TableReaderCaller, uint64_t> cf_caller_num_accesses_map;
std::map<TableReaderCaller, std::map<uint64_t, uint64_t>>
cf_caller_level_num_accesses_map;
std::map<BlockCacheLookupCaller, std::map<uint64_t, uint64_t>>
std::map<TableReaderCaller, std::map<uint64_t, uint64_t>>
cf_caller_file_num_accesses_map;
std::map<BlockCacheLookupCaller, std::map<TraceType, uint64_t>>
std::map<TableReaderCaller, std::map<TraceType, uint64_t>>
cf_caller_bt_num_accesses_map;
total_num_files += cf_aggregates.second.fd_aggregates_map.size();
for (auto const& file_aggregates : cf_aggregates.second.fd_aggregates_map) {
@ -492,7 +510,7 @@ void BlockCacheTraceAnalyzer::PrintStatsSummary() const {
for (auto const& stats :
block_access_info.second.caller_num_access_map) {
// Stats per caller.
const BlockCacheLookupCaller caller = stats.first;
const TableReaderCaller caller = stats.first;
const uint64_t num_accesses = stats.second;
// Overall stats.
total_num_accesses += num_accesses;

@ -72,7 +72,7 @@ struct BlockAccessInfo {
std::map<std::string, uint64_t>
non_exist_key_num_access_map; // for keys do not exist in this block.
uint64_t num_referenced_key_exist_in_block = 0;
std::map<BlockCacheLookupCaller, uint64_t> caller_num_access_map;
std::map<TableReaderCaller, uint64_t> caller_num_access_map;
void AddAccess(const BlockCacheTraceRecord& access) {
if (first_access_time == 0) {

@ -61,23 +61,23 @@ class BlockCacheTracerTest : public testing::Test {
EXPECT_OK(env_->DeleteDir(test_path_));
}
BlockCacheLookupCaller GetCaller(uint32_t key_id) {
TableReaderCaller GetCaller(uint32_t key_id) {
uint32_t n = key_id % 5;
switch (n) {
case 0:
return BlockCacheLookupCaller::kPrefetch;
return TableReaderCaller::kPrefetch;
case 1:
return BlockCacheLookupCaller::kCompaction;
return TableReaderCaller::kCompaction;
case 2:
return BlockCacheLookupCaller::kUserGet;
return TableReaderCaller::kUserGet;
case 3:
return BlockCacheLookupCaller::kUserMGet;
return TableReaderCaller::kUserMultiGet;
case 4:
return BlockCacheLookupCaller::kUserIterator;
return TableReaderCaller::kUserIterator;
}
// This cannot happend.
assert(false);
return BlockCacheLookupCaller::kUserGet;
return TableReaderCaller::kMaxBlockCacheLookupCaller;
}
void WriteBlockAccess(BlockCacheTraceWriter* writer, uint32_t from_key_id,
@ -124,15 +124,15 @@ class BlockCacheTracerTest : public testing::Test {
ASSERT_GT(block_access_info.first_access_time, 0);
ASSERT_GT(block_access_info.last_access_time, 0);
ASSERT_EQ(1, block_access_info.caller_num_access_map.size());
BlockCacheLookupCaller expected_caller = GetCaller(key_id);
TableReaderCaller expected_caller = GetCaller(key_id);
ASSERT_TRUE(block_access_info.caller_num_access_map.find(expected_caller) !=
block_access_info.caller_num_access_map.end());
ASSERT_EQ(
1,
block_access_info.caller_num_access_map.find(expected_caller)->second);
if ((expected_caller == BlockCacheLookupCaller::kUserGet ||
expected_caller == BlockCacheLookupCaller::kUserMGet) &&
if ((expected_caller == TableReaderCaller::kUserGet ||
expected_caller == TableReaderCaller::kUserMultiGet) &&
type == TraceType::kBlockTraceDataBlock) {
ASSERT_EQ(kNumKeysInBlock, block_access_info.num_keys);
ASSERT_EQ(1, block_access_info.key_num_access_map.size());

@ -143,7 +143,7 @@ Status SstFileDumper::NewTableReader(
}
Status SstFileDumper::VerifyChecksum() {
return table_reader_->VerifyChecksum();
return table_reader_->VerifyChecksum(TableReaderCaller::kSSTDumpTool);
}
Status SstFileDumper::DumpTable(const std::string& out_filename) {
@ -173,7 +173,8 @@ uint64_t SstFileDumper::CalculateCompressedTableSize(
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
dest_writer.get()));
std::unique_ptr<InternalIterator> iter(table_reader_->NewIterator(
ReadOptions(), moptions_.prefix_extractor.get()));
ReadOptions(), moptions_.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kSSTDumpTool));
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
if (!iter->status().ok()) {
fputs(iter->status().ToString().c_str(), stderr);
@ -299,7 +300,9 @@ Status SstFileDumper::ReadSequential(bool print_kv, uint64_t read_num,
}
InternalIterator* iter = table_reader_->NewIterator(
ReadOptions(verify_checksum_, false), moptions_.prefix_extractor.get());
ReadOptions(verify_checksum_, false), moptions_.prefix_extractor.get(),
/*arena=*/nullptr, /*skip_filters=*/false,
TableReaderCaller::kSSTDumpTool);
uint64_t i = 0;
if (has_from) {
InternalKey ikey;

@ -31,11 +31,11 @@ bool ShouldTrace(const Slice& block_key, const TraceOptions& trace_options) {
const std::string BlockCacheTraceHelper::kUnknownColumnFamilyName =
"UnknownColumnFamily";
bool BlockCacheTraceHelper::ShouldTraceReferencedKey(
TraceType block_type, BlockCacheLookupCaller caller) {
bool BlockCacheTraceHelper::ShouldTraceReferencedKey(TraceType block_type,
TableReaderCaller caller) {
return (block_type == TraceType::kBlockTraceDataBlock) &&
(caller == BlockCacheLookupCaller::kUserGet ||
caller == BlockCacheLookupCaller::kUserMGet);
(caller == TableReaderCaller::kUserGet ||
caller == TableReaderCaller::kUserMultiGet);
}
BlockCacheTraceWriter::BlockCacheTraceWriter(
@ -182,7 +182,7 @@ Status BlockCacheTraceReader::ReadAccess(BlockCacheTraceRecord* record) {
return Status::Incomplete(
"Incomplete access record: Failed to read caller.");
}
record->caller = static_cast<BlockCacheLookupCaller>(enc_slice[0]);
record->caller = static_cast<TableReaderCaller>(enc_slice[0]);
enc_slice.remove_prefix(kCharSize);
if (enc_slice.empty()) {
return Status::Incomplete(

@ -11,21 +11,11 @@
#include "rocksdb/env.h"
#include "rocksdb/options.h"
#include "rocksdb/trace_reader_writer.h"
#include "table/table_reader_caller.h"
#include "trace_replay/trace_replay.h"
namespace rocksdb {
enum BlockCacheLookupCaller : char {
kUserGet = 1,
kUserMGet = 2,
kUserIterator = 3,
kUserApproximateSize = 4,
kPrefetch = 5,
kCompaction = 6,
// All callers should be added before kMaxBlockCacheLookupCaller.
kMaxBlockCacheLookupCaller
};
// Lookup context for tracing block cache accesses.
// We trace block accesses at five places:
// 1. BlockBasedTable::GetFilter
@ -46,9 +36,8 @@ enum BlockCacheLookupCaller : char {
// 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or
// kUserApproximateSize).
struct BlockCacheLookupContext {
BlockCacheLookupContext(const BlockCacheLookupCaller& _caller)
: caller(_caller) {}
const BlockCacheLookupCaller caller;
BlockCacheLookupContext(const TableReaderCaller& _caller) : caller(_caller) {}
const TableReaderCaller caller;
// These are populated when we perform lookup/insert on block cache. The block
// cache tracer uses these inforation when logging the block access at
// BlockBasedTable::GET and BlockBasedTable::MultiGet.
@ -84,8 +73,7 @@ struct BlockCacheTraceRecord {
std::string cf_name;
uint32_t level = 0;
uint64_t sst_fd_number = 0;
BlockCacheLookupCaller caller =
BlockCacheLookupCaller::kMaxBlockCacheLookupCaller;
TableReaderCaller caller = TableReaderCaller::kMaxBlockCacheLookupCaller;
Boolean is_cache_hit = Boolean::kFalse;
Boolean no_insert = Boolean::kFalse;
@ -100,7 +88,7 @@ struct BlockCacheTraceRecord {
BlockCacheTraceRecord(uint64_t _access_timestamp, std::string _block_key,
TraceType _block_type, uint64_t _block_size,
uint64_t _cf_id, std::string _cf_name, uint32_t _level,
uint64_t _sst_fd_number, BlockCacheLookupCaller _caller,
uint64_t _sst_fd_number, TableReaderCaller _caller,
bool _is_cache_hit, bool _no_insert,
std::string _referenced_key = "",
uint64_t _referenced_data_size = 0,
@ -134,7 +122,7 @@ struct BlockCacheTraceHeader {
class BlockCacheTraceHelper {
public:
static bool ShouldTraceReferencedKey(TraceType block_type,
BlockCacheLookupCaller caller);
TableReaderCaller caller);
static const std::string kUnknownColumnFamilyName;
};

@ -37,19 +37,19 @@ class BlockCacheTracerTest : public testing::Test {
EXPECT_OK(env_->DeleteDir(test_path_));
}
BlockCacheLookupCaller GetCaller(uint32_t key_id) {
TableReaderCaller GetCaller(uint32_t key_id) {
uint32_t n = key_id % 5;
switch (n) {
case 0:
return BlockCacheLookupCaller::kPrefetch;
return TableReaderCaller::kPrefetch;
case 1:
return BlockCacheLookupCaller::kCompaction;
return TableReaderCaller::kCompaction;
case 2:
return BlockCacheLookupCaller::kUserGet;
return TableReaderCaller::kUserGet;
case 3:
return BlockCacheLookupCaller::kUserMGet;
return TableReaderCaller::kUserMultiGet;
case 4:
return BlockCacheLookupCaller::kUserIterator;
return TableReaderCaller::kUserIterator;
}
assert(false);
}
@ -121,8 +121,8 @@ class BlockCacheTracerTest : public testing::Test {
ASSERT_EQ(Boolean::kFalse, record.is_cache_hit);
ASSERT_EQ(Boolean::kFalse, record.no_insert);
if (block_type == TraceType::kBlockTraceDataBlock &&
(record.caller == BlockCacheLookupCaller::kUserGet ||
record.caller == BlockCacheLookupCaller::kUserMGet)) {
(record.caller == TableReaderCaller::kUserGet ||
record.caller == TableReaderCaller::kUserMultiGet)) {
ASSERT_EQ(kRefKeyPrefix + std::to_string(key_id),
record.referenced_key);
ASSERT_EQ(Boolean::kTrue, record.referenced_key_exist_in_block);

Loading…
Cancel
Save