diff --git a/db/blob/blob_file_reader.cc b/db/blob/blob_file_reader.cc index 1a4a4152fb2b..3c961d4faf7d 100644 --- a/db/blob/blob_file_reader.cc +++ b/db/blob/blob_file_reader.cc @@ -126,9 +126,9 @@ Status BlobFileReader::OpenFile( file_reader->reset(new RandomAccessFileReader( std::move(file), blob_file_path, immutable_options.clock, io_tracer, - immutable_options.stats, BLOB_DB_BLOB_FILE_READ_MICROS, - blob_file_read_hist, immutable_options.rate_limiter.get(), - immutable_options.listeners)); + immutable_options.stats, IOActivity::kBlob, + Histograms::HISTOGRAM_ENUM_MAX, blob_file_read_hist, + immutable_options.rate_limiter.get(), immutable_options.listeners)); return Status::OK(); } diff --git a/db/builder.cc b/db/builder.cc index 1508a28001b0..1610e1750644 100644 --- a/db/builder.cc +++ b/db/builder.cc @@ -57,7 +57,7 @@ Status BuildTable( const std::string& dbname, VersionSet* versions, const ImmutableDBOptions& db_options, const TableBuilderOptions& tboptions, const FileOptions& file_options, TableCache* table_cache, - InternalIterator* iter, + TableReaderCaller table_reader_caller, InternalIterator* iter, std::vector> range_del_iters, FileMetaData* meta, std::vector* blob_file_additions, @@ -256,8 +256,7 @@ Status BuildTable( approx_opts.files_size_error_margin = 0.1; meta->compensated_range_deletion_size += versions->ApproximateSize( approx_opts, version, kv.first.Encode(), tombstone_end.Encode(), - 0 /* start_level */, -1 /* end_level */, - TableReaderCaller::kFlush); + 0 /* start_level */, -1 /* end_level */, table_reader_caller); } last_tombstone_start_user_key = range_del_it->start_key(); } @@ -376,7 +375,7 @@ Status BuildTable( nullptr, (internal_stats == nullptr) ? nullptr : internal_stats->GetFileReadHist(0), - TableReaderCaller::kFlush, /*arena=*/nullptr, + table_reader_caller, /*arena=*/nullptr, /*skip_filter=*/false, tboptions.level_at_creation, MaxFileSizeForL0MetaPin(mutable_cf_options), /*smallest_compaction_key=*/nullptr, diff --git a/db/builder.h b/db/builder.h index 063da5ca9eda..c313ff145891 100644 --- a/db/builder.h +++ b/db/builder.h @@ -54,7 +54,7 @@ extern Status BuildTable( const std::string& dbname, VersionSet* versions, const ImmutableDBOptions& db_options, const TableBuilderOptions& tboptions, const FileOptions& file_options, TableCache* table_cache, - InternalIterator* iter, + TableReaderCaller table_reader_caller, InternalIterator* iter, std::vector> range_del_iters, FileMetaData* meta, std::vector* blob_file_additions, diff --git a/db/convenience.cc b/db/convenience.cc index 4bc419422949..427798e77274 100644 --- a/db/convenience.cc +++ b/db/convenience.cc @@ -57,7 +57,8 @@ Status VerifySstFileChecksum(const Options& options, new RandomAccessFileReader( std::move(file), file_path, ioptions.clock, nullptr /* io_tracer */, nullptr /* stats */, IOActivity::kUnknown, - nullptr /* file_read_hist */, ioptions.rate_limiter.get())); + Histograms::HISTOGRAM_ENUM_MAX, nullptr /* file_read_hist */, + ioptions.rate_limiter.get())); const bool kImmortal = true; auto reader_options = TableReaderOptions( ioptions, options.prefix_extractor, env_options, internal_comparator, diff --git a/db/db_impl/db_impl_open.cc b/db/db_impl/db_impl_open.cc index 94f36e862980..564f954f880d 100644 --- a/db/db_impl/db_impl_open.cc +++ b/db/db_impl/db_impl_open.cc @@ -1538,16 +1538,17 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, SeqnoToTimeMapping empty_seqno_time_mapping; Version* version = cfd->current(); version->Ref(); - s = BuildTable( - dbname_, versions_.get(), immutable_db_options_, tboptions, - file_options_for_compaction_, cfd->table_cache(), iter.get(), - std::move(range_del_iters), &meta, &blob_file_additions, - snapshot_seqs, earliest_write_conflict_snapshot, kMaxSequenceNumber, - snapshot_checker, paranoid_file_checks, cfd->internal_stats(), &io_s, - io_tracer_, BlobFileCreationReason::kRecovery, - empty_seqno_time_mapping, &event_logger_, job_id, Env::IO_HIGH, - nullptr /* table_properties */, write_hint, - nullptr /*full_history_ts_low*/, &blob_callback_, version); + s = BuildTable(dbname_, versions_.get(), immutable_db_options_, tboptions, + file_options_for_compaction_, cfd->table_cache(), + TableReaderCaller::kFlush, iter.get(), + std::move(range_del_iters), &meta, &blob_file_additions, + snapshot_seqs, earliest_write_conflict_snapshot, + kMaxSequenceNumber, snapshot_checker, paranoid_file_checks, + cfd->internal_stats(), &io_s, io_tracer_, + BlobFileCreationReason::kRecovery, + empty_seqno_time_mapping, &event_logger_, job_id, + Env::IO_HIGH, nullptr /* table_properties */, write_hint, + nullptr /*full_history_ts_low*/, &blob_callback_, version); version->Unref(); LogFlush(immutable_db_options_.info_log); ROCKS_LOG_DEBUG(immutable_db_options_.info_log, diff --git a/db/flush_job.cc b/db/flush_job.cc index 8193f594f8b4..5512326863bd 100644 --- a/db/flush_job.cc +++ b/db/flush_job.cc @@ -930,17 +930,18 @@ Status FlushJob::WriteLevel0Table() { meta_.fd.GetNumber()); const SequenceNumber job_snapshot_seq = job_context_->GetJobSnapshotSequence(); - s = BuildTable( - dbname_, versions_, db_options_, tboptions, file_options_, - cfd_->table_cache(), iter.get(), std::move(range_del_iters), &meta_, - &blob_file_additions, existing_snapshots_, - earliest_write_conflict_snapshot_, job_snapshot_seq, - snapshot_checker_, mutable_cf_options_.paranoid_file_checks, - cfd_->internal_stats(), &io_s, io_tracer_, - BlobFileCreationReason::kFlush, seqno_to_time_mapping_, event_logger_, - job_context_->job_id, io_priority, &table_properties_, write_hint, - full_history_ts_low, blob_callback_, base_, &num_input_entries, - &memtable_payload_bytes, &memtable_garbage_bytes); + s = BuildTable(dbname_, versions_, db_options_, tboptions, file_options_, + cfd_->table_cache(), TableReaderCaller::kFlush, iter.get(), + std::move(range_del_iters), &meta_, &blob_file_additions, + existing_snapshots_, earliest_write_conflict_snapshot_, + job_snapshot_seq, snapshot_checker_, + mutable_cf_options_.paranoid_file_checks, + cfd_->internal_stats(), &io_s, io_tracer_, + BlobFileCreationReason::kFlush, seqno_to_time_mapping_, + event_logger_, job_context_->job_id, io_priority, + &table_properties_, write_hint, full_history_ts_low, + blob_callback_, base_, &num_input_entries, + &memtable_payload_bytes, &memtable_garbage_bytes); // TODO: Cleanup io_status in BuildTable and table builders assert(!s.ok() || io_s.ok()); io_s.PermitUncheckedError(); diff --git a/db/repair.cc b/db/repair.cc index 0b3e120c9b24..f6b0833a6dea 100644 --- a/db/repair.cc +++ b/db/repair.cc @@ -456,13 +456,14 @@ class Repairer { SeqnoToTimeMapping empty_seqno_time_mapping; status = BuildTable( dbname_, /* versions */ nullptr, immutable_db_options_, tboptions, - file_options_, table_cache_.get(), iter.get(), - std::move(range_del_iters), &meta, nullptr /* blob_file_additions */, - {}, kMaxSequenceNumber, kMaxSequenceNumber, snapshot_checker, - false /* paranoid_file_checks*/, nullptr /* internal_stats */, &io_s, - nullptr /*IOTracer*/, BlobFileCreationReason::kRecovery, - empty_seqno_time_mapping, nullptr /* event_logger */, 0 /* job_id */, - Env::IO_HIGH, nullptr /* table_properties */, write_hint); + file_options_, table_cache_.get(), TableReaderCaller::kFlush, + iter.get(), std::move(range_del_iters), &meta, + nullptr /* blob_file_additions */, {}, kMaxSequenceNumber, + kMaxSequenceNumber, snapshot_checker, false /* paranoid_file_checks*/, + nullptr /* internal_stats */, &io_s, nullptr /*IOTracer*/, + BlobFileCreationReason::kRecovery, empty_seqno_time_mapping, + nullptr /* event_logger */, 0 /* job_id */, Env::IO_HIGH, + nullptr /* table_properties */, write_hint); ROCKS_LOG_INFO(db_options_.info_log, "Log #%" PRIu64 ": %d ops saved to Table #%" PRIu64 " %s", log, counter, meta.fd.GetNumber(), @@ -809,4 +810,3 @@ Status RepairDB(const std::string& dbname, const Options& options) { } } // namespace ROCKSDB_NAMESPACE - diff --git a/db/table_cache.cc b/db/table_cache.cc index f456260bc670..4967c6008e08 100644 --- a/db/table_cache.cc +++ b/db/table_cache.cc @@ -92,6 +92,7 @@ Status TableCache::GetTableReader( const InternalKeyComparator& internal_comparator, const FileMetaData& file_meta, bool sequential_mode, bool record_read_stats, HistogramImpl* file_read_hist, std::unique_ptr* table_reader, + TableReaderCaller caller, const std::shared_ptr& prefix_extractor, bool skip_filters, int level, bool prefetch_index_and_filter_in_cache, size_t max_file_size_for_l0_meta_pin, Temperature file_temperature) { @@ -124,11 +125,23 @@ Status TableCache::GetTableReader( if (!sequential_mode && ioptions_.advise_random_on_open) { file->Hint(FSRandomAccessFile::kRandom); } - StopWatch sw(ioptions_.clock, ioptions_.stats, TABLE_OPEN_IO_MICROS); + IOActivity io_activity; + if (caller == TableReaderCaller::kFlush) { + io_activity = IOActivity::kFlush; + } else if (caller == TableReaderCaller::kCompaction || + caller == TableReaderCaller::kCompactionRefill) { + io_activity = IOActivity::kCompaction; + } else { + io_activity = IOActivity::kUnknown; + } + StopWatch sw(ioptions_.clock, ioptions_.stats, {TABLE_OPEN_IO_MICROS}); std::unique_ptr file_reader( new RandomAccessFileReader( std::move(file), fname, ioptions_.clock, io_tracer_, - record_read_stats ? ioptions_.stats : nullptr, SST_READ_MICROS, + record_read_stats ? ioptions_.stats : nullptr, io_activity, + io_activity == IOActivity::kUnknown + ? Histograms::SST_READ_MICROS + : Histograms::HISTOGRAM_ENUM_MAX, file_read_hist, ioptions_.rate_limiter.get(), ioptions_.listeners, file_temperature, level == ioptions_.num_levels - 1)); UniqueId64x2 expected_unique_id; @@ -156,6 +169,7 @@ Status TableCache::FindTable( const ReadOptions& ro, const FileOptions& file_options, const InternalKeyComparator& internal_comparator, const FileMetaData& file_meta, TypedHandle** handle, + TableReaderCaller caller, const std::shared_ptr& prefix_extractor, const bool no_io, bool record_read_stats, HistogramImpl* file_read_hist, bool skip_filters, int level, bool prefetch_index_and_filter_in_cache, @@ -182,7 +196,7 @@ Status TableCache::FindTable( Status s = GetTableReader(ro, file_options, internal_comparator, file_meta, false /* sequential mode */, record_read_stats, - file_read_hist, &table_reader, prefix_extractor, + file_read_hist, &table_reader, caller, prefix_extractor, skip_filters, level, prefetch_index_and_filter_in_cache, max_file_size_for_l0_meta_pin, file_temperature); if (!s.ok()) { @@ -226,7 +240,7 @@ InternalIterator* TableCache::NewIterator( table_reader = fd.table_reader; if (table_reader == nullptr) { s = FindTable( - options, file_options, icomparator, file_meta, &handle, + options, file_options, icomparator, file_meta, &handle, caller, prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */, !for_compaction /* record_read_stats */, file_read_hist, skip_filters, level, true /* prefetch_index_and_filter_in_cache */, @@ -317,7 +331,7 @@ Status TableCache::GetRangeTombstoneIterator( TypedHandle* handle = nullptr; if (t == nullptr) { s = FindTable(options, file_options_, internal_comparator, file_meta, - &handle); + &handle, TableReaderCaller::kUncategorized); if (s.ok()) { t = cache_.Value(handle); } @@ -430,7 +444,8 @@ Status TableCache::Get( assert(s.ok()); if (t == nullptr) { s = FindTable(options, file_options_, internal_comparator, file_meta, - &handle, prefix_extractor, + &handle, TableReaderCaller::kUncategorized, + prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */, true /* record_read_stats */, file_read_hist, skip_filters, level, true /* prefetch_index_and_filter_in_cache */, @@ -531,12 +546,13 @@ Status TableCache::MultiGetFilter( MultiGetContext::Range tombstone_range(*mget_range, mget_range->begin(), mget_range->end()); if (t == nullptr) { - s = FindTable( - options, file_options_, internal_comparator, file_meta, &handle, - prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */, - true /* record_read_stats */, file_read_hist, /*skip_filters=*/false, - level, true /* prefetch_index_and_filter_in_cache */, - /*max_file_size_for_l0_meta_pin=*/0, file_meta.temperature); + s = FindTable(options, file_options_, internal_comparator, file_meta, + &handle, TableReaderCaller::kUncategorized, prefix_extractor, + options.read_tier == kBlockCacheTier /* no_io */, + true /* record_read_stats */, file_read_hist, + /*skip_filters=*/false, level, + true /* prefetch_index_and_filter_in_cache */, + /*max_file_size_for_l0_meta_pin=*/0, file_meta.temperature); if (s.ok()) { t = cache_.Value(handle); } @@ -574,8 +590,10 @@ Status TableCache::GetTableProperties( } TypedHandle* table_handle = nullptr; - Status s = FindTable(ReadOptions(), file_options, internal_comparator, - file_meta, &table_handle, prefix_extractor, no_io); + Status s = + FindTable(ReadOptions(), file_options, internal_comparator, file_meta, + &table_handle, TableReaderCaller::kUncategorized, + prefix_extractor, no_io); if (!s.ok()) { return s; } @@ -593,7 +611,8 @@ Status TableCache::ApproximateKeyAnchors( TableReader* t = file_meta.fd.table_reader; TypedHandle* handle = nullptr; if (t == nullptr) { - s = FindTable(ro, file_options_, internal_comparator, file_meta, &handle); + s = FindTable(ro, file_options_, internal_comparator, file_meta, &handle, + TableReaderCaller::kUncategorized); if (s.ok()) { t = cache_.Value(handle); } @@ -619,8 +638,9 @@ size_t TableCache::GetMemoryUsageByTableReader( } TypedHandle* table_handle = nullptr; - Status s = FindTable(ReadOptions(), file_options, internal_comparator, - file_meta, &table_handle, prefix_extractor, true); + Status s = FindTable( + ReadOptions(), file_options, internal_comparator, file_meta, + &table_handle, TableReaderCaller::kUncategorized, prefix_extractor, true); if (!s.ok()) { return 0; } @@ -644,10 +664,10 @@ uint64_t TableCache::ApproximateOffsetOf( TypedHandle* table_handle = nullptr; if (table_reader == nullptr) { const bool for_compaction = (caller == TableReaderCaller::kCompaction); - Status s = - FindTable(ReadOptions(), file_options_, internal_comparator, file_meta, - &table_handle, prefix_extractor, false /* no_io */, - !for_compaction /* record_read_stats */); + Status s = FindTable( + ReadOptions(), file_options_, internal_comparator, file_meta, + &table_handle, TableReaderCaller::kUncategorized, prefix_extractor, + false /* no_io */, !for_compaction /* record_read_stats */); if (s.ok()) { table_reader = cache_.Value(table_handle); } @@ -672,10 +692,10 @@ uint64_t TableCache::ApproximateSize( TypedHandle* table_handle = nullptr; if (table_reader == nullptr) { const bool for_compaction = (caller == TableReaderCaller::kCompaction); - Status s = - FindTable(ReadOptions(), file_options_, internal_comparator, file_meta, - &table_handle, prefix_extractor, false /* no_io */, - !for_compaction /* record_read_stats */); + Status s = FindTable( + ReadOptions(), file_options_, internal_comparator, file_meta, + &table_handle, TableReaderCaller::kUncategorized, prefix_extractor, + false /* no_io */, !for_compaction /* record_read_stats */); if (s.ok()) { table_reader = cache_.Value(table_handle); } diff --git a/db/table_cache.h b/db/table_cache.h index 66282bf41f04..24bb634d613f 100644 --- a/db/table_cache.h +++ b/db/table_cache.h @@ -164,7 +164,7 @@ class TableCache { Status FindTable( const ReadOptions& ro, const FileOptions& toptions, const InternalKeyComparator& internal_comparator, - const FileMetaData& file_meta, TypedHandle**, + const FileMetaData& file_meta, TypedHandle**, TableReaderCaller caller, const std::shared_ptr& prefix_extractor = nullptr, const bool no_io = false, bool record_read_stats = true, HistogramImpl* file_read_hist = nullptr, bool skip_filters = false, @@ -234,7 +234,7 @@ class TableCache { const InternalKeyComparator& internal_comparator, const FileMetaData& file_meta, bool sequential_mode, bool record_read_stats, HistogramImpl* file_read_hist, - std::unique_ptr* table_reader, + std::unique_ptr* table_reader, TableReaderCaller caller, const std::shared_ptr& prefix_extractor = nullptr, bool skip_filters = false, int level = -1, bool prefetch_index_and_filter_in_cache = true, diff --git a/db/table_cache_sync_and_async.h b/db/table_cache_sync_and_async.h index b1ab73247ce0..183a53367674 100644 --- a/db/table_cache_sync_and_async.h +++ b/db/table_cache_sync_and_async.h @@ -65,7 +65,8 @@ DEFINE_SYNC_AND_ASYNC(Status, TableCache::MultiGet) if (t == nullptr) { assert(handle == nullptr); s = FindTable(options, file_options_, internal_comparator, file_meta, - &handle, prefix_extractor, + &handle, TableReaderCaller::kUncategorized, + prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */, true /* record_read_stats */, file_read_hist, skip_filters, level, true /* prefetch_index_and_filter_in_cache */, diff --git a/db/version_builder.cc b/db/version_builder.cc index 4f0e3a8413c0..41fbfc679e66 100644 --- a/db/version_builder.cc +++ b/db/version_builder.cc @@ -1326,7 +1326,8 @@ class VersionBuilder::Rep { statuses[file_idx] = table_cache_->FindTable( ReadOptions(), file_options_, *(base_vstorage_->InternalComparator()), *file_meta, &handle, - prefix_extractor, false /*no_io */, true /* record_read_stats */, + TableReaderCaller::kUncategorized, prefix_extractor, + false /*no_io */, true /* record_read_stats */, internal_stats->GetFileReadHist(level), false, level, prefetch_index_and_filter_in_cache, max_file_size_for_l0_meta_pin, file_meta->temperature); diff --git a/db/version_set.cc b/db/version_set.cc index 945adc22cc52..e9d50b4ec8fc 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -1567,8 +1567,8 @@ Status Version::GetTableProperties(std::shared_ptr* tp, new RandomAccessFileReader( std::move(file), file_name, nullptr /* env */, io_tracer_, nullptr /* stats */, IOActivity::kUnknown, - nullptr /* file_read_hist */, nullptr /* rate_limiter */, - ioptions->listeners)); + Histograms::HISTOGRAM_ENUM_MAX, nullptr /* file_read_hist */, + nullptr /* rate_limiter */, ioptions->listeners)); std::unique_ptr props; s = ReadTableProperties( file_reader.get(), file_meta->fd.GetFileSize(), @@ -6927,7 +6927,8 @@ Status VersionSet::VerifyFileMetadata(ColumnFamilyData* cfd, TableCache::TypedHandle* handle = nullptr; FileMetaData meta_copy = meta; status = table_cache->FindTable( - ReadOptions(), file_opts, *icmp, meta_copy, &handle, pe, + ReadOptions(), file_opts, *icmp, meta_copy, &handle, + TableReaderCaller::kUncategorized, pe, /*no_io=*/false, /*record_read_stats=*/true, internal_stats->GetFileReadHist(level), false, level, /*prefetch_index_and_filter_in_cache*/ false, max_sz_for_l0_meta_pin, diff --git a/file/file_util.cc b/file/file_util.cc index 4211189a5374..edbcd6cd78c1 100644 --- a/file/file_util.cc +++ b/file/file_util.cc @@ -187,7 +187,8 @@ IOStatus GenerateOneFileChecksum( } reader.reset(new RandomAccessFileReader( std::move(r_file), file_path, nullptr /*Env*/, io_tracer, nullptr, - IOActivity::kUnknown, nullptr, rate_limiter)); + IOActivity::kUnknown, Histograms::HISTOGRAM_ENUM_MAX, nullptr, + rate_limiter)); } // Found that 256 KB readahead size provides the best performance, based on diff --git a/file/random_access_file_reader.cc b/file/random_access_file_reader.cc index 93090fc678de..832bd6bdbd64 100644 --- a/file/random_access_file_reader.cc +++ b/file/random_access_file_reader.cc @@ -94,7 +94,7 @@ IOStatus RandomAccessFileReader::Read(const IOOptions& opts, uint64_t offset, IOStatus io_s; uint64_t elapsed = 0; { - Histograms hist_type = GetHistType(io_activity_, op_io_activity); + Histograms hist_type = GetHistType(op_io_activity); StopWatch sw(clock_, stats_, GetHistTypes(hist_type), (stats_ != nullptr) ? &elapsed : nullptr, true /*overwrite*/, true /*delay_enabled*/); @@ -290,7 +290,7 @@ IOStatus RandomAccessFileReader::MultiRead( IOStatus io_s; uint64_t elapsed = 0; { - Histograms hist_type = GetHistType(io_activity_, op_io_activity); + Histograms hist_type = GetHistType(op_io_activity); StopWatch sw(clock_, stats_, GetHistTypes(hist_type), (stats_ != nullptr) ? &elapsed : nullptr, true /*overwrite*/, true /*delay_enabled*/); @@ -480,15 +480,15 @@ IOStatus RandomAccessFileReader::ReadAsync( assert(read_async_info->buf_.CurrentSize() == 0); - StopWatch sw(clock_, nullptr /*stats*/, - {Histograms::HISTOGRAM_ENUM_MAX} /*hist_type*/, &elapsed, - true /*overwrite*/, true /*delay_enabled*/); + // TODO(hx235): Replace hard-coded `hist_type` like ::Read() + StopWatch sw(clock_, nullptr /*stats*/, {Histograms::DB_GET} /*hist_type*/, + &elapsed, true /*overwrite*/, true /*delay_enabled*/); s = file_->ReadAsync(aligned_req, opts, read_async_callback, read_async_info, io_handle, del_fn, nullptr /*dbg*/); } else { - StopWatch sw(clock_, nullptr /*stats*/, - {Histograms::HISTOGRAM_ENUM_MAX} /*hist_type*/, &elapsed, - true /*overwrite*/, true /*delay_enabled*/); + // TODO(hx235): Replace hard-coded `hist_type` like ::Read() + StopWatch sw(clock_, nullptr /*stats*/, {Histograms::DB_GET} /*hist_type*/, + &elapsed, true /*overwrite*/, true /*delay_enabled*/); s = file_->ReadAsync(req, opts, read_async_callback, read_async_info, io_handle, del_fn, nullptr /*dbg*/); } @@ -588,16 +588,16 @@ void RandomAccessFileReader::ReadAsyncCallback(const FSReadRequest& req, delete read_async_info; } -Histograms RandomAccessFileReader::GetHistType(IOActivity io_activity, - IOActivity op_io_activity) { +Histograms RandomAccessFileReader::GetHistType( + IOActivity op_io_activity) const { IOActivity io_activity_used; if (op_io_activity != IOActivity::kUnknown) { io_activity_used = op_io_activity; } else { - io_activity_used = io_activity; + io_activity_used = io_activity_; } - Histograms hist_type = Histograms::HISTOGRAM_ENUM_MAX; + Histograms hist_type; if (io_activity_used == IOActivity::kFlush) { hist_type = Histograms::SST_READ_FLUSH_MICROS; } else if (io_activity_used == IOActivity::kCompaction) { @@ -606,6 +606,7 @@ Histograms RandomAccessFileReader::GetHistType(IOActivity io_activity, hist_type = Histograms::BLOB_DB_BLOB_FILE_READ_MICROS; } else { assert(io_activity_used == IOActivity::kUnknown); + hist_type = fall_back_read_hist_type_; } return hist_type; diff --git a/file/random_access_file_reader.h b/file/random_access_file_reader.h index a34ab521a9c7..d63264e5e07e 100644 --- a/file/random_access_file_reader.h +++ b/file/random_access_file_reader.h @@ -85,6 +85,9 @@ class RandomAccessFileReader { SystemClock* clock_; Statistics* stats_; IOActivity io_activity_; + // TODO(hx235): eliminate `fall_back_hist_type_` once we establish clear + // mapping from `IOActivity` to `read_hist_type` + Histograms fall_back_read_hist_type_; HistogramImpl* file_read_hist_; RateLimiter* rate_limiter_; std::vector> listeners_; @@ -125,6 +128,7 @@ class RandomAccessFileReader { const std::shared_ptr& io_tracer = nullptr, Statistics* stats = nullptr, IOActivity io_activity = IOActivity::kUnknown, + Histograms fall_back_read_hist_type = Histograms::HISTOGRAM_ENUM_MAX, HistogramImpl* file_read_hist = nullptr, RateLimiter* rate_limiter = nullptr, const std::vector>& listeners = {}, @@ -135,6 +139,7 @@ class RandomAccessFileReader { clock_(clock), stats_(stats), io_activity_(io_activity), + fall_back_read_hist_type_(fall_back_read_hist_type), file_read_hist_(file_read_hist), rate_limiter_(rate_limiter), listeners_(), @@ -211,8 +216,7 @@ class RandomAccessFileReader { void ReadAsyncCallback(const FSReadRequest& req, void* cb_arg); private: - static Histograms GetHistType(IOActivity io_activity, - IOActivity op_io_activity); + Histograms GetHistType(IOActivity op_io_activity) const; static std::unordered_set GetHistTypes( Histograms hist_type_to_use); }; diff --git a/include/rocksdb/statistics.h b/include/rocksdb/statistics.h index ce550f5bc15c..e5dbfef92e2c 100644 --- a/include/rocksdb/statistics.h +++ b/include/rocksdb/statistics.h @@ -458,6 +458,9 @@ enum Histograms : uint32_t { DB_SEEK, WRITE_STALL, SST_READ_MICROS, + SST_READ_FLUSH_MICROS, + SST_READ_COMPACTION_MICROS, + // The number of subcompactions actually scheduled during a compaction NUM_SUBCOMPACTIONS_SCHEDULED, // Value size distribution in each operation diff --git a/include/rocksdb/table_reader_caller.h b/include/rocksdb/table_reader_caller.h index 10ec08130fbb..a9fddf99339d 100644 --- a/include/rocksdb/table_reader_caller.h +++ b/include/rocksdb/table_reader_caller.h @@ -22,7 +22,7 @@ enum TableReaderCaller : char { kSSTDumpTool = 6, kExternalSSTIngestion = 7, kRepair = 8, - kPrefetch = 9, + kPrefetch = 9, // Used for BlockCacheLookupContext only kCompaction = 10, // A compaction job may refill the block cache with blocks in the new SST // files if paranoid_file_checks is true. diff --git a/java/rocksjni/portal.h b/java/rocksjni/portal.h index 9d36ec863c65..7e1c769a8487 100644 --- a/java/rocksjni/portal.h +++ b/java/rocksjni/portal.h @@ -5609,6 +5609,10 @@ class HistogramTypeJni { return 0x37; case ASYNC_PREFETCH_ABORT_MICROS: return 0x38; + case ROCKSDB_NAMESPACE::Histograms::SST_READ_FLUSH_MICROS: + return 0x39; + case ROCKSDB_NAMESPACE::Histograms::SST_READ_COMPACTION_MICROS: + return 0x3A; case ROCKSDB_NAMESPACE::Histograms::HISTOGRAM_ENUM_MAX: // 0x1F for backwards compatibility on current minor version. return 0x1F; @@ -5725,6 +5729,10 @@ class HistogramTypeJni { return ROCKSDB_NAMESPACE::Histograms::NUM_LEVEL_READ_PER_MULTIGET; case 0x38: return ROCKSDB_NAMESPACE::Histograms::ASYNC_PREFETCH_ABORT_MICROS; + case 0x39: + return ROCKSDB_NAMESPACE::Histograms::SST_READ_FLUSH_MICROS; + case 0x3A: + return ROCKSDB_NAMESPACE::Histograms::SST_READ_COMPACTION_MICROS; case 0x1F: // 0x1F for backwards compatibility on current minor version. return ROCKSDB_NAMESPACE::Histograms::HISTOGRAM_ENUM_MAX; diff --git a/java/src/main/java/org/rocksdb/HistogramType.java b/java/src/main/java/org/rocksdb/HistogramType.java index 1f8b7a78a2c2..a9f9d4f64f41 100644 --- a/java/src/main/java/org/rocksdb/HistogramType.java +++ b/java/src/main/java/org/rocksdb/HistogramType.java @@ -162,6 +162,10 @@ public enum HistogramType { ASYNC_READ_BYTES((byte) 0x33), + SST_READ_FLUSH_MICROS((byte) 0x39), + + SST_READ_COMPACTION_MICROS((byte) 0x3A), + // 0x1F for backwards compatibility on current minor version. HISTOGRAM_ENUM_MAX((byte) 0x1F); diff --git a/microbench/db_basic_bench.cc b/microbench/db_basic_bench.cc index aa9acee9067b..6d963ae28231 100644 --- a/microbench/db_basic_bench.cc +++ b/microbench/db_basic_bench.cc @@ -1541,8 +1541,8 @@ static void RandomAccessFileReaderRead(benchmark::State& state) { : Temperature::kCold; readers.emplace_back(new RandomAccessFileReader( std::move(f), fname, env->GetSystemClock().get(), nullptr, statistics, - IOActivity::kUnknown, nullptr, nullptr, {}, temperature, - rand_num == 1)); + IOActivity::kUnknown, Histograms::HISTOGRAM_ENUM_MAX, nullptr, nullptr, + {}, temperature, rand_num == 1)); } IOOptions io_options; diff --git a/monitoring/statistics.cc b/monitoring/statistics.cc index 4766947468b6..e2a3656e9e5a 100644 --- a/monitoring/statistics.cc +++ b/monitoring/statistics.cc @@ -237,6 +237,8 @@ const std::vector> HistogramsNameMap = { {DB_SEEK, "rocksdb.db.seek.micros"}, {WRITE_STALL, "rocksdb.db.write.stall"}, {SST_READ_MICROS, "rocksdb.sst.read.micros"}, + {SST_READ_FLUSH_MICROS, "rocksdb.sst.read.flush.micros"}, + {SST_READ_COMPACTION_MICROS, "rocksdb.sst.read.compaction.micros"}, {NUM_SUBCOMPACTIONS_SCHEDULED, "rocksdb.num.subcompactions.scheduled"}, {BYTES_PER_READ, "rocksdb.bytes.per.read"}, {BYTES_PER_WRITE, "rocksdb.bytes.per.write"}, diff --git a/table/block_fetcher.cc b/table/block_fetcher.cc index 8df0850b3dfc..492c7f7b21c0 100644 --- a/table/block_fetcher.cc +++ b/table/block_fetcher.cc @@ -262,9 +262,10 @@ IOStatus BlockFetcher::ReadBlockContents() { } else { PrepareBufferForBlockFromFile(); PERF_TIMER_GUARD(block_read_time); - io_status_ = file_->Read(opts, handle_.offset(), - block_size_with_trailer_, &slice_, used_buf_, - nullptr, read_options_.rate_limiter_priority); + io_status_ = file_->Read( + opts, handle_.offset(), block_size_with_trailer_, &slice_, + used_buf_, nullptr, read_options_.rate_limiter_priority, + for_compaction_ ? IOActivity::kCompaction : IOActivity::kUnknown); PERF_COUNTER_ADD(block_read_count, 1); #ifndef NDEBUG if (slice_.data() == &stack_buf_[0]) {