diff --git a/CMakeLists.txt b/CMakeLists.txt index 2e4c66a86..57a7789bb 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -642,6 +642,7 @@ set(SOURCES db/write_batch_base.cc db/write_controller.cc db/write_thread.cc + env/composite_env.cc env/env.cc env/env_chroot.cc env/env_encryption.cc diff --git a/HISTORY.md b/HISTORY.md index 23120ce98..db3bb9f3d 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -12,7 +12,7 @@ ### Public API Change * Add a public API WriteBufferManager::dummy_entries_in_cache_usage() which reports the size of dummy entries stored in cache (passed to WriteBufferManager). Dummy entries are used to account for DataBlocks. - +* Add a SystemClock class that contains the time-related methods from Env. The original methods in Env may be deprecated in a future release. This class will allow easier testing, development, and expansion of time-related features. ## 6.16.0 (12/18/2020) ### Behavior Changes * Attempting to write a merge operand without explicitly configuring `merge_operator` now fails immediately, causing the DB to enter read-only mode. Previously, failure was deferred until the `merge_operator` was needed by a user read or a background operation. diff --git a/TARGETS b/TARGETS index 020b3e985..4b9460f05 100644 --- a/TARGETS +++ b/TARGETS @@ -206,6 +206,7 @@ cpp_library( "db/write_batch_base.cc", "db/write_controller.cc", "db/write_thread.cc", + "env/composite_env.cc", "env/env.cc", "env/env_chroot.cc", "env/env_encryption.cc", @@ -510,6 +511,7 @@ cpp_library( "db/write_batch_base.cc", "db/write_controller.cc", "db/write_thread.cc", + "env/composite_env.cc", "env/env.cc", "env/env_chroot.cc", "env/env_encryption.cc", diff --git a/cache/cache_bench.cc b/cache/cache_bench.cc index 7d2467596..96d07d864 100644 --- a/cache/cache_bench.cc +++ b/cache/cache_bench.cc @@ -13,6 +13,7 @@ int main() { #include #include + #include #include @@ -20,6 +21,7 @@ int main() { #include "rocksdb/cache.h" #include "rocksdb/db.h" #include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "util/coding.h" #include "util/gflags_compat.h" #include "util/hash.h" @@ -210,6 +212,7 @@ class CacheBench { bool Run() { ROCKSDB_NAMESPACE::Env* env = ROCKSDB_NAMESPACE::Env::Default(); + const auto& clock = env->GetSystemClock(); PrintEnv(); SharedState shared(this); @@ -224,7 +227,7 @@ class CacheBench { shared.GetCondVar()->Wait(); } // Record start time - uint64_t start_time = env->NowMicros(); + uint64_t start_time = clock->NowMicros(); // Start all threads shared.SetStart(); @@ -236,7 +239,7 @@ class CacheBench { } // Record end time - uint64_t end_time = env->NowMicros(); + uint64_t end_time = clock->NowMicros(); double elapsed = static_cast(end_time - start_time) * 1e-6; uint32_t qps = static_cast( static_cast(FLAGS_threads * FLAGS_ops_per_thread) / elapsed); diff --git a/db/blob/blob_file_builder.cc b/db/blob/blob_file_builder.cc index 6f2901739..ba1c7a740 100644 --- a/db/blob/blob_file_builder.cc +++ b/db/blob/blob_file_builder.cc @@ -49,7 +49,6 @@ BlobFileBuilder::BlobFileBuilder( std::vector* blob_file_paths, std::vector* blob_file_additions) : file_number_generator_(std::move(file_number_generator)), - env_(env), fs_(fs), immutable_cf_options_(immutable_cf_options), min_blob_size_(mutable_cf_options->min_blob_size), @@ -66,7 +65,7 @@ BlobFileBuilder::BlobFileBuilder( blob_count_(0), blob_bytes_(0) { assert(file_number_generator_); - assert(env_); + assert(env); assert(fs_); assert(immutable_cf_options_); assert(file_options_); @@ -74,6 +73,7 @@ BlobFileBuilder::BlobFileBuilder( assert(blob_file_paths_->empty()); assert(blob_file_additions_); assert(blob_file_additions_->empty()); + clock_ = env->GetSystemClock(); } BlobFileBuilder::~BlobFileBuilder() = default; @@ -181,14 +181,14 @@ Status BlobFileBuilder::OpenBlobFileIfNeeded() { Statistics* const statistics = immutable_cf_options_->statistics; std::unique_ptr file_writer(new WritableFileWriter( - std::move(file), blob_file_paths_->back(), *file_options_, env_, + std::move(file), blob_file_paths_->back(), *file_options_, clock_, nullptr /*IOTracer*/, statistics, immutable_cf_options_->listeners, immutable_cf_options_->file_checksum_gen_factory)); constexpr bool do_flush = false; std::unique_ptr blob_log_writer(new BlobLogWriter( - std::move(file_writer), env_, statistics, blob_file_number, + std::move(file_writer), clock_, statistics, blob_file_number, immutable_cf_options_->use_fsync, do_flush)); constexpr bool has_ttl = false; diff --git a/db/blob/blob_file_builder.h b/db/blob/blob_file_builder.h index 755ab4350..524a878c4 100644 --- a/db/blob/blob_file_builder.h +++ b/db/blob/blob_file_builder.h @@ -18,6 +18,7 @@ namespace ROCKSDB_NAMESPACE { class VersionSet; class FileSystem; +class SystemClock; struct ImmutableCFOptions; struct MutableCFOptions; struct FileOptions; @@ -69,8 +70,8 @@ class BlobFileBuilder { Status CloseBlobFileIfNeeded(); std::function file_number_generator_; - Env* env_; FileSystem* fs_; + std::shared_ptr clock_; const ImmutableCFOptions* immutable_cf_options_; uint64_t min_blob_size_; uint64_t blob_file_size_; diff --git a/db/blob/blob_file_builder_test.cc b/db/blob/blob_file_builder_test.cc index 134ca3cb2..d6068b64f 100644 --- a/db/blob/blob_file_builder_test.cc +++ b/db/blob/blob_file_builder_test.cc @@ -39,8 +39,10 @@ class TestFileNumberGenerator { class BlobFileBuilderTest : public testing::Test { protected: - BlobFileBuilderTest() - : mock_env_(Env::Default()), fs_(mock_env_.GetFileSystem().get()) {} + BlobFileBuilderTest() : mock_env_(Env::Default()) { + fs_ = mock_env_.GetFileSystem().get(); + clock_ = mock_env_.GetSystemClock(); + } void VerifyBlobFile(uint64_t blob_file_number, const std::string& blob_file_path, @@ -57,11 +59,10 @@ class BlobFileBuilderTest : public testing::Test { fs_->NewRandomAccessFile(blob_file_path, file_options_, &file, dbg)); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(file), blob_file_path, - &mock_env_)); + new RandomAccessFileReader(std::move(file), blob_file_path, clock_)); constexpr Statistics* statistics = nullptr; - BlobLogSequentialReader blob_log_reader(std::move(file_reader), &mock_env_, + BlobLogSequentialReader blob_log_reader(std::move(file_reader), clock_, statistics); BlobLogHeader header; @@ -109,6 +110,7 @@ class BlobFileBuilderTest : public testing::Test { MockEnv mock_env_; FileSystem* fs_; + std::shared_ptr clock_; FileOptions file_options_; }; diff --git a/db/blob/blob_file_cache_test.cc b/db/blob/blob_file_cache_test.cc index e1dd21d98..c325eb4cb 100644 --- a/db/blob/blob_file_cache_test.cc +++ b/db/blob/blob_file_cache_test.cc @@ -42,15 +42,15 @@ void WriteBlobFile(uint32_t column_family_id, std::unique_ptr file_writer( new WritableFileWriter(std::move(file), blob_file_path, FileOptions(), - immutable_cf_options.env)); + immutable_cf_options.env->GetSystemClock())); constexpr Statistics* statistics = nullptr; constexpr bool use_fsync = false; constexpr bool do_flush = false; - BlobLogWriter blob_log_writer(std::move(file_writer), - immutable_cf_options.env, statistics, - blob_file_number, use_fsync, do_flush); + BlobLogWriter blob_log_writer( + std::move(file_writer), immutable_cf_options.env->GetSystemClock(), + statistics, blob_file_number, use_fsync, do_flush); constexpr bool has_ttl = false; constexpr ExpirationRange expiration_range; diff --git a/db/blob/blob_file_reader.cc b/db/blob/blob_file_reader.cc index 0cae4eb53..fd394107d 100644 --- a/db/blob/blob_file_reader.cc +++ b/db/blob/blob_file_reader.cc @@ -117,10 +117,11 @@ Status BlobFileReader::OpenFile( } file_reader->reset(new RandomAccessFileReader( - std::move(file), blob_file_path, immutable_cf_options.env, - std::shared_ptr(), immutable_cf_options.statistics, - BLOB_DB_BLOB_FILE_READ_MICROS, blob_file_read_hist, - immutable_cf_options.rate_limiter, immutable_cf_options.listeners)); + std::move(file), blob_file_path, + immutable_cf_options.env->GetSystemClock(), std::shared_ptr(), + immutable_cf_options.statistics, BLOB_DB_BLOB_FILE_READ_MICROS, + blob_file_read_hist, immutable_cf_options.rate_limiter, + immutable_cf_options.listeners)); return Status::OK(); } diff --git a/db/blob/blob_file_reader_test.cc b/db/blob/blob_file_reader_test.cc index a7b7fc878..f31fb3dae 100644 --- a/db/blob/blob_file_reader_test.cc +++ b/db/blob/blob_file_reader_test.cc @@ -50,15 +50,15 @@ void WriteBlobFile(const ImmutableCFOptions& immutable_cf_options, std::unique_ptr file_writer( new WritableFileWriter(std::move(file), blob_file_path, FileOptions(), - immutable_cf_options.env)); + immutable_cf_options.env->GetSystemClock())); constexpr Statistics* statistics = nullptr; constexpr bool use_fsync = false; constexpr bool do_flush = false; - BlobLogWriter blob_log_writer(std::move(file_writer), - immutable_cf_options.env, statistics, - blob_file_number, use_fsync, do_flush); + BlobLogWriter blob_log_writer( + std::move(file_writer), immutable_cf_options.env->GetSystemClock(), + statistics, blob_file_number, use_fsync, do_flush); BlobLogHeader header(column_family_id, compression_type, has_ttl, expiration_range_header); @@ -260,15 +260,15 @@ TEST_F(BlobFileReaderTest, Malformed) { std::unique_ptr file_writer( new WritableFileWriter(std::move(file), blob_file_path, FileOptions(), - immutable_cf_options.env)); + immutable_cf_options.env->GetSystemClock())); constexpr Statistics* statistics = nullptr; constexpr bool use_fsync = false; constexpr bool do_flush = false; - BlobLogWriter blob_log_writer(std::move(file_writer), - immutable_cf_options.env, statistics, - blob_file_number, use_fsync, do_flush); + BlobLogWriter blob_log_writer( + std::move(file_writer), immutable_cf_options.env->GetSystemClock(), + statistics, blob_file_number, use_fsync, do_flush); BlobLogHeader header(column_family_id, kNoCompression, has_ttl, expiration_range); diff --git a/db/blob/blob_log_sequential_reader.cc b/db/blob/blob_log_sequential_reader.cc index 0a0262ee7..cb328ee56 100644 --- a/db/blob/blob_log_sequential_reader.cc +++ b/db/blob/blob_log_sequential_reader.cc @@ -13,10 +13,10 @@ namespace ROCKSDB_NAMESPACE { BlobLogSequentialReader::BlobLogSequentialReader( - std::unique_ptr&& file_reader, Env* env, - Statistics* statistics) + std::unique_ptr&& file_reader, + const std::shared_ptr& clock, Statistics* statistics) : file_(std::move(file_reader)), - env_(env), + clock_(clock), statistics_(statistics), next_byte_(0) {} @@ -27,7 +27,7 @@ Status BlobLogSequentialReader::ReadSlice(uint64_t size, Slice* slice, assert(slice); assert(file_); - StopWatch read_sw(env_, statistics_, BLOB_DB_BLOB_FILE_READ_MICROS); + StopWatch read_sw(clock_, statistics_, BLOB_DB_BLOB_FILE_READ_MICROS); Status s = file_->Read(IOOptions(), next_byte_, static_cast(size), slice, buf, nullptr); next_byte_ += size; diff --git a/db/blob/blob_log_sequential_reader.h b/db/blob/blob_log_sequential_reader.h index 62c707b4d..f7aada62d 100644 --- a/db/blob/blob_log_sequential_reader.h +++ b/db/blob/blob_log_sequential_reader.h @@ -16,6 +16,7 @@ class RandomAccessFileReader; class Env; class Statistics; class Status; +class SystemClock; /** * BlobLogSequentialReader is a general purpose log stream reader @@ -35,7 +36,8 @@ class BlobLogSequentialReader { // Create a reader that will return log records from "*file_reader". BlobLogSequentialReader(std::unique_ptr&& file_reader, - Env* env, Statistics* statistics); + const std::shared_ptr& clock, + Statistics* statistics); // No copying allowed BlobLogSequentialReader(const BlobLogSequentialReader&) = delete; @@ -63,7 +65,8 @@ class BlobLogSequentialReader { Status ReadSlice(uint64_t size, Slice* slice, char* buf); const std::unique_ptr file_; - Env* env_; + std::shared_ptr clock_; + Statistics* statistics_; Slice buffer_; diff --git a/db/blob/blob_log_writer.cc b/db/blob/blob_log_writer.cc index 859cbfc12..ce31bafdb 100644 --- a/db/blob/blob_log_writer.cc +++ b/db/blob/blob_log_writer.cc @@ -11,7 +11,7 @@ #include "db/blob/blob_log_format.h" #include "file/writable_file_writer.h" #include "monitoring/statistics.h" -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "util/coding.h" #include "util/stop_watch.h" @@ -19,11 +19,11 @@ namespace ROCKSDB_NAMESPACE { BlobLogWriter::BlobLogWriter(std::unique_ptr&& dest, - Env* env, Statistics* statistics, - uint64_t log_number, bool use_fs, bool do_flush, - uint64_t boffset) + const std::shared_ptr& clock, + Statistics* statistics, uint64_t log_number, + bool use_fs, bool do_flush, uint64_t boffset) : dest_(std::move(dest)), - env_(env), + clock_(clock), statistics_(statistics), log_number_(log_number), block_offset_(boffset), @@ -36,7 +36,7 @@ BlobLogWriter::~BlobLogWriter() = default; Status BlobLogWriter::Sync() { TEST_SYNC_POINT("BlobLogWriter::Sync"); - StopWatch sync_sw(env_, statistics_, BLOB_DB_BLOB_FILE_SYNC_MICROS); + StopWatch sync_sw(clock_, statistics_, BLOB_DB_BLOB_FILE_SYNC_MICROS); Status s = dest_->Sync(use_fsync_); RecordTick(statistics_, BLOB_DB_BLOB_FILE_SYNCED); return s; @@ -148,7 +148,7 @@ Status BlobLogWriter::EmitPhysicalRecord(const std::string& headerbuf, const Slice& key, const Slice& val, uint64_t* key_offset, uint64_t* blob_offset) { - StopWatch write_sw(env_, statistics_, BLOB_DB_BLOB_FILE_WRITE_MICROS); + StopWatch write_sw(clock_, statistics_, BLOB_DB_BLOB_FILE_WRITE_MICROS); Status s = dest_->Append(Slice(headerbuf)); if (s.ok()) { s = dest_->Append(key); diff --git a/db/blob/blob_log_writer.h b/db/blob/blob_log_writer.h index a2f28d901..17d4e06f0 100644 --- a/db/blob/blob_log_writer.h +++ b/db/blob/blob_log_writer.h @@ -9,7 +9,6 @@ #include #include "db/blob/blob_log_format.h" -#include "rocksdb/env.h" #include "rocksdb/slice.h" #include "rocksdb/statistics.h" #include "rocksdb/status.h" @@ -18,7 +17,7 @@ namespace ROCKSDB_NAMESPACE { class WritableFileWriter; - +class SystemClock; /** * BlobLogWriter is the blob log stream writer. It provides an append-only * abstraction for writing blob data. @@ -32,7 +31,8 @@ class BlobLogWriter { // Create a writer that will append data to "*dest". // "*dest" must be initially empty. // "*dest" must remain live while this BlobLogWriter is in use. - BlobLogWriter(std::unique_ptr&& dest, Env* env, + BlobLogWriter(std::unique_ptr&& dest, + const std::shared_ptr& clock, Statistics* statistics, uint64_t log_number, bool use_fsync, bool do_flush, uint64_t boffset = 0); // No copying allowed @@ -69,7 +69,7 @@ class BlobLogWriter { private: std::unique_ptr dest_; - Env* env_; + std::shared_ptr clock_; Statistics* statistics_; uint64_t log_number_; uint64_t block_offset_; // Current offset in block diff --git a/db/builder.cc b/db/builder.cc index 9a78e0f4f..f277058b5 100644 --- a/db/builder.cc +++ b/db/builder.cc @@ -124,6 +124,8 @@ Status BuildTable( assert(env); FileSystem* fs = db_options.fs.get(); assert(fs); + const auto& clock = env->GetSystemClock(); + TableProperties tp; if (iter->Valid() || !range_del_agg->IsEmpty()) { TableBuilder* builder; @@ -151,7 +153,7 @@ Status BuildTable( file->SetWriteLifeTimeHint(write_hint); file_writer.reset(new WritableFileWriter( - std::move(file), fname, file_options, env, io_tracer, + std::move(file), fname, file_options, clock, io_tracer, ioptions.statistics, ioptions.listeners, ioptions.file_checksum_gen_factory)); @@ -254,7 +256,7 @@ Status BuildTable( // Finish and check for file errors TEST_SYNC_POINT("BuildTable:BeforeSyncTable"); if (s.ok() && !empty) { - StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS); + StopWatch sw(clock, ioptions.statistics, TABLE_SYNC_MICROS); *io_status = file_writer->Sync(ioptions.use_fsync); } TEST_SYNC_POINT("BuildTable:BeforeCloseTableFile"); diff --git a/db/column_family.cc b/db/column_family.cc index 98b2927d0..b4ea8355e 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -556,8 +556,8 @@ ColumnFamilyData::ColumnFamilyData( // if _dummy_versions is nullptr, then this is a dummy column family. if (_dummy_versions != nullptr) { - internal_stats_.reset( - new InternalStats(ioptions_.num_levels, db_options.env, this)); + internal_stats_.reset(new InternalStats( + ioptions_.num_levels, db_options.env->GetSystemClock(), this)); table_cache_.reset(new TableCache(ioptions_, file_options, _table_cache, block_cache_tracer, io_tracer)); blob_file_cache_.reset( diff --git a/db/compaction/compaction_iterator.cc b/db/compaction/compaction_iterator.cc index 720cdbb65..cbe9291ce 100644 --- a/db/compaction/compaction_iterator.cc +++ b/db/compaction/compaction_iterator.cc @@ -80,6 +80,7 @@ CompactionIterator::CompactionIterator( earliest_write_conflict_snapshot_(earliest_write_conflict_snapshot), snapshot_checker_(snapshot_checker), env_(env), + clock_(env_->GetSystemClock()), report_detailed_time_(report_detailed_time), expect_valid_internal_key_(expect_valid_internal_key), range_del_agg_(range_del_agg), @@ -219,7 +220,7 @@ bool CompactionIterator::InvokeFilterIfNeeded(bool* need_skip, // to get sequence number. Slice& filter_key = ikey_.type == kTypeValue ? ikey_.user_key : key_; { - StopWatchNano timer(env_, report_detailed_time_); + StopWatchNano timer(clock_, report_detailed_time_); filter = compaction_filter_->FilterV2( compaction_->level(), filter_key, value_type, value_, &compaction_filter_value_, compaction_filter_skip_until_.rep()); diff --git a/db/compaction/compaction_iterator.h b/db/compaction/compaction_iterator.h index a669b16cb..a3073ee2e 100644 --- a/db/compaction/compaction_iterator.h +++ b/db/compaction/compaction_iterator.h @@ -248,6 +248,7 @@ class CompactionIterator { const SequenceNumber earliest_write_conflict_snapshot_; const SnapshotChecker* const snapshot_checker_; Env* env_; + std::shared_ptr clock_; bool report_detailed_time_; bool expect_valid_internal_key_; CompactionRangeDelAggregator* range_del_agg_; diff --git a/db/compaction/compaction_job.cc b/db/compaction/compaction_job.cc index 7edde82b1..f8b7c4967 100644 --- a/db/compaction/compaction_job.cc +++ b/db/compaction/compaction_job.cc @@ -320,6 +320,7 @@ CompactionJob::CompactionJob( db_options_(db_options), file_options_(file_options), env_(db_options.env), + clock_(env_->GetSystemClock()), io_tracer_(io_tracer), fs_(db_options.fs, io_tracer), file_options_for_read_( @@ -419,7 +420,7 @@ void CompactionJob::Prepare() { if (c->ShouldFormSubcompactions()) { { - StopWatch sw(env_, stats_, SUBCOMPACTION_SETUP_TIME); + StopWatch sw(clock_, stats_, SUBCOMPACTION_SETUP_TIME); GenSubcompactionBoundaries(); } assert(sizes_.size() == boundaries_.size() + 1); @@ -585,7 +586,7 @@ Status CompactionJob::Run() { const size_t num_threads = compact_->sub_compact_states.size(); assert(num_threads > 0); - const uint64_t start_micros = env_->NowMicros(); + const uint64_t start_micros = clock_->NowMicros(); // Launch a thread for each of subcompactions 1...num_threads-1 std::vector thread_pool; @@ -604,7 +605,7 @@ Status CompactionJob::Run() { thread.join(); } - compaction_stats_.micros = env_->NowMicros() - start_micros; + compaction_stats_.micros = clock_->NowMicros() - start_micros; compaction_stats_.cpu_micros = 0; for (size_t i = 0; i < compact_->sub_compact_states.size(); i++) { compaction_stats_.cpu_micros += @@ -894,7 +895,7 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) { assert(sub_compact); assert(sub_compact->compaction); - uint64_t prev_cpu_micros = env_->NowCPUNanos() / 1000; + uint64_t prev_cpu_micros = clock_->CPUNanos() / 1000; ColumnFamilyData* cfd = sub_compact->compaction->column_family_data(); @@ -1184,7 +1185,7 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) { } sub_compact->compaction_job_stats.cpu_micros = - env_->NowCPUNanos() / 1000 - prev_cpu_micros; + clock_->CPUNanos() / 1000 - prev_cpu_micros; if (measure_io_stats_) { sub_compact->compaction_job_stats.file_write_nanos += @@ -1463,7 +1464,7 @@ Status CompactionJob::FinishCompactionOutputFile( // Finish and check for file errors if (s.ok()) { - StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS); + StopWatch sw(clock_, stats_, COMPACTION_OUTFILE_SYNC_MICROS); io_s = sub_compact->outfile->Sync(db_options_.use_fsync); } if (s.ok() && io_s.ok()) { @@ -1738,7 +1739,7 @@ Status CompactionJob::OpenCompactionOutputFile( const auto& listeners = sub_compact->compaction->immutable_cf_options()->listeners; sub_compact->outfile.reset(new WritableFileWriter( - std::move(writable_file), fname, file_options_, env_, io_tracer_, + std::move(writable_file), fname, file_options_, clock_, io_tracer_, db_options_.statistics.get(), listeners, db_options_.file_checksum_gen_factory.get())); diff --git a/db/compaction/compaction_job.h b/db/compaction/compaction_job.h index bbd6547da..24449d45e 100644 --- a/db/compaction/compaction_job.h +++ b/db/compaction/compaction_job.h @@ -50,6 +50,7 @@ class Arena; class ErrorHandler; class MemTable; class SnapshotChecker; +class SystemClock; class TableCache; class Version; class VersionEdit; @@ -159,6 +160,7 @@ class CompactionJob { const FileOptions file_options_; Env* env_; + std::shared_ptr clock_; std::shared_ptr io_tracer_; FileSystemPtr fs_; // env_option optimized for compaction table reads diff --git a/db/db_compaction_test.cc b/db/db_compaction_test.cc index ea1dc075a..2480538f4 100644 --- a/db/db_compaction_test.cc +++ b/db/db_compaction_test.cc @@ -11,6 +11,7 @@ #include "db/blob/blob_index.h" #include "db/db_test_util.h" +#include "env/mock_env.h" #include "port/port.h" #include "port/stack_trace.h" #include "rocksdb/concurrent_task_limiter.h" diff --git a/db/db_dynamic_level_test.cc b/db/db_dynamic_level_test.cc index 955004304..6e2f6a283 100644 --- a/db/db_dynamic_level_test.cc +++ b/db/db_dynamic_level_test.cc @@ -13,6 +13,7 @@ #if !defined(ROCKSDB_LITE) #include "db/db_test_util.h" +#include "env/mock_env.h" #include "port/port.h" #include "port/stack_trace.h" #include "util/random.h" diff --git a/db/db_flush_test.cc b/db/db_flush_test.cc index df8108463..a5e4f7a81 100644 --- a/db/db_flush_test.cc +++ b/db/db_flush_test.cc @@ -11,6 +11,7 @@ #include "db/db_impl/db_impl.h" #include "db/db_test_util.h" +#include "env/mock_env.h" #include "file/filename.h" #include "port/port.h" #include "port/stack_trace.h" diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index f4e8e8e31..470fec7de 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -151,12 +151,13 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, own_info_log_(options.info_log == nullptr), initial_db_options_(SanitizeOptions(dbname, options)), env_(initial_db_options_.env), + clock_(initial_db_options_.env->GetSystemClock()), io_tracer_(std::make_shared()), immutable_db_options_(initial_db_options_), fs_(immutable_db_options_.fs, io_tracer_), mutable_db_options_(initial_db_options_), stats_(immutable_db_options_.statistics.get()), - mutex_(stats_, env_, DB_MUTEX_WAIT_MICROS, + mutex_(stats_, clock_, DB_MUTEX_WAIT_MICROS, immutable_db_options_.use_adaptive_mutex), default_cf_handle_(nullptr), max_total_in_memory_state_(0), @@ -191,7 +192,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, bg_purge_scheduled_(0), disable_delete_obsolete_files_(0), pending_purge_obsolete_files_(0), - delete_obsolete_files_last_run_(env_->NowMicros()), + delete_obsolete_files_last_run_(clock_->NowMicros()), last_stats_dump_time_microsec_(0), next_job_id_(1), has_unpersisted_data_(false), @@ -752,7 +753,7 @@ void DBImpl::PersistStats() { return; } TEST_SYNC_POINT("DBImpl::PersistStats:StartRunning"); - uint64_t now_seconds = env_->NowMicros() / kMicrosInSecond; + uint64_t now_seconds = clock_->NowMicros() / kMicrosInSecond; Statistics* statistics = immutable_db_options_.statistics.get(); if (!statistics) { @@ -1653,8 +1654,8 @@ Status DBImpl::GetImpl(const ReadOptions& read_options, const Slice& key, } #endif // NDEBUG - PERF_CPU_TIMER_GUARD(get_cpu_nanos, env_); - StopWatch sw(env_, stats_, DB_GET); + PERF_CPU_TIMER_GUARD(get_cpu_nanos, clock_); + StopWatch sw(clock_, stats_, DB_GET); PERF_TIMER_GUARD(get_snapshot_time); auto cfh = static_cast_with_check( @@ -1842,8 +1843,8 @@ std::vector DBImpl::MultiGet( const std::vector& column_family, const std::vector& keys, std::vector* values, std::vector* timestamps) { - PERF_CPU_TIMER_GUARD(get_cpu_nanos, env_); - StopWatch sw(env_, stats_, DB_MULTIGET); + PERF_CPU_TIMER_GUARD(get_cpu_nanos, clock_); + StopWatch sw(clock_, stats_, DB_MULTIGET); PERF_TIMER_GUARD(get_snapshot_time); #ifndef NDEBUG @@ -1974,7 +1975,7 @@ std::vector DBImpl::MultiGet( } if (read_options.deadline.count() && - env_->NowMicros() > + clock_->NowMicros() > static_cast(read_options.deadline.count())) { break; } @@ -1983,8 +1984,8 @@ std::vector DBImpl::MultiGet( if (keys_read < num_keys) { // The only reason to break out of the loop is when the deadline is // exceeded - assert(env_->NowMicros() > - static_cast(read_options.deadline.count())); + assert(clock_->NowMicros() > + static_cast(read_options.deadline.count())); for (++keys_read; keys_read < num_keys; ++keys_read) { stat_list[keys_read] = Status::TimedOut(); } @@ -2422,8 +2423,8 @@ Status DBImpl::MultiGetImpl( autovector* sorted_keys, SuperVersion* super_version, SequenceNumber snapshot, ReadCallback* callback) { - PERF_CPU_TIMER_GUARD(get_cpu_nanos, env_); - StopWatch sw(env_, stats_, DB_MULTIGET); + PERF_CPU_TIMER_GUARD(get_cpu_nanos, clock_); + StopWatch sw(clock_, stats_, DB_MULTIGET); // For each of the given keys, apply the entire "get" process as follows: // First look in the memtable, then in the immutable memtable (if any). @@ -2434,7 +2435,7 @@ Status DBImpl::MultiGetImpl( uint64_t curr_value_size = 0; while (keys_left) { if (read_options.deadline.count() && - env_->NowMicros() > + clock_->NowMicros() > static_cast(read_options.deadline.count())) { s = Status::TimedOut(); break; @@ -3137,7 +3138,8 @@ FileSystem* DBImpl::GetFileSystem() const { Status DBImpl::StartIOTrace(Env* env, const TraceOptions& trace_options, std::unique_ptr&& trace_writer) { assert(trace_writer != nullptr); - return io_tracer_->StartIOTrace(env, trace_options, std::move(trace_writer)); + return io_tracer_->StartIOTrace(env->GetSystemClock(), trace_options, + std::move(trace_writer)); } Status DBImpl::EndIOTrace() { @@ -4417,7 +4419,7 @@ Status DBImpl::IngestExternalFiles( for (const auto& arg : args) { auto* cfd = static_cast(arg.column_family)->cfd(); ingestion_jobs.emplace_back( - env_, versions_.get(), cfd, immutable_db_options_, file_options_, + clock_, versions_.get(), cfd, immutable_db_options_, file_options_, &snapshots_, arg.options, &directories_, &event_logger_, io_tracer_); } @@ -4685,7 +4687,7 @@ Status DBImpl::CreateColumnFamilyWithImport( // Import sst files from metadata. auto cfh = static_cast_with_check(*handle); auto cfd = cfh->cfd(); - ImportColumnFamilyJob import_job(env_, versions_.get(), cfd, + ImportColumnFamilyJob import_job(clock_, versions_.get(), cfd, immutable_db_options_, file_options_, import_options, metadata.files, io_tracer_); @@ -4941,7 +4943,7 @@ void DBImpl::WaitForIngestFile() { Status DBImpl::StartTrace(const TraceOptions& trace_options, std::unique_ptr&& trace_writer) { InstrumentedMutexLock lock(&trace_mutex_); - tracer_.reset(new Tracer(env_, trace_options, std::move(trace_writer))); + tracer_.reset(new Tracer(clock_, trace_options, std::move(trace_writer))); return Status::OK(); } diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index 0a09aa1a4..71ef9505e 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -54,9 +54,6 @@ #include "rocksdb/transaction_log.h" #include "rocksdb/write_buffer_manager.h" #include "table/scoped_arena_iterator.h" -#include "trace_replay/block_cache_tracer.h" -#include "trace_replay/io_tracer.h" -#include "trace_replay/trace_replay.h" #include "util/autovector.h" #include "util/hash.h" #include "util/repeatable_thread.h" @@ -1053,6 +1050,7 @@ class DBImpl : public DB { bool own_info_log_; const DBOptions initial_db_options_; Env* const env_; + std::shared_ptr clock_; std::shared_ptr io_tracer_; const ImmutableDBOptions immutable_db_options_; FileSystemPtr fs_; diff --git a/db/db_impl/db_impl_compaction_flush.cc b/db/db_impl/db_impl_compaction_flush.cc index 49f6b6593..28f570d54 100644 --- a/db/db_impl/db_impl_compaction_flush.cc +++ b/db/db_impl/db_impl_compaction_flush.cc @@ -2529,7 +2529,7 @@ void DBImpl::BackgroundCallFlush(Env::Priority thread_pri) { s.ToString().c_str(), error_cnt); log_buffer.FlushBufferToLog(); LogFlush(immutable_db_options_.info_log); - env_->SleepForMicroseconds(1000000); + clock_->SleepForMicroseconds(1000000); mutex_.Lock(); } @@ -2602,7 +2602,7 @@ void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction, if (s.IsBusy()) { bg_cv_.SignalAll(); // In case a waiter can proceed despite the error mutex_.Unlock(); - env_->SleepForMicroseconds(10000); // prevent hot loop + clock_->SleepForMicroseconds(10000); // prevent hot loop mutex_.Lock(); } else if (!s.ok() && !s.IsShutdownInProgress() && !s.IsManualCompactionPaused() && !s.IsColumnFamilyDropped()) { @@ -2620,7 +2620,7 @@ void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction, "Accumulated background error counts: %" PRIu64, s.ToString().c_str(), error_cnt); LogFlush(immutable_db_options_.info_log); - env_->SleepForMicroseconds(1000000); + clock_->SleepForMicroseconds(1000000); mutex_.Lock(); } else if (s.IsManualCompactionPaused()) { ManualCompactionState* m = prepicked_compaction->manual_compaction_state; diff --git a/db/db_impl/db_impl_files.cc b/db/db_impl/db_impl_files.cc index 74bffffbe..791549a23 100644 --- a/db/db_impl/db_impl_files.cc +++ b/db/db_impl/db_impl_files.cc @@ -120,7 +120,7 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, mutable_db_options_.delete_obsolete_files_period_micros == 0) { doing_the_full_scan = true; } else { - const uint64_t now_micros = env_->NowMicros(); + const uint64_t now_micros = clock_->NowMicros(); if ((delete_obsolete_files_last_run_ + mutable_db_options_.delete_obsolete_files_period_micros) < now_micros) { diff --git a/db/db_impl/db_impl_open.cc b/db/db_impl/db_impl_open.cc index fe9f0ce74..63b539a97 100644 --- a/db/db_impl/db_impl_open.cc +++ b/db/db_impl/db_impl_open.cc @@ -292,14 +292,14 @@ Status DBImpl::NewDB(std::vector* new_filenames) { file->SetPreallocationBlockSize( immutable_db_options_.manifest_preallocation_size); std::unique_ptr file_writer(new WritableFileWriter( - std::move(file), manifest, file_options, env_, io_tracer_, + std::move(file), manifest, file_options, clock_, io_tracer_, nullptr /* stats */, immutable_db_options_.listeners)); log::Writer log(std::move(file_writer), 0, false); std::string record; new_db.EncodeTo(&record); s = log.AddRecord(record); if (s.ok()) { - s = SyncManifest(env_, &immutable_db_options_, log.file()); + s = SyncManifest(clock_, &immutable_db_options_, log.file()); } } if (s.ok()) { @@ -1295,7 +1295,7 @@ Status DBImpl::RestoreAliveLogFiles(const std::vector& wal_numbers) { Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, MemTable* mem, VersionEdit* edit) { mutex_.AssertHeld(); - const uint64_t start_micros = env_->NowMicros(); + const uint64_t start_micros = clock_->NowMicros(); FileMetaData meta; std::vector blob_file_additions; @@ -1395,7 +1395,7 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, } InternalStats::CompactionStats stats(CompactionReason::kFlush, 1); - stats.micros = env_->NowMicros() - start_micros; + stats.micros = clock_->NowMicros() - start_micros; if (has_output) { stats.bytes_written = meta.fd.GetFileSize(); @@ -1486,7 +1486,7 @@ IOStatus DBImpl::CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number, const auto& listeners = immutable_db_options_.listeners; std::unique_ptr file_writer(new WritableFileWriter( - std::move(lfile), log_fname, opt_file_options, env_, io_tracer_, + std::move(lfile), log_fname, opt_file_options, clock_, io_tracer_, nullptr /* stats */, listeners)); *new_log = new log::Writer(std::move(file_writer), log_file_num, immutable_db_options_.recycle_log_file_num > 0, diff --git a/db/db_impl/db_impl_secondary.cc b/db/db_impl/db_impl_secondary.cc index 69ea75cdb..e5669f9c4 100644 --- a/db/db_impl/db_impl_secondary.cc +++ b/db/db_impl/db_impl_secondary.cc @@ -327,8 +327,8 @@ Status DBImplSecondary::GetImpl(const ReadOptions& read_options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* pinnable_val) { assert(pinnable_val != nullptr); - PERF_CPU_TIMER_GUARD(get_cpu_nanos, env_); - StopWatch sw(env_, stats_, DB_GET); + PERF_CPU_TIMER_GUARD(get_cpu_nanos, clock_); + StopWatch sw(clock_, stats_, DB_GET); PERF_TIMER_GUARD(get_snapshot_time); auto cfh = static_cast(column_family); diff --git a/db/db_impl/db_impl_write.cc b/db/db_impl/db_impl_write.cc index d3dd7a0a9..083cd92d1 100644 --- a/db/db_impl/db_impl_write.cc +++ b/db/db_impl/db_impl_write.cc @@ -160,7 +160,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, RecordTick(stats_, WRITE_WITH_WAL); } - StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE); + StopWatch write_sw(clock_, immutable_db_options_.statistics.get(), DB_WRITE); write_thread_.JoinBatchGroup(&w); if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { @@ -465,7 +465,7 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, uint64_t* log_used, uint64_t log_ref, bool disable_memtable, uint64_t* seq_used) { PERF_TIMER_GUARD(write_pre_and_post_process_time); - StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE); + StopWatch write_sw(clock_, immutable_db_options_.statistics.get(), DB_WRITE); WriteContext write_context; @@ -621,7 +621,7 @@ Status DBImpl::UnorderedWriteMemtable(const WriteOptions& write_options, SequenceNumber seq, const size_t sub_batch_cnt) { PERF_TIMER_GUARD(write_pre_and_post_process_time); - StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE); + StopWatch write_sw(clock_, immutable_db_options_.statistics.get(), DB_WRITE); WriteThread::Writer w(write_options, my_batch, callback, log_ref, false /*disable_memtable*/); @@ -676,7 +676,7 @@ Status DBImpl::WriteImplWALOnly( WriteThread::Writer w(write_options, my_batch, callback, log_ref, disable_memtable, sub_batch_cnt, pre_release_callback); RecordTick(stats_, WRITE_WITH_WAL); - StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE); + StopWatch write_sw(clock_, immutable_db_options_.statistics.get(), DB_WRITE); write_thread->JoinBatchGroup(&w); assert(w.state != WriteThread::STATE_PARALLEL_MEMTABLE_WRITER); @@ -1093,7 +1093,7 @@ IOStatus DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group, } if (io_s.ok() && need_log_sync) { - StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS); + StopWatch sw(clock_, stats_, WAL_FILE_SYNC_MICROS); // It's safe to access logs_ with unlocked mutex_ here because: // - we've set getting_synced=true for all logs, // so other threads won't pop from logs_ while we're here, @@ -1457,8 +1457,8 @@ Status DBImpl::DelayWrite(uint64_t num_bytes, uint64_t time_delayed = 0; bool delayed = false; { - StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed); - uint64_t delay = write_controller_.GetDelay(env_, num_bytes); + StopWatch sw(clock_, stats_, WRITE_STALL, &time_delayed); + uint64_t delay = write_controller_.GetDelay(clock_, num_bytes); if (delay > 0) { if (write_options.no_slowdown) { return Status::Incomplete("Write stall"); @@ -1475,14 +1475,14 @@ Status DBImpl::DelayWrite(uint64_t num_bytes, const uint64_t kDelayInterval = 1000; uint64_t stall_end = sw.start_time() + delay; while (write_controller_.NeedsDelay()) { - if (env_->NowMicros() >= stall_end) { + if (clock_->NowMicros() >= stall_end) { // We already delayed this write `delay` microseconds break; } delayed = true; // Sleep for 0.001 seconds - env_->SleepForMicroseconds(kDelayInterval); + clock_->SleepForMicroseconds(kDelayInterval); } mutex_.Lock(); write_thread_.EndWriteStall(); diff --git a/db/db_iter.cc b/db/db_iter.cc index 00b230326..dc67aa309 100644 --- a/db/db_iter.cc +++ b/db/db_iter.cc @@ -8,9 +8,10 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. #include "db/db_iter.h" -#include + #include #include +#include #include "db/dbformat.h" #include "db/merge_context.h" @@ -24,6 +25,7 @@ #include "rocksdb/iterator.h" #include "rocksdb/merge_operator.h" #include "rocksdb/options.h" +#include "rocksdb/system_clock.h" #include "table/internal_iterator.h" #include "table/iterator_wrapper.h" #include "trace_replay/trace_replay.h" @@ -43,6 +45,7 @@ DBIter::DBIter(Env* _env, const ReadOptions& read_options, ColumnFamilyData* cfd, bool expose_blob_index) : prefix_extractor_(mutable_cf_options.prefix_extractor.get()), env_(_env), + clock_(_env->GetSystemClock()), logger_(cf_options.info_log), user_comparator_(cmp), merge_operator_(cf_options.merge_operator), @@ -127,7 +130,7 @@ void DBIter::Next() { assert(valid_); assert(status_.ok()); - PERF_CPU_TIMER_GUARD(iter_next_cpu_nanos, env_); + PERF_CPU_TIMER_GUARD(iter_next_cpu_nanos, clock_); // Release temporarily pinned blocks from last operation ReleaseTempPinnedData(); local_stats_.skip_count_ += num_internal_keys_skipped_; @@ -574,7 +577,7 @@ bool DBIter::MergeValuesNewToOld() { const Slice val = iter_.value(); Status s = MergeHelper::TimedFullMerge( merge_operator_, ikey.user_key, &val, merge_context_.GetOperands(), - &saved_value_, logger_, statistics_, env_, &pinned_value_, true); + &saved_value_, logger_, statistics_, clock_, &pinned_value_, true); if (!s.ok()) { valid_ = false; status_ = s; @@ -617,7 +620,7 @@ bool DBIter::MergeValuesNewToOld() { // client can differentiate this scenario and do things accordingly. Status s = MergeHelper::TimedFullMerge( merge_operator_, saved_key_.GetUserKey(), nullptr, - merge_context_.GetOperands(), &saved_value_, logger_, statistics_, env_, + merge_context_.GetOperands(), &saved_value_, logger_, statistics_, clock_, &pinned_value_, true); if (!s.ok()) { valid_ = false; @@ -640,7 +643,7 @@ void DBIter::Prev() { assert(valid_); assert(status_.ok()); - PERF_CPU_TIMER_GUARD(iter_prev_cpu_nanos, env_); + PERF_CPU_TIMER_GUARD(iter_prev_cpu_nanos, clock_); ReleaseTempPinnedData(); ResetInternalKeysSkippedCounter(); bool ok = true; @@ -921,7 +924,7 @@ bool DBIter::FindValueForCurrentKey() { s = MergeHelper::TimedFullMerge( merge_operator_, saved_key_.GetUserKey(), nullptr, merge_context_.GetOperands(), &saved_value_, logger_, statistics_, - env_, &pinned_value_, true); + clock_, &pinned_value_, true); } else if (last_not_merge_type == kTypeBlobIndex) { status_ = Status::NotSupported("BlobDB does not support merge operator."); @@ -932,7 +935,7 @@ bool DBIter::FindValueForCurrentKey() { s = MergeHelper::TimedFullMerge( merge_operator_, saved_key_.GetUserKey(), &pinned_value_, merge_context_.GetOperands(), &saved_value_, logger_, statistics_, - env_, &pinned_value_, true); + clock_, &pinned_value_, true); } break; case kTypeValue: @@ -1070,7 +1073,7 @@ bool DBIter::FindValueForCurrentKeyUsingSeek() { Status s = MergeHelper::TimedFullMerge( merge_operator_, saved_key_.GetUserKey(), &val, merge_context_.GetOperands(), &saved_value_, logger_, statistics_, - env_, &pinned_value_, true); + clock_, &pinned_value_, true); if (!s.ok()) { valid_ = false; status_ = s; @@ -1097,7 +1100,7 @@ bool DBIter::FindValueForCurrentKeyUsingSeek() { Status s = MergeHelper::TimedFullMerge( merge_operator_, saved_key_.GetUserKey(), nullptr, - merge_context_.GetOperands(), &saved_value_, logger_, statistics_, env_, + merge_context_.GetOperands(), &saved_value_, logger_, statistics_, clock_, &pinned_value_, true); if (!s.ok()) { valid_ = false; @@ -1248,8 +1251,8 @@ void DBIter::SetSavedKeyToSeekForPrevTarget(const Slice& target) { } void DBIter::Seek(const Slice& target) { - PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_); - StopWatch sw(env_, statistics_, DB_SEEK); + PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_); + StopWatch sw(clock_, statistics_, DB_SEEK); #ifndef ROCKSDB_LITE if (db_impl_ != nullptr && cfd_ != nullptr) { @@ -1310,8 +1313,8 @@ void DBIter::Seek(const Slice& target) { } void DBIter::SeekForPrev(const Slice& target) { - PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_); - StopWatch sw(env_, statistics_, DB_SEEK); + PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_); + StopWatch sw(clock_, statistics_, DB_SEEK); #ifndef ROCKSDB_LITE if (db_impl_ != nullptr && cfd_ != nullptr) { @@ -1378,7 +1381,7 @@ void DBIter::SeekToFirst() { Seek(*iterate_lower_bound_); return; } - PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_); + PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_); // Don't use iter_::Seek() if we set a prefix extractor // because prefix seek will be used. if (!expect_total_order_inner_iter()) { @@ -1439,7 +1442,7 @@ void DBIter::SeekToLast() { return; } - PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, env_); + PERF_CPU_TIMER_GUARD(iter_seek_cpu_nanos, clock_); // Don't use iter_::Seek() if we set a prefix extractor // because prefix seek will be used. if (!expect_total_order_inner_iter()) { diff --git a/db/db_iter.h b/db/db_iter.h index 1e5fb3c3b..9a7d8b948 100644 --- a/db/db_iter.h +++ b/db/db_iter.h @@ -21,7 +21,7 @@ #include "util/autovector.h" namespace ROCKSDB_NAMESPACE { - +class SystemClock_; class Version; // This file declares the factory functions of DBIter, in its original form @@ -298,6 +298,7 @@ class DBIter final : public Iterator { const SliceTransform* prefix_extractor_; Env* const env_; + std::shared_ptr clock_; Logger* logger_; UserComparatorWrapper user_comparator_; const MergeOperator* const merge_operator_; diff --git a/db/db_log_iter_test.cc b/db/db_log_iter_test.cc index 04fbe08be..9baf72037 100644 --- a/db/db_log_iter_test.cc +++ b/db/db_log_iter_test.cc @@ -13,6 +13,7 @@ #if !defined(ROCKSDB_LITE) #include "db/db_test_util.h" +#include "env/mock_env.h" #include "port/stack_trace.h" namespace ROCKSDB_NAMESPACE { diff --git a/db/db_test_util.cc b/db/db_test_util.cc index 93e1a42ed..3a104346d 100644 --- a/db/db_test_util.cc +++ b/db/db_test_util.cc @@ -10,6 +10,7 @@ #include "db/db_test_util.h" #include "db/forward_iterator.h" +#include "env/mock_env.h" #include "rocksdb/convenience.h" #include "rocksdb/env_encryption.h" #include "rocksdb/utilities/object_registry.h" diff --git a/db/db_test_util.h b/db/db_test_util.h index 41eb7b020..7bee90be1 100644 --- a/db/db_test_util.h +++ b/db/db_test_util.h @@ -23,7 +23,6 @@ #include "db/db_impl/db_impl.h" #include "db/dbformat.h" -#include "env/mock_env.h" #include "file/filename.h" #include "memtable/hash_linklist_rep.h" #include "rocksdb/cache.h" @@ -40,7 +39,6 @@ #include "rocksdb/utilities/checkpoint.h" #include "table/mock_table.h" #include "table/scoped_arena_iterator.h" -#include "test_util/mock_time_env.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" #include "util/cast_util.h" @@ -50,6 +48,7 @@ #include "utilities/merge_operators.h" namespace ROCKSDB_NAMESPACE { +class MockEnv; namespace anon { class AtomicCounter { diff --git a/db/error_handler.cc b/db/error_handler.cc index cc313f0a9..221ca6d1a 100644 --- a/db/error_handler.cc +++ b/db/error_handler.cc @@ -670,7 +670,7 @@ void ErrorHandler::RecoverFromRetryableBGIOError() { // a period of time and redo auto resume if it is allowed. TEST_SYNC_POINT("RecoverFromRetryableBGIOError:BeforeWait0"); TEST_SYNC_POINT("RecoverFromRetryableBGIOError:BeforeWait1"); - int64_t wait_until = db_->env_->NowMicros() + wait_interval; + int64_t wait_until = db_->clock_->NowMicros() + wait_interval; cv_.TimedWait(wait_until); TEST_SYNC_POINT("RecoverFromRetryableBGIOError:AfterWait0"); } else { diff --git a/db/external_sst_file_ingestion_job.cc b/db/external_sst_file_ingestion_job.cc index 39aab214b..23d457ade 100644 --- a/db/external_sst_file_ingestion_job.cc +++ b/db/external_sst_file_ingestion_job.cc @@ -293,12 +293,13 @@ Status ExternalSstFileIngestionJob::Prepare( // TODO: The following is duplicated with Cleanup(). if (!status.ok()) { + IOOptions io_opts; // We failed, remove all files that we copied into the db for (IngestedFileInfo& f : files_to_ingest_) { if (f.internal_file_path.empty()) { continue; } - Status s = env_->DeleteFile(f.internal_file_path); + Status s = fs_->DeleteFile(f.internal_file_path, io_opts, nullptr); if (!s.ok()) { ROCKS_LOG_WARN(db_options_.info_log, "AddFile() clean up for file %s failed : %s", @@ -392,7 +393,7 @@ Status ExternalSstFileIngestionJob::Run() { int64_t temp_current_time = 0; uint64_t current_time = kUnknownFileCreationTime; uint64_t oldest_ancester_time = kUnknownOldestAncesterTime; - if (env_->GetCurrentTime(&temp_current_time).ok()) { + if (clock_->GetCurrentTime(&temp_current_time).ok()) { current_time = oldest_ancester_time = static_cast(temp_current_time); } @@ -410,7 +411,7 @@ void ExternalSstFileIngestionJob::UpdateStats() { // Update internal stats for new ingested files uint64_t total_keys = 0; uint64_t total_l0_files = 0; - uint64_t total_time = env_->NowMicros() - job_start_time_; + uint64_t total_time = clock_->NowMicros() - job_start_time_; EventLoggerStream stream = event_logger_->Log(); stream << "event" @@ -466,6 +467,7 @@ void ExternalSstFileIngestionJob::UpdateStats() { } void ExternalSstFileIngestionJob::Cleanup(const Status& status) { + IOOptions io_opts; if (!status.ok()) { // We failed to add the files to the database // remove all the files we copied @@ -473,7 +475,7 @@ void ExternalSstFileIngestionJob::Cleanup(const Status& status) { if (f.internal_file_path.empty()) { continue; } - Status s = env_->DeleteFile(f.internal_file_path); + Status s = fs_->DeleteFile(f.internal_file_path, io_opts, nullptr); if (!s.ok()) { ROCKS_LOG_WARN(db_options_.info_log, "AddFile() clean up for file %s failed : %s", @@ -485,7 +487,7 @@ void ExternalSstFileIngestionJob::Cleanup(const Status& status) { } else if (status.ok() && ingestion_options_.move_files) { // The files were moved and added successfully, remove original file links for (IngestedFileInfo& f : files_to_ingest_) { - Status s = env_->DeleteFile(f.external_file_path); + Status s = fs_->DeleteFile(f.external_file_path, io_opts, nullptr); if (!s.ok()) { ROCKS_LOG_WARN( db_options_.info_log, diff --git a/db/external_sst_file_ingestion_job.h b/db/external_sst_file_ingestion_job.h index a2782f54a..f46e7356c 100644 --- a/db/external_sst_file_ingestion_job.h +++ b/db/external_sst_file_ingestion_job.h @@ -16,13 +16,14 @@ #include "logging/event_logger.h" #include "options/db_options.h" #include "rocksdb/db.h" -#include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/sst_file_writer.h" #include "util/autovector.h" namespace ROCKSDB_NAMESPACE { class Directories; +class SystemClock; struct IngestedFileInfo { // External file path @@ -73,13 +74,13 @@ struct IngestedFileInfo { class ExternalSstFileIngestionJob { public: ExternalSstFileIngestionJob( - Env* env, VersionSet* versions, ColumnFamilyData* cfd, - const ImmutableDBOptions& db_options, const EnvOptions& env_options, - SnapshotList* db_snapshots, + const std::shared_ptr& clock, VersionSet* versions, + ColumnFamilyData* cfd, const ImmutableDBOptions& db_options, + const EnvOptions& env_options, SnapshotList* db_snapshots, const IngestExternalFileOptions& ingestion_options, Directories* directories, EventLogger* event_logger, const std::shared_ptr& io_tracer) - : env_(env), + : clock_(clock), fs_(db_options.fs, io_tracer), versions_(versions), cfd_(cfd), @@ -89,7 +90,7 @@ class ExternalSstFileIngestionJob { ingestion_options_(ingestion_options), directories_(directories), event_logger_(event_logger), - job_start_time_(env_->NowMicros()), + job_start_time_(clock_->NowMicros()), consumed_seqno_count_(0), io_tracer_(io_tracer) { assert(directories != nullptr); @@ -169,7 +170,7 @@ class ExternalSstFileIngestionJob { template Status SyncIngestedFile(TWritableFile* file); - Env* env_; + std::shared_ptr clock_; FileSystemPtr fs_; VersionSet* versions_; ColumnFamilyData* cfd_; diff --git a/db/flush_job.cc b/db/flush_job.cc index d596dc06b..d356b2bb6 100644 --- a/db/flush_job.cc +++ b/db/flush_job.cc @@ -127,6 +127,7 @@ FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd, pick_memtable_called(false), thread_pri_(thread_pri), io_tracer_(io_tracer), + clock_(db_options_.env->GetSystemClock()), full_history_ts_low_(std::move(full_history_ts_low)) { // Update the thread status to indicate flush. ReportStartedFlush(); @@ -309,8 +310,8 @@ Status FlushJob::WriteLevel0Table() { AutoThreadOperationStageUpdater stage_updater( ThreadStatus::STAGE_FLUSH_WRITE_L0); db_mutex_->AssertHeld(); - const uint64_t start_micros = db_options_.env->NowMicros(); - const uint64_t start_cpu_micros = db_options_.env->NowCPUNanos() / 1000; + const uint64_t start_micros = clock_->NowMicros(); + const uint64_t start_cpu_micros = clock_->CPUNanos() / 1000; Status s; std::vector blob_file_additions; @@ -371,7 +372,7 @@ Status FlushJob::WriteLevel0Table() { TEST_SYNC_POINT_CALLBACK("FlushJob::WriteLevel0Table:output_compression", &output_compression_); int64_t _current_time = 0; - auto status = db_options_.env->GetCurrentTime(&_current_time); + auto status = clock_->GetCurrentTime(&_current_time); // Safe to proceed even if GetCurrentTime fails. So, log and proceed. if (!status.ok()) { ROCKS_LOG_WARN( @@ -466,8 +467,8 @@ Status FlushJob::WriteLevel0Table() { // Note that here we treat flush as level 0 compaction in internal stats InternalStats::CompactionStats stats(CompactionReason::kFlush, 1); - stats.micros = db_options_.env->NowMicros() - start_micros; - stats.cpu_micros = db_options_.env->NowCPUNanos() / 1000 - start_cpu_micros; + stats.micros = clock_->NowMicros() - start_micros; + stats.cpu_micros = clock_->CPUNanos() / 1000 - start_cpu_micros; if (has_output) { stats.bytes_written = meta_.fd.GetFileSize(); diff --git a/db/flush_job.h b/db/flush_job.h index e3623209f..35ef4a00c 100644 --- a/db/flush_job.h +++ b/db/flush_job.h @@ -162,6 +162,7 @@ class FlushJob { IOStatus io_status_; const std::shared_ptr io_tracer_; + const std::shared_ptr clock_; const std::string full_history_ts_low_; }; diff --git a/db/import_column_family_job.cc b/db/import_column_family_job.cc index 7854ceff6..7c8c44e4e 100644 --- a/db/import_column_family_job.cc +++ b/db/import_column_family_job.cc @@ -140,7 +140,7 @@ Status ImportColumnFamilyJob::Run() { int64_t temp_current_time = 0; uint64_t oldest_ancester_time = kUnknownOldestAncesterTime; uint64_t current_time = kUnknownOldestAncesterTime; - if (env_->GetCurrentTime(&temp_current_time).ok()) { + if (clock_->GetCurrentTime(&temp_current_time).ok()) { current_time = oldest_ancester_time = static_cast(temp_current_time); } diff --git a/db/import_column_family_job.h b/db/import_column_family_job.h index 6cdde2473..0896b4fad 100644 --- a/db/import_column_family_job.h +++ b/db/import_column_family_job.h @@ -9,24 +9,26 @@ #include "db/snapshot_impl.h" #include "options/db_options.h" #include "rocksdb/db.h" -#include "rocksdb/env.h" #include "rocksdb/metadata.h" #include "rocksdb/sst_file_writer.h" #include "util/autovector.h" namespace ROCKSDB_NAMESPACE { +struct EnvOptions; +class SystemClock; // Imports a set of sst files as is into a new column family. Logic is similar // to ExternalSstFileIngestionJob. class ImportColumnFamilyJob { public: - ImportColumnFamilyJob(Env* env, VersionSet* versions, ColumnFamilyData* cfd, + ImportColumnFamilyJob(const std::shared_ptr& clock, + VersionSet* versions, ColumnFamilyData* cfd, const ImmutableDBOptions& db_options, const EnvOptions& env_options, const ImportColumnFamilyOptions& import_options, const std::vector& metadata, const std::shared_ptr& io_tracer) - : env_(env), + : clock_(clock), versions_(versions), cfd_(cfd), db_options_(db_options), @@ -59,7 +61,7 @@ class ImportColumnFamilyJob { IngestedFileInfo* file_to_import, SuperVersion* sv); - Env* env_; + std::shared_ptr clock_; VersionSet* versions_; ColumnFamilyData* cfd_; const ImmutableDBOptions& db_options_; diff --git a/db/internal_stats.cc b/db/internal_stats.cc index 512bc1b01..fc5981d91 100644 --- a/db/internal_stats.cc +++ b/db/internal_stats.cc @@ -990,7 +990,7 @@ bool InternalStats::HandleBlockCachePinnedUsage(uint64_t* value, DBImpl* /*db*/, void InternalStats::DumpDBStats(std::string* value) { char buf[1000]; // DB-level stats, only available from default column family - double seconds_up = (env_->NowMicros() - started_at_ + 1) / kMicrosInSec; + double seconds_up = (clock_->NowMicros() - started_at_ + 1) / kMicrosInSec; double interval_seconds_up = seconds_up - db_stats_snapshot_.seconds_up; snprintf(buf, sizeof(buf), "\n** DB Stats **\nUptime(secs): %.1f total, %.1f interval\n", @@ -1313,7 +1313,7 @@ void InternalStats::DumpCFStatsNoFileHistogram(std::string* value) { } } - double seconds_up = (env_->NowMicros() - started_at_ + 1) / kMicrosInSec; + double seconds_up = (clock_->NowMicros() - started_at_ + 1) / kMicrosInSec; double interval_seconds_up = seconds_up - cf_stats_snapshot_.seconds_up; snprintf(buf, sizeof(buf), "Uptime(secs): %.1f total, %.1f interval\n", seconds_up, interval_seconds_up); diff --git a/db/internal_stats.h b/db/internal_stats.h index 056719c5c..28ed50087 100644 --- a/db/internal_stats.h +++ b/db/internal_stats.h @@ -14,6 +14,7 @@ #include #include "db/version_set.h" +#include "rocksdb/system_clock.h" class ColumnFamilyData; @@ -122,7 +123,8 @@ class InternalStats { kIntStatsNumMax, }; - InternalStats(int num_levels, Env* env, ColumnFamilyData* cfd) + InternalStats(int num_levels, const std::shared_ptr& clock, + ColumnFamilyData* cfd) : db_stats_{}, cf_stats_value_{}, cf_stats_count_{}, @@ -131,9 +133,9 @@ class InternalStats { file_read_latency_(num_levels), bg_error_count_(0), number_levels_(num_levels), - env_(env), + clock_(clock), cfd_(cfd), - started_at_(env->NowMicros()) {} + started_at_(clock->NowMicros()) {} // Per level compaction stats. comp_stats_[level] stores the stats for // compactions that produced data for the specified "level". @@ -341,7 +343,7 @@ class InternalStats { cf_stats_snapshot_.Clear(); db_stats_snapshot_.Clear(); bg_error_count_ = 0; - started_at_ = env_->NowMicros(); + started_at_ = clock_->NowMicros(); } void AddCompactionStats(int level, Env::Priority thread_pri, @@ -602,7 +604,7 @@ class InternalStats { uint64_t bg_error_count_; const int number_levels_; - Env* env_; + const std::shared_ptr clock_; ColumnFamilyData* cfd_; uint64_t started_at_; }; @@ -641,7 +643,9 @@ class InternalStats { kIntStatsNumMax, }; - InternalStats(int /*num_levels*/, Env* /*env*/, ColumnFamilyData* /*cfd*/) {} + InternalStats(int /*num_levels*/, + const std::shared_ptr& /*clock*/, + ColumnFamilyData* /*cfd*/) {} struct CompactionStats { uint64_t micros; diff --git a/db/memtable.cc b/db/memtable.cc index 49f0a4c9c..6517fa882 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -102,7 +102,7 @@ MemTable::MemTable(const InternalKeyComparator& cmp, : 0), prefix_extractor_(mutable_cf_options.prefix_extractor.get()), flush_state_(FLUSH_NOT_REQUESTED), - env_(ioptions.env), + clock_(ioptions.env->GetSystemClock()), insert_with_hint_prefix_extractor_( ioptions.memtable_insert_with_hint_prefix_extractor), oldest_key_time_(std::numeric_limits::max()), @@ -221,7 +221,7 @@ void MemTable::UpdateOldestKeyTime() { uint64_t oldest_key_time = oldest_key_time_.load(std::memory_order_relaxed); if (oldest_key_time == std::numeric_limits::max()) { int64_t current_time = 0; - auto s = env_->GetCurrentTime(¤t_time); + auto s = clock_->GetCurrentTime(¤t_time); if (s.ok()) { assert(current_time >= 0); // If fail, the timestamp is already set. @@ -628,7 +628,8 @@ struct Saver { Statistics* statistics; bool inplace_update_support; bool do_merge; - Env* env_; + std::shared_ptr clock; + ReadCallback* callback_; bool* is_blob_index; bool allow_data_in_errors; @@ -712,7 +713,7 @@ static bool SaveValue(void* arg, const char* entry) { *(s->status) = MergeHelper::TimedFullMerge( merge_operator, s->key->user_key(), &v, merge_context->GetOperands(), s->value, s->logger, - s->statistics, s->env_, nullptr /* result_operand */, true); + s->statistics, s->clock, nullptr /* result_operand */, true); } } else { // Preserve the value with the goal of returning it as part of @@ -751,7 +752,7 @@ static bool SaveValue(void* arg, const char* entry) { *(s->status) = MergeHelper::TimedFullMerge( merge_operator, s->key->user_key(), nullptr, merge_context->GetOperands(), s->value, s->logger, - s->statistics, s->env_, nullptr /* result_operand */, true); + s->statistics, s->clock, nullptr /* result_operand */, true); } } else { *(s->status) = Status::NotFound(); @@ -779,7 +780,7 @@ static bool SaveValue(void* arg, const char* entry) { *(s->status) = MergeHelper::TimedFullMerge( merge_operator, s->key->user_key(), nullptr, merge_context->GetOperands(), s->value, s->logger, s->statistics, - s->env_, nullptr /* result_operand */, true); + s->clock, nullptr /* result_operand */, true); *(s->found_final_value) = true; return false; } @@ -887,7 +888,7 @@ void MemTable::GetFromTable(const LookupKey& key, saver.logger = moptions_.info_log; saver.inplace_update_support = moptions_.inplace_update_support; saver.statistics = moptions_.statistics; - saver.env_ = env_; + saver.clock = clock_; saver.callback_ = callback; saver.is_blob_index = is_blob_index; saver.do_merge = do_merge; diff --git a/db/memtable.h b/db/memtable.h index 525582698..ab03a2214 100644 --- a/db/memtable.h +++ b/db/memtable.h @@ -24,7 +24,6 @@ #include "monitoring/instrumented_mutex.h" #include "options/cf_options.h" #include "rocksdb/db.h" -#include "rocksdb/env.h" #include "rocksdb/memtablerep.h" #include "table/multiget_context.h" #include "util/dynamic_bloom.h" @@ -36,6 +35,7 @@ struct FlushJobInfo; class Mutex; class MemTableIterator; class MergeContext; +class SystemClock; struct ImmutableMemTableOptions { explicit ImmutableMemTableOptions(const ImmutableCFOptions& ioptions, @@ -504,7 +504,7 @@ class MemTable { std::atomic flush_state_; - Env* env_; + std::shared_ptr clock_; // Extract sequential insert prefixes. const SliceTransform* insert_with_hint_prefix_extractor_; diff --git a/db/merge_helper.cc b/db/merge_helper.cc index 8497d23b0..dd9e1c4d3 100644 --- a/db/merge_helper.cc +++ b/db/merge_helper.cc @@ -14,6 +14,7 @@ #include "rocksdb/comparator.h" #include "rocksdb/db.h" #include "rocksdb/merge_operator.h" +#include "rocksdb/system_clock.h" #include "table/format.h" #include "table/internal_iterator.h" @@ -28,6 +29,7 @@ MergeHelper::MergeHelper(Env* env, const Comparator* user_comparator, Statistics* stats, const std::atomic* shutting_down) : env_(env), + clock_(env->GetSystemClock()), user_comparator_(user_comparator), user_merge_operator_(user_merge_operator), compaction_filter_(compaction_filter), @@ -39,7 +41,7 @@ MergeHelper::MergeHelper(Env* env, const Comparator* user_comparator, snapshot_checker_(snapshot_checker), level_(level), keys_(), - filter_timer_(env_), + filter_timer_(clock_), total_filter_time_(0U), stats_(stats) { assert(user_comparator_ != nullptr); @@ -48,13 +50,11 @@ MergeHelper::MergeHelper(Env* env, const Comparator* user_comparator, } } -Status MergeHelper::TimedFullMerge(const MergeOperator* merge_operator, - const Slice& key, const Slice* value, - const std::vector& operands, - std::string* result, Logger* logger, - Statistics* statistics, Env* env, - Slice* result_operand, - bool update_num_ops_stats) { +Status MergeHelper::TimedFullMerge( + const MergeOperator* merge_operator, const Slice& key, const Slice* value, + const std::vector& operands, std::string* result, Logger* logger, + Statistics* statistics, const std::shared_ptr& clock, + Slice* result_operand, bool update_num_ops_stats) { assert(merge_operator != nullptr); if (operands.size() == 0) { @@ -75,7 +75,7 @@ Status MergeHelper::TimedFullMerge(const MergeOperator* merge_operator, MergeOperator::MergeOperationOutput merge_out(*result, tmp_result_operand); { // Setup to time the merge - StopWatchNano timer(env, statistics != nullptr); + StopWatchNano timer(clock, statistics != nullptr); PERF_TIMER_GUARD(merge_operator_time_nanos); // Do the merge @@ -213,7 +213,7 @@ Status MergeHelper::MergeUntil(InternalIterator* iter, std::string merge_result; s = TimedFullMerge(user_merge_operator_, ikey.user_key, val_ptr, merge_context_.GetOperands(), &merge_result, logger_, - stats_, env_); + stats_, clock_); // We store the result in keys_.back() and operands_.back() // if nothing went wrong (i.e.: no operand corruption on disk) @@ -324,7 +324,7 @@ Status MergeHelper::MergeUntil(InternalIterator* iter, std::string merge_result; s = TimedFullMerge(user_merge_operator_, orig_ikey.user_key, nullptr, merge_context_.GetOperands(), &merge_result, logger_, - stats_, env_); + stats_, clock_); if (s.ok()) { // The original key encountered // We are certain that keys_ is not empty here (see assertions couple of @@ -347,7 +347,7 @@ Status MergeHelper::MergeUntil(InternalIterator* iter, bool merge_success = false; std::string merge_result; { - StopWatchNano timer(env_, stats_ != nullptr); + StopWatchNano timer(clock_, stats_ != nullptr); PERF_TIMER_GUARD(merge_operator_time_nanos); merge_success = user_merge_operator_->PartialMergeMulti( orig_ikey.user_key, diff --git a/db/merge_helper.h b/db/merge_helper.h index 30759effa..1d6d59585 100644 --- a/db/merge_helper.h +++ b/db/merge_helper.h @@ -25,6 +25,7 @@ class Iterator; class Logger; class MergeOperator; class Statistics; +class SystemClock; class MergeHelper { public: @@ -44,13 +45,11 @@ class MergeHelper { // Returns one of the following statuses: // - OK: Entries were successfully merged. // - Corruption: Merge operator reported unsuccessful merge. - static Status TimedFullMerge(const MergeOperator* merge_operator, - const Slice& key, const Slice* value, - const std::vector& operands, - std::string* result, Logger* logger, - Statistics* statistics, Env* env, - Slice* result_operand = nullptr, - bool update_num_ops_stats = false); + static Status TimedFullMerge( + const MergeOperator* merge_operator, const Slice& key, const Slice* value, + const std::vector& operands, std::string* result, Logger* logger, + Statistics* statistics, const std::shared_ptr& clock, + Slice* result_operand = nullptr, bool update_num_ops_stats = false); // Merge entries until we hit // - a corrupted key @@ -140,6 +139,7 @@ class MergeHelper { private: Env* env_; + std::shared_ptr clock_; const Comparator* user_comparator_; const MergeOperator* user_merge_operator_; const CompactionFilter* compaction_filter_; diff --git a/db/perf_context_test.cc b/db/perf_context_test.cc index 0df8b462a..95da75e64 100644 --- a/db/perf_context_test.cc +++ b/db/perf_context_test.cc @@ -3,6 +3,8 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). // +#include "rocksdb/perf_context.h" + #include #include #include @@ -15,8 +17,8 @@ #include "port/port.h" #include "rocksdb/db.h" #include "rocksdb/memtablerep.h" -#include "rocksdb/perf_context.h" #include "rocksdb/slice_transform.h" +#include "rocksdb/system_clock.h" #include "test_util/testharness.h" #include "util/stop_watch.h" #include "util/string_util.h" @@ -91,7 +93,7 @@ TEST_F(PerfContextTest, SeekIntoDeletion) { std::string value; get_perf_context()->Reset(); - StopWatchNano timer(Env::Default()); + StopWatchNano timer(SystemClock::Default()); timer.Start(); auto status = db->Get(read_options, key, &value); auto elapsed_nanos = timer.ElapsedNanos(); @@ -110,7 +112,7 @@ TEST_F(PerfContextTest, SeekIntoDeletion) { std::unique_ptr iter(db->NewIterator(read_options)); get_perf_context()->Reset(); - StopWatchNano timer(Env::Default(), true); + StopWatchNano timer(SystemClock::Default(), true); iter->SeekToFirst(); hist_seek_to_first.Add(get_perf_context()->user_key_comparison_count); auto elapsed_nanos = timer.ElapsedNanos(); @@ -131,7 +133,7 @@ TEST_F(PerfContextTest, SeekIntoDeletion) { std::string key = "k" + ToString(i); get_perf_context()->Reset(); - StopWatchNano timer(Env::Default(), true); + StopWatchNano timer(SystemClock::Default(), true); iter->Seek(key); auto elapsed_nanos = timer.ElapsedNanos(); hist_seek.Add(get_perf_context()->user_key_comparison_count); @@ -145,7 +147,7 @@ TEST_F(PerfContextTest, SeekIntoDeletion) { get_perf_context()->Reset(); ASSERT_TRUE(iter->Valid()); - StopWatchNano timer2(Env::Default(), true); + StopWatchNano timer2(SystemClock::Default(), true); iter->Next(); auto elapsed_nanos2 = timer2.ElapsedNanos(); if (FLAGS_verbose) { @@ -164,7 +166,7 @@ TEST_F(PerfContextTest, StopWatchNanoOverhead) { const int kTotalIterations = 1000000; std::vector timings(kTotalIterations); - StopWatchNano timer(Env::Default(), true); + StopWatchNano timer(SystemClock::Default(), true); for (auto& timing : timings) { timing = timer.ElapsedNanos(true /* reset */); } @@ -185,7 +187,7 @@ TEST_F(PerfContextTest, StopWatchOverhead) { uint64_t elapsed = 0; std::vector timings(kTotalIterations); - StopWatch timer(Env::Default(), nullptr, 0, &elapsed); + StopWatch timer(SystemClock::Default(), nullptr, 0, &elapsed); for (auto& timing : timings) { timing = elapsed; } @@ -539,7 +541,7 @@ TEST_F(PerfContextTest, SeekKeyComparison) { HistogramImpl hist_time_diff; SetPerfLevel(kEnableTime); - StopWatchNano timer(Env::Default()); + StopWatchNano timer(SystemClock::Default()); for (const int i : keys) { std::string key = "k" + ToString(i); std::string value = "v" + ToString(i); @@ -592,25 +594,25 @@ TEST_F(PerfContextTest, DBMutexLockCounter) { for (PerfLevel perf_level_test : {PerfLevel::kEnableTimeExceptForMutex, PerfLevel::kEnableTime}) { for (int c = 0; c < 2; ++c) { - InstrumentedMutex mutex(nullptr, Env::Default(), stats_code[c]); - mutex.Lock(); - ROCKSDB_NAMESPACE::port::Thread child_thread([&] { - SetPerfLevel(perf_level_test); - get_perf_context()->Reset(); - ASSERT_EQ(get_perf_context()->db_mutex_lock_nanos, 0); + InstrumentedMutex mutex(nullptr, SystemClock::Default(), stats_code[c]); mutex.Lock(); - mutex.Unlock(); - if (perf_level_test == PerfLevel::kEnableTimeExceptForMutex || - stats_code[c] != DB_MUTEX_WAIT_MICROS) { + ROCKSDB_NAMESPACE::port::Thread child_thread([&] { + SetPerfLevel(perf_level_test); + get_perf_context()->Reset(); ASSERT_EQ(get_perf_context()->db_mutex_lock_nanos, 0); - } else { - // increment the counter only when it's a DB Mutex - ASSERT_GT(get_perf_context()->db_mutex_lock_nanos, 0); - } - }); - Env::Default()->SleepForMicroseconds(100); - mutex.Unlock(); - child_thread.join(); + mutex.Lock(); + mutex.Unlock(); + if (perf_level_test == PerfLevel::kEnableTimeExceptForMutex || + stats_code[c] != DB_MUTEX_WAIT_MICROS) { + ASSERT_EQ(get_perf_context()->db_mutex_lock_nanos, 0); + } else { + // increment the counter only when it's a DB Mutex + ASSERT_GT(get_perf_context()->db_mutex_lock_nanos, 0); + } + }); + SystemClock::Default()->SleepForMicroseconds(100); + mutex.Unlock(); + child_thread.join(); } } } @@ -619,7 +621,7 @@ TEST_F(PerfContextTest, FalseDBMutexWait) { SetPerfLevel(kEnableTime); int stats_code[] = {0, static_cast(DB_MUTEX_WAIT_MICROS)}; for (int c = 0; c < 2; ++c) { - InstrumentedMutex mutex(nullptr, Env::Default(), stats_code[c]); + InstrumentedMutex mutex(nullptr, SystemClock::Default(), stats_code[c]); InstrumentedCondVar lock(&mutex); get_perf_context()->Reset(); mutex.Lock(); @@ -824,8 +826,8 @@ TEST_F(PerfContextTest, PerfContextByLevelGetSet) { } TEST_F(PerfContextTest, CPUTimer) { - if (Env::Default()->NowCPUNanos() == 0) { - ROCKSDB_GTEST_SKIP("Target without NowCPUNanos support"); + if (SystemClock::Default()->CPUNanos() == 0) { + ROCKSDB_GTEST_SKIP("Target without CPUNanos support"); return; } diff --git a/db/periodic_work_scheduler.cc b/db/periodic_work_scheduler.cc index da0bc1e4b..ebca23d51 100644 --- a/db/periodic_work_scheduler.cc +++ b/db/periodic_work_scheduler.cc @@ -6,12 +6,14 @@ #include "db/periodic_work_scheduler.h" #include "db/db_impl/db_impl.h" +#include "rocksdb/system_clock.h" #ifndef ROCKSDB_LITE namespace ROCKSDB_NAMESPACE { -PeriodicWorkScheduler::PeriodicWorkScheduler(Env* env) : timer_mu_(env) { - timer = std::unique_ptr(new Timer(env)); +PeriodicWorkScheduler::PeriodicWorkScheduler( + const std::shared_ptr& clock) { + timer = std::unique_ptr(new Timer(clock)); } void PeriodicWorkScheduler::Register(DBImpl* dbi, @@ -52,10 +54,10 @@ void PeriodicWorkScheduler::Unregister(DBImpl* dbi) { } PeriodicWorkScheduler* PeriodicWorkScheduler::Default() { - // Always use the default Env for the scheduler, as we only use the NowMicros - // which is the same for all env. - // The Env could only be overridden in test. - static PeriodicWorkScheduler scheduler(Env::Default()); + // Always use the default SystemClock for the scheduler, as we only use the + // NowMicros which is the same for all clocks. The Env could only be + // overridden in test. + static PeriodicWorkScheduler scheduler(SystemClock::Default()); return &scheduler; } @@ -69,12 +71,13 @@ std::string PeriodicWorkScheduler::GetTaskName(DBImpl* dbi, #ifndef NDEBUG -// Get the static scheduler. For a new env, it needs to re-create the internal -// timer, so only re-create it when there's no running task. Otherwise, return -// the existing scheduler. Which means if the unittest needs to update MockEnv, -// Close all db instances and then re-open them. -PeriodicWorkTestScheduler* PeriodicWorkTestScheduler::Default(Env* env) { - static PeriodicWorkTestScheduler scheduler(env); +// Get the static scheduler. For a new SystemClock, it needs to re-create the +// internal timer, so only re-create it when there's no running task. Otherwise, +// return the existing scheduler. Which means if the unittest needs to update +// MockClock, Close all db instances and then re-open them. +PeriodicWorkTestScheduler* PeriodicWorkTestScheduler::Default( + const std::shared_ptr& clock) { + static PeriodicWorkTestScheduler scheduler(clock); static port::Mutex mutex; { MutexLock l(&mutex); @@ -84,7 +87,7 @@ PeriodicWorkTestScheduler* PeriodicWorkTestScheduler::Default(Env* env) { MutexLock timer_mu_guard(&scheduler.timer_mu_); scheduler.timer->Shutdown(); } - scheduler.timer.reset(new Timer(env)); + scheduler.timer.reset(new Timer(clock)); } } return &scheduler; @@ -104,8 +107,9 @@ size_t PeriodicWorkTestScheduler::TEST_GetValidTaskNum() const { return 0; } -PeriodicWorkTestScheduler::PeriodicWorkTestScheduler(Env* env) - : PeriodicWorkScheduler(env) {} +PeriodicWorkTestScheduler::PeriodicWorkTestScheduler( + const std::shared_ptr& clock) + : PeriodicWorkScheduler(clock) {} #endif // !NDEBUG } // namespace ROCKSDB_NAMESPACE diff --git a/db/periodic_work_scheduler.h b/db/periodic_work_scheduler.h index 9382adc44..fe89ff567 100644 --- a/db/periodic_work_scheduler.h +++ b/db/periodic_work_scheduler.h @@ -11,6 +11,7 @@ #include "util/timer.h" namespace ROCKSDB_NAMESPACE { +class SystemClock; // PeriodicWorkScheduler is a singleton object, which is scheduling/running // DumpStats(), PersistStats(), and FlushInfoLog() for all DB instances. All DB @@ -49,25 +50,26 @@ class PeriodicWorkScheduler { // the `Timer::Cancel()`s and `Timer::Shutdown()` run atomically. port::Mutex timer_mu_; - explicit PeriodicWorkScheduler(Env* env); + explicit PeriodicWorkScheduler(const std::shared_ptr& clock); private: std::string GetTaskName(DBImpl* dbi, const std::string& func_name); }; #ifndef NDEBUG -// PeriodicWorkTestScheduler is for unittest, which can specify the Env like -// SafeMockTimeEnv. It also contains functions for unittest. +// PeriodicWorkTestScheduler is for unittest, which can specify the SystemClock +// It also contains functions for unittest. class PeriodicWorkTestScheduler : public PeriodicWorkScheduler { public: - static PeriodicWorkTestScheduler* Default(Env* env); + static PeriodicWorkTestScheduler* Default( + const std::shared_ptr& clock); void TEST_WaitForRun(std::function callback) const; size_t TEST_GetValidTaskNum() const; private: - explicit PeriodicWorkTestScheduler(Env* env); + explicit PeriodicWorkTestScheduler(const std::shared_ptr& clock); }; #endif // !NDEBUG diff --git a/db/periodic_work_scheduler_test.cc b/db/periodic_work_scheduler_test.cc index 28e7de783..a92b8730f 100644 --- a/db/periodic_work_scheduler_test.cc +++ b/db/periodic_work_scheduler_test.cc @@ -6,6 +6,8 @@ #include "db/periodic_work_scheduler.h" #include "db/db_test_util.h" +#include "env/composite_env_wrapper.h" +#include "test_util/mock_time_env.h" namespace ROCKSDB_NAMESPACE { @@ -14,20 +16,22 @@ class PeriodicWorkSchedulerTest : public DBTestBase { public: PeriodicWorkSchedulerTest() : DBTestBase("/periodic_work_scheduler_test", /*env_do_fsync=*/true) { - mock_env_.reset(new MockTimeEnv(env_)); + mock_clock_ = std::make_shared(env_->GetSystemClock()); + mock_env_.reset(new CompositeEnvWrapper(env_, mock_clock_)); } protected: - std::unique_ptr mock_env_; + std::unique_ptr mock_env_; + std::shared_ptr mock_clock_; void SetUp() override { - mock_env_->InstallTimedWaitFixCallback(); + mock_clock_->InstallTimedWaitFixCallback(); SyncPoint::GetInstance()->SetCallBack( "DBImpl::StartPeriodicWorkScheduler:Init", [&](void* arg) { auto* periodic_work_scheduler_ptr = reinterpret_cast(arg); *periodic_work_scheduler_ptr = - PeriodicWorkTestScheduler::Default(mock_env_.get()); + PeriodicWorkTestScheduler::Default(mock_clock_); }); } }; @@ -63,7 +67,7 @@ TEST_F(PeriodicWorkSchedulerTest, Basic) { ASSERT_GT(kPeriodSec, 1u); dbfull()->TEST_WaitForStatsDumpRun([&] { - mock_env_->MockSleepForSeconds(static_cast(kPeriodSec) - 1); + mock_clock_->MockSleepForSeconds(static_cast(kPeriodSec) - 1); }); auto scheduler = dbfull()->TEST_GetPeriodicWorkScheduler(); @@ -75,14 +79,14 @@ TEST_F(PeriodicWorkSchedulerTest, Basic) { ASSERT_EQ(1, flush_info_log_counter); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(static_cast(kPeriodSec)); }); + [&] { mock_clock_->MockSleepForSeconds(static_cast(kPeriodSec)); }); ASSERT_EQ(2, dump_st_counter); ASSERT_EQ(2, pst_st_counter); ASSERT_EQ(2, flush_info_log_counter); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(static_cast(kPeriodSec)); }); + [&] { mock_clock_->MockSleepForSeconds(static_cast(kPeriodSec)); }); ASSERT_EQ(3, dump_st_counter); ASSERT_EQ(3, pst_st_counter); @@ -96,7 +100,7 @@ TEST_F(PeriodicWorkSchedulerTest, Basic) { // Info log flush should still run. dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(static_cast(kPeriodSec)); }); + [&] { mock_clock_->MockSleepForSeconds(static_cast(kPeriodSec)); }); ASSERT_EQ(3, dump_st_counter); ASSERT_EQ(3, pst_st_counter); ASSERT_EQ(4, flush_info_log_counter); @@ -114,7 +118,7 @@ TEST_F(PeriodicWorkSchedulerTest, Basic) { ASSERT_EQ(2, scheduler->TEST_GetValidTaskNum()); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(static_cast(kPeriodSec)); }); + [&] { mock_clock_->MockSleepForSeconds(static_cast(kPeriodSec)); }); ASSERT_EQ(4, dump_st_counter); ASSERT_EQ(3, pst_st_counter); ASSERT_EQ(5, flush_info_log_counter); @@ -154,19 +158,19 @@ TEST_F(PeriodicWorkSchedulerTest, MultiInstances) { int expected_run = kInstanceNum; dbi->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); ASSERT_EQ(expected_run, dump_st_counter); ASSERT_EQ(expected_run, pst_st_counter); expected_run += kInstanceNum; dbi->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); ASSERT_EQ(expected_run, dump_st_counter); ASSERT_EQ(expected_run, pst_st_counter); expected_run += kInstanceNum; dbi->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); ASSERT_EQ(expected_run, dump_st_counter); ASSERT_EQ(expected_run, pst_st_counter); @@ -178,9 +182,9 @@ TEST_F(PeriodicWorkSchedulerTest, MultiInstances) { expected_run += (kInstanceNum - half) * 2; dbi->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); dbi->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); ASSERT_EQ(expected_run, dump_st_counter); ASSERT_EQ(expected_run, pst_st_counter); @@ -202,7 +206,8 @@ TEST_F(PeriodicWorkSchedulerTest, MultiEnv) { Reopen(options1); - std::unique_ptr mock_env2(new MockTimeEnv(Env::Default())); + std::unique_ptr mock_env2( + new CompositeEnvWrapper(Env::Default(), mock_clock_)); Options options2; options2.stats_dump_period_sec = kDumpPeriodSec; options2.stats_persist_period_sec = kPersistPeriodSec; diff --git a/db/prefix_test.cc b/db/prefix_test.cc index d4808c78f..24b153575 100644 --- a/db/prefix_test.cc +++ b/db/prefix_test.cc @@ -25,6 +25,7 @@ int main() { #include "rocksdb/memtablerep.h" #include "rocksdb/perf_context.h" #include "rocksdb/slice_transform.h" +#include "rocksdb/system_clock.h" #include "rocksdb/table.h" #include "test_util/testharness.h" #include "util/cast_util.h" @@ -608,7 +609,7 @@ TEST_F(PrefixTest, DynamicPrefixIterator) { std::string value(FLAGS_value_size, 0); get_perf_context()->Reset(); - StopWatchNano timer(Env::Default(), true); + StopWatchNano timer(SystemClock::Default(), true); ASSERT_OK(db->Put(write_options, key, value)); hist_put_time.Add(timer.ElapsedNanos()); hist_put_comparison.Add(get_perf_context()->user_key_comparison_count); @@ -631,7 +632,7 @@ TEST_F(PrefixTest, DynamicPrefixIterator) { std::string value = "v" + ToString(0); get_perf_context()->Reset(); - StopWatchNano timer(Env::Default(), true); + StopWatchNano timer(SystemClock::Default(), true); auto key_prefix = options.prefix_extractor->Transform(key); uint64_t total_keys = 0; for (iter->Seek(key); @@ -665,7 +666,7 @@ TEST_F(PrefixTest, DynamicPrefixIterator) { Slice key = TestKeyToSlice(s, test_key); get_perf_context()->Reset(); - StopWatchNano timer(Env::Default(), true); + StopWatchNano timer(SystemClock::Default(), true); iter->Seek(key); hist_no_seek_time.Add(timer.ElapsedNanos()); hist_no_seek_comparison.Add(get_perf_context()->user_key_comparison_count); diff --git a/db/range_del_aggregator_bench.cc b/db/range_del_aggregator_bench.cc index 3f3135f2e..a92efa132 100644 --- a/db/range_del_aggregator_bench.cc +++ b/db/range_del_aggregator_bench.cc @@ -11,8 +11,8 @@ int main() { } #else -#include #include +#include #include #include #include @@ -22,14 +22,13 @@ int main() { #include "db/range_del_aggregator.h" #include "db/range_tombstone_fragmenter.h" #include "rocksdb/comparator.h" -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "test_util/testutil.h" #include "util/coding.h" +#include "util/gflags_compat.h" #include "util/random.h" #include "util/stop_watch.h" -#include "util/gflags_compat.h" - using GFLAGS_NAMESPACE::ParseCommandLineFlags; DEFINE_int32(num_range_tombstones, 1000, "number of range tombstones created"); @@ -220,7 +219,7 @@ int main(int argc, char** argv) { ROCKSDB_NAMESPACE::kMaxSequenceNumber)); ROCKSDB_NAMESPACE::StopWatchNano stop_watch_add_tombstones( - ROCKSDB_NAMESPACE::Env::Default(), true /* auto_start */); + ROCKSDB_NAMESPACE::SystemClock::Default(), true /* auto_start */); range_del_agg.AddTombstones(std::move(fragmented_range_del_iter)); stats.time_add_tombstones += stop_watch_add_tombstones.ElapsedNanos(); } @@ -237,7 +236,7 @@ int main(int argc, char** argv) { parsed_key.user_key = key_string; ROCKSDB_NAMESPACE::StopWatchNano stop_watch_should_delete( - ROCKSDB_NAMESPACE::Env::Default(), true /* auto_start */); + ROCKSDB_NAMESPACE::SystemClock::Default(), true /* auto_start */); range_del_agg.ShouldDelete(parsed_key, mode); uint64_t call_time = stop_watch_should_delete.ElapsedNanos(); diff --git a/db/table_cache.cc b/db/table_cache.cc index c47d62891..49ae124a2 100644 --- a/db/table_cache.cc +++ b/db/table_cache.cc @@ -106,14 +106,15 @@ Status TableCache::GetTableReader( TableFileName(ioptions_.cf_paths, fd.GetNumber(), fd.GetPathId()); std::unique_ptr file; FileOptions fopts = file_options; - Status s = PrepareIOFromReadOptions(ro, ioptions_.env, fopts.io_options); + const auto& clock = ioptions_.env->GetSystemClock(); + Status s = PrepareIOFromReadOptions(ro, clock, fopts.io_options); if (s.ok()) { s = ioptions_.fs->NewRandomAccessFile(fname, fopts, &file, nullptr); } RecordTick(ioptions_.statistics, NO_FILE_OPENS); if (s.IsPathNotFound()) { fname = Rocks2LevelTableFileName(fname); - s = PrepareIOFromReadOptions(ro, ioptions_.env, fopts.io_options); + s = PrepareIOFromReadOptions(ro, clock, fopts.io_options); if (s.ok()) { s = ioptions_.fs->NewRandomAccessFile(fname, file_options, &file, nullptr); @@ -125,10 +126,10 @@ Status TableCache::GetTableReader( if (!sequential_mode && ioptions_.advise_random_on_open) { file->Hint(FSRandomAccessFile::kRandom); } - StopWatch sw(ioptions_.env, ioptions_.statistics, TABLE_OPEN_IO_MICROS); + StopWatch sw(clock, ioptions_.statistics, TABLE_OPEN_IO_MICROS); std::unique_ptr file_reader( new RandomAccessFileReader( - std::move(file), fname, ioptions_.env, io_tracer_, + std::move(file), fname, clock, io_tracer_, record_read_stats ? ioptions_.statistics : nullptr, SST_READ_MICROS, file_read_hist, ioptions_.rate_limiter, ioptions_.listeners)); s = ioptions_.table_factory->NewTableReader( @@ -161,7 +162,8 @@ Status TableCache::FindTable(const ReadOptions& ro, HistogramImpl* file_read_hist, bool skip_filters, int level, bool prefetch_index_and_filter_in_cache, size_t max_file_size_for_l0_meta_pin) { - PERF_TIMER_GUARD_WITH_ENV(find_table_nanos, ioptions_.env); + PERF_TIMER_GUARD_WITH_CLOCK(find_table_nanos, + ioptions_.env->GetSystemClock()); uint64_t number = fd.GetNumber(); Slice key = GetSliceForFileNumber(&number); *handle = cache_->Lookup(key); diff --git a/db/version_set.cc b/db/version_set.cc index 836acf0c4..3c03117f8 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -1761,6 +1761,7 @@ Version::Version(ColumnFamilyData* column_family_data, VersionSet* vset, const std::shared_ptr& io_tracer, uint64_t version_number) : env_(vset->env_), + clock_(env_->GetSystemClock()), cfd_(column_family_data), info_log_((cfd_ == nullptr) ? nullptr : cfd_->ioptions()->info_log), db_statistics_((cfd_ == nullptr) ? nullptr @@ -1879,7 +1880,7 @@ void Version::Get(const ReadOptions& read_options, const LookupKey& k, user_comparator(), merge_operator_, info_log_, db_statistics_, status->ok() ? GetContext::kNotFound : GetContext::kMerge, user_key, do_merge ? value : nullptr, do_merge ? timestamp : nullptr, value_found, - merge_context, do_merge, max_covering_tombstone_seq, this->env_, seq, + merge_context, do_merge, max_covering_tombstone_seq, clock_, seq, merge_operator_ ? &pinned_iters_mgr : nullptr, callback, is_blob_to_use, tracing_get_id); @@ -1907,7 +1908,7 @@ void Version::Get(const ReadOptions& read_options, const LookupKey& k, bool timer_enabled = GetPerfLevel() >= PerfLevel::kEnableTimeExceptForMutex && get_perf_context()->per_level_perf_context_enabled; - StopWatchNano timer(env_, timer_enabled /* auto_start */); + StopWatchNano timer(clock_, timer_enabled /* auto_start */); *status = table_cache_->Get( read_options, *internal_comparator(), *f->file_metadata, ikey, &get_context, mutable_cf_options_.prefix_extractor.get(), @@ -1996,7 +1997,7 @@ void Version::Get(const ReadOptions& read_options, const LookupKey& k, std::string* str_value = value != nullptr ? value->GetSelf() : nullptr; *status = MergeHelper::TimedFullMerge( merge_operator_, user_key, nullptr, merge_context->GetOperands(), - str_value, info_log_, db_statistics_, env_, + str_value, info_log_, db_statistics_, clock_, nullptr /* result_operand */, true); if (LIKELY(value != nullptr)) { value->PinSelf(); @@ -2033,9 +2034,9 @@ void Version::MultiGet(const ReadOptions& read_options, MultiGetRange* range, user_comparator(), merge_operator_, info_log_, db_statistics_, iter->s->ok() ? GetContext::kNotFound : GetContext::kMerge, iter->ukey_with_ts, iter->value, iter->timestamp, nullptr, - &(iter->merge_context), true, &iter->max_covering_tombstone_seq, - this->env_, nullptr, merge_operator_ ? &pinned_iters_mgr : nullptr, - callback, &iter->is_blob_index, tracing_mget_id); + &(iter->merge_context), true, &iter->max_covering_tombstone_seq, clock_, + nullptr, merge_operator_ ? &pinned_iters_mgr : nullptr, callback, + &iter->is_blob_index, tracing_mget_id); // MergeInProgress status, if set, has been transferred to the get_context // state, so we set status to ok here. From now on, the iter status will // be used for IO errors, and get_context state will be used for any @@ -2065,7 +2066,7 @@ void Version::MultiGet(const ReadOptions& read_options, MultiGetRange* range, bool timer_enabled = GetPerfLevel() >= PerfLevel::kEnableTimeExceptForMutex && get_perf_context()->per_level_perf_context_enabled; - StopWatchNano timer(env_, timer_enabled /* auto_start */); + StopWatchNano timer(clock_, timer_enabled /* auto_start */); s = table_cache_->MultiGet( read_options, *internal_comparator(), *f->file_metadata, &file_range, mutable_cf_options_.prefix_extractor.get(), @@ -2228,7 +2229,7 @@ void Version::MultiGet(const ReadOptions& read_options, MultiGetRange* range, iter->value != nullptr ? iter->value->GetSelf() : nullptr; *status = MergeHelper::TimedFullMerge( merge_operator_, user_key, nullptr, iter->merge_context.GetOperands(), - str_value, info_log_, db_statistics_, env_, + str_value, info_log_, db_statistics_, clock_, nullptr /* result_operand */, true); if (LIKELY(iter->value != nullptr)) { iter->value->PinSelf(); @@ -3782,6 +3783,7 @@ VersionSet::VersionSet(const std::string& dbname, table_cache_(table_cache), env_(_db_options->env), fs_(_db_options->fs, io_tracer), + clock_(env_->GetSystemClock()), dbname_(dbname), db_options_(_db_options), next_file_number_(2), @@ -4119,7 +4121,7 @@ Status VersionSet::ProcessManifestWrites( db_options_->manifest_preallocation_size); std::unique_ptr file_writer(new WritableFileWriter( - std::move(descriptor_file), descriptor_fname, opt_file_opts, env_, + std::move(descriptor_file), descriptor_fname, opt_file_opts, clock_, io_tracer_, nullptr, db_options_->listeners)); descriptor_log_.reset( new log::Writer(std::move(file_writer), 0, false)); @@ -4167,7 +4169,7 @@ Status VersionSet::ProcessManifestWrites( } } if (s.ok()) { - io_s = SyncManifest(env_, db_options_, descriptor_log_->file()); + io_s = SyncManifest(clock_, db_options_, descriptor_log_->file()); TEST_SYNC_POINT_CALLBACK( "VersionSet::ProcessManifestWrites:AfterSyncManifest", &io_s); } @@ -6302,7 +6304,7 @@ Status ReactiveVersionSet::MaybeSwitchManifest( "ReactiveVersionSet::MaybeSwitchManifest:" "AfterGetCurrentManifestPath:1"); s = fs_->NewSequentialFile(manifest_path, - env_->OptimizeForManifestRead(file_options_), + fs_->OptimizeForManifestRead(file_options_), &manifest_file, nullptr); } else { // No need to switch manifest. diff --git a/db/version_set.h b/db/version_set.h index 7cada5f46..98c15af5c 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -71,6 +71,7 @@ class WriteBufferManager; class MergeContext; class ColumnFamilySet; class MergeIteratorBuilder; +class SystemClock; // VersionEdit is always supposed to be valid and it is used to point at // entries in Manifest. Ideally it should not be used as a container to @@ -779,6 +780,8 @@ class Version { private: Env* env_; + std::shared_ptr clock_; + friend class ReactiveVersionSet; friend class VersionSet; friend class VersionEditHandler; @@ -1346,6 +1349,7 @@ class VersionSet { Cache* table_cache_; Env* const env_; FileSystemPtr const fs_; + const std::shared_ptr clock_; const std::string dbname_; std::string db_id_; const ImmutableDBOptions* const db_options_; diff --git a/db/version_set_test.cc b/db/version_set_test.cc index bd94f223a..ecc47b207 100644 --- a/db/version_set_test.cc +++ b/db/version_set_test.cc @@ -2786,8 +2786,8 @@ class VersionSetTestMissingFiles : public VersionSetTestBase, std::unique_ptr file; Status s = fs_->NewWritableFile(fname, FileOptions(), &file, nullptr); ASSERT_OK(s); - std::unique_ptr fwriter( - new WritableFileWriter(std::move(file), fname, FileOptions(), env_)); + std::unique_ptr fwriter(new WritableFileWriter( + std::move(file), fname, FileOptions(), env_->GetSystemClock())); std::vector> int_tbl_prop_collector_factories; diff --git a/db/write_batch.cc b/db/write_batch.cc index 8600b24b3..2004a371e 100644 --- a/db/write_batch.cc +++ b/db/write_batch.cc @@ -55,6 +55,7 @@ #include "monitoring/statistics.h" #include "port/lang.h" #include "rocksdb/merge_operator.h" +#include "rocksdb/system_clock.h" #include "util/autovector.h" #include "util/cast_util.h" #include "util/coding.h" @@ -1804,7 +1805,7 @@ class MemTableInserter : public WriteBatch::Handler { Status merge_status = MergeHelper::TimedFullMerge( merge_operator, key, &get_value_slice, {value}, &new_value, - moptions->info_log, moptions->statistics, Env::Default()); + moptions->info_log, moptions->statistics, SystemClock::Default()); if (!merge_status.ok()) { // Failed to merge! diff --git a/db/write_controller.cc b/db/write_controller.cc index 5480aabd1..a10e98973 100644 --- a/db/write_controller.cc +++ b/db/write_controller.cc @@ -8,7 +8,8 @@ #include #include #include -#include "rocksdb/env.h" + +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { @@ -42,7 +43,8 @@ bool WriteController::IsStopped() const { // If it turns out to be a performance issue, we can redesign the thread // synchronization model here. // The function trust caller will sleep micros returned. -uint64_t WriteController::GetDelay(Env* env, uint64_t num_bytes) { +uint64_t WriteController::GetDelay(const std::shared_ptr& clock, + uint64_t num_bytes) { if (total_stopped_.load(std::memory_order_relaxed) > 0) { return 0; } @@ -59,7 +61,7 @@ uint64_t WriteController::GetDelay(Env* env, uint64_t num_bytes) { } // The frequency to get time inside DB mutex is less than one per refill // interval. - auto time_now = NowMicrosMonotonic(env); + auto time_now = NowMicrosMonotonic(clock); uint64_t sleep_debt = 0; uint64_t time_since_last_refill = 0; @@ -106,8 +108,9 @@ uint64_t WriteController::GetDelay(Env* env, uint64_t num_bytes) { return sleep_amount; } -uint64_t WriteController::NowMicrosMonotonic(Env* env) { - return env->NowNanos() / std::milli::den; +uint64_t WriteController::NowMicrosMonotonic( + const std::shared_ptr& clock) { + return clock->NowNanos() / std::milli::den; } StopWriteToken::~StopWriteToken() { diff --git a/db/write_controller.h b/db/write_controller.h index 785ae6896..9a661d13e 100644 --- a/db/write_controller.h +++ b/db/write_controller.h @@ -13,7 +13,7 @@ namespace ROCKSDB_NAMESPACE { -class Env; +class SystemClock; class WriteControllerToken; // WriteController is controlling write stalls in our write code-path. Write @@ -57,7 +57,8 @@ class WriteController { // return how many microseconds the caller needs to sleep after the call // num_bytes: how many number of bytes to put into the DB. // Prerequisite: DB mutex held. - uint64_t GetDelay(Env* env, uint64_t num_bytes); + uint64_t GetDelay(const std::shared_ptr& clock, + uint64_t num_bytes); void set_delayed_write_rate(uint64_t write_rate) { // avoid divide 0 if (write_rate == 0) { @@ -85,7 +86,7 @@ class WriteController { RateLimiter* low_pri_rate_limiter() { return low_pri_rate_limiter_.get(); } private: - uint64_t NowMicrosMonotonic(Env* env); + uint64_t NowMicrosMonotonic(const std::shared_ptr& clock); friend class WriteControllerToken; friend class StopWriteToken; diff --git a/db/write_controller_test.cc b/db/write_controller_test.cc index 72d116798..af984befd 100644 --- a/db/write_controller_test.cc +++ b/db/write_controller_test.cc @@ -3,46 +3,50 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). // -#include - #include "db/write_controller.h" -#include "rocksdb/env.h" +#include + +#include "rocksdb/system_clock.h" #include "test_util/testharness.h" namespace ROCKSDB_NAMESPACE { - -class WriteControllerTest : public testing::Test {}; - -class TimeSetEnv : public EnvWrapper { +namespace { +class TimeSetClock : public SystemClockWrapper { public: - explicit TimeSetEnv() : EnvWrapper(nullptr) {} + explicit TimeSetClock() : SystemClockWrapper(nullptr) {} + const char* Name() const override { return "TimeSetClock"; } uint64_t now_micros_ = 6666; uint64_t NowNanos() override { return now_micros_ * std::milli::den; } }; +} // namespace +class WriteControllerTest : public testing::Test { + public: + WriteControllerTest() { clock_ = std::make_shared(); } + std::shared_ptr clock_; +}; TEST_F(WriteControllerTest, ChangeDelayRateTest) { - TimeSetEnv env; WriteController controller(40000000u); // also set max delayed rate controller.set_delayed_write_rate(10000000u); auto delay_token_0 = controller.GetDelayToken(controller.delayed_write_rate()); ASSERT_EQ(static_cast(2000000), - controller.GetDelay(&env, 20000000u)); + controller.GetDelay(clock_, 20000000u)); auto delay_token_1 = controller.GetDelayToken(2000000u); ASSERT_EQ(static_cast(10000000), - controller.GetDelay(&env, 20000000u)); + controller.GetDelay(clock_, 20000000u)); auto delay_token_2 = controller.GetDelayToken(1000000u); ASSERT_EQ(static_cast(20000000), - controller.GetDelay(&env, 20000000u)); + controller.GetDelay(clock_, 20000000u)); auto delay_token_3 = controller.GetDelayToken(20000000u); ASSERT_EQ(static_cast(1000000), - controller.GetDelay(&env, 20000000u)); + controller.GetDelay(clock_, 20000000u)); // This is more than max rate. Max delayed rate will be used. auto delay_token_4 = controller.GetDelayToken(controller.delayed_write_rate() * 3); ASSERT_EQ(static_cast(500000), - controller.GetDelay(&env, 20000000u)); + controller.GetDelay(clock_, 20000000u)); } TEST_F(WriteControllerTest, SanityTest) { @@ -56,73 +60,71 @@ TEST_F(WriteControllerTest, SanityTest) { stop_token_2.reset(); ASSERT_FALSE(controller.IsStopped()); - TimeSetEnv env; - auto delay_token_1 = controller.GetDelayToken(10000000u); ASSERT_EQ(static_cast(2000000), - controller.GetDelay(&env, 20000000u)); + controller.GetDelay(clock_, 20000000u)); - env.now_micros_ += 1999900u; // sleep debt 1000 + clock_->now_micros_ += 1999900u; // sleep debt 1000 auto delay_token_2 = controller.GetDelayToken(10000000u); // Rate reset after changing the token. ASSERT_EQ(static_cast(2000000), - controller.GetDelay(&env, 20000000u)); + controller.GetDelay(clock_, 20000000u)); - env.now_micros_ += 1999900u; // sleep debt 1000 + clock_->now_micros_ += 1999900u; // sleep debt 1000 // One refill: 10240 bytes allowed, 1000 used, 9240 left - ASSERT_EQ(static_cast(1124), controller.GetDelay(&env, 1000u)); - env.now_micros_ += 1124u; // sleep debt 0 + ASSERT_EQ(static_cast(1124), controller.GetDelay(clock_, 1000u)); + clock_->now_micros_ += 1124u; // sleep debt 0 delay_token_2.reset(); // 1000 used, 8240 left - ASSERT_EQ(static_cast(0), controller.GetDelay(&env, 1000u)); + ASSERT_EQ(static_cast(0), controller.GetDelay(clock_, 1000u)); - env.now_micros_ += 100u; // sleep credit 100 + clock_->now_micros_ += 100u; // sleep credit 100 // 1000 used, 7240 left - ASSERT_EQ(static_cast(0), controller.GetDelay(&env, 1000u)); + ASSERT_EQ(static_cast(0), controller.GetDelay(clock_, 1000u)); - env.now_micros_ += 100u; // sleep credit 200 + clock_->now_micros_ += 100u; // sleep credit 200 // One refill: 10240 fileed, sleep credit generates 2000. 8000 used // 7240 + 10240 + 2000 - 8000 = 11480 left - ASSERT_EQ(static_cast(1024u), controller.GetDelay(&env, 8000u)); + ASSERT_EQ(static_cast(1024u), controller.GetDelay(clock_, 8000u)); - env.now_micros_ += 200u; // sleep debt 824 + clock_->now_micros_ += 200u; // sleep debt 824 // 1000 used, 10480 left. - ASSERT_EQ(static_cast(0), controller.GetDelay(&env, 1000u)); + ASSERT_EQ(static_cast(0), controller.GetDelay(clock_, 1000u)); - env.now_micros_ += 200u; // sleep debt 624 + clock_->now_micros_ += 200u; // sleep debt 624 // Out of bound sleep, still 10480 left ASSERT_EQ(static_cast(3000624u), - controller.GetDelay(&env, 30000000u)); + controller.GetDelay(clock_, 30000000u)); - env.now_micros_ += 3000724u; // sleep credit 100 + clock_->now_micros_ += 3000724u; // sleep credit 100 // 6000 used, 4480 left. - ASSERT_EQ(static_cast(0), controller.GetDelay(&env, 6000u)); + ASSERT_EQ(static_cast(0), controller.GetDelay(clock_, 6000u)); - env.now_micros_ += 200u; // sleep credit 300 + clock_->now_micros_ += 200u; // sleep credit 300 // One refill, credit 4480 balance + 3000 credit + 10240 refill // Use 8000, 9720 left - ASSERT_EQ(static_cast(1024u), controller.GetDelay(&env, 8000u)); + ASSERT_EQ(static_cast(1024u), controller.GetDelay(clock_, 8000u)); - env.now_micros_ += 3024u; // sleep credit 2000 + clock_->now_micros_ += 3024u; // sleep credit 2000 // 1720 left - ASSERT_EQ(static_cast(0u), controller.GetDelay(&env, 8000u)); + ASSERT_EQ(static_cast(0u), controller.GetDelay(clock_, 8000u)); // 1720 balance + 20000 credit = 20170 left // Use 8000, 12170 left - ASSERT_EQ(static_cast(0u), controller.GetDelay(&env, 8000u)); + ASSERT_EQ(static_cast(0u), controller.GetDelay(clock_, 8000u)); // 4170 left - ASSERT_EQ(static_cast(0u), controller.GetDelay(&env, 8000u)); + ASSERT_EQ(static_cast(0u), controller.GetDelay(clock_, 8000u)); // Need a refill - ASSERT_EQ(static_cast(1024u), controller.GetDelay(&env, 9000u)); + ASSERT_EQ(static_cast(1024u), controller.GetDelay(clock_, 9000u)); delay_token_1.reset(); - ASSERT_EQ(static_cast(0), controller.GetDelay(&env, 30000000u)); + ASSERT_EQ(static_cast(0), controller.GetDelay(clock_, 30000000u)); delay_token_1.reset(); ASSERT_FALSE(controller.IsStopped()); } diff --git a/env/composite_env.cc b/env/composite_env.cc new file mode 100644 index 000000000..16f6c7e90 --- /dev/null +++ b/env/composite_env.cc @@ -0,0 +1,369 @@ +// Copyright (c) 2019-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). +// +#include "env/composite_env_wrapper.h" + +namespace ROCKSDB_NAMESPACE { +namespace { +// The CompositeEnvWrapper class provides an interface that is compatible +// with the old monolithic Env API, and an implementation that wraps around +// the new Env that provides threading and other OS related functionality, and +// the new FileSystem API that provides storage functionality. By +// providing the old Env interface, it allows the rest of RocksDB code to +// be agnostic of whether the underlying Env implementation is a monolithic +// Env or an Env + FileSystem. In the former case, the user will specify +// Options::env only, whereas in the latter case, the user will specify +// Options::env and Options::file_system. + +class CompositeSequentialFileWrapper : public SequentialFile { + public: + explicit CompositeSequentialFileWrapper( + std::unique_ptr& target) + : target_(std::move(target)) {} + + Status Read(size_t n, Slice* result, char* scratch) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Read(n, io_opts, result, scratch, &dbg); + } + Status Skip(uint64_t n) override { return target_->Skip(n); } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + Status InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + Status PositionedRead(uint64_t offset, size_t n, Slice* result, + char* scratch) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->PositionedRead(offset, n, io_opts, result, scratch, &dbg); + } + + private: + std::unique_ptr target_; +}; + +class CompositeRandomAccessFileWrapper : public RandomAccessFile { + public: + explicit CompositeRandomAccessFileWrapper( + std::unique_ptr& target) + : target_(std::move(target)) {} + + Status Read(uint64_t offset, size_t n, Slice* result, + char* scratch) const override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Read(offset, n, io_opts, result, scratch, &dbg); + } + Status MultiRead(ReadRequest* reqs, size_t num_reqs) override { + IOOptions io_opts; + IODebugContext dbg; + std::vector fs_reqs; + Status status; + + fs_reqs.resize(num_reqs); + for (size_t i = 0; i < num_reqs; ++i) { + fs_reqs[i].offset = reqs[i].offset; + fs_reqs[i].len = reqs[i].len; + fs_reqs[i].scratch = reqs[i].scratch; + fs_reqs[i].status = IOStatus::OK(); + } + status = target_->MultiRead(fs_reqs.data(), num_reqs, io_opts, &dbg); + for (size_t i = 0; i < num_reqs; ++i) { + reqs[i].result = fs_reqs[i].result; + reqs[i].status = fs_reqs[i].status; + } + return status; + } + Status Prefetch(uint64_t offset, size_t n) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Prefetch(offset, n, io_opts, &dbg); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + void Hint(AccessPattern pattern) override { + target_->Hint((FSRandomAccessFile::AccessPattern)pattern); + } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + Status InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + + private: + std::unique_ptr target_; +}; + +class CompositeWritableFileWrapper : public WritableFile { + public: + explicit CompositeWritableFileWrapper(std::unique_ptr& t) + : target_(std::move(t)) {} + + Status Append(const Slice& data) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Append(data, io_opts, &dbg); + } + Status PositionedAppend(const Slice& data, uint64_t offset) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->PositionedAppend(data, offset, io_opts, &dbg); + } + Status Truncate(uint64_t size) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Truncate(size, io_opts, &dbg); + } + Status Close() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Close(io_opts, &dbg); + } + Status Flush() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Flush(io_opts, &dbg); + } + Status Sync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Sync(io_opts, &dbg); + } + Status Fsync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Fsync(io_opts, &dbg); + } + bool IsSyncThreadSafe() const override { return target_->IsSyncThreadSafe(); } + + bool use_direct_io() const override { return target_->use_direct_io(); } + + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + + void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) override { + target_->SetWriteLifeTimeHint(hint); + } + + Env::WriteLifeTimeHint GetWriteLifeTimeHint() override { + return target_->GetWriteLifeTimeHint(); + } + + uint64_t GetFileSize() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->GetFileSize(io_opts, &dbg); + } + + void SetPreallocationBlockSize(size_t size) override { + target_->SetPreallocationBlockSize(size); + } + + void GetPreallocationStatus(size_t* block_size, + size_t* last_allocated_block) override { + target_->GetPreallocationStatus(block_size, last_allocated_block); + } + + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + Status InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + + Status RangeSync(uint64_t offset, uint64_t nbytes) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->RangeSync(offset, nbytes, io_opts, &dbg); + } + + void PrepareWrite(size_t offset, size_t len) override { + IOOptions io_opts; + IODebugContext dbg; + target_->PrepareWrite(offset, len, io_opts, &dbg); + } + + Status Allocate(uint64_t offset, uint64_t len) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Allocate(offset, len, io_opts, &dbg); + } + + std::unique_ptr* target() { return &target_; } + + private: + std::unique_ptr target_; +}; + +class CompositeRandomRWFileWrapper : public RandomRWFile { + public: + explicit CompositeRandomRWFileWrapper(std::unique_ptr& target) + : target_(std::move(target)) {} + + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + Status Write(uint64_t offset, const Slice& data) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Write(offset, data, io_opts, &dbg); + } + Status Read(uint64_t offset, size_t n, Slice* result, + char* scratch) const override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Read(offset, n, io_opts, result, scratch, &dbg); + } + Status Flush() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Flush(io_opts, &dbg); + } + Status Sync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Sync(io_opts, &dbg); + } + Status Fsync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Fsync(io_opts, &dbg); + } + Status Close() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Close(io_opts, &dbg); + } + + private: + std::unique_ptr target_; +}; + +class CompositeDirectoryWrapper : public Directory { + public: + explicit CompositeDirectoryWrapper(std::unique_ptr& target) + : target_(std::move(target)) {} + + Status Fsync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Fsync(io_opts, &dbg); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + private: + std::unique_ptr target_; +}; +} // namespace + +Status CompositeEnv::NewSequentialFile(const std::string& f, + std::unique_ptr* r, + const EnvOptions& options) { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = + file_system_->NewSequentialFile(f, FileOptions(options), &file, &dbg); + if (status.ok()) { + r->reset(new CompositeSequentialFileWrapper(file)); + } + return status; +} + +Status CompositeEnv::NewRandomAccessFile(const std::string& f, + std::unique_ptr* r, + const EnvOptions& options) { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = + file_system_->NewRandomAccessFile(f, FileOptions(options), &file, &dbg); + if (status.ok()) { + r->reset(new CompositeRandomAccessFileWrapper(file)); + } + return status; +} + +Status CompositeEnv::NewWritableFile(const std::string& f, + std::unique_ptr* r, + const EnvOptions& options) { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = file_system_->NewWritableFile(f, FileOptions(options), &file, &dbg); + if (status.ok()) { + r->reset(new CompositeWritableFileWrapper(file)); + } + return status; +} + +Status CompositeEnv::ReopenWritableFile(const std::string& fname, + std::unique_ptr* result, + const EnvOptions& options) { + IODebugContext dbg; + Status status; + std::unique_ptr file; + status = file_system_->ReopenWritableFile(fname, FileOptions(options), &file, + &dbg); + if (status.ok()) { + result->reset(new CompositeWritableFileWrapper(file)); + } + return status; +} + +Status CompositeEnv::ReuseWritableFile(const std::string& fname, + const std::string& old_fname, + std::unique_ptr* r, + const EnvOptions& options) { + IODebugContext dbg; + Status status; + std::unique_ptr file; + status = file_system_->ReuseWritableFile(fname, old_fname, + FileOptions(options), &file, &dbg); + if (status.ok()) { + r->reset(new CompositeWritableFileWrapper(file)); + } + return status; +} + +Status CompositeEnv::NewRandomRWFile(const std::string& fname, + std::unique_ptr* result, + const EnvOptions& options) { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = + file_system_->NewRandomRWFile(fname, FileOptions(options), &file, &dbg); + if (status.ok()) { + result->reset(new CompositeRandomRWFileWrapper(file)); + } + return status; +} + +Status CompositeEnv::NewDirectory(const std::string& name, + std::unique_ptr* result) { + IOOptions io_opts; + IODebugContext dbg; + std::unique_ptr dir; + Status status; + status = file_system_->NewDirectory(name, io_opts, &dir, &dbg); + if (status.ok()) { + result->reset(new CompositeDirectoryWrapper(dir)); + } + return status; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/env/composite_env_wrapper.h b/env/composite_env_wrapper.h index f634eee52..5eafa43b6 100644 --- a/env/composite_env_wrapper.h +++ b/env/composite_env_wrapper.h @@ -7,6 +7,7 @@ #include "rocksdb/env.h" #include "rocksdb/file_system.h" +#include "rocksdb/system_clock.h" #ifdef _WIN32 // Windows API macro interference @@ -17,272 +18,14 @@ namespace ROCKSDB_NAMESPACE { -// The CompositeEnvWrapper class provides an interface that is compatible -// with the old monolithic Env API, and an implementation that wraps around -// the new Env that provides threading and other OS related functionality, and -// the new FileSystem API that provides storage functionality. By -// providing the old Env interface, it allows the rest of RocksDB code to -// be agnostic of whether the underlying Env implementation is a monolithic -// Env or an Env + FileSystem. In the former case, the user will specify -// Options::env only, whereas in the latter case, the user will specify -// Options::env and Options::file_system. - -class CompositeSequentialFileWrapper : public SequentialFile { - public: - explicit CompositeSequentialFileWrapper( - std::unique_ptr& target) - : target_(std::move(target)) {} - - Status Read(size_t n, Slice* result, char* scratch) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Read(n, io_opts, result, scratch, &dbg); - } - Status Skip(uint64_t n) override { return target_->Skip(n); } - bool use_direct_io() const override { return target_->use_direct_io(); } - size_t GetRequiredBufferAlignment() const override { - return target_->GetRequiredBufferAlignment(); - } - Status InvalidateCache(size_t offset, size_t length) override { - return target_->InvalidateCache(offset, length); - } - Status PositionedRead(uint64_t offset, size_t n, Slice* result, - char* scratch) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->PositionedRead(offset, n, io_opts, result, scratch, &dbg); - } - - private: - std::unique_ptr target_; -}; - -class CompositeRandomAccessFileWrapper : public RandomAccessFile { - public: - explicit CompositeRandomAccessFileWrapper( - std::unique_ptr& target) - : target_(std::move(target)) {} - - Status Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Read(offset, n, io_opts, result, scratch, &dbg); - } - Status MultiRead(ReadRequest* reqs, size_t num_reqs) override { - IOOptions io_opts; - IODebugContext dbg; - std::vector fs_reqs; - Status status; - - fs_reqs.resize(num_reqs); - for (size_t i = 0; i < num_reqs; ++i) { - fs_reqs[i].offset = reqs[i].offset; - fs_reqs[i].len = reqs[i].len; - fs_reqs[i].scratch = reqs[i].scratch; - fs_reqs[i].status = IOStatus::OK(); - } - status = target_->MultiRead(fs_reqs.data(), num_reqs, io_opts, &dbg); - for (size_t i = 0; i < num_reqs; ++i) { - reqs[i].result = fs_reqs[i].result; - reqs[i].status = fs_reqs[i].status; - } - return status; - } - Status Prefetch(uint64_t offset, size_t n) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Prefetch(offset, n, io_opts, &dbg); - } - size_t GetUniqueId(char* id, size_t max_size) const override { - return target_->GetUniqueId(id, max_size); - }; - void Hint(AccessPattern pattern) override { - target_->Hint((FSRandomAccessFile::AccessPattern)pattern); - } - bool use_direct_io() const override { return target_->use_direct_io(); } - size_t GetRequiredBufferAlignment() const override { - return target_->GetRequiredBufferAlignment(); - } - Status InvalidateCache(size_t offset, size_t length) override { - return target_->InvalidateCache(offset, length); - } - - private: - std::unique_ptr target_; -}; - -class CompositeWritableFileWrapper : public WritableFile { - public: - explicit CompositeWritableFileWrapper(std::unique_ptr& t) - : target_(std::move(t)) {} - - Status Append(const Slice& data) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Append(data, io_opts, &dbg); - } - Status PositionedAppend(const Slice& data, uint64_t offset) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->PositionedAppend(data, offset, io_opts, &dbg); - } - Status Truncate(uint64_t size) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Truncate(size, io_opts, &dbg); - } - Status Close() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Close(io_opts, &dbg); - } - Status Flush() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Flush(io_opts, &dbg); - } - Status Sync() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Sync(io_opts, &dbg); - } - Status Fsync() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Fsync(io_opts, &dbg); - } - bool IsSyncThreadSafe() const override { return target_->IsSyncThreadSafe(); } - - bool use_direct_io() const override { return target_->use_direct_io(); } - - size_t GetRequiredBufferAlignment() const override { - return target_->GetRequiredBufferAlignment(); - } - - void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) override { - target_->SetWriteLifeTimeHint(hint); - } - - Env::WriteLifeTimeHint GetWriteLifeTimeHint() override { - return target_->GetWriteLifeTimeHint(); - } - - uint64_t GetFileSize() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->GetFileSize(io_opts, &dbg); - } - - void SetPreallocationBlockSize(size_t size) override { - target_->SetPreallocationBlockSize(size); - } - - void GetPreallocationStatus(size_t* block_size, - size_t* last_allocated_block) override { - target_->GetPreallocationStatus(block_size, last_allocated_block); - } - - size_t GetUniqueId(char* id, size_t max_size) const override { - return target_->GetUniqueId(id, max_size); - } - - Status InvalidateCache(size_t offset, size_t length) override { - return target_->InvalidateCache(offset, length); - } - - Status RangeSync(uint64_t offset, uint64_t nbytes) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->RangeSync(offset, nbytes, io_opts, &dbg); - } - - void PrepareWrite(size_t offset, size_t len) override { - IOOptions io_opts; - IODebugContext dbg; - target_->PrepareWrite(offset, len, io_opts, &dbg); - } - - Status Allocate(uint64_t offset, uint64_t len) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Allocate(offset, len, io_opts, &dbg); - } - - std::unique_ptr* target() { return &target_; } - - private: - std::unique_ptr target_; -}; - -class CompositeRandomRWFileWrapper : public RandomRWFile { - public: - explicit CompositeRandomRWFileWrapper(std::unique_ptr& target) - : target_(std::move(target)) {} - - bool use_direct_io() const override { return target_->use_direct_io(); } - size_t GetRequiredBufferAlignment() const override { - return target_->GetRequiredBufferAlignment(); - } - Status Write(uint64_t offset, const Slice& data) override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Write(offset, data, io_opts, &dbg); - } - Status Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Read(offset, n, io_opts, result, scratch, &dbg); - } - Status Flush() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Flush(io_opts, &dbg); - } - Status Sync() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Sync(io_opts, &dbg); - } - Status Fsync() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Fsync(io_opts, &dbg); - } - Status Close() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Close(io_opts, &dbg); - } - - private: - std::unique_ptr target_; -}; - -class CompositeDirectoryWrapper : public Directory { - public: - explicit CompositeDirectoryWrapper(std::unique_ptr& target) - : target_(std::move(target)) {} - - Status Fsync() override { - IOOptions io_opts; - IODebugContext dbg; - return target_->Fsync(io_opts, &dbg); - } - size_t GetUniqueId(char* id, size_t max_size) const override { - return target_->GetUniqueId(id, max_size); - } - - private: - std::unique_ptr target_; -}; class CompositeEnv : public Env { public: // Initialize a CompositeEnvWrapper that delegates all thread/time related // calls to env, and all file operations to fs - explicit CompositeEnv(const std::shared_ptr& fs) : Env(fs) {} + explicit CompositeEnv(const std::shared_ptr& fs, + const std::shared_ptr& clock) + : Env(fs, clock) {} Status RegisterDbPaths(const std::vector& paths) override { return file_system_->RegisterDbPaths(paths); @@ -294,99 +37,37 @@ class CompositeEnv : public Env { // The following text is boilerplate that forwards all methods to target() Status NewSequentialFile(const std::string& f, std::unique_ptr* r, - const EnvOptions& options) override { - IODebugContext dbg; - std::unique_ptr file; - Status status; - status = - file_system_->NewSequentialFile(f, FileOptions(options), &file, &dbg); - if (status.ok()) { - r->reset(new CompositeSequentialFileWrapper(file)); - } - return status; - } + const EnvOptions& options) override; + Status NewRandomAccessFile(const std::string& f, std::unique_ptr* r, - const EnvOptions& options) override { - IODebugContext dbg; - std::unique_ptr file; - Status status; - status = - file_system_->NewRandomAccessFile(f, FileOptions(options), &file, &dbg); - if (status.ok()) { - r->reset(new CompositeRandomAccessFileWrapper(file)); - } - return status; - } + const EnvOptions& options) override; + Status NewWritableFile(const std::string& f, std::unique_ptr* r, - const EnvOptions& options) override { - IODebugContext dbg; - std::unique_ptr file; - Status status; - status = - file_system_->NewWritableFile(f, FileOptions(options), &file, &dbg); - if (status.ok()) { - r->reset(new CompositeWritableFileWrapper(file)); - } - return status; - } + const EnvOptions& options) override; + Status ReopenWritableFile(const std::string& fname, std::unique_ptr* result, - const EnvOptions& options) override { - IODebugContext dbg; - Status status; - std::unique_ptr file; - status = file_system_->ReopenWritableFile(fname, FileOptions(options), - &file, &dbg); - if (status.ok()) { - result->reset(new CompositeWritableFileWrapper(file)); - } - return status; - } + const EnvOptions& options) override; + Status ReuseWritableFile(const std::string& fname, const std::string& old_fname, std::unique_ptr* r, - const EnvOptions& options) override { - IODebugContext dbg; - Status status; - std::unique_ptr file; - status = file_system_->ReuseWritableFile(fname, old_fname, - FileOptions(options), &file, &dbg); - if (status.ok()) { - r->reset(new CompositeWritableFileWrapper(file)); - } - return status; - } + const EnvOptions& options) override; + Status NewRandomRWFile(const std::string& fname, std::unique_ptr* result, - const EnvOptions& options) override { - IODebugContext dbg; - std::unique_ptr file; - Status status; - status = - file_system_->NewRandomRWFile(fname, FileOptions(options), &file, &dbg); - if (status.ok()) { - result->reset(new CompositeRandomRWFileWrapper(file)); - } - return status; - } + const EnvOptions& options) override; + Status NewMemoryMappedFileBuffer( const std::string& fname, std::unique_ptr* result) override { return file_system_->NewMemoryMappedFileBuffer(fname, result); } + Status NewDirectory(const std::string& name, - std::unique_ptr* result) override { - IOOptions io_opts; - IODebugContext dbg; - std::unique_ptr dir; - Status status; - status = file_system_->NewDirectory(name, io_opts, &dir, &dbg); - if (status.ok()) { - result->reset(new CompositeDirectoryWrapper(dir)); - } - return status; - } + std::unique_ptr* result) override; + Status FileExists(const std::string& f) override { IOOptions io_opts; IODebugContext dbg; @@ -548,6 +229,21 @@ class CompositeEnv : public Env { IODebugContext dbg; return file_system_->GetFreeSpace(path, io_opts, diskfree, &dbg); } + uint64_t NowMicros() override { return system_clock_->NowMicros(); } + uint64_t NowNanos() override { return system_clock_->NowNanos(); } + + uint64_t NowCPUNanos() override { return system_clock_->CPUNanos(); } + + void SleepForMicroseconds(int micros) override { + system_clock_->SleepForMicroseconds(micros); + } + + Status GetCurrentTime(int64_t* unix_time) override { + return system_clock_->GetCurrentTime(unix_time); + } + std::string TimeToString(uint64_t time) override { + return system_clock_->TimeToString(time); + } }; class CompositeEnvWrapper : public CompositeEnv { @@ -555,7 +251,14 @@ class CompositeEnvWrapper : public CompositeEnv { // Initialize a CompositeEnvWrapper that delegates all thread/time related // calls to env, and all file operations to fs explicit CompositeEnvWrapper(Env* env, const std::shared_ptr& fs) - : CompositeEnv(fs), env_target_(env) {} + : CompositeEnvWrapper(env, fs, env->GetSystemClock()) {} + + explicit CompositeEnvWrapper(Env* env, const std::shared_ptr& sc) + : CompositeEnvWrapper(env, env->GetFileSystem(), sc) {} + + explicit CompositeEnvWrapper(Env* env, const std::shared_ptr& fs, + const std::shared_ptr& sc) + : CompositeEnv(fs, sc), env_target_(env) {} // Return the target to which this Env forwards all calls Env* env_target() const { return env_target_; } @@ -585,19 +288,9 @@ class CompositeEnvWrapper : public CompositeEnv { return env_target_->GetThreadPoolQueueLen(pri); } - uint64_t NowMicros() override { return env_target_->NowMicros(); } - uint64_t NowNanos() override { return env_target_->NowNanos(); } - uint64_t NowCPUNanos() override { return env_target_->NowCPUNanos(); } - - void SleepForMicroseconds(int micros) override { - env_target_->SleepForMicroseconds(micros); - } Status GetHostName(char* name, uint64_t len) override { return env_target_->GetHostName(name, len); } - Status GetCurrentTime(int64_t* unix_time) override { - return env_target_->GetCurrentTime(unix_time); - } void SetBackgroundThreads(int num, Priority pri) override { return env_target_->SetBackgroundThreads(num, pri); } @@ -625,10 +318,6 @@ class CompositeEnvWrapper : public CompositeEnv { return env_target_->LowerThreadPoolCPUPriority(pool, pri); } - std::string TimeToString(uint64_t time) override { - return env_target_->TimeToString(time); - } - Status GetThreadList(std::vector* thread_list) override { return env_target_->GetThreadList(thread_list); } diff --git a/env/env.cc b/env/env.cc index 7371fa328..04d64fa3b 100644 --- a/env/env.cc +++ b/env/env.cc @@ -10,17 +10,58 @@ #include "rocksdb/env.h" #include + #include "env/composite_env_wrapper.h" #include "logging/env_logger.h" #include "memory/arena.h" #include "options/db_options.h" #include "port/port.h" #include "rocksdb/options.h" +#include "rocksdb/system_clock.h" #include "rocksdb/utilities/object_registry.h" #include "util/autovector.h" namespace ROCKSDB_NAMESPACE { namespace { +class LegacySystemClock : public SystemClock { + private: + Env* env_; + + public: + explicit LegacySystemClock(Env* env) : env_(env) {} + const char* Name() const override { return "Legacy System Clock"; } + + // Returns the number of micro-seconds since some fixed point in time. + // It is often used as system time such as in GenericRateLimiter + // and other places so a port needs to return system time in order to work. + uint64_t NowMicros() override { return env_->NowMicros(); } + + // Returns the number of nano-seconds since some fixed point in time. Only + // useful for computing deltas of time in one run. + // Default implementation simply relies on NowMicros. + // In platform-specific implementations, NowNanos() should return time points + // that are MONOTONIC. + uint64_t NowNanos() override { return env_->NowNanos(); } + + uint64_t CPUMicros() override { return CPUNanos() / 1000; } + uint64_t CPUNanos() override { return env_->NowCPUNanos(); } + + // Sleep/delay the thread for the prescribed number of micro-seconds. + void SleepForMicroseconds(int micros) override { + env_->SleepForMicroseconds(micros); + } + + // Get the number of seconds since the Epoch, 1970-01-01 00:00:00 (UTC). + // Only overwrites *unix_time on success. + Status GetCurrentTime(int64_t* unix_time) override { + return env_->GetCurrentTime(unix_time); + } + // Converts seconds-since-Jan-01-1970 to a printable string + std::string TimeToString(uint64_t time) override { + return env_->TimeToString(time); + } +}; + class LegacyFileSystemWrapper : public FileSystem { public: // Initialize an EnvWrapper that delegates all calls to *t @@ -265,11 +306,17 @@ class LegacyFileSystemWrapper : public FileSystem { Env::Env() : thread_status_updater_(nullptr) { file_system_ = std::make_shared(this); + system_clock_ = std::make_shared(this); } -Env::Env(std::shared_ptr fs) - : thread_status_updater_(nullptr), - file_system_(fs) {} +Env::Env(const std::shared_ptr& fs) + : thread_status_updater_(nullptr), file_system_(fs) { + system_clock_ = std::make_shared(this); +} + +Env::Env(const std::shared_ptr& fs, + const std::shared_ptr& clock) + : thread_status_updater_(nullptr), file_system_(fs), system_clock_(clock) {} Env::~Env() { } @@ -730,4 +777,8 @@ Status NewEnvLogger(const std::string& fname, Env* env, const std::shared_ptr& Env::GetFileSystem() const { return file_system_; } + +const std::shared_ptr& Env::GetSystemClock() const { + return system_clock_; +} } // namespace ROCKSDB_NAMESPACE diff --git a/env/env_posix.cc b/env/env_posix.cc index b28e606eb..8c20f1eb5 100644 --- a/env/env_posix.cc +++ b/env/env_posix.cc @@ -56,8 +56,10 @@ #include "monitoring/iostats_context_imp.h" #include "monitoring/thread_status_updater.h" #include "port/port.h" +#include "rocksdb/env.h" #include "rocksdb/options.h" #include "rocksdb/slice.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "util/coding.h" #include "util/compression_context_cache.h" @@ -121,6 +123,82 @@ class PosixDynamicLibrary : public DynamicLibrary { void* handle_; }; #endif // !ROCKSDB_NO_DYNAMIC_EXTENSION +class PosixClock : public SystemClock { + public: + const char* Name() const override { return "PosixClock"; } + uint64_t NowMicros() override { + struct timeval tv; + gettimeofday(&tv, nullptr); + return static_cast(tv.tv_sec) * 1000000 + tv.tv_usec; + } + + uint64_t NowNanos() override { +#if defined(OS_LINUX) || defined(OS_FREEBSD) || defined(OS_GNU_KFREEBSD) || \ + defined(OS_AIX) + struct timespec ts; + clock_gettime(CLOCK_MONOTONIC, &ts); + return static_cast(ts.tv_sec) * 1000000000 + ts.tv_nsec; +#elif defined(OS_SOLARIS) + return gethrtime(); +#elif defined(__MACH__) + clock_serv_t cclock; + mach_timespec_t ts; + host_get_clock_service(mach_host_self(), CALENDAR_CLOCK, &cclock); + clock_get_time(cclock, &ts); + mach_port_deallocate(mach_task_self(), cclock); + return static_cast(ts.tv_sec) * 1000000000 + ts.tv_nsec; +#else + return std::chrono::duration_cast( + std::chrono::steady_clock::now().time_since_epoch()) + .count(); +#endif + } + + uint64_t CPUMicros() override { +#if defined(OS_LINUX) || defined(OS_FREEBSD) || defined(OS_GNU_KFREEBSD) || \ + defined(OS_AIX) || (defined(__MACH__) && defined(__MAC_10_12)) + struct timespec ts; + clock_gettime(CLOCK_THREAD_CPUTIME_ID, &ts); + return static_cast(ts.tv_sec) * 1000000000; +#endif + return 0; + } + + uint64_t CPUNanos() override { +#if defined(OS_LINUX) || defined(OS_FREEBSD) || defined(OS_GNU_KFREEBSD) || \ + defined(OS_AIX) || (defined(__MACH__) && defined(__MAC_10_12)) + struct timespec ts; + clock_gettime(CLOCK_THREAD_CPUTIME_ID, &ts); + return static_cast(ts.tv_sec) * 1000000000 + ts.tv_nsec; +#endif + return 0; + } + + void SleepForMicroseconds(int micros) override { usleep(micros); } + + Status GetCurrentTime(int64_t* unix_time) override { + time_t ret = time(nullptr); + if (ret == (time_t)-1) { + return IOError("GetCurrentTime", "", errno); + } + *unix_time = (int64_t)ret; + return Status::OK(); + } + + std::string TimeToString(uint64_t secondsSince1970) override { + const time_t seconds = (time_t)secondsSince1970; + struct tm t; + int maxsize = 64; + std::string dummy; + dummy.reserve(maxsize); + dummy.resize(maxsize); + char* p = &dummy[0]; + localtime_r(&seconds, &t); + snprintf(p, maxsize, "%04d/%02d/%02d-%02d:%02d:%02d ", t.tm_year + 1900, + t.tm_mon + 1, t.tm_mday, t.tm_hour, t.tm_min, t.tm_sec); + return dummy; + } +}; class PosixEnv : public CompositeEnv { public: @@ -232,45 +310,6 @@ class PosixEnv : public CompositeEnv { uint64_t GetThreadID() const override { return gettid(pthread_self()); } - uint64_t NowMicros() override { - struct timeval tv; - gettimeofday(&tv, nullptr); - return static_cast(tv.tv_sec) * 1000000 + tv.tv_usec; - } - - uint64_t NowNanos() override { -#if defined(OS_LINUX) || defined(OS_FREEBSD) || defined(OS_GNU_KFREEBSD) || \ - defined(OS_AIX) - struct timespec ts; - clock_gettime(CLOCK_MONOTONIC, &ts); - return static_cast(ts.tv_sec) * 1000000000 + ts.tv_nsec; -#elif defined(OS_SOLARIS) - return gethrtime(); -#elif defined(__MACH__) - clock_serv_t cclock; - mach_timespec_t ts; - host_get_clock_service(mach_host_self(), CALENDAR_CLOCK, &cclock); - clock_get_time(cclock, &ts); - mach_port_deallocate(mach_task_self(), cclock); - return static_cast(ts.tv_sec) * 1000000000 + ts.tv_nsec; -#else - return std::chrono::duration_cast( - std::chrono::steady_clock::now().time_since_epoch()).count(); -#endif - } - - uint64_t NowCPUNanos() override { -#if defined(OS_LINUX) || defined(OS_FREEBSD) || defined(OS_GNU_KFREEBSD) || \ - defined(OS_AIX) || (defined(__MACH__) && defined(__MAC_10_12)) - struct timespec ts; - clock_gettime(CLOCK_THREAD_CPUTIME_ID, &ts); - return static_cast(ts.tv_sec) * 1000000000 + ts.tv_nsec; -#endif - return 0; - } - - void SleepForMicroseconds(int micros) override { usleep(micros); } - Status GetHostName(char* name, uint64_t len) override { int ret = gethostname(name, static_cast(len)); if (ret < 0) { @@ -283,15 +322,6 @@ class PosixEnv : public CompositeEnv { return Status::OK(); } - Status GetCurrentTime(int64_t* unix_time) override { - time_t ret = time(nullptr); - if (ret == (time_t) -1) { - return IOError("GetCurrentTime", "", errno); - } - *unix_time = (int64_t) ret; - return Status::OK(); - } - ThreadStatusUpdater* GetThreadStatusUpdater() const override { return Env::GetThreadStatusUpdater(); } @@ -340,26 +370,6 @@ class PosixEnv : public CompositeEnv { return Status::OK(); } - std::string TimeToString(uint64_t secondsSince1970) override { - const time_t seconds = (time_t)secondsSince1970; - struct tm t; - int maxsize = 64; - std::string dummy; - dummy.reserve(maxsize); - dummy.resize(maxsize); - char* p = &dummy[0]; - localtime_r(&seconds, &t); - snprintf(p, maxsize, - "%04d/%02d/%02d-%02d:%02d:%02d ", - t.tm_year + 1900, - t.tm_mon + 1, - t.tm_mday, - t.tm_hour, - t.tm_min, - t.tm_sec); - return dummy; - } - private: friend Env* Env::Default(); // Constructs the default Env, a singleton @@ -382,7 +392,7 @@ class PosixEnv : public CompositeEnv { }; PosixEnv::PosixEnv() - : CompositeEnv(FileSystem::Default()), + : CompositeEnv(FileSystem::Default(), SystemClock::Default()), thread_pools_storage_(Priority::TOTAL), allow_non_owner_access_storage_(true), thread_pools_(thread_pools_storage_), @@ -401,7 +411,7 @@ PosixEnv::PosixEnv() PosixEnv::PosixEnv(const PosixEnv* default_env, const std::shared_ptr& fs) - : CompositeEnv(fs), + : CompositeEnv(fs, default_env->GetSystemClock()), thread_pools_(default_env->thread_pools_), mu_(default_env->mu_), threads_to_join_(default_env->threads_to_join_), @@ -509,6 +519,14 @@ std::unique_ptr NewCompositeEnv(const std::shared_ptr& fs) { return std::unique_ptr(new PosixEnv(default_env, fs)); } +// +// Default Posix SystemClock +// +const std::shared_ptr& SystemClock::Default() { + static std::shared_ptr default_clock = + std::make_shared(); + return default_clock; +} } // namespace ROCKSDB_NAMESPACE #endif diff --git a/env/env_test.cc b/env/env_test.cc index bfde71a97..ab5ec6649 100644 --- a/env/env_test.cc +++ b/env/env_test.cc @@ -35,6 +35,7 @@ #include "port/malloc.h" #include "port/port.h" #include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" #include "test_util/testutil.h" @@ -2213,7 +2214,8 @@ TEST_F(EnvTest, IsDirectory) { ASSERT_OK(s); std::unique_ptr fwriter; fwriter.reset(new WritableFileWriter(std::move(wfile), test_file_path, - FileOptions(), Env::Default())); + FileOptions(), + SystemClock::Default())); constexpr char buf[] = "test"; s = fwriter->Append(buf); ASSERT_OK(s); diff --git a/env/file_system_tracer.cc b/env/file_system_tracer.cc index 28ee9d85c..2a639f152 100644 --- a/env/file_system_tracer.cc +++ b/env/file_system_tracer.cc @@ -5,18 +5,19 @@ #include "env/file_system_tracer.h" -#include "rocksdb/env.h" +#include "rocksdb/file_system.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { IOStatus FileSystemTracingWrapper::NewSequentialFile( const std::string& fname, const FileOptions& file_opts, std::unique_ptr* result, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->NewSequentialFile(fname, file_opts, result, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -26,11 +27,11 @@ IOStatus FileSystemTracingWrapper::NewSequentialFile( IOStatus FileSystemTracingWrapper::NewRandomAccessFile( const std::string& fname, const FileOptions& file_opts, std::unique_ptr* result, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->NewRandomAccessFile(fname, file_opts, result, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -40,11 +41,11 @@ IOStatus FileSystemTracingWrapper::NewRandomAccessFile( IOStatus FileSystemTracingWrapper::NewWritableFile( const std::string& fname, const FileOptions& file_opts, std::unique_ptr* result, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->NewWritableFile(fname, file_opts, result, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -54,11 +55,11 @@ IOStatus FileSystemTracingWrapper::NewWritableFile( IOStatus FileSystemTracingWrapper::ReopenWritableFile( const std::string& fname, const FileOptions& file_opts, std::unique_ptr* result, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->ReopenWritableFile(fname, file_opts, result, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -69,12 +70,12 @@ IOStatus FileSystemTracingWrapper::ReuseWritableFile( const std::string& fname, const std::string& old_fname, const FileOptions& file_opts, std::unique_ptr* result, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->ReuseWritableFile(fname, old_fname, file_opts, result, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -84,11 +85,11 @@ IOStatus FileSystemTracingWrapper::ReuseWritableFile( IOStatus FileSystemTracingWrapper::NewRandomRWFile( const std::string& fname, const FileOptions& file_opts, std::unique_ptr* result, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->NewRandomRWFile(fname, file_opts, result, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -98,11 +99,11 @@ IOStatus FileSystemTracingWrapper::NewRandomRWFile( IOStatus FileSystemTracingWrapper::NewDirectory( const std::string& name, const IOOptions& io_opts, std::unique_ptr* result, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->NewDirectory(name, io_opts, result, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), name.substr(name.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -113,11 +114,11 @@ IOStatus FileSystemTracingWrapper::GetChildren(const std::string& dir, const IOOptions& io_opts, std::vector* r, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->GetChildren(dir, io_opts, r, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), dir.substr(dir.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -127,11 +128,11 @@ IOStatus FileSystemTracingWrapper::GetChildren(const std::string& dir, IOStatus FileSystemTracingWrapper::DeleteFile(const std::string& fname, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->DeleteFile(fname, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -141,11 +142,11 @@ IOStatus FileSystemTracingWrapper::DeleteFile(const std::string& fname, IOStatus FileSystemTracingWrapper::CreateDir(const std::string& dirname, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->CreateDir(dirname, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), dirname.substr(dirname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -154,11 +155,11 @@ IOStatus FileSystemTracingWrapper::CreateDir(const std::string& dirname, IOStatus FileSystemTracingWrapper::CreateDirIfMissing( const std::string& dirname, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->CreateDirIfMissing(dirname, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), dirname.substr(dirname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -168,11 +169,11 @@ IOStatus FileSystemTracingWrapper::CreateDirIfMissing( IOStatus FileSystemTracingWrapper::DeleteDir(const std::string& dirname, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->DeleteDir(dirname, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), dirname.substr(dirname.find_last_of("/\\") + 1)); io_tracer_->WriteIOOp(io_record); @@ -183,14 +184,14 @@ IOStatus FileSystemTracingWrapper::GetFileSize(const std::string& fname, const IOOptions& options, uint64_t* file_size, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->GetFileSize(fname, options, file_size, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOFileSize); IOTraceRecord io_record( - env_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, + clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1), *file_size); io_tracer_->WriteIOOp(io_record); return s; @@ -200,13 +201,13 @@ IOStatus FileSystemTracingWrapper::Truncate(const std::string& fname, size_t size, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Truncate(fname, size, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOFileSize); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), fname.substr(fname.find_last_of("/\\") + 1), size); io_tracer_->WriteIOOp(io_record); @@ -217,13 +218,13 @@ IOStatus FSSequentialFileTracingWrapper::Read(size_t n, const IOOptions& options, Slice* result, char* scratch, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Read(n, options, result, scratch, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, result->size(), 0 /*Offset*/); io_tracer_->WriteIOOp(io_record); @@ -232,14 +233,14 @@ IOStatus FSSequentialFileTracingWrapper::Read(size_t n, IOStatus FSSequentialFileTracingWrapper::InvalidateCache(size_t offset, size_t length) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->InvalidateCache(offset, length); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, length, offset); io_tracer_->WriteIOOp(io_record); @@ -249,7 +250,7 @@ IOStatus FSSequentialFileTracingWrapper::InvalidateCache(size_t offset, IOStatus FSSequentialFileTracingWrapper::PositionedRead( uint64_t offset, size_t n, const IOOptions& options, Slice* result, char* scratch, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->PositionedRead(offset, n, options, result, scratch, dbg); @@ -257,7 +258,7 @@ IOStatus FSSequentialFileTracingWrapper::PositionedRead( uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, result->size(), offset); io_tracer_->WriteIOOp(io_record); @@ -268,14 +269,14 @@ IOStatus FSRandomAccessFileTracingWrapper::Read(uint64_t offset, size_t n, const IOOptions& options, Slice* result, char* scratch, IODebugContext* dbg) const { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Read(offset, n, options, result, scratch, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, n, offset); io_tracer_->WriteIOOp(io_record); @@ -286,7 +287,7 @@ IOStatus FSRandomAccessFileTracingWrapper::MultiRead(FSReadRequest* reqs, size_t num_reqs, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->MultiRead(reqs, num_reqs, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); @@ -295,9 +296,9 @@ IOStatus FSRandomAccessFileTracingWrapper::MultiRead(FSReadRequest* reqs, io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); for (size_t i = 0; i < num_reqs; i++) { - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, - __func__, latency, reqs[i].status.ToString(), - file_name_, reqs[i].len, reqs[i].offset); + IOTraceRecord io_record( + clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, latency, + reqs[i].status.ToString(), file_name_, reqs[i].len, reqs[i].offset); io_tracer_->WriteIOOp(io_record); } return s; @@ -306,14 +307,14 @@ IOStatus FSRandomAccessFileTracingWrapper::MultiRead(FSReadRequest* reqs, IOStatus FSRandomAccessFileTracingWrapper::Prefetch(uint64_t offset, size_t n, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Prefetch(offset, n, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, n, offset); io_tracer_->WriteIOOp(io_record); @@ -322,14 +323,14 @@ IOStatus FSRandomAccessFileTracingWrapper::Prefetch(uint64_t offset, size_t n, IOStatus FSRandomAccessFileTracingWrapper::InvalidateCache(size_t offset, size_t length) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->InvalidateCache(offset, length); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, length, static_cast(offset)); io_tracer_->WriteIOOp(io_record); @@ -339,13 +340,13 @@ IOStatus FSRandomAccessFileTracingWrapper::InvalidateCache(size_t offset, IOStatus FSWritableFileTracingWrapper::Append(const Slice& data, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Append(data, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, data.size(), 0 /*Offset*/); io_tracer_->WriteIOOp(io_record); @@ -355,14 +356,14 @@ IOStatus FSWritableFileTracingWrapper::Append(const Slice& data, IOStatus FSWritableFileTracingWrapper::PositionedAppend( const Slice& data, uint64_t offset, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->PositionedAppend(data, offset, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, data.size(), offset); io_tracer_->WriteIOOp(io_record); @@ -372,13 +373,13 @@ IOStatus FSWritableFileTracingWrapper::PositionedAppend( IOStatus FSWritableFileTracingWrapper::Truncate(uint64_t size, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Truncate(size, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, size, 0 /*Offset*/); io_tracer_->WriteIOOp(io_record); @@ -387,11 +388,11 @@ IOStatus FSWritableFileTracingWrapper::Truncate(uint64_t size, IOStatus FSWritableFileTracingWrapper::Close(const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Close(options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), file_name_); io_tracer_->WriteIOOp(io_record); @@ -400,13 +401,13 @@ IOStatus FSWritableFileTracingWrapper::Close(const IOOptions& options, uint64_t FSWritableFileTracingWrapper::GetFileSize(const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); uint64_t file_size = target()->GetFileSize(options, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOFileSize); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, "OK", file_name_, file_size); io_tracer_->WriteIOOp(io_record); return file_size; @@ -414,14 +415,14 @@ uint64_t FSWritableFileTracingWrapper::GetFileSize(const IOOptions& options, IOStatus FSWritableFileTracingWrapper::InvalidateCache(size_t offset, size_t length) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->InvalidateCache(offset, length); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, length, static_cast(offset)); io_tracer_->WriteIOOp(io_record); @@ -431,14 +432,14 @@ IOStatus FSWritableFileTracingWrapper::InvalidateCache(size_t offset, IOStatus FSRandomRWFileTracingWrapper::Write(uint64_t offset, const Slice& data, const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Write(offset, data, options, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, data.size(), offset); io_tracer_->WriteIOOp(io_record); @@ -449,14 +450,14 @@ IOStatus FSRandomRWFileTracingWrapper::Read(uint64_t offset, size_t n, const IOOptions& options, Slice* result, char* scratch, IODebugContext* dbg) const { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Read(offset, n, options, result, scratch, dbg); uint64_t elapsed = timer.ElapsedNanos(); uint64_t io_op_data = 0; io_op_data |= (1 << IOTraceOp::kIOLen); io_op_data |= (1 << IOTraceOp::kIOOffset); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, io_op_data, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, io_op_data, __func__, elapsed, s.ToString(), file_name_, n, offset); io_tracer_->WriteIOOp(io_record); @@ -465,11 +466,11 @@ IOStatus FSRandomRWFileTracingWrapper::Read(uint64_t offset, size_t n, IOStatus FSRandomRWFileTracingWrapper::Flush(const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Flush(options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), file_name_); io_tracer_->WriteIOOp(io_record); @@ -478,11 +479,11 @@ IOStatus FSRandomRWFileTracingWrapper::Flush(const IOOptions& options, IOStatus FSRandomRWFileTracingWrapper::Close(const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Close(options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), file_name_); io_tracer_->WriteIOOp(io_record); @@ -491,11 +492,11 @@ IOStatus FSRandomRWFileTracingWrapper::Close(const IOOptions& options, IOStatus FSRandomRWFileTracingWrapper::Sync(const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Sync(options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), file_name_); io_tracer_->WriteIOOp(io_record); @@ -504,11 +505,11 @@ IOStatus FSRandomRWFileTracingWrapper::Sync(const IOOptions& options, IOStatus FSRandomRWFileTracingWrapper::Fsync(const IOOptions& options, IODebugContext* dbg) { - StopWatchNano timer(env_); + StopWatchNano timer(clock_); timer.Start(); IOStatus s = target()->Fsync(options, dbg); uint64_t elapsed = timer.ElapsedNanos(); - IOTraceRecord io_record(env_->NowNanos(), TraceType::kIOTracer, + IOTraceRecord io_record(clock_->NowNanos(), TraceType::kIOTracer, 0 /*io_op_data*/, __func__, elapsed, s.ToString(), file_name_); io_tracer_->WriteIOOp(io_record); diff --git a/env/file_system_tracer.h b/env/file_system_tracer.h index 4f45957d5..faa99ec32 100644 --- a/env/file_system_tracer.h +++ b/env/file_system_tracer.h @@ -6,6 +6,7 @@ #pragma once #include "rocksdb/file_system.h" +#include "rocksdb/system_clock.h" #include "trace_replay/io_tracer.h" namespace ROCKSDB_NAMESPACE { @@ -18,9 +19,11 @@ namespace ROCKSDB_NAMESPACE { // overridden. class FileSystemTracingWrapper : public FileSystemWrapper { public: - FileSystemTracingWrapper(std::shared_ptr t, - std::shared_ptr io_tracer) - : FileSystemWrapper(t), io_tracer_(io_tracer), env_(Env::Default()) {} + FileSystemTracingWrapper(const std::shared_ptr& t, + const std::shared_ptr& io_tracer) + : FileSystemWrapper(t), + io_tracer_(io_tracer), + clock_(SystemClock::Default()) {} ~FileSystemTracingWrapper() override {} @@ -83,7 +86,7 @@ class FileSystemTracingWrapper : public FileSystemWrapper { private: std::shared_ptr io_tracer_; - Env* env_; + std::shared_ptr clock_; }; // The FileSystemPtr is a wrapper class that takes pointer to storage systems @@ -135,7 +138,7 @@ class FSSequentialFileTracingWrapper : public FSSequentialFileWrapper { const std::string& file_name) : FSSequentialFileWrapper(t), io_tracer_(io_tracer), - env_(Env::Default()), + clock_(SystemClock::Default()), file_name_(file_name) {} ~FSSequentialFileTracingWrapper() override {} @@ -151,7 +154,7 @@ class FSSequentialFileTracingWrapper : public FSSequentialFileWrapper { private: std::shared_ptr io_tracer_; - Env* env_; + std::shared_ptr clock_; std::string file_name_; }; @@ -207,7 +210,7 @@ class FSRandomAccessFileTracingWrapper : public FSRandomAccessFileWrapper { const std::string& file_name) : FSRandomAccessFileWrapper(t), io_tracer_(io_tracer), - env_(Env::Default()), + clock_(SystemClock::Default()), file_name_(file_name) {} ~FSRandomAccessFileTracingWrapper() override {} @@ -226,7 +229,7 @@ class FSRandomAccessFileTracingWrapper : public FSRandomAccessFileWrapper { private: std::shared_ptr io_tracer_; - Env* env_; + std::shared_ptr clock_; // Stores file name instead of full path. std::string file_name_; }; @@ -282,7 +285,7 @@ class FSWritableFileTracingWrapper : public FSWritableFileWrapper { const std::string& file_name) : FSWritableFileWrapper(t), io_tracer_(io_tracer), - env_(Env::Default()), + clock_(SystemClock::Default()), file_name_(file_name) {} ~FSWritableFileTracingWrapper() override {} @@ -316,7 +319,7 @@ class FSWritableFileTracingWrapper : public FSWritableFileWrapper { private: std::shared_ptr io_tracer_; - Env* env_; + std::shared_ptr clock_; // Stores file name instead of full path. std::string file_name_; }; @@ -379,7 +382,7 @@ class FSRandomRWFileTracingWrapper : public FSRandomRWFileWrapper { const std::string& file_name) : FSRandomRWFileWrapper(t), io_tracer_(io_tracer), - env_(Env::Default()), + clock_(SystemClock::Default()), file_name_(file_name) {} ~FSRandomRWFileTracingWrapper() override {} @@ -401,7 +404,7 @@ class FSRandomRWFileTracingWrapper : public FSRandomRWFileWrapper { private: std::shared_ptr io_tracer_; - Env* env_; + std::shared_ptr clock_; // Stores file name instead of full path. std::string file_name_; }; diff --git a/file/delete_scheduler.cc b/file/delete_scheduler.cc index 2693baf9a..003bd2cc9 100644 --- a/file/delete_scheduler.cc +++ b/file/delete_scheduler.cc @@ -15,17 +15,20 @@ #include "logging/logging.h" #include "port/port.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "util/mutexlock.h" namespace ROCKSDB_NAMESPACE { -DeleteScheduler::DeleteScheduler(Env* env, FileSystem* fs, - int64_t rate_bytes_per_sec, Logger* info_log, +DeleteScheduler::DeleteScheduler(const std::shared_ptr& clock, + FileSystem* fs, int64_t rate_bytes_per_sec, + Logger* info_log, SstFileManagerImpl* sst_file_manager, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk) - : env_(env), + : clock_(clock), fs_(fs), total_trash_size_(0), rate_bytes_per_sec_(rate_bytes_per_sec), @@ -223,14 +226,14 @@ void DeleteScheduler::BackgroundEmptyTrash() { } // Delete all files in queue_ - uint64_t start_time = env_->NowMicros(); + uint64_t start_time = clock_->NowMicros(); uint64_t total_deleted_bytes = 0; int64_t current_delete_rate = rate_bytes_per_sec_.load(); while (!queue_.empty() && !closing_) { if (current_delete_rate != rate_bytes_per_sec_.load()) { // User changed the delete rate current_delete_rate = rate_bytes_per_sec_.load(); - start_time = env_->NowMicros(); + start_time = clock_->NowMicros(); total_deleted_bytes = 0; ROCKS_LOG_INFO(info_log_, "rate_bytes_per_sec is changed to %" PRIi64, current_delete_rate); diff --git a/file/delete_scheduler.h b/file/delete_scheduler.h index 7c5ae8e00..9981d13bf 100644 --- a/file/delete_scheduler.h +++ b/file/delete_scheduler.h @@ -15,14 +15,15 @@ #include "monitoring/instrumented_mutex.h" #include "port/port.h" -#include "rocksdb/file_system.h" #include "rocksdb/status.h" namespace ROCKSDB_NAMESPACE { class Env; +class FileSystem; class Logger; class SstFileManagerImpl; +class SystemClock; // DeleteScheduler allows the DB to enforce a rate limit on file deletion, // Instead of deleteing files immediately, files are marked as trash @@ -33,8 +34,9 @@ class SstFileManagerImpl; // case DeleteScheduler will delete files immediately. class DeleteScheduler { public: - DeleteScheduler(Env* env, FileSystem* fs, int64_t rate_bytes_per_sec, - Logger* info_log, SstFileManagerImpl* sst_file_manager, + DeleteScheduler(const std::shared_ptr& clock, FileSystem* fs, + int64_t rate_bytes_per_sec, Logger* info_log, + SstFileManagerImpl* sst_file_manager, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk); ~DeleteScheduler(); @@ -99,7 +101,7 @@ class DeleteScheduler { void MaybeCreateBackgroundThread(); - Env* env_; + const std::shared_ptr clock_; FileSystem* fs_; // total size of trash files diff --git a/file/delete_scheduler_test.cc b/file/delete_scheduler_test.cc index e6cc240b6..4981336f5 100644 --- a/file/delete_scheduler_test.cc +++ b/file/delete_scheduler_test.cc @@ -95,9 +95,10 @@ class DeleteSchedulerTest : public testing::Test { // Tests in this file are for DeleteScheduler component and don't create any // DBs, so we need to set max_trash_db_ratio to 100% (instead of default // 25%) - sst_file_mgr_.reset(new SstFileManagerImpl( - env_, env_->GetFileSystem(), nullptr, rate_bytes_per_sec_, - /* max_trash_db_ratio= */ 1.1, 128 * 1024)); + sst_file_mgr_.reset( + new SstFileManagerImpl(env_->GetSystemClock(), env_->GetFileSystem(), + nullptr, rate_bytes_per_sec_, + /* max_trash_db_ratio= */ 1.1, 128 * 1024)); delete_scheduler_ = sst_file_mgr_->delete_scheduler(); sst_file_mgr_->SetStatisticsPtr(stats_); } diff --git a/file/file_util.h b/file/file_util.h index ef5ca922f..b015e81e5 100644 --- a/file/file_util.h +++ b/file/file_util.h @@ -12,6 +12,7 @@ #include "rocksdb/file_system.h" #include "rocksdb/sst_file_writer.h" #include "rocksdb/status.h" +#include "rocksdb/system_clock.h" #include "rocksdb/types.h" #include "trace_replay/io_tracer.h" @@ -67,14 +68,12 @@ inline IOStatus GenerateOneFileChecksum( allow_mmap_reads, io_tracer); } -inline IOStatus PrepareIOFromReadOptions(const ReadOptions& ro, Env* env, - IOOptions& opts) { - if (!env) { - env = Env::Default(); - } - +inline IOStatus PrepareIOFromReadOptions( + const ReadOptions& ro, const std::shared_ptr& clock, + IOOptions& opts) { if (ro.deadline.count()) { - std::chrono::microseconds now = std::chrono::microseconds(env->NowMicros()); + std::chrono::microseconds now = + std::chrono::microseconds(clock->NowMicros()); // Ensure there is atleast 1us available. We don't want to pass a value of // 0 as that means no timeout if (now >= ro.deadline) { diff --git a/file/filename.cc b/file/filename.cc index a581b466f..f1b943619 100644 --- a/file/filename.cc +++ b/file/filename.cc @@ -419,15 +419,17 @@ Status SetIdentityFile(Env* env, const std::string& dbname, return s; } -IOStatus SyncManifest(Env* env, const ImmutableDBOptions* db_options, +IOStatus SyncManifest(const std::shared_ptr& clock, + const ImmutableDBOptions* db_options, WritableFileWriter* file) { TEST_KILL_RANDOM("SyncManifest:0", rocksdb_kill_odds * REDUCE_ODDS2); - StopWatch sw(env, db_options->statistics.get(), MANIFEST_FILE_SYNC_MICROS); + StopWatch sw(clock, db_options->statistics.get(), MANIFEST_FILE_SYNC_MICROS); return file->Sync(db_options->use_fsync); } -Status GetInfoLogFiles(Env* env, const std::string& db_log_dir, - const std::string& dbname, std::string* parent_dir, +Status GetInfoLogFiles(const std::shared_ptr& fs, + const std::string& db_log_dir, const std::string& dbname, + std::string* parent_dir, std::vector* info_log_list) { assert(parent_dir != nullptr); assert(info_log_list != nullptr); @@ -443,7 +445,7 @@ Status GetInfoLogFiles(Env* env, const std::string& db_log_dir, InfoLogPrefix info_log_prefix(!db_log_dir.empty(), dbname); std::vector file_names; - Status s = env->GetChildren(*parent_dir, &file_names); + Status s = fs->GetChildren(*parent_dir, IOOptions(), &file_names, nullptr); if (!s.ok()) { return s; diff --git a/file/filename.h b/file/filename.h index 188305a6d..912ef6735 100644 --- a/file/filename.h +++ b/file/filename.h @@ -27,6 +27,7 @@ namespace ROCKSDB_NAMESPACE { class Env; class Directory; +class SystemClock; class WritableFileWriter; #ifdef OS_WIN @@ -166,14 +167,16 @@ extern Status SetIdentityFile(Env* env, const std::string& dbname, const std::string& db_id = {}); // Sync manifest file `file`. -extern IOStatus SyncManifest(Env* env, const ImmutableDBOptions* db_options, +extern IOStatus SyncManifest(const std::shared_ptr& clock, + const ImmutableDBOptions* db_options, WritableFileWriter* file); // Return list of file names of info logs in `file_names`. // The list only contains file name. The parent directory name is stored // in `parent_dir`. // `db_log_dir` should be the one as in options.db_log_dir -extern Status GetInfoLogFiles(Env* env, const std::string& db_log_dir, +extern Status GetInfoLogFiles(const std::shared_ptr& fs, + const std::string& db_log_dir, const std::string& dbname, std::string* parent_dir, std::vector* file_names); diff --git a/file/random_access_file_reader.cc b/file/random_access_file_reader.cc index 646c039b5..e330db342 100644 --- a/file/random_access_file_reader.cc +++ b/file/random_access_file_reader.cc @@ -12,6 +12,7 @@ #include #include +#include "file/file_util.h" #include "monitoring/histogram.h" #include "monitoring/iostats_context_imp.h" #include "port/port.h" @@ -32,7 +33,7 @@ Status RandomAccessFileReader::Read(const IOOptions& opts, uint64_t offset, Status s; uint64_t elapsed = 0; { - StopWatch sw(env_, stats_, hist_type_, + StopWatch sw(clock_, stats_, hist_type_, (stats_ != nullptr) ? &elapsed : nullptr, true /*overwrite*/, true /*delay_enabled*/); auto prev_perf_level = GetPerfLevel(); @@ -68,7 +69,7 @@ Status RandomAccessFileReader::Read(const IOOptions& opts, uint64_t offset, } { - IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, env_); + IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_); // Only user reads are expected to specify a timeout. And user reads // are not subjected to rate_limiter and should go through only // one iteration of this loop, so we don't need to check and adjust @@ -128,7 +129,7 @@ Status RandomAccessFileReader::Read(const IOOptions& opts, uint64_t offset, #endif { - IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, env_); + IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_); // Only user reads are expected to specify a timeout. And user reads // are not subjected to rate_limiter and should go through only // one iteration of this loop, so we don't need to check and adjust @@ -205,7 +206,7 @@ Status RandomAccessFileReader::MultiRead(const IOOptions& opts, Status s; uint64_t elapsed = 0; { - StopWatch sw(env_, stats_, hist_type_, + StopWatch sw(clock_, stats_, hist_type_, (stats_ != nullptr) ? &elapsed : nullptr, true /*overwrite*/, true /*delay_enabled*/); auto prev_perf_level = GetPerfLevel(); @@ -267,7 +268,7 @@ Status RandomAccessFileReader::MultiRead(const IOOptions& opts, #endif // ROCKSDB_LITE { - IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, env_); + IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_); s = file_->MultiRead(fs_reqs, num_fs_reqs, opts, nullptr); } @@ -312,4 +313,12 @@ Status RandomAccessFileReader::MultiRead(const IOOptions& opts, return s; } +IOStatus RandomAccessFileReader::PrepareIOOptions(const ReadOptions& ro, + IOOptions& opts) { + if (clock_.get() != nullptr) { + return PrepareIOFromReadOptions(ro, clock_, opts); + } else { + return PrepareIOFromReadOptions(ro, SystemClock::Default(), opts); + } +} } // namespace ROCKSDB_NAMESPACE diff --git a/file/random_access_file_reader.h b/file/random_access_file_reader.h index 8efe8650d..c13b4ceb1 100644 --- a/file/random_access_file_reader.h +++ b/file/random_access_file_reader.h @@ -14,7 +14,6 @@ #include "env/file_system_tracer.h" #include "port/port.h" -#include "rocksdb/env.h" #include "rocksdb/file_system.h" #include "rocksdb/listener.h" #include "rocksdb/options.h" @@ -24,6 +23,7 @@ namespace ROCKSDB_NAMESPACE { class Statistics; class HistogramImpl; +class SystemClock; using AlignedBuf = std::unique_ptr; @@ -67,7 +67,7 @@ class RandomAccessFileReader { FSRandomAccessFilePtr file_; std::string file_name_; - Env* env_; + std::shared_ptr clock_; Statistics* stats_; uint32_t hist_type_; HistogramImpl* file_read_hist_; @@ -77,14 +77,15 @@ class RandomAccessFileReader { public: explicit RandomAccessFileReader( std::unique_ptr&& raf, const std::string& _file_name, - Env* _env = nullptr, const std::shared_ptr& io_tracer = nullptr, + const std::shared_ptr& clock = nullptr, + const std::shared_ptr& io_tracer = nullptr, Statistics* stats = nullptr, uint32_t hist_type = 0, HistogramImpl* file_read_hist = nullptr, RateLimiter* rate_limiter = nullptr, const std::vector>& listeners = {}) : file_(std::move(raf), io_tracer, _file_name), file_name_(std::move(_file_name)), - env_(_env), + clock_(clock), stats_(stats), hist_type_(hist_type), file_read_hist_(file_read_hist), @@ -137,6 +138,6 @@ class RandomAccessFileReader { bool use_direct_io() const { return file_->use_direct_io(); } - Env* env() const { return env_; } + IOStatus PrepareIOOptions(const ReadOptions& ro, IOOptions& opts); }; } // namespace ROCKSDB_NAMESPACE diff --git a/file/random_access_file_reader_test.cc b/file/random_access_file_reader_test.cc index 72a1133a3..22857f8ef 100644 --- a/file/random_access_file_reader_test.cc +++ b/file/random_access_file_reader_test.cc @@ -42,7 +42,8 @@ class RandomAccessFileReaderTest : public testing::Test { std::string fpath = Path(fname); std::unique_ptr f; ASSERT_OK(fs_->NewRandomAccessFile(fpath, opts, &f, nullptr)); - (*reader).reset(new RandomAccessFileReader(std::move(f), fpath, env_)); + (*reader).reset(new RandomAccessFileReader(std::move(f), fpath, + env_->GetSystemClock())); } void AssertResult(const std::string& content, diff --git a/file/sst_file_manager_impl.cc b/file/sst_file_manager_impl.cc index 359ebf999..c53160e80 100644 --- a/file/sst_file_manager_impl.cc +++ b/file/sst_file_manager_impl.cc @@ -18,12 +18,12 @@ namespace ROCKSDB_NAMESPACE { #ifndef ROCKSDB_LITE -SstFileManagerImpl::SstFileManagerImpl(Env* env, std::shared_ptr fs, - std::shared_ptr logger, - int64_t rate_bytes_per_sec, - double max_trash_db_ratio, - uint64_t bytes_max_delete_chunk) - : env_(env), +SstFileManagerImpl::SstFileManagerImpl( + const std::shared_ptr& clock, + const std::shared_ptr& fs, + const std::shared_ptr& logger, int64_t rate_bytes_per_sec, + double max_trash_db_ratio, uint64_t bytes_max_delete_chunk) + : clock_(clock), fs_(fs), logger_(logger), total_files_size_(0), @@ -31,8 +31,8 @@ SstFileManagerImpl::SstFileManagerImpl(Env* env, std::shared_ptr fs, compaction_buffer_size_(0), cur_compactions_reserved_size_(0), max_allowed_space_(0), - delete_scheduler_(env, fs_.get(), rate_bytes_per_sec, logger.get(), this, - max_trash_db_ratio, bytes_max_delete_chunk), + delete_scheduler_(clock_, fs_.get(), rate_bytes_per_sec, logger.get(), + this, max_trash_db_ratio, bytes_max_delete_chunk), cv_(&mu_), closing_(false), bg_thread_(nullptr), @@ -347,7 +347,7 @@ void SstFileManagerImpl::ClearError() { if (!error_handler_list_.empty()) { // If there are more instances to be recovered, reschedule after 5 // seconds - int64_t wait_until = env_->NowMicros() + 5000000; + int64_t wait_until = clock_->NowMicros() + 5000000; cv_.TimedWait(wait_until); } @@ -485,7 +485,6 @@ SstFileManager* NewSstFileManager(Env* env, std::shared_ptr info_log, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk) { const auto& fs = env->GetFileSystem(); - return NewSstFileManager(env, fs, info_log, trash_dir, rate_bytes_per_sec, delete_existing_trash, status, max_trash_db_ratio, bytes_max_delete_chunk); @@ -498,8 +497,9 @@ SstFileManager* NewSstFileManager(Env* env, std::shared_ptr fs, bool delete_existing_trash, Status* status, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk) { + const auto& clock = env->GetSystemClock(); SstFileManagerImpl* res = - new SstFileManagerImpl(env, fs, info_log, rate_bytes_per_sec, + new SstFileManagerImpl(clock, fs, info_log, rate_bytes_per_sec, max_trash_db_ratio, bytes_max_delete_chunk); // trash_dir is deprecated and not needed anymore, but if user passed it diff --git a/file/sst_file_manager_impl.h b/file/sst_file_manager_impl.h index cb0e7fbbb..52ef1917e 100644 --- a/file/sst_file_manager_impl.h +++ b/file/sst_file_manager_impl.h @@ -12,14 +12,13 @@ #include "port/port.h" #include "db/compaction/compaction.h" -#include "db/error_handler.h" #include "file/delete_scheduler.h" -#include "rocksdb/file_system.h" #include "rocksdb/sst_file_manager.h" namespace ROCKSDB_NAMESPACE { - -class Env; +class ErrorHandler; +class FileSystem; +class SystemClock; class Logger; // SstFileManager is used to track SST files in the DB and control their @@ -27,8 +26,9 @@ class Logger; // All SstFileManager public functions are thread-safe. class SstFileManagerImpl : public SstFileManager { public: - explicit SstFileManagerImpl(Env* env, std::shared_ptr fs, - std::shared_ptr logger, + explicit SstFileManagerImpl(const std::shared_ptr& clock, + const std::shared_ptr& fs, + const std::shared_ptr& logger, int64_t rate_bytes_per_sec, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk); @@ -152,7 +152,7 @@ class SstFileManagerImpl : public SstFileManager { return bg_err_.severity() == Status::Severity::kSoftError; } - Env* env_; + std::shared_ptr clock_; std::shared_ptr fs_; std::shared_ptr logger_; // Mutex to protect tracked_files_, total_files_size_ diff --git a/file/writable_file_writer.cc b/file/writable_file_writer.cc index eafd8b66a..1c18b3eb1 100644 --- a/file/writable_file_writer.cc +++ b/file/writable_file_writer.cc @@ -16,6 +16,7 @@ #include "monitoring/histogram.h" #include "monitoring/iostats_context_imp.h" #include "port/port.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "util/random.h" #include "util/rate_limiter.h" @@ -331,7 +332,7 @@ IOStatus WritableFileWriter::SyncInternal(bool use_fsync) { IOSTATS_TIMER_GUARD(fsync_nanos); TEST_SYNC_POINT("WritableFileWriter::SyncInternal:0"); auto prev_perf_level = GetPerfLevel(); - IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, env_); + IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_); #ifndef ROCKSDB_LITE FileOperationInfo::StartTimePoint start_ts; if (ShouldNotifyListeners()) { @@ -406,7 +407,7 @@ IOStatus WritableFileWriter::WriteBuffered(const char* data, size_t size) { #endif { auto prev_perf_level = GetPerfLevel(); - IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, env_); + IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_); s = writable_file_->Append(Slice(src, allowed), IOOptions(), nullptr); SetPerfLevel(prev_perf_level); } diff --git a/file/writable_file_writer.h b/file/writable_file_writer.h index 0904bf37f..c1375af5a 100644 --- a/file/writable_file_writer.h +++ b/file/writable_file_writer.h @@ -14,7 +14,6 @@ #include "db/version_edit.h" #include "env/file_system_tracer.h" #include "port/port.h" -#include "rocksdb/env.h" #include "rocksdb/file_checksum.h" #include "rocksdb/file_system.h" #include "rocksdb/io_status.h" @@ -25,6 +24,7 @@ namespace ROCKSDB_NAMESPACE { class Statistics; +class SystemClock; // WritableFileWriter is a wrapper on top of Env::WritableFile. It provides // facilities to: @@ -121,7 +121,7 @@ class WritableFileWriter { std::string file_name_; FSWritableFilePtr writable_file_; - Env* env_; + std::shared_ptr clock_; AlignedBuffer buf_; size_t max_buffer_size_; // Actually written data size can be used for truncate @@ -145,14 +145,15 @@ class WritableFileWriter { public: WritableFileWriter( std::unique_ptr&& file, const std::string& _file_name, - const FileOptions& options, Env* env = nullptr, + const FileOptions& options, + const std::shared_ptr& clock = nullptr, const std::shared_ptr& io_tracer = nullptr, Statistics* stats = nullptr, const std::vector>& listeners = {}, FileChecksumGenFactory* file_checksum_gen_factory = nullptr) : file_name_(_file_name), writable_file_(std::move(file), io_tracer, _file_name), - env_(env), + clock_(clock), buf_(), max_buffer_size_(options.writable_file_max_buffer_size), filesize_(0), diff --git a/include/rocksdb/env.h b/include/rocksdb/env.h index f8eec967d..f31abeb19 100644 --- a/include/rocksdb/env.h +++ b/include/rocksdb/env.h @@ -59,6 +59,7 @@ class RateLimiter; class ThreadStatusUpdater; struct ThreadStatus; class FileSystem; +class SystemClock; const size_t kDefaultPageSize = 4 * 1024; @@ -150,8 +151,11 @@ class Env { }; Env(); - // Construct an Env with a separate FileSystem implementation - Env(std::shared_ptr fs); + // Construct an Env with a separate FileSystem and/or SystemClock + // implementation + explicit Env(const std::shared_ptr& fs); + Env(const std::shared_ptr& fs, + const std::shared_ptr& clock); // No copying allowed Env(const Env&) = delete; void operator=(const Env&) = delete; @@ -576,6 +580,10 @@ class Env { // could be a fully implemented one, or a wrapper class around the Env const std::shared_ptr& GetFileSystem() const; + // Get the SystemClock implementation this Env was constructed with. It + // could be a fully implemented one, or a wrapper class around the Env + const std::shared_ptr& GetSystemClock() const; + // If you're adding methods here, remember to add them to EnvWrapper too. protected: @@ -586,6 +594,9 @@ class Env { // Pointer to the underlying FileSystem implementation std::shared_ptr file_system_; + // Pointer to the underlying SystemClock implementation + std::shared_ptr system_clock_; + private: static const size_t kMaxHostNameLen = 256; }; diff --git a/include/rocksdb/system_clock.h b/include/rocksdb/system_clock.h new file mode 100644 index 000000000..e03d195ee --- /dev/null +++ b/include/rocksdb/system_clock.h @@ -0,0 +1,102 @@ +// 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). +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#pragma once +#include +#include +#include + +#include + +#ifdef _WIN32 +// Windows API macro interference +#undef GetCurrentTime +#endif + +namespace ROCKSDB_NAMESPACE { +struct ConfigOptions; + +// A SystemClock is an interface used by the rocksdb implementation to access +// operating system time-related functionality. +class SystemClock { + public: + virtual ~SystemClock() {} + + static const char* Type() { return "SystemClock"; } + + // The name of this system clock + virtual const char* Name() const = 0; + + // Return a default SystemClock suitable for the current operating + // system. + static const std::shared_ptr& Default(); + + // Returns the number of micro-seconds since some fixed point in time. + // It is often used as system time such as in GenericRateLimiter + // and other places so a port needs to return system time in order to work. + virtual uint64_t NowMicros() = 0; + + // Returns the number of nano-seconds since some fixed point in time. Only + // useful for computing deltas of time in one run. + // Default implementation simply relies on NowMicros. + // In platform-specific implementations, NowNanos() should return time points + // that are MONOTONIC. + virtual uint64_t NowNanos() { return NowMicros() * 1000; } + + // Returns the number of micro-seconds of CPU time used by the current thread. + // 0 indicates not supported. + virtual uint64_t CPUMicros() { return 0; } + + // Returns the number of nano-seconds of CPU time used by the current thread. + // Default implementation simply relies on CPUMicros. + // 0 indicates not supported. + virtual uint64_t CPUNanos() { return CPUMicros() * 1000; } + + // Sleep/delay the thread for the prescribed number of micro-seconds. + virtual void SleepForMicroseconds(int micros) = 0; + + // Get the number of seconds since the Epoch, 1970-01-01 00:00:00 (UTC). + // Only overwrites *unix_time on success. + virtual Status GetCurrentTime(int64_t* unix_time) = 0; + + // Converts seconds-since-Jan-01-1970 to a printable string + virtual std::string TimeToString(uint64_t time) = 0; +}; + +// Wrapper class for a SystemClock. Redirects all methods (except Name) +// of the SystemClock interface to the target/wrapped class. +class SystemClockWrapper : public SystemClock { + public: + explicit SystemClockWrapper(const std::shared_ptr& t) + : target_(t) {} + + uint64_t NowMicros() override { return target_->NowMicros(); } + + uint64_t NowNanos() override { return target_->NowNanos(); } + + uint64_t CPUMicros() override { return target_->CPUMicros(); } + + uint64_t CPUNanos() override { return target_->CPUNanos(); } + + virtual void SleepForMicroseconds(int micros) override { + return target_->SleepForMicroseconds(micros); + } + + Status GetCurrentTime(int64_t* unix_time) override { + return target_->GetCurrentTime(unix_time); + } + + std::string TimeToString(uint64_t time) override { + return target_->TimeToString(time); + } + + protected: + std::shared_ptr target_; +}; + +} // end namespace ROCKSDB_NAMESPACE diff --git a/logging/auto_roll_logger.cc b/logging/auto_roll_logger.cc index 3533724ba..1ff08c1ad 100644 --- a/logging/auto_roll_logger.cc +++ b/logging/auto_roll_logger.cc @@ -6,8 +6,12 @@ #include "logging/auto_roll_logger.h" #include + #include "file/filename.h" #include "logging/logging.h" +#include "rocksdb/env.h" +#include "rocksdb/file_system.h" +#include "rocksdb/system_clock.h" #include "util/mutexlock.h" namespace ROCKSDB_NAMESPACE { @@ -15,7 +19,9 @@ namespace ROCKSDB_NAMESPACE { #ifndef ROCKSDB_LITE // -- AutoRollLogger -AutoRollLogger::AutoRollLogger(Env* env, const std::string& dbname, +AutoRollLogger::AutoRollLogger(const std::shared_ptr& fs, + const std::shared_ptr& clock, + const std::string& dbname, const std::string& db_log_dir, size_t log_max_size, size_t log_file_time_to_roll, @@ -24,24 +30,26 @@ AutoRollLogger::AutoRollLogger(Env* env, const std::string& dbname, : Logger(log_level), dbname_(dbname), db_log_dir_(db_log_dir), - env_(env), + fs_(fs), + clock_(clock), status_(Status::OK()), kMaxLogFileSize(log_max_size), kLogFileTimeToRoll(log_file_time_to_roll), kKeepLogFileNum(keep_log_file_num), - cached_now(static_cast(env_->NowMicros() * 1e-6)), + cached_now(static_cast(clock_->NowMicros() * 1e-6)), ctime_(cached_now), cached_now_access_count(0), call_NowMicros_every_N_records_(100), mutex_() { - Status s = env->GetAbsolutePath(dbname, &db_absolute_path_); + Status s = fs->GetAbsolutePath(dbname, io_options_, &db_absolute_path_, + &io_context_); if (s.IsNotSupported()) { db_absolute_path_ = dbname; } else { status_ = s; } log_fname_ = InfoLogFileName(dbname_, db_absolute_path_, db_log_dir_); - if (env_->FileExists(log_fname_).ok()) { + if (fs_->FileExists(log_fname_, io_options_, &io_context_).ok()) { RollLogFile(); } GetExistingFiles(); @@ -53,7 +61,7 @@ AutoRollLogger::AutoRollLogger(Env* env, const std::string& dbname, Status AutoRollLogger::ResetLogger() { TEST_SYNC_POINT("AutoRollLogger::ResetLogger:BeforeNewLogger"); - status_ = env_->NewLogger(log_fname_, &logger_); + status_ = fs_->NewLogger(log_fname_, io_options_, &logger_, &io_context_); TEST_SYNC_POINT("AutoRollLogger::ResetLogger:AfterNewLogger"); if (!status_.ok()) { @@ -67,7 +75,7 @@ Status AutoRollLogger::ResetLogger() { "The underlying logger doesn't support GetLogFileSize()"); } if (status_.ok()) { - cached_now = static_cast(env_->NowMicros() * 1e-6); + cached_now = static_cast(clock_->NowMicros() * 1e-6); ctime_ = cached_now; cached_now_access_count = 0; } @@ -79,14 +87,14 @@ void AutoRollLogger::RollLogFile() { // This function is called when log is rotating. Two rotations // can happen quickly (NowMicro returns same value). To not overwrite // previous log file we increment by one micro second and try again. - uint64_t now = env_->NowMicros(); + uint64_t now = clock_->NowMicros(); std::string old_fname; do { old_fname = OldInfoLogFileName( dbname_, now, db_absolute_path_, db_log_dir_); now++; - } while (env_->FileExists(old_fname).ok()); - Status s = env_->RenameFile(log_fname_, old_fname); + } while (fs_->FileExists(old_fname, io_options_, &io_context_).ok()); + Status s = fs_->RenameFile(log_fname_, old_fname, io_options_, &io_context_); if (!s.ok()) { // What should we do on error? } @@ -103,7 +111,7 @@ void AutoRollLogger::GetExistingFiles() { std::string parent_dir; std::vector info_log_files; Status s = - GetInfoLogFiles(env_, db_log_dir_, dbname_, &parent_dir, &info_log_files); + GetInfoLogFiles(fs_, db_log_dir_, dbname_, &parent_dir, &info_log_files); if (status_.ok()) { status_ = s; } @@ -117,7 +125,7 @@ void AutoRollLogger::GetExistingFiles() { } Status AutoRollLogger::TrimOldLogFiles() { - // Here we directly list info files and delete them through Env. + // Here we directly list info files and delete them through FileSystem. // The deletion isn't going through DB, so there are shortcomes: // 1. the deletion is not rate limited by SstFileManager // 2. there is a chance that an I/O will be issued here @@ -130,7 +138,8 @@ Status AutoRollLogger::TrimOldLogFiles() { // it's essentially the same thing, and checking empty before accessing // the queue feels safer. while (!old_log_files_.empty() && old_log_files_.size() >= kKeepLogFileNum) { - Status s = env_->DeleteFile(old_log_files_.front()); + Status s = + fs_->DeleteFile(old_log_files_.front(), io_options_, &io_context_); // Remove the file from the tracking anyway. It's possible that // DB cleaned up the old log file, or people cleaned it up manually. old_log_files_.pop(); @@ -241,7 +250,7 @@ void AutoRollLogger::LogHeader(const char* format, va_list args) { bool AutoRollLogger::LogExpired() { if (cached_now_access_count >= call_NowMicros_every_N_records_) { - cached_now = static_cast(env_->NowMicros() * 1e-6); + cached_now = static_cast(clock_->NowMicros() * 1e-6); cached_now_access_count = 0; } @@ -267,15 +276,16 @@ Status CreateLoggerFromOptions(const std::string& dbname, std::string fname = InfoLogFileName(dbname, db_absolute_path, options.db_log_dir); + const auto& clock = env->GetSystemClock(); env->CreateDirIfMissing(dbname) .PermitUncheckedError(); // In case it does not exist // Currently we only support roll by time-to-roll and log size #ifndef ROCKSDB_LITE if (options.log_file_time_to_roll > 0 || options.max_log_file_size > 0) { AutoRollLogger* result = new AutoRollLogger( - env, dbname, options.db_log_dir, options.max_log_file_size, - options.log_file_time_to_roll, options.keep_log_file_num, - options.info_log_level); + env->GetFileSystem(), clock, dbname, options.db_log_dir, + options.max_log_file_size, options.log_file_time_to_roll, + options.keep_log_file_num, options.info_log_level); s = result->GetStatus(); if (!s.ok()) { delete result; @@ -286,9 +296,9 @@ Status CreateLoggerFromOptions(const std::string& dbname, } #endif // !ROCKSDB_LITE // Open a log file in the same directory as the db - env->RenameFile(fname, - OldInfoLogFileName(dbname, env->NowMicros(), db_absolute_path, - options.db_log_dir)) + env->RenameFile( + fname, OldInfoLogFileName(dbname, clock->NowMicros(), db_absolute_path, + options.db_log_dir)) .PermitUncheckedError(); s = env->NewLogger(fname, logger); if (logger->get() != nullptr) { diff --git a/logging/auto_roll_logger.h b/logging/auto_roll_logger.h index 2b63fc9d9..ccbce1d99 100644 --- a/logging/auto_roll_logger.h +++ b/logging/auto_roll_logger.h @@ -18,14 +18,18 @@ #include "util/mutexlock.h" namespace ROCKSDB_NAMESPACE { +class FileSystem; +class SystemClock; #ifndef ROCKSDB_LITE // Rolls the log file by size and/or time class AutoRollLogger : public Logger { public: - AutoRollLogger(Env* env, const std::string& dbname, - const std::string& db_log_dir, size_t log_max_size, - size_t log_file_time_to_roll, size_t keep_log_file_num, + AutoRollLogger(const std::shared_ptr& fs, + const std::shared_ptr& clock, + const std::string& dbname, const std::string& db_log_dir, + size_t log_max_size, size_t log_file_time_to_roll, + size_t keep_log_file_num, const InfoLogLevel log_level = InfoLogLevel::INFO_LEVEL); using Logger::Logv; @@ -134,7 +138,8 @@ class AutoRollLogger : public Logger { std::string dbname_; std::string db_log_dir_; std::string db_absolute_path_; - Env* env_; + std::shared_ptr fs_; + std::shared_ptr clock_; std::shared_ptr logger_; // current status of the logger Status status_; @@ -148,11 +153,13 @@ class AutoRollLogger : public Logger { // Full path is stored here. It consumes signifianctly more memory // than only storing file name. Can optimize if it causes a problem. std::queue old_log_files_; - // to avoid frequent env->NowMicros() calls, we cached the current time + // to avoid frequent clock->NowMicros() calls, we cached the current time uint64_t cached_now; uint64_t ctime_; uint64_t cached_now_access_count; uint64_t call_NowMicros_every_N_records_; + IOOptions io_options_; + IODebugContext io_context_; mutable port::Mutex mutex_; }; #endif // !ROCKSDB_LITE diff --git a/logging/auto_roll_logger_test.cc b/logging/auto_roll_logger_test.cc index e5d384800..59e0ebac6 100644 --- a/logging/auto_roll_logger_test.cc +++ b/logging/auto_roll_logger_test.cc @@ -7,7 +7,9 @@ #ifndef ROCKSDB_LITE #include "logging/auto_roll_logger.h" + #include + #include #include #include @@ -16,18 +18,24 @@ #include #include #include + #include "logging/logging.h" #include "port/port.h" #include "rocksdb/db.h" +#include "rocksdb/file_system.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" #include "test_util/testutil.h" namespace ROCKSDB_NAMESPACE { namespace { -class NoSleepEnv : public EnvWrapper { +class NoSleepClock : public SystemClockWrapper { public: - NoSleepEnv(Env* base) : EnvWrapper(base) {} + NoSleepClock( + const std::shared_ptr& base = SystemClock::Default()) + : SystemClockWrapper(base) {} + const char* Name() const override { return "NoSleepClock"; } void SleepForMicroseconds(int micros) override { fake_time_ += static_cast(micros); } @@ -75,7 +83,9 @@ class AutoRollLoggerTest : public testing::Test { void RollLogFileBySizeTest(AutoRollLogger* logger, size_t log_max_size, const std::string& log_message); - void RollLogFileByTimeTest(Env*, AutoRollLogger* logger, size_t time, + void RollLogFileByTimeTest(const std::shared_ptr& fs, + const std::shared_ptr& sc, + AutoRollLogger* logger, size_t time, const std::string& log_message); // return list of files under kTestDir that contains "LOG" std::vector GetLogFiles() { @@ -156,21 +166,22 @@ void AutoRollLoggerTest::RollLogFileBySizeTest(AutoRollLogger* logger, ASSERT_TRUE(message_size == logger->GetLogFileSize()); } -void AutoRollLoggerTest::RollLogFileByTimeTest(Env* env, AutoRollLogger* logger, - size_t time, - const std::string& log_message) { +void AutoRollLoggerTest::RollLogFileByTimeTest( + const std::shared_ptr& fs, + const std::shared_ptr& sc, AutoRollLogger* logger, size_t time, + const std::string& log_message) { uint64_t expected_ctime; uint64_t actual_ctime; uint64_t total_log_size; - EXPECT_OK(env->GetFileSize(kLogFile, &total_log_size)); + EXPECT_OK(fs->GetFileSize(kLogFile, IOOptions(), &total_log_size, nullptr)); expected_ctime = logger->TEST_ctime(); logger->SetCallNowMicrosEveryNRecords(0); // -- Write to the log for several times, which is supposed // to be finished before time. for (int i = 0; i < 10; ++i) { - env->SleepForMicroseconds(50000); + sc->SleepForMicroseconds(50000); LogMessage(logger, log_message.c_str()); EXPECT_OK(logger->GetStatus()); // Make sure we always write to the same log file (by @@ -185,7 +196,7 @@ void AutoRollLoggerTest::RollLogFileByTimeTest(Env* env, AutoRollLogger* logger, } // -- Make the log file expire - env->SleepForMicroseconds(static_cast(time * 1000000)); + sc->SleepForMicroseconds(static_cast(time * 1000000)); LogMessage(logger, log_message.c_str()); // At this time, the new log file should be created. @@ -199,15 +210,15 @@ TEST_F(AutoRollLoggerTest, RollLogFileBySize) { size_t log_max_size = 1024 * 5; size_t keep_log_file_num = 10; - AutoRollLogger logger(Env::Default(), kTestDir, "", log_max_size, 0, - keep_log_file_num); + AutoRollLogger logger(FileSystem::Default(), SystemClock::Default(), + kTestDir, "", log_max_size, 0, keep_log_file_num); RollLogFileBySizeTest(&logger, log_max_size, kSampleMessage + ":RollLogFileBySize"); } TEST_F(AutoRollLoggerTest, RollLogFileByTime) { - NoSleepEnv nse(Env::Default()); + auto nsc = std::make_shared(); size_t time = 2; size_t log_size = 1024 * 5; @@ -216,10 +227,11 @@ TEST_F(AutoRollLoggerTest, RollLogFileByTime) { InitTestDb(); // -- Test the existence of file during the server restart. ASSERT_EQ(Status::NotFound(), default_env->FileExists(kLogFile)); - AutoRollLogger logger(&nse, kTestDir, "", log_size, time, keep_log_file_num); + AutoRollLogger logger(default_env->GetFileSystem(), nsc, kTestDir, "", + log_size, time, keep_log_file_num); ASSERT_OK(default_env->FileExists(kLogFile)); - RollLogFileByTimeTest(&nse, &logger, time, + RollLogFileByTimeTest(default_env->GetFileSystem(), nsc, &logger, time, kSampleMessage + ":RollLogFileByTime"); } @@ -254,15 +266,17 @@ TEST_F(AutoRollLoggerTest, OpenLogFilesMultipleTimesWithOptionLog_max_size) { size_t log_size = 1024; size_t keep_log_file_num = 10; - AutoRollLogger* logger = new AutoRollLogger(Env::Default(), kTestDir, "", - log_size, 0, keep_log_file_num); + AutoRollLogger* logger = + new AutoRollLogger(FileSystem::Default(), SystemClock::Default(), + kTestDir, "", log_size, 0, keep_log_file_num); LogMessage(logger, kSampleMessage.c_str()); ASSERT_GT(logger->GetLogFileSize(), kZero); delete logger; // reopens the log file and an empty log file will be created. - logger = new AutoRollLogger(Env::Default(), kTestDir, "", log_size, 0, 10); + logger = new AutoRollLogger(FileSystem::Default(), SystemClock::Default(), + kTestDir, "", log_size, 0, 10); ASSERT_EQ(logger->GetLogFileSize(), kZero); delete logger; } @@ -273,16 +287,16 @@ TEST_F(AutoRollLoggerTest, CompositeRollByTimeAndSizeLogger) { InitTestDb(); - NoSleepEnv nse(Env::Default()); - AutoRollLogger logger(&nse, kTestDir, "", log_max_size, time, - keep_log_file_num); + auto nsc = std::make_shared(); + AutoRollLogger logger(FileSystem::Default(), nsc, kTestDir, "", log_max_size, + time, keep_log_file_num); // Test the ability to roll by size RollLogFileBySizeTest(&logger, log_max_size, kSampleMessage + ":CompositeRollByTimeAndSizeLogger"); // Test the ability to roll by Time - RollLogFileByTimeTest(&nse, &logger, time, + RollLogFileByTimeTest(FileSystem::Default(), nsc, &logger, time, kSampleMessage + ":CompositeRollByTimeAndSizeLogger"); } @@ -291,7 +305,9 @@ TEST_F(AutoRollLoggerTest, CompositeRollByTimeAndSizeLogger) { // port TEST_F(AutoRollLoggerTest, CreateLoggerFromOptions) { DBOptions options; - NoSleepEnv nse(Env::Default()); + auto nsc = std::make_shared(); + std::unique_ptr nse(new CompositeEnvWrapper(Env::Default(), nsc)); + std::shared_ptr logger; // Normal logger @@ -310,14 +326,15 @@ TEST_F(AutoRollLoggerTest, CreateLoggerFromOptions) { kSampleMessage + ":CreateLoggerFromOptions - size"); // Only roll by Time - options.env = &nse; + options.env = nse.get(); InitTestDb(); options.max_log_file_size = 0; options.log_file_time_to_roll = 2; ASSERT_OK(CreateLoggerFromOptions(kTestDir, options, &logger)); auto_roll_logger = dynamic_cast(logger.get()); - RollLogFileByTimeTest(&nse, auto_roll_logger, options.log_file_time_to_roll, + RollLogFileByTimeTest(options.env->GetFileSystem(), nsc, auto_roll_logger, + options.log_file_time_to_roll, kSampleMessage + ":CreateLoggerFromOptions - time"); // roll by both Time and size @@ -329,7 +346,8 @@ TEST_F(AutoRollLoggerTest, CreateLoggerFromOptions) { dynamic_cast(logger.get()); RollLogFileBySizeTest(auto_roll_logger, options.max_log_file_size, kSampleMessage + ":CreateLoggerFromOptions - both"); - RollLogFileByTimeTest(&nse, auto_roll_logger, options.log_file_time_to_roll, + RollLogFileByTimeTest(options.env->GetFileSystem(), nsc, auto_roll_logger, + options.log_file_time_to_roll, kSampleMessage + ":CreateLoggerFromOptions - both"); // Set keep_log_file_num @@ -402,8 +420,8 @@ TEST_F(AutoRollLoggerTest, AutoDeleting) { const size_t kMaxFileSize = 512; { size_t log_num = 8; - AutoRollLogger logger(Env::Default(), dbname, db_log_dir, kMaxFileSize, 0, - log_num); + AutoRollLogger logger(FileSystem::Default(), SystemClock::Default(), + dbname, db_log_dir, kMaxFileSize, 0, log_num); RollNTimesBySize(&logger, log_num, kMaxFileSize); ASSERT_EQ(log_num, GetLogFiles().size()); @@ -411,8 +429,8 @@ TEST_F(AutoRollLoggerTest, AutoDeleting) { // Shrink number of files { size_t log_num = 5; - AutoRollLogger logger(Env::Default(), dbname, db_log_dir, kMaxFileSize, 0, - log_num); + AutoRollLogger logger(FileSystem::Default(), SystemClock::Default(), + dbname, db_log_dir, kMaxFileSize, 0, log_num); ASSERT_EQ(log_num, GetLogFiles().size()); RollNTimesBySize(&logger, 3, kMaxFileSize); @@ -422,8 +440,8 @@ TEST_F(AutoRollLoggerTest, AutoDeleting) { // Increase number of files again. { size_t log_num = 7; - AutoRollLogger logger(Env::Default(), dbname, db_log_dir, kMaxFileSize, 0, - log_num); + AutoRollLogger logger(FileSystem::Default(), SystemClock::Default(), + dbname, db_log_dir, kMaxFileSize, 0, log_num); ASSERT_EQ(6, GetLogFiles().size()); RollNTimesBySize(&logger, 3, kMaxFileSize); @@ -485,7 +503,8 @@ TEST_F(AutoRollLoggerTest, InfoLogLevel) { // an extra-scope to force the AutoRollLogger to flush the log file when it // becomes out of scope. { - AutoRollLogger logger(Env::Default(), kTestDir, "", log_size, 0, 10); + AutoRollLogger logger(FileSystem::Default(), SystemClock::Default(), + kTestDir, "", log_size, 0, 10); for (int log_level = InfoLogLevel::HEADER_LEVEL; log_level >= InfoLogLevel::DEBUG_LEVEL; log_level--) { logger.SetInfoLogLevel((InfoLogLevel)log_level); @@ -523,7 +542,8 @@ TEST_F(AutoRollLoggerTest, Close) { size_t log_size = 8192; size_t log_lines = 0; - AutoRollLogger logger(Env::Default(), kTestDir, "", log_size, 0, 10); + AutoRollLogger logger(FileSystem::Default(), SystemClock::Default(), kTestDir, + "", log_size, 0, 10); for (int log_level = InfoLogLevel::HEADER_LEVEL; log_level >= InfoLogLevel::DEBUG_LEVEL; log_level--) { logger.SetInfoLogLevel((InfoLogLevel)log_level); @@ -590,8 +610,9 @@ TEST_F(AutoRollLoggerTest, LogHeaderTest) { InitTestDb(); - AutoRollLogger logger(Env::Default(), kTestDir, /*db_log_dir=*/"", - LOG_MAX_SIZE, /*log_file_time_to_roll=*/0, + AutoRollLogger logger(FileSystem::Default(), SystemClock::Default(), + kTestDir, /*db_log_dir=*/"", LOG_MAX_SIZE, + /*log_file_time_to_roll=*/0, /*keep_log_file_num=*/10); if (test_num == 0) { diff --git a/logging/env_logger.h b/logging/env_logger.h index 669f69a76..eb0e55df8 100644 --- a/logging/env_logger.h +++ b/logging/env_logger.h @@ -31,7 +31,7 @@ class EnvLogger : public Logger { const std::string& fname, const EnvOptions& options, Env* env, InfoLogLevel log_level = InfoLogLevel::ERROR_LEVEL) : Logger(log_level), - file_(std::move(writable_file), fname, options, env), + file_(std::move(writable_file), fname, options, env->GetSystemClock()), last_flush_micros_(0), env_(env), flush_pending_(false) {} diff --git a/memtable/memtablerep_bench.cc b/memtable/memtablerep_bench.cc index 0f6203042..68d6727fa 100644 --- a/memtable/memtablerep_bench.cc +++ b/memtable/memtablerep_bench.cc @@ -31,6 +31,7 @@ int main() { #include "rocksdb/memtablerep.h" #include "rocksdb/options.h" #include "rocksdb/slice_transform.h" +#include "rocksdb/system_clock.h" #include "rocksdb/write_buffer_manager.h" #include "test_util/testutil.h" #include "util/gflags_compat.h" @@ -417,7 +418,7 @@ class Benchmark { uint64_t bytes_written = 0; uint64_t bytes_read = 0; uint64_t read_hits = 0; - StopWatchNano timer(Env::Default(), true); + StopWatchNano timer(SystemClock::Default(), true); RunThreads(&threads, &bytes_written, &bytes_read, true, &read_hits); auto elapsed_time = static_cast(timer.ElapsedNanos() / 1000); std::cout << "Elapsed time: " << static_cast(elapsed_time) << " us" diff --git a/memtable/write_buffer_manager_test.cc b/memtable/write_buffer_manager_test.cc index e9377a2a7..f58320588 100644 --- a/memtable/write_buffer_manager_test.cc +++ b/memtable/write_buffer_manager_test.cc @@ -11,10 +11,11 @@ #include "test_util/testharness.h" namespace ROCKSDB_NAMESPACE { -const size_t kSizeDummyEntry = 256 * 1024; class WriteBufferManagerTest : public testing::Test {}; #ifndef ROCKSDB_LITE +const size_t kSizeDummyEntry = 256 * 1024; + TEST_F(WriteBufferManagerTest, ShouldFlush) { // A write buffer manager of size 10MB std::unique_ptr wbf( diff --git a/monitoring/histogram_test.cc b/monitoring/histogram_test.cc index 3d0233a2b..fd7c00437 100644 --- a/monitoring/histogram_test.cc +++ b/monitoring/histogram_test.cc @@ -8,6 +8,7 @@ #include #include "monitoring/histogram_windowing.h" +#include "rocksdb/system_clock.h" #include "test_util/mock_time_env.h" #include "test_util/testharness.h" #include "util/random.h" @@ -19,7 +20,8 @@ class HistogramTest : public testing::Test {}; namespace { const double kIota = 0.1; const HistogramBucketMapper bucketMapper; - MockTimeEnv* env = new MockTimeEnv(Env::Default()); + std::shared_ptr clock = + std::make_shared(SystemClock::Default()); } void PopulateHistogram(Histogram& histogram, @@ -29,11 +31,11 @@ void PopulateHistogram(Histogram& histogram, for (uint64_t i = low; i <= high; i++) { histogram.Add(i); // sleep a random microseconds [0-10) - env->MockSleepForMicroseconds(rnd.Uniform(10)); + clock->MockSleepForMicroseconds(rnd.Uniform(10)); } } // make sure each data population at least take some time - env->MockSleepForMicroseconds(1); + clock->MockSleepForMicroseconds(1); } void BasicOperation(Histogram& histogram) { @@ -139,23 +141,23 @@ TEST_F(HistogramTest, HistogramWindowingExpire) { HistogramWindowingImpl histogramWindowing(num_windows, micros_per_window, min_num_per_window); - histogramWindowing.TEST_UpdateEnv(env); + histogramWindowing.TEST_UpdateClock(clock); PopulateHistogram(histogramWindowing, 1, 1, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); ASSERT_EQ(histogramWindowing.num(), 100); ASSERT_EQ(histogramWindowing.min(), 1); ASSERT_EQ(histogramWindowing.max(), 1); ASSERT_EQ(histogramWindowing.Average(), 1); PopulateHistogram(histogramWindowing, 2, 2, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); ASSERT_EQ(histogramWindowing.num(), 200); ASSERT_EQ(histogramWindowing.min(), 1); ASSERT_EQ(histogramWindowing.max(), 2); ASSERT_EQ(histogramWindowing.Average(), 1.5); PopulateHistogram(histogramWindowing, 3, 3, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); ASSERT_EQ(histogramWindowing.num(), 300); ASSERT_EQ(histogramWindowing.min(), 1); ASSERT_EQ(histogramWindowing.max(), 3); @@ -163,7 +165,7 @@ TEST_F(HistogramTest, HistogramWindowingExpire) { // dropping oldest window with value 1, remaining 2 ~ 4 PopulateHistogram(histogramWindowing, 4, 4, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); ASSERT_EQ(histogramWindowing.num(), 300); ASSERT_EQ(histogramWindowing.min(), 2); ASSERT_EQ(histogramWindowing.max(), 4); @@ -171,7 +173,7 @@ TEST_F(HistogramTest, HistogramWindowingExpire) { // dropping oldest window with value 2, remaining 3 ~ 5 PopulateHistogram(histogramWindowing, 5, 5, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); ASSERT_EQ(histogramWindowing.num(), 300); ASSERT_EQ(histogramWindowing.min(), 3); ASSERT_EQ(histogramWindowing.max(), 5); @@ -187,20 +189,20 @@ TEST_F(HistogramTest, HistogramWindowingMerge) { histogramWindowing(num_windows, micros_per_window, min_num_per_window); HistogramWindowingImpl otherWindowing(num_windows, micros_per_window, min_num_per_window); - histogramWindowing.TEST_UpdateEnv(env); - otherWindowing.TEST_UpdateEnv(env); + histogramWindowing.TEST_UpdateClock(clock); + otherWindowing.TEST_UpdateClock(clock); PopulateHistogram(histogramWindowing, 1, 1, 100); PopulateHistogram(otherWindowing, 1, 1, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); PopulateHistogram(histogramWindowing, 2, 2, 100); PopulateHistogram(otherWindowing, 2, 2, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); PopulateHistogram(histogramWindowing, 3, 3, 100); PopulateHistogram(otherWindowing, 3, 3, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); histogramWindowing.Merge(otherWindowing); ASSERT_EQ(histogramWindowing.num(), 600); @@ -210,14 +212,14 @@ TEST_F(HistogramTest, HistogramWindowingMerge) { // dropping oldest window with value 1, remaining 2 ~ 4 PopulateHistogram(histogramWindowing, 4, 4, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); ASSERT_EQ(histogramWindowing.num(), 500); ASSERT_EQ(histogramWindowing.min(), 2); ASSERT_EQ(histogramWindowing.max(), 4); // dropping oldest window with value 2, remaining 3 ~ 5 PopulateHistogram(histogramWindowing, 5, 5, 100); - env->MockSleepForMicroseconds(micros_per_window); + clock->MockSleepForMicroseconds(micros_per_window); ASSERT_EQ(histogramWindowing.num(), 400); ASSERT_EQ(histogramWindowing.min(), 3); ASSERT_EQ(histogramWindowing.max(), 5); diff --git a/monitoring/histogram_windowing.cc b/monitoring/histogram_windowing.cc index e114a6686..f31bbe06a 100644 --- a/monitoring/histogram_windowing.cc +++ b/monitoring/histogram_windowing.cc @@ -8,15 +8,17 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. #include "monitoring/histogram_windowing.h" -#include "monitoring/histogram.h" -#include "util/cast_util.h" #include +#include "monitoring/histogram.h" +#include "rocksdb/system_clock.h" +#include "util/cast_util.h" + namespace ROCKSDB_NAMESPACE { HistogramWindowingImpl::HistogramWindowingImpl() { - env_ = Env::Default(); + clock_ = SystemClock::Default(); window_stats_.reset(new HistogramStat[static_cast(num_windows_)]); Clear(); } @@ -28,7 +30,7 @@ HistogramWindowingImpl::HistogramWindowingImpl( num_windows_(num_windows), micros_per_window_(micros_per_window), min_num_per_window_(min_num_per_window) { - env_ = Env::Default(); + clock_ = SystemClock::Default(); window_stats_.reset(new HistogramStat[static_cast(num_windows_)]); Clear(); } @@ -44,7 +46,7 @@ void HistogramWindowingImpl::Clear() { window_stats_[i].Clear(); } current_window_.store(0, std::memory_order_relaxed); - last_swap_time_.store(env_->NowMicros(), std::memory_order_relaxed); + last_swap_time_.store(clock_->NowMicros(), std::memory_order_relaxed); } bool HistogramWindowingImpl::Empty() const { return stats_.Empty(); } @@ -129,7 +131,7 @@ void HistogramWindowingImpl::Data(HistogramData * const data) const { } void HistogramWindowingImpl::TimerTick() { - uint64_t curr_time = env_->NowMicros(); + uint64_t curr_time = clock_->NowMicros(); size_t curr_window_ = static_cast(current_window()); if (curr_time - last_swap_time() > micros_per_window_ && window_stats_[curr_window_].num() >= min_num_per_window_) { @@ -144,7 +146,7 @@ void HistogramWindowingImpl::SwapHistoryBucket() { // If mutex is held by Merge() or Clear(), next Add() will take care of the // swap, if needed. if (mutex_.try_lock()) { - last_swap_time_.store(env_->NowMicros(), std::memory_order_relaxed); + last_swap_time_.store(clock_->NowMicros(), std::memory_order_relaxed); uint64_t curr_window = current_window(); uint64_t next_window = (curr_window == num_windows_ - 1) ? diff --git a/monitoring/histogram_windowing.h b/monitoring/histogram_windowing.h index 12db237a1..f8da07b36 100644 --- a/monitoring/histogram_windowing.h +++ b/monitoring/histogram_windowing.h @@ -10,9 +10,9 @@ #pragma once #include "monitoring/histogram.h" -#include "rocksdb/env.h" namespace ROCKSDB_NAMESPACE { +class SystemClock; class HistogramWindowingImpl : public Histogram { @@ -45,7 +45,9 @@ public: virtual void Data(HistogramData* const data) const override; #ifndef NDEBUG - void TEST_UpdateEnv(Env* env) { env_ = env; } + void TEST_UpdateClock(const std::shared_ptr& clock) { + clock_ = clock; + } #endif // NDEBUG private: @@ -58,7 +60,7 @@ public: return last_swap_time_.load(std::memory_order_relaxed); } - Env* env_; + std::shared_ptr clock_; std::mutex mutex_; // Aggregated stats over windows_stats_, all the computation is done diff --git a/monitoring/instrumented_mutex.cc b/monitoring/instrumented_mutex.cc index d82e594c1..989c091c3 100644 --- a/monitoring/instrumented_mutex.cc +++ b/monitoring/instrumented_mutex.cc @@ -4,15 +4,18 @@ // (found in the LICENSE.Apache file in the root directory). #include "monitoring/instrumented_mutex.h" + #include "monitoring/perf_context_imp.h" #include "monitoring/thread_status_util.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" namespace ROCKSDB_NAMESPACE { namespace { #ifndef NPERF_CONTEXT -Statistics* stats_for_report(Env* env, Statistics* stats) { - if (env != nullptr && stats != nullptr && +Statistics* stats_for_report(const std::shared_ptr& clock, + Statistics* stats) { + if (clock.get() != nullptr && stats != nullptr && stats->get_stats_level() > kExceptTimeForMutex) { return stats; } else { @@ -25,7 +28,7 @@ Statistics* stats_for_report(Env* env, Statistics* stats) { void InstrumentedMutex::Lock() { PERF_CONDITIONAL_TIMER_FOR_MUTEX_GUARD( db_mutex_lock_nanos, stats_code_ == DB_MUTEX_WAIT_MICROS, - stats_for_report(env_, stats_), stats_code_); + stats_for_report(clock_, stats_), stats_code_); LockInternal(); } @@ -39,7 +42,7 @@ void InstrumentedMutex::LockInternal() { void InstrumentedCondVar::Wait() { PERF_CONDITIONAL_TIMER_FOR_MUTEX_GUARD( db_condition_wait_nanos, stats_code_ == DB_MUTEX_WAIT_MICROS, - stats_for_report(env_, stats_), stats_code_); + stats_for_report(clock_, stats_), stats_code_); WaitInternal(); } @@ -53,7 +56,7 @@ void InstrumentedCondVar::WaitInternal() { bool InstrumentedCondVar::TimedWait(uint64_t abs_time_us) { PERF_CONDITIONAL_TIMER_FOR_MUTEX_GUARD( db_condition_wait_nanos, stats_code_ == DB_MUTEX_WAIT_MICROS, - stats_for_report(env_, stats_), stats_code_); + stats_for_report(clock_, stats_), stats_code_); return TimedWaitInternal(abs_time_us); } diff --git a/monitoring/instrumented_mutex.h b/monitoring/instrumented_mutex.h index 50c1f29c8..bd68825cc 100644 --- a/monitoring/instrumented_mutex.h +++ b/monitoring/instrumented_mutex.h @@ -7,8 +7,8 @@ #include "monitoring/statistics.h" #include "port/port.h" -#include "rocksdb/env.h" #include "rocksdb/statistics.h" +#include "rocksdb/system_clock.h" #include "rocksdb/thread_status.h" #include "util/stop_watch.h" @@ -20,13 +20,18 @@ class InstrumentedCondVar; class InstrumentedMutex { public: explicit InstrumentedMutex(bool adaptive = false) - : mutex_(adaptive), stats_(nullptr), env_(nullptr), - stats_code_(0) {} - - InstrumentedMutex( - Statistics* stats, Env* env, - int stats_code, bool adaptive = false) - : mutex_(adaptive), stats_(stats), env_(env), + : mutex_(adaptive), stats_(nullptr), clock_(nullptr), stats_code_(0) {} + + explicit InstrumentedMutex(const std::shared_ptr& clock, + bool adaptive = false) + : mutex_(adaptive), stats_(nullptr), clock_(clock), stats_code_(0) {} + + InstrumentedMutex(Statistics* stats, + const std::shared_ptr& clock, int stats_code, + bool adaptive = false) + : mutex_(adaptive), + stats_(stats), + clock_(clock), stats_code_(stats_code) {} void Lock(); @@ -44,7 +49,7 @@ class InstrumentedMutex { friend class InstrumentedCondVar; port::Mutex mutex_; Statistics* stats_; - Env* env_; + std::shared_ptr clock_; int stats_code_; }; @@ -71,7 +76,7 @@ class InstrumentedCondVar { explicit InstrumentedCondVar(InstrumentedMutex* instrumented_mutex) : cond_(&(instrumented_mutex->mutex_)), stats_(instrumented_mutex->stats_), - env_(instrumented_mutex->env_), + clock_(instrumented_mutex->clock_), stats_code_(instrumented_mutex->stats_code_) {} void Wait(); @@ -91,7 +96,7 @@ class InstrumentedCondVar { bool TimedWaitInternal(uint64_t abs_time_us); port::CondVar cond_; Statistics* stats_; - Env* env_; + const std::shared_ptr clock_; int stats_code_; }; diff --git a/monitoring/iostats_context_imp.h b/monitoring/iostats_context_imp.h index a7f095d6e..92a3f7126 100644 --- a/monitoring/iostats_context_imp.h +++ b/monitoring/iostats_context_imp.h @@ -38,9 +38,9 @@ extern __thread IOStatsContext iostats_context; iostats_step_timer_##metric.Start(); // Declare and set start time of the timer -#define IOSTATS_CPU_TIMER_GUARD(metric, env) \ +#define IOSTATS_CPU_TIMER_GUARD(metric, clock) \ PerfStepTimer iostats_step_timer_##metric( \ - &(iostats_context.metric), env, true, \ + &(iostats_context.metric), clock, true, \ PerfLevel::kEnableTimeAndCPUTimeExceptForMutex); \ iostats_step_timer_##metric.Start(); @@ -55,6 +55,6 @@ extern __thread IOStatsContext iostats_context; #define IOSTATS(metric) 0 #define IOSTATS_TIMER_GUARD(metric) -#define IOSTATS_CPU_TIMER_GUARD(metric, env) static_cast(env) +#define IOSTATS_CPU_TIMER_GUARD(metric, clock) static_cast(clock) #endif // ROCKSDB_SUPPORT_THREAD_LOCAL diff --git a/monitoring/perf_context_imp.h b/monitoring/perf_context_imp.h index cdca27621..b7a56adef 100644 --- a/monitoring/perf_context_imp.h +++ b/monitoring/perf_context_imp.h @@ -25,8 +25,8 @@ extern thread_local PerfContext perf_context; #define PERF_TIMER_STOP(metric) #define PERF_TIMER_START(metric) #define PERF_TIMER_GUARD(metric) -#define PERF_TIMER_GUARD_WITH_ENV(metric, env) -#define PERF_CPU_TIMER_GUARD(metric, env) +#define PERF_TIMER_GUARD_WITH_CLOCK(metric, clock) +#define PERF_CPU_TIMER_GUARD(metric, clock) #define PERF_CONDITIONAL_TIMER_FOR_MUTEX_GUARD(metric, condition, stats, \ ticker_type) #define PERF_TIMER_MEASURE(metric) @@ -46,14 +46,14 @@ extern thread_local PerfContext perf_context; perf_step_timer_##metric.Start(); // Declare and set start time of the timer -#define PERF_TIMER_GUARD_WITH_ENV(metric, env) \ - PerfStepTimer perf_step_timer_##metric(&(perf_context.metric), env); \ +#define PERF_TIMER_GUARD_WITH_CLOCK(metric, clock) \ + PerfStepTimer perf_step_timer_##metric(&(perf_context.metric), clock); \ perf_step_timer_##metric.Start(); // Declare and set start time of the timer -#define PERF_CPU_TIMER_GUARD(metric, env) \ +#define PERF_CPU_TIMER_GUARD(metric, clock) \ PerfStepTimer perf_step_timer_##metric( \ - &(perf_context.metric), env, true, \ + &(perf_context.metric), clock, true, \ PerfLevel::kEnableTimeAndCPUTimeExceptForMutex); \ perf_step_timer_##metric.Start(); diff --git a/monitoring/perf_step_timer.h b/monitoring/perf_step_timer.h index f2d35d9d6..9f1adbda5 100644 --- a/monitoring/perf_step_timer.h +++ b/monitoring/perf_step_timer.h @@ -5,22 +5,23 @@ // #pragma once #include "monitoring/perf_level_imp.h" -#include "rocksdb/env.h" -#include "util/stop_watch.h" +#include "monitoring/statistics.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { class PerfStepTimer { public: explicit PerfStepTimer( - uint64_t* metric, Env* env = nullptr, bool use_cpu_time = false, + uint64_t* metric, const std::shared_ptr& clock = nullptr, + bool use_cpu_time = false, PerfLevel enable_level = PerfLevel::kEnableTimeExceptForMutex, Statistics* statistics = nullptr, uint32_t ticker_type = 0) : perf_counter_enabled_(perf_level >= enable_level), use_cpu_time_(use_cpu_time), - env_((perf_counter_enabled_ || statistics != nullptr) - ? ((env != nullptr) ? env : Env::Default()) - : nullptr), + clock_((perf_counter_enabled_ || statistics != nullptr) + ? ((clock.get() != nullptr) ? clock : SystemClock::Default()) + : nullptr), start_(0), metric_(metric), statistics_(statistics), @@ -36,14 +37,6 @@ class PerfStepTimer { } } - uint64_t time_now() { - if (!use_cpu_time_) { - return env_->NowNanos(); - } else { - return env_->NowCPUNanos(); - } - } - void Measure() { if (start_) { uint64_t now = time_now(); @@ -67,9 +60,17 @@ class PerfStepTimer { } private: + uint64_t time_now() { + if (!use_cpu_time_) { + return clock_->NowNanos(); + } else { + return clock_->CPUNanos(); + } + } + const bool perf_counter_enabled_; const bool use_cpu_time_; - Env* const env_; + std::shared_ptr clock_; uint64_t start_; uint64_t* metric_; Statistics* statistics_; diff --git a/monitoring/stats_history_test.cc b/monitoring/stats_history_test.cc index 7a1df6128..391f7c442 100644 --- a/monitoring/stats_history_test.cc +++ b/monitoring/stats_history_test.cc @@ -22,6 +22,7 @@ #include "rocksdb/cache.h" #include "rocksdb/convenience.h" #include "rocksdb/rate_limiter.h" +#include "test_util/mock_time_env.h" #include "test_util/sync_point.h" #include "test_util/testutil.h" #include "util/random.h" @@ -33,20 +34,22 @@ class StatsHistoryTest : public DBTestBase { public: StatsHistoryTest() : DBTestBase("/stats_history_test", /*env_do_fsync=*/true) { - mock_env_.reset(new MockTimeEnv(env_)); + mock_clock_ = std::make_shared(env_->GetSystemClock()); + mock_env_.reset(new CompositeEnvWrapper(env_, mock_clock_)); } protected: - std::unique_ptr mock_env_; + std::shared_ptr mock_clock_; + std::unique_ptr mock_env_; void SetUp() override { - mock_env_->InstallTimedWaitFixCallback(); + mock_clock_->InstallTimedWaitFixCallback(); SyncPoint::GetInstance()->SetCallBack( "DBImpl::StartPeriodicWorkScheduler:Init", [&](void* arg) { auto* periodic_work_scheduler_ptr = reinterpret_cast(arg); *periodic_work_scheduler_ptr = - PeriodicWorkTestScheduler::Default(mock_env_.get()); + PeriodicWorkTestScheduler::Default(mock_clock_); }); } }; @@ -66,17 +69,17 @@ TEST_F(StatsHistoryTest, RunStatsDumpPeriodSec) { // Wait for the first stats persist to finish, as the initial delay could be // different. dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); ASSERT_GE(counter, 1); // Test cancel job through SetOptions ASSERT_OK(dbfull()->SetDBOptions({{"stats_dump_period_sec", "0"}})); int old_val = counter; for (int i = 1; i < 20; ++i) { - mock_env_->MockSleepForSeconds(kPeriodSec); + mock_clock_->MockSleepForSeconds(kPeriodSec); } ASSERT_EQ(counter, old_val); Close(); @@ -98,17 +101,17 @@ TEST_F(StatsHistoryTest, StatsPersistScheduling) { // Wait for the first stats persist to finish, as the initial delay could be // different. dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); ASSERT_GE(counter, 1); // Test cancel job through SetOptions ASSERT_OK(dbfull()->SetDBOptions({{"stats_persist_period_sec", "0"}})); int old_val = counter; dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec * 2); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec * 2); }); ASSERT_EQ(counter, old_val); Close(); @@ -130,7 +133,7 @@ TEST_F(StatsHistoryTest, PersistentStatsFreshInstall) { ASSERT_EQ(kPeriodSec, dbfull()->GetDBOptions().stats_persist_period_sec); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); ASSERT_GE(counter, 1); Close(); } @@ -149,30 +152,31 @@ TEST_F(StatsHistoryTest, GetStatsHistoryInMemory) { // make sure the first stats persist to finish dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); // Wait for stats persist to finish dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); std::unique_ptr stats_iter; - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds() + 1, &stats_iter)); + ASSERT_OK( + db_->GetStatsHistory(0, mock_clock_->NowSeconds() + 1, &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); // disabled stats snapshots ASSERT_OK(dbfull()->SetDBOptions({{"stats_persist_period_sec", "0"}})); size_t stats_count = 0; for (; stats_iter->Valid(); stats_iter->Next()) { auto stats_map = stats_iter->GetStatsMap(); - ASSERT_EQ(stats_iter->GetStatsTime(), mock_env_->NowSeconds()); + ASSERT_EQ(stats_iter->GetStatsTime(), mock_clock_->NowSeconds()); stats_count += stats_map.size(); } ASSERT_GT(stats_count, 0); // Wait a bit and verify no more stats are found for (int i = 0; i < 10; ++i) { dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(1); }); + [&] { mock_clock_->MockSleepForSeconds(1); }); } - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds(), &stats_iter)); + ASSERT_OK(db_->GetStatsHistory(0, mock_clock_->NowSeconds(), &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); size_t stats_count_new = 0; for (; stats_iter->Valid(); stats_iter->Next()) { @@ -225,11 +229,12 @@ TEST_F(StatsHistoryTest, InMemoryStatsHistoryPurging) { const int kIterations = 10; for (int i = 0; i < kIterations; ++i) { dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); } std::unique_ptr stats_iter; - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds() + 1, &stats_iter)); + ASSERT_OK( + db_->GetStatsHistory(0, mock_clock_->NowSeconds() + 1, &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); size_t stats_count = 0; int slice_count = 0; @@ -248,10 +253,11 @@ TEST_F(StatsHistoryTest, InMemoryStatsHistoryPurging) { // Wait for stats persist to finish for (int i = 0; i < kIterations; ++i) { dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); } - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds() + 1, &stats_iter)); + ASSERT_OK( + db_->GetStatsHistory(0, mock_clock_->NowSeconds() + 1, &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); size_t stats_count_reopen = 0; slice_count = 0; @@ -296,11 +302,11 @@ TEST_F(StatsHistoryTest, GetStatsHistoryFromDisk) { // Wait for the first stats persist to finish, as the initial delay could be // different. dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); // Wait for stats persist to finish dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); auto iter = db_->NewIterator(ReadOptions(), dbfull()->PersistentStatsColumnFamily()); @@ -308,14 +314,14 @@ TEST_F(StatsHistoryTest, GetStatsHistoryFromDisk) { delete iter; dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); iter = db_->NewIterator(ReadOptions(), dbfull()->PersistentStatsColumnFamily()); int key_count2 = countkeys(iter); delete iter; dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); iter = db_->NewIterator(ReadOptions(), dbfull()->PersistentStatsColumnFamily()); int key_count3 = countkeys(iter); @@ -324,7 +330,8 @@ TEST_F(StatsHistoryTest, GetStatsHistoryFromDisk) { ASSERT_GE(key_count3, key_count2); ASSERT_EQ(key_count3 - key_count2, key_count2 - key_count1); std::unique_ptr stats_iter; - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds() + 1, &stats_iter)); + ASSERT_OK( + db_->GetStatsHistory(0, mock_clock_->NowSeconds() + 1, &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); size_t stats_count = 0; int slice_count = 0; @@ -345,7 +352,8 @@ TEST_F(StatsHistoryTest, GetStatsHistoryFromDisk) { ASSERT_EQ(stats_count, key_count3 - 2); // verify reopen will not cause data loss ReopenWithColumnFamilies({"default", "pikachu"}, options); - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds() + 1, &stats_iter)); + ASSERT_OK( + db_->GetStatsHistory(0, mock_clock_->NowSeconds() + 1, &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); size_t stats_count_reopen = 0; int slice_count_reopen = 0; @@ -387,37 +395,38 @@ TEST_F(StatsHistoryTest, PersitentStatsVerifyValue) { // Wait for the first stats persist to finish, as the initial delay could be // different. dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); // Wait for stats persist to finish dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); auto iter = db_->NewIterator(ReadOptions(), dbfull()->PersistentStatsColumnFamily()); countkeys(iter); delete iter; dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); iter = db_->NewIterator(ReadOptions(), dbfull()->PersistentStatsColumnFamily()); countkeys(iter); delete iter; dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); iter = db_->NewIterator(ReadOptions(), dbfull()->PersistentStatsColumnFamily()); countkeys(iter); delete iter; dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); std::map stats_map_after; ASSERT_TRUE(options.statistics->getTickerMap(&stats_map_after)); std::unique_ptr stats_iter; - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds() + 1, &stats_iter)); + ASSERT_OK( + db_->GetStatsHistory(0, mock_clock_->NowSeconds() + 1, &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); std::string sample = "rocksdb.num.iterator.deleted"; uint64_t recovered_value = 0; @@ -434,7 +443,8 @@ TEST_F(StatsHistoryTest, PersitentStatsVerifyValue) { // test stats value retains after recovery ReopenWithColumnFamilies({"default", "pikachu"}, options); - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds() + 1, &stats_iter)); + ASSERT_OK( + db_->GetStatsHistory(0, mock_clock_->NowSeconds() + 1, &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); uint64_t new_recovered_value = 0; for (int i = 2; stats_iter->Valid(); stats_iter->Next(), i++) { @@ -474,10 +484,10 @@ TEST_F(StatsHistoryTest, PersistentStatsCreateColumnFamilies) { // make sure the first stats persist to finish dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); auto iter = db_->NewIterator(ReadOptions(), dbfull()->PersistentStatsColumnFamily()); int key_count = countkeys(iter); @@ -486,7 +496,7 @@ TEST_F(StatsHistoryTest, PersistentStatsCreateColumnFamilies) { uint64_t num_write_wal = 0; std::string sample = "rocksdb.write.wal"; std::unique_ptr stats_iter; - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds(), &stats_iter)); + ASSERT_OK(db_->GetStatsHistory(0, mock_clock_->NowSeconds(), &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); for (; stats_iter->Valid(); stats_iter->Next()) { auto stats_map = stats_iter->GetStatsMap(); @@ -522,7 +532,7 @@ TEST_F(StatsHistoryTest, PersistentStatsCreateColumnFamilies) { ASSERT_NOK(db_->CreateColumnFamily(cf_opts, kPersistentStatsColumnFamilyName, &handle)); // verify stats is not affected by prior failed CF creation - ASSERT_OK(db_->GetStatsHistory(0, mock_env_->NowSeconds(), &stats_iter)); + ASSERT_OK(db_->GetStatsHistory(0, mock_clock_->NowSeconds(), &stats_iter)); ASSERT_TRUE(stats_iter != nullptr); num_write_wal = 0; for (; stats_iter->Valid(); stats_iter->Next()) { @@ -574,7 +584,7 @@ TEST_F(StatsHistoryTest, ForceManualFlushStatsCF) { // Wait for the first stats persist to finish, as the initial delay could be // different. dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec - 1); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec - 1); }); ColumnFamilyData* cfd_default = static_cast(dbfull()->DefaultColumnFamily()) @@ -593,7 +603,7 @@ TEST_F(StatsHistoryTest, ForceManualFlushStatsCF) { ASSERT_EQ("v0", Get(1, "Eevee")); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); // writing to all three cf, flush default cf // LogNumbers: default: 14, stats: 4, pikachu: 4 ASSERT_OK(Flush()); @@ -618,7 +628,7 @@ TEST_F(StatsHistoryTest, ForceManualFlushStatsCF) { ASSERT_EQ("v2", Get("foo2")); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); // writing to default and stats cf, flushing default cf // LogNumbers: default: 19, stats: 19, pikachu: 19 ASSERT_OK(Flush()); @@ -633,7 +643,7 @@ TEST_F(StatsHistoryTest, ForceManualFlushStatsCF) { ASSERT_EQ("v3", Get(1, "Jolteon")); dbfull()->TEST_WaitForStatsDumpRun( - [&] { mock_env_->MockSleepForSeconds(kPeriodSec); }); + [&] { mock_clock_->MockSleepForSeconds(kPeriodSec); }); // writing to all three cf, flushing test cf // LogNumbers: default: 19, stats: 19, pikachu: 22 ASSERT_OK(Flush(1)); diff --git a/monitoring/thread_status_updater.cc b/monitoring/thread_status_updater.cc index 7e4b299a8..267a0c0b0 100644 --- a/monitoring/thread_status_updater.cc +++ b/monitoring/thread_status_updater.cc @@ -4,9 +4,12 @@ // (found in the LICENSE.Apache file in the root directory). #include "monitoring/thread_status_updater.h" + #include + #include "port/likely.h" #include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "util/mutexlock.h" namespace ROCKSDB_NAMESPACE { @@ -159,7 +162,7 @@ Status ThreadStatusUpdater::GetThreadList( std::vector* thread_list) { thread_list->clear(); std::vector> valid_list; - uint64_t now_micros = Env::Default()->NowMicros(); + uint64_t now_micros = SystemClock::Default()->NowMicros(); std::lock_guard lck(thread_list_mutex_); for (auto* thread_data : thread_data_set_) { diff --git a/monitoring/thread_status_util.cc b/monitoring/thread_status_util.cc index 13a79163c..37fcb9f48 100644 --- a/monitoring/thread_status_util.cc +++ b/monitoring/thread_status_util.cc @@ -7,6 +7,7 @@ #include "monitoring/thread_status_updater.h" #include "rocksdb/env.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { @@ -57,7 +58,7 @@ void ThreadStatusUtil::SetThreadOperation(ThreadStatus::OperationType op) { } if (op != ThreadStatus::OP_UNKNOWN) { - uint64_t current_time = Env::Default()->NowMicros(); + uint64_t current_time = SystemClock::Default()->NowMicros(); thread_updater_local_cache_->SetOperationStartTime(current_time); } else { // TDOO(yhchiang): we could report the time when we set operation to diff --git a/monitoring/thread_status_util_debug.cc b/monitoring/thread_status_util_debug.cc index 375fe8c0a..c493ddca5 100644 --- a/monitoring/thread_status_util_debug.cc +++ b/monitoring/thread_status_util_debug.cc @@ -7,7 +7,7 @@ #include "monitoring/thread_status_updater.h" #include "monitoring/thread_status_util.h" -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { @@ -23,7 +23,7 @@ void ThreadStatusUtil::TEST_SetStateDelay( void ThreadStatusUtil::TEST_StateDelay(const ThreadStatus::StateType state) { auto delay = states_delay[state].load(std::memory_order_relaxed); if (delay > 0) { - Env::Default()->SleepForMicroseconds(delay); + SystemClock::Default()->SleepForMicroseconds(delay); } } diff --git a/port/win/env_win.cc b/port/win/env_win.cc index 1e6e9d65b..05bab78b2 100644 --- a/port/win/env_win.cc +++ b/port/win/env_win.cc @@ -93,10 +93,6 @@ WinClock::WinClock() } } -const std::shared_ptr& WinClock::Default() { - static std::shared_ptr clock = std::make_shared(); - return clock; -} void WinClock::SleepForMicroseconds(int micros) { std::this_thread::sleep_for(std::chrono::microseconds(micros)); } @@ -186,7 +182,7 @@ Status WinClock::GetCurrentTime(int64_t* unix_time) { return Status::OK(); } -WinFileSystem::WinFileSystem(const std::shared_ptr& clock) +WinFileSystem::WinFileSystem(const std::shared_ptr& clock) : clock_(clock), page_size_(4 * 1024), allocation_granularity_(page_size_) { SYSTEM_INFO sinfo; GetSystemInfo(&sinfo); @@ -1343,11 +1339,10 @@ void WinEnvThreads::IncBackgroundThreadsIfNeeded(int num, Env::Priority pri) { // WinEnv WinEnv::WinEnv() - : CompositeEnv(WinFileSystem::Default()), + : CompositeEnv(WinFileSystem::Default(), WinClock::Default()), winenv_io_(this), winenv_threads_(this) { // Protected member of the base class - clock_ = WinClock::Default(); thread_status_updater_ = CreateThreadStatusUpdater(); } @@ -1362,22 +1357,9 @@ Status WinEnv::GetThreadList(std::vector* thread_list) { return thread_status_updater_->GetThreadList(thread_list); } -Status WinEnv::GetCurrentTime(int64_t* unix_time) { - return clock_->GetCurrentTime(unix_time); -} - -uint64_t WinEnv::NowMicros() { return clock_->NowMicros(); } - -uint64_t WinEnv::NowNanos() { return clock_->NowNanos(); } - Status WinEnv::GetHostName(char* name, uint64_t len) { return winenv_io_.GetHostName(name, len); } - -std::string WinEnv::TimeToString(uint64_t secondsSince1970) { - return clock_->TimeToString(secondsSince1970); -} - void WinEnv::Schedule(void (*function)(void*), void* arg, Env::Priority pri, void* tag, void (*unschedFunction)(void* arg)) { return winenv_threads_.Schedule(function, arg, pri, tag, unschedFunction); @@ -1399,10 +1381,6 @@ unsigned int WinEnv::GetThreadPoolQueueLen(Env::Priority pri) const { uint64_t WinEnv::GetThreadID() const { return winenv_threads_.GetThreadID(); } -void WinEnv::SleepForMicroseconds(int micros) { - return clock_->SleepForMicroseconds(micros); -} - // Allow increasing the number of worker threads. void WinEnv::SetBackgroundThreads(int num, Env::Priority pri) { return winenv_threads_.SetBackgroundThreads(num, pri); @@ -1441,6 +1419,12 @@ std::shared_ptr FileSystem::Default() { return port::WinFileSystem::Default(); } +const std::shared_ptr& SystemClock::Default() { + static std::shared_ptr clock = + std::make_shared(); + return clock; +} + std::unique_ptr NewCompositeEnv(const std::shared_ptr& fs) { return std::unique_ptr(new CompositeEnvWrapper(Env::Default(), fs)); } diff --git a/port/win/env_win.h b/port/win/env_win.h index da4e955ac..54d3e7dbf 100644 --- a/port/win/env_win.h +++ b/port/win/env_win.h @@ -26,6 +26,7 @@ #include "port/win/win_thread.h" #include "rocksdb/env.h" #include "rocksdb/file_system.h" +#include "rocksdb/system_clock.h" #include "util/threadpool_imp.h" #undef GetCurrentTime @@ -73,19 +74,22 @@ class WinEnvThreads { std::vector threads_to_join_; }; -class WinClock { +class WinClock : public SystemClock { public: - static const std::shared_ptr& Default(); WinClock(); virtual ~WinClock() {} - virtual uint64_t NowMicros(); + const char* Name() const override { return "WindowsClock"; } - virtual uint64_t NowNanos(); + uint64_t NowMicros() override; + + uint64_t NowNanos() override; - virtual void SleepForMicroseconds(int micros); + // 0 indicates not supported + uint64_t CPUMicros() override { return 0; } + void SleepForMicroseconds(int micros) override; - virtual Status GetCurrentTime(int64_t* unix_time); + Status GetCurrentTime(int64_t* unix_time) override; // Converts seconds-since-Jan-01-1970 to a printable string virtual std::string TimeToString(uint64_t time); @@ -102,7 +106,7 @@ class WinClock { class WinFileSystem : public FileSystem { public: static const std::shared_ptr& Default(); - WinFileSystem(const std::shared_ptr& clock); + WinFileSystem(const std::shared_ptr& clock); ~WinFileSystem() {} const char* Name() const { return "WinFS"; } static size_t GetSectorSize(const std::string& fname); @@ -227,7 +231,7 @@ class WinFileSystem : public FileSystem { bool reopen); private: - std::shared_ptr clock_; + std::shared_ptr clock_; size_t page_size_; size_t allocation_granularity_; }; @@ -251,16 +255,9 @@ class WinEnv : public CompositeEnv { WinEnv(); ~WinEnv(); - Status GetCurrentTime(int64_t* unix_time) override; - - uint64_t NowMicros() override; - - uint64_t NowNanos() override; Status GetHostName(char* name, uint64_t len) override; - std::string TimeToString(uint64_t secondsSince1970) override; - Status GetThreadList(std::vector* thread_list) override; void Schedule(void (*function)(void*), void* arg, Env::Priority pri, @@ -276,8 +273,6 @@ class WinEnv : public CompositeEnv { uint64_t GetThreadID() const override; - void SleepForMicroseconds(int micros) override; - // Allow increasing the number of worker threads. void SetBackgroundThreads(int num, Env::Priority pri) override; int GetBackgroundThreads(Env::Priority pri) override; @@ -285,7 +280,6 @@ class WinEnv : public CompositeEnv { void IncBackgroundThreadsIfNeeded(int num, Env::Priority pri) override; private: - std::shared_ptr clock_; WinEnvIO winenv_io_; WinEnvThreads winenv_threads_; }; diff --git a/port/win/win_logger.cc b/port/win/win_logger.cc index aee78b771..352e38a59 100644 --- a/port/win/win_logger.cc +++ b/port/win/win_logger.cc @@ -26,13 +26,14 @@ #include "port/win/env_win.h" #include "port/win/io_win.h" #include "rocksdb/env.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { namespace port { WinLogger::WinLogger(uint64_t (*gettid)(), - const std::shared_ptr& clock, HANDLE file, + const std::shared_ptr& clock, HANDLE file, const InfoLogLevel log_level) : Logger(log_level), file_(file), diff --git a/port/win/win_logger.h b/port/win/win_logger.h index a11b7b864..08be85089 100644 --- a/port/win/win_logger.h +++ b/port/win/win_logger.h @@ -21,13 +21,13 @@ #include "rocksdb/env.h" namespace ROCKSDB_NAMESPACE { +class SystemClock; namespace port { -class WinClock; class WinLogger : public ROCKSDB_NAMESPACE::Logger { public: - WinLogger(uint64_t (*gettid)(), const std::shared_ptr& clock, + WinLogger(uint64_t (*gettid)(), const std::shared_ptr& clock, HANDLE file, const InfoLogLevel log_level = InfoLogLevel::ERROR_LEVEL); @@ -55,7 +55,7 @@ protected: uint64_t (*gettid_)(); // Return the thread id for the current thread std::atomic_size_t log_size_; std::atomic_uint_fast64_t last_flush_micros_; - std::shared_ptr clock_; + std::shared_ptr clock_; bool flush_pending_; Status CloseInternal(); diff --git a/src.mk b/src.mk index 2bb45f3eb..14e252440 100644 --- a/src.mk +++ b/src.mk @@ -78,6 +78,7 @@ LIB_SOURCES = \ db/write_batch_base.cc \ db/write_controller.cc \ db/write_thread.cc \ + env/composite_env.cc \ env/env.cc \ env/env_chroot.cc \ env/env_encryption.cc \ diff --git a/table/block_based/block_based_table_builder.cc b/table/block_based/block_based_table_builder.cc index a28639932..235b53576 100644 --- a/table/block_based/block_based_table_builder.cc +++ b/table/block_based/block_based_table_builder.cc @@ -1069,7 +1069,7 @@ void BlockBasedTableBuilder::CompressAndVerifyBlock( bool abort_compression = false; StopWatchNano timer( - r->ioptions.env, + r->ioptions.env->GetSystemClock(), ShouldReportDetailedTime(r->ioptions.env, r->ioptions.statistics)); if (is_status_ok && raw_block_contents.size() < kCompressionSizeLimit) { @@ -1163,7 +1163,8 @@ void BlockBasedTableBuilder::WriteRawBlock(const Slice& block_contents, Rep* r = rep_; Status s = Status::OK(); IOStatus io_s = IOStatus::OK(); - StopWatch sw(r->ioptions.env, r->ioptions.statistics, WRITE_RAW_BLOCK_MICROS); + StopWatch sw(r->ioptions.env->GetSystemClock(), r->ioptions.statistics, + WRITE_RAW_BLOCK_MICROS); handle->set_offset(r->get_offset()); handle->set_size(block_contents.size()); assert(status().ok()); diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index 0aa380965..2f22cd8ea 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -16,7 +16,6 @@ #include #include "cache/sharded_cache.h" - #include "db/dbformat.h" #include "db/pinned_iterators_manager.h" #include "file/file_prefetch_buffer.h" @@ -24,6 +23,7 @@ #include "file/random_access_file_reader.h" #include "monitoring/perf_context_imp.h" #include "options/options_helper.h" +#include "port/lang.h" #include "rocksdb/cache.h" #include "rocksdb/comparator.h" #include "rocksdb/env.h" @@ -32,6 +32,7 @@ #include "rocksdb/iterator.h" #include "rocksdb/options.h" #include "rocksdb/statistics.h" +#include "rocksdb/system_clock.h" #include "rocksdb/table.h" #include "rocksdb/table_properties.h" #include "table/block_based/binary_search_index_reader.h" @@ -54,9 +55,6 @@ #include "table/persistent_cache_helper.h" #include "table/sst_file_writer_collectors.h" #include "table/two_level_iterator.h" - -#include "monitoring/perf_context_imp.h" -#include "port/lang.h" #include "test_util/sync_point.h" #include "util/coding.h" #include "util/crc32c.h" @@ -629,7 +627,7 @@ Status BlockBasedTable::Open( // 6. [meta block: index] // 7. [meta block: filter] IOOptions opts; - s = PrepareIOFromReadOptions(ro, file->env(), opts); + s = file->PrepareIOOptions(ro, opts); if (s.ok()) { s = ReadFooterFromFile(opts, file.get(), prefetch_buffer.get(), file_size, &footer, kBlockBasedTableMagicNumber); @@ -762,7 +760,7 @@ Status BlockBasedTable::PrefetchTail( // Use `FilePrefetchBuffer` prefetch_buffer->reset(new FilePrefetchBuffer(nullptr, 0, 0, true, true)); IOOptions opts; - Status s = PrepareIOFromReadOptions(ro, file->env(), opts); + Status s = file->PrepareIOOptions(ro, opts); if (s.ok()) { s = (*prefetch_buffer)->Prefetch(opts, file, prefetch_off, prefetch_len); } @@ -1508,7 +1506,7 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache( CompressionType raw_block_comp_type; BlockContents raw_block_contents; if (!contents) { - StopWatch sw(rep_->ioptions.env, statistics, READ_BLOCK_GET_MICROS); + StopWatch sw(rep_->clock, statistics, READ_BLOCK_GET_MICROS); BlockFetcher block_fetcher( rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, &raw_block_contents, rep_->ioptions, do_uncompress, @@ -1597,7 +1595,7 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache( // Avoid making copy of block_key and cf_name when constructing the access // record. BlockCacheTraceRecord access_record( - rep_->ioptions.env->NowMicros(), + rep_->clock->NowMicros(), /*block_key=*/"", trace_block_type, /*block_size=*/usage, rep_->cf_id_for_tracing(), /*cf_name=*/"", rep_->level_for_tracing(), @@ -1740,7 +1738,7 @@ void BlockBasedTable::RetrieveMultipleBlocks( AlignedBuf direct_io_buf; { IOOptions opts; - IOStatus s = PrepareIOFromReadOptions(options, file->env(), opts); + IOStatus s = file->PrepareIOOptions(options, opts); if (s.IsTimedOut()) { for (FSReadRequest& req : read_reqs) { req.status = s; @@ -1942,8 +1940,7 @@ Status BlockBasedTable::RetrieveBlock( std::unique_ptr block; { - StopWatch sw(rep_->ioptions.env, rep_->ioptions.statistics, - READ_BLOCK_GET_MICROS); + StopWatch sw(rep_->clock, rep_->ioptions.statistics, READ_BLOCK_GET_MICROS); s = ReadBlockFromFile( rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, &block, rep_->ioptions, do_uncompress, maybe_compressed, block_type, @@ -2435,7 +2432,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key, referenced_key = key; } BlockCacheTraceRecord access_record( - rep_->ioptions.env->NowMicros(), + rep_->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(), @@ -2771,7 +2768,7 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options, referenced_key = key; } BlockCacheTraceRecord access_record( - rep_->ioptions.env->NowMicros(), + rep_->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(), diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index 2923b482d..37a325dbc 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -32,6 +32,7 @@ class Footer; class InternalKeyComparator; class Iterator; class FSRandomAccessFile; +class SystemClock; class TableCache; class TableReader; class WritableFile; @@ -522,13 +523,16 @@ struct BlockBasedTable::Rep { global_seqno(kDisableGlobalSequenceNumber), file_size(_file_size), level(_level), - immortal_table(_immortal_table) {} + immortal_table(_immortal_table) { + clock = ioptions.env->GetSystemClock(); + } ~Rep() { status.PermitUncheckedError(); } const ImmutableCFOptions& ioptions; const EnvOptions& env_options; const BlockBasedTableOptions table_options; const FilterPolicy* const filter_policy; const InternalKeyComparator& internal_comparator; + std::shared_ptr clock; Status status; std::unique_ptr file; char cache_key_prefix[kMaxCacheKeyPrefixSize]; diff --git a/table/block_based/block_based_table_reader_test.cc b/table/block_based/block_based_table_reader_test.cc index f714ce868..36ec3c4d3 100644 --- a/table/block_based/block_based_table_reader_test.cc +++ b/table/block_based/block_based_table_reader_test.cc @@ -135,7 +135,8 @@ class BlockBasedTableReaderTest std::string path = Path(filename); std::unique_ptr f; ASSERT_OK(fs_->NewRandomAccessFile(path, opt, &f, nullptr)); - reader->reset(new RandomAccessFileReader(std::move(f), path, env_)); + reader->reset( + new RandomAccessFileReader(std::move(f), path, env_->GetSystemClock())); } std::string ToInternalKey(const std::string& key) { diff --git a/table/block_based/partitioned_filter_block.cc b/table/block_based/partitioned_filter_block.cc index e43d5efdf..694ce8199 100644 --- a/table/block_based/partitioned_filter_block.cc +++ b/table/block_based/partitioned_filter_block.cc @@ -7,7 +7,7 @@ #include -#include "file/file_util.h" +#include "file/random_access_file_reader.h" #include "monitoring/perf_context_imp.h" #include "port/malloc.h" #include "port/port.h" @@ -460,7 +460,7 @@ Status PartitionedFilterBlockReader::CacheDependencies(const ReadOptions& ro, rep->CreateFilePrefetchBuffer(0, 0, &prefetch_buffer); IOOptions opts; - s = PrepareIOFromReadOptions(ro, rep->file->env(), opts); + s = rep->file->PrepareIOOptions(ro, opts); if (s.ok()) { s = prefetch_buffer->Prefetch(opts, rep->file.get(), prefetch_off, static_cast(prefetch_len)); diff --git a/table/block_based/partitioned_index_reader.cc b/table/block_based/partitioned_index_reader.cc index 0dbd132b1..ec8562e81 100644 --- a/table/block_based/partitioned_index_reader.cc +++ b/table/block_based/partitioned_index_reader.cc @@ -8,7 +8,7 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. #include "table/block_based/partitioned_index_reader.h" -#include "file/file_util.h" +#include "file/random_access_file_reader.h" #include "table/block_based/partitioned_index_iterator.h" namespace ROCKSDB_NAMESPACE { @@ -148,7 +148,7 @@ Status PartitionIndexReader::CacheDependencies(const ReadOptions& ro, std::unique_ptr prefetch_buffer; rep->CreateFilePrefetchBuffer(0, 0, &prefetch_buffer); IOOptions opts; - s = PrepareIOFromReadOptions(ro, rep->file->env(), opts); + s = rep->file->PrepareIOOptions(ro, opts); if (s.ok()) { s = prefetch_buffer->Prefetch(opts, rep->file.get(), prefetch_off, static_cast(prefetch_len)); diff --git a/table/block_fetcher.cc b/table/block_fetcher.cc index e9df0f7af..588dae474 100644 --- a/table/block_fetcher.cc +++ b/table/block_fetcher.cc @@ -12,7 +12,6 @@ #include #include -#include "file/file_util.h" #include "logging/logging.h" #include "memory/memory_allocator.h" #include "monitoring/perf_context_imp.h" @@ -60,7 +59,7 @@ inline bool BlockFetcher::TryGetUncompressBlockFromPersistentCache() { inline bool BlockFetcher::TryGetFromPrefetchBuffer() { if (prefetch_buffer_ != nullptr) { IOOptions opts; - Status s = PrepareIOFromReadOptions(read_options_, file_->env(), opts); + Status s = file_->PrepareIOOptions(read_options_, opts); if (s.ok() && prefetch_buffer_->TryReadFromCache( opts, handle_.offset(), block_size_with_trailer_, &slice_, &s, for_compaction_)) { @@ -230,7 +229,7 @@ Status BlockFetcher::ReadBlockContents() { } } else if (!TryGetCompressedBlockFromPersistentCache()) { IOOptions opts; - status_ = PrepareIOFromReadOptions(read_options_, file_->env(), opts); + status_ = file_->PrepareIOOptions(read_options_, opts); // Actual file read if (status_.ok()) { if (file_->use_direct_io()) { diff --git a/table/block_fetcher_test.cc b/table/block_fetcher_test.cc index 0786730af..5d3c90858 100644 --- a/table/block_fetcher_test.cc +++ b/table/block_fetcher_test.cc @@ -260,7 +260,8 @@ class BlockFetcherTest : public testing::Test { std::string path = Path(filename); std::unique_ptr f; ASSERT_OK(fs_->NewRandomAccessFile(path, opt, &f, nullptr)); - reader->reset(new RandomAccessFileReader(std::move(f), path, env_)); + reader->reset( + new RandomAccessFileReader(std::move(f), path, env_->GetSystemClock())); } void NewTableReader(const ImmutableCFOptions& ioptions, diff --git a/table/format.cc b/table/format.cc index b87274700..66acb8837 100644 --- a/table/format.cc +++ b/table/format.cc @@ -353,8 +353,9 @@ Status UncompressBlockContentsForCompressionType( assert(uncompression_info.type() != kNoCompression && "Invalid compression type"); - StopWatchNano timer(ioptions.env, ShouldReportDetailedTime( - ioptions.env, ioptions.statistics)); + StopWatchNano timer( + ioptions.env->GetSystemClock(), + ShouldReportDetailedTime(ioptions.env, ioptions.statistics)); size_t uncompressed_size = 0; CacheAllocationPtr ubuf = UncompressData(uncompression_info, data, n, &uncompressed_size, diff --git a/table/get_context.cc b/table/get_context.cc index 78901dca4..2211e440b 100644 --- a/table/get_context.cc +++ b/table/get_context.cc @@ -4,15 +4,16 @@ // (found in the LICENSE.Apache file in the root directory). #include "table/get_context.h" + #include "db/merge_helper.h" #include "db/pinned_iterators_manager.h" #include "db/read_callback.h" #include "monitoring/file_read_sample.h" #include "monitoring/perf_context_imp.h" #include "monitoring/statistics.h" -#include "rocksdb/env.h" #include "rocksdb/merge_operator.h" #include "rocksdb/statistics.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { @@ -43,7 +44,8 @@ GetContext::GetContext( Statistics* statistics, GetState init_state, const Slice& user_key, PinnableSlice* pinnable_val, std::string* timestamp, bool* value_found, MergeContext* merge_context, bool do_merge, - SequenceNumber* _max_covering_tombstone_seq, Env* env, SequenceNumber* seq, + SequenceNumber* _max_covering_tombstone_seq, + const std::shared_ptr& clock, SequenceNumber* seq, PinnedIteratorsManager* _pinned_iters_mgr, ReadCallback* callback, bool* is_blob_index, uint64_t tracing_get_id) : ucmp_(ucmp), @@ -57,7 +59,7 @@ GetContext::GetContext( value_found_(value_found), merge_context_(merge_context), max_covering_tombstone_seq_(_max_covering_tombstone_seq), - env_(env), + clock_(clock), seq_(seq), replay_log_(nullptr), pinned_iters_mgr_(_pinned_iters_mgr), @@ -75,12 +77,13 @@ GetContext::GetContext( const Comparator* ucmp, const MergeOperator* merge_operator, Logger* logger, Statistics* statistics, GetState init_state, const Slice& user_key, PinnableSlice* pinnable_val, bool* value_found, MergeContext* merge_context, - bool do_merge, SequenceNumber* _max_covering_tombstone_seq, Env* env, - SequenceNumber* seq, PinnedIteratorsManager* _pinned_iters_mgr, - ReadCallback* callback, bool* is_blob_index, uint64_t tracing_get_id) + bool do_merge, SequenceNumber* _max_covering_tombstone_seq, + const std::shared_ptr& clock, SequenceNumber* seq, + PinnedIteratorsManager* _pinned_iters_mgr, ReadCallback* callback, + bool* is_blob_index, uint64_t tracing_get_id) : GetContext(ucmp, merge_operator, logger, statistics, init_state, user_key, pinnable_val, nullptr, value_found, merge_context, do_merge, - _max_covering_tombstone_seq, env, seq, _pinned_iters_mgr, + _max_covering_tombstone_seq, clock, seq, _pinned_iters_mgr, callback, is_blob_index, tracing_get_id) {} // Called from TableCache::Get and Table::Get when file/block in which @@ -277,7 +280,7 @@ bool GetContext::SaveValue(const ParsedInternalKey& parsed_key, Status merge_status = MergeHelper::TimedFullMerge( merge_operator_, user_key_, &value, merge_context_->GetOperands(), pinnable_val_->GetSelf(), - logger_, statistics_, env_); + logger_, statistics_, clock_); pinnable_val_->PinSelf(); if (!merge_status.ok()) { state_ = kCorrupt; @@ -318,7 +321,7 @@ bool GetContext::SaveValue(const ParsedInternalKey& parsed_key, Status merge_status = MergeHelper::TimedFullMerge( merge_operator_, user_key_, nullptr, merge_context_->GetOperands(), pinnable_val_->GetSelf(), - logger_, statistics_, env_); + logger_, statistics_, clock_); pinnable_val_->PinSelf(); if (!merge_status.ok()) { state_ = kCorrupt; @@ -346,7 +349,7 @@ bool GetContext::SaveValue(const ParsedInternalKey& parsed_key, Status merge_status = MergeHelper::TimedFullMerge( merge_operator_, user_key_, nullptr, merge_context_->GetOperands(), pinnable_val_->GetSelf(), - logger_, statistics_, env_); + logger_, statistics_, clock_); pinnable_val_->PinSelf(); if (!merge_status.ok()) { state_ = kCorrupt; diff --git a/table/get_context.h b/table/get_context.h index f330580db..34434e10d 100644 --- a/table/get_context.h +++ b/table/get_context.h @@ -8,7 +8,6 @@ #include "db/dbformat.h" #include "db/merge_context.h" #include "db/read_callback.h" -#include "rocksdb/env.h" #include "rocksdb/statistics.h" #include "rocksdb/types.h" #include "table/block_based/block.h" @@ -16,6 +15,7 @@ namespace ROCKSDB_NAMESPACE { class MergeContext; class PinnedIteratorsManager; +class SystemClock; // Data structure for accumulating statistics during a point lookup. At the // end of the point lookup, the corresponding ticker stats are updated. This @@ -97,9 +97,10 @@ class GetContext { // merge_context and they are never merged. The value pointer is untouched. GetContext(const Comparator* ucmp, const MergeOperator* merge_operator, Logger* logger, Statistics* statistics, GetState init_state, - const Slice& user_key, PinnableSlice* value, - bool* value_found, MergeContext* merge_context, bool do_merge, - SequenceNumber* max_covering_tombstone_seq, Env* env, + const Slice& user_key, PinnableSlice* value, bool* value_found, + MergeContext* merge_context, bool do_merge, + SequenceNumber* max_covering_tombstone_seq, + const std::shared_ptr& clock, SequenceNumber* seq = nullptr, PinnedIteratorsManager* _pinned_iters_mgr = nullptr, ReadCallback* callback = nullptr, bool* is_blob_index = nullptr, @@ -109,7 +110,8 @@ class GetContext { const Slice& user_key, PinnableSlice* value, std::string* timestamp, bool* value_found, MergeContext* merge_context, bool do_merge, - SequenceNumber* max_covering_tombstone_seq, Env* env, + SequenceNumber* max_covering_tombstone_seq, + const std::shared_ptr& clock, SequenceNumber* seq = nullptr, PinnedIteratorsManager* _pinned_iters_mgr = nullptr, ReadCallback* callback = nullptr, bool* is_blob_index = nullptr, @@ -183,7 +185,7 @@ class GetContext { bool* value_found_; // Is value set correctly? Used by KeyMayExist MergeContext* merge_context_; SequenceNumber* max_covering_tombstone_seq_; - Env* env_; + std::shared_ptr clock_; // If a key is found, seq_ will be set to the SequenceNumber of most recent // write to the key or kMaxSequenceNumber if unknown SequenceNumber* seq_; diff --git a/table/sst_file_writer.cc b/table/sst_file_writer.cc index ab1ee7c4e..d90f97881 100644 --- a/table/sst_file_writer.cc +++ b/table/sst_file_writer.cc @@ -257,8 +257,8 @@ Status SstFileWriter::Open(const std::string& file_path) { 0 /* file_creation_time */, "SST Writer" /* db_id */, db_session_id); r->file_writer.reset(new WritableFileWriter( NewLegacyWritableFileWrapper(std::move(sst_file)), file_path, - r->env_options, r->ioptions.env, nullptr /* io_tracer */, - nullptr /* stats */, r->ioptions.listeners, + r->env_options, r->ioptions.env->GetSystemClock(), + nullptr /* io_tracer */, nullptr /* stats */, r->ioptions.listeners, r->ioptions.file_checksum_gen_factory)); // TODO(tec) : If table_factory is using compressed block cache, we will diff --git a/table/table_reader_bench.cc b/table/table_reader_bench.cc index f1fd605aa..f2825dccb 100644 --- a/table/table_reader_bench.cc +++ b/table/table_reader_bench.cc @@ -18,6 +18,7 @@ int main() { #include "monitoring/histogram.h" #include "rocksdb/db.h" #include "rocksdb/slice_transform.h" +#include "rocksdb/system_clock.h" #include "rocksdb/table.h" #include "table/block_based/block_based_table_factory.h" #include "table/get_context.h" @@ -50,8 +51,9 @@ static std::string MakeKey(int i, int j, bool through_db) { return key.Encode().ToString(); } -uint64_t Now(Env* env, bool measured_by_nanosecond) { - return measured_by_nanosecond ? env->NowNanos() : env->NowMicros(); +uint64_t Now(const std::shared_ptr& clock, + bool measured_by_nanosecond) { + return measured_by_nanosecond ? clock->NowNanos() : clock->NowMicros(); } } // namespace @@ -81,6 +83,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, std::string dbname = test::PerThreadDBPath("rocksdb_table_reader_bench_db"); WriteOptions wo; Env* env = Env::Default(); + const auto& clock = env->GetSystemClock(); TableBuilder* tb = nullptr; DB* db = nullptr; Status s; @@ -168,7 +171,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, if (!for_iterator) { // Query one existing key; std::string key = MakeKey(r1, r2, through_db); - uint64_t start_time = Now(env, measured_by_nanosecond); + uint64_t start_time = Now(clock, measured_by_nanosecond); if (!through_db) { PinnableSlice value; MergeContext merge_context; @@ -177,12 +180,12 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, ioptions.merge_operator, ioptions.info_log, ioptions.statistics, GetContext::kNotFound, Slice(key), &value, nullptr, &merge_context, - true, &max_covering_tombstone_seq, env); + true, &max_covering_tombstone_seq, clock); s = table_reader->Get(read_options, key, &get_context, nullptr); } else { s = db->Get(read_options, key, &result); } - hist.Add(Now(env, measured_by_nanosecond) - start_time); + hist.Add(Now(clock, measured_by_nanosecond) - start_time); } else { int r2_len; if (if_query_empty_keys) { @@ -196,7 +199,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, std::string start_key = MakeKey(r1, r2, through_db); std::string end_key = MakeKey(r1, r2 + r2_len, through_db); uint64_t total_time = 0; - uint64_t start_time = Now(env, measured_by_nanosecond); + uint64_t start_time = Now(clock, measured_by_nanosecond); Iterator* iter = nullptr; InternalIterator* iiter = nullptr; if (!through_db) { @@ -214,10 +217,10 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, break; } // verify key; - total_time += Now(env, measured_by_nanosecond) - start_time; + total_time += Now(clock, measured_by_nanosecond) - start_time; assert(Slice(MakeKey(r1, r2 + count, through_db)) == (through_db ? iter->key() : iiter->key())); - start_time = Now(env, measured_by_nanosecond); + start_time = Now(clock, measured_by_nanosecond); if (++count >= r2_len) { break; } @@ -229,7 +232,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, assert(false); } delete iter; - total_time += Now(env, measured_by_nanosecond) - start_time; + total_time += Now(clock, measured_by_nanosecond) - start_time; hist.Add(total_time); } } diff --git a/test_util/mock_time_env.cc b/test_util/mock_time_env.cc index 8316406ec..23888e69e 100644 --- a/test_util/mock_time_env.cc +++ b/test_util/mock_time_env.cc @@ -12,7 +12,7 @@ namespace ROCKSDB_NAMESPACE { // TODO: this is a workaround for the different behavior on different platform // for timedwait timeout. Ideally timedwait API should be moved to env. // details: PR #7101. -void MockTimeEnv::InstallTimedWaitFixCallback() { +void MockSystemClock::InstallTimedWaitFixCallback() { #ifndef NDEBUG SyncPoint::GetInstance()->DisableProcessing(); SyncPoint::GetInstance()->ClearAllCallBacks(); diff --git a/test_util/mock_time_env.h b/test_util/mock_time_env.h index 1f454144a..61dc4e443 100644 --- a/test_util/mock_time_env.h +++ b/test_util/mock_time_env.h @@ -6,18 +6,21 @@ #pragma once #include +#include -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { // NOTE: SpecialEnv offers most of this functionality, along with hooks // for safe DB behavior under a mock time environment, so should be used -// instead of MockTimeEnv for DB tests. -class MockTimeEnv : public EnvWrapper { +// instead of MockSystemClock for DB tests. +class MockSystemClock : public SystemClockWrapper { public: - explicit MockTimeEnv(Env* base) : EnvWrapper(base) {} + explicit MockSystemClock(const std::shared_ptr& base) + : SystemClockWrapper(base) {} + const char* Name() const override { return "MockSystemClock"; } virtual Status GetCurrentTime(int64_t* time_sec) override { assert(time_sec != nullptr); *time_sec = static_cast(current_time_us_ / kMicrosInSecond); @@ -33,9 +36,9 @@ class MockTimeEnv : public EnvWrapper { return current_time_us_ * 1000; } - uint64_t RealNowMicros() { return target()->NowMicros(); } + uint64_t RealNowMicros() { return target_->NowMicros(); } - void set_current_time(uint64_t time_sec) { + void SetCurrentTime(uint64_t time_sec) { assert(time_sec < std::numeric_limits::max() / kMicrosInSecond); assert(time_sec * kMicrosInSecond >= current_time_us_); current_time_us_ = time_sec * kMicrosInSecond; diff --git a/test_util/testutil.cc b/test_util/testutil.cc index 6a7c10e4b..34586bd24 100644 --- a/test_util/testutil.cc +++ b/test_util/testutil.cc @@ -24,6 +24,7 @@ #include "file/writable_file_writer.h" #include "port/port.h" #include "rocksdb/convenience.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "util/random.h" @@ -196,6 +197,28 @@ std::string KeyStr(uint64_t ts, const std::string& user_key, return KeyStr(user_key_with_ts, seq, t, corrupt); } +bool SleepingBackgroundTask::TimedWaitUntilSleeping(uint64_t wait_time) { + auto abs_time = SystemClock::Default()->NowMicros() + wait_time; + MutexLock l(&mutex_); + while (!sleeping_ || !should_sleep_) { + if (bg_cv_.TimedWait(abs_time)) { + return true; + } + } + return false; +} + +bool SleepingBackgroundTask::TimedWaitUntilDone(uint64_t wait_time) { + auto abs_time = SystemClock::Default()->NowMicros() + wait_time; + MutexLock l(&mutex_); + while (!done_with_sleep_) { + if (bg_cv_.TimedWait(abs_time)) { + return true; + } + } + return false; +} + std::string RandomName(Random* rnd, const size_t len) { std::stringstream ss; for (size_t i = 0; i < len; ++i) { diff --git a/test_util/testutil.h b/test_util/testutil.h index c53c35944..5992783b9 100644 --- a/test_util/testutil.h +++ b/test_util/testutil.h @@ -453,16 +453,8 @@ class SleepingBackgroundTask { // otherwise times out. // wait_time is in microseconds. // Returns true when times out, false otherwise. - bool TimedWaitUntilSleeping(uint64_t wait_time) { - auto abs_time = Env::Default()->NowMicros() + wait_time; - MutexLock l(&mutex_); - while (!sleeping_ || !should_sleep_) { - if (bg_cv_.TimedWait(abs_time)) { - return true; - } - } - return false; - } + bool TimedWaitUntilSleeping(uint64_t wait_time); + void WakeUp() { MutexLock l(&mutex_); should_sleep_ = false; @@ -476,16 +468,8 @@ class SleepingBackgroundTask { } // Similar to TimedWaitUntilSleeping. // Waits until the task is done. - bool TimedWaitUntilDone(uint64_t wait_time) { - auto abs_time = Env::Default()->NowMicros() + wait_time; - MutexLock l(&mutex_); - while (!done_with_sleep_) { - if (bg_cv_.TimedWait(abs_time)) { - return true; - } - } - return false; - } + bool TimedWaitUntilDone(uint64_t wait_time); + bool WokenUp() { MutexLock l(&mutex_); return should_sleep_ == false; diff --git a/tools/io_tracer_parser_test.cc b/tools/io_tracer_parser_test.cc index 2e62a6b0a..a65d4267e 100644 --- a/tools/io_tracer_parser_test.cc +++ b/tools/io_tracer_parser_test.cc @@ -18,6 +18,7 @@ int main() { #include "rocksdb/env.h" #include "rocksdb/status.h" +#include "rocksdb/trace_reader_writer.h" #include "test_util/testharness.h" #include "test_util/testutil.h" #include "tools/io_tracer_parser_tool.h" diff --git a/tools/io_tracer_parser_tool.cc b/tools/io_tracer_parser_tool.cc index df7700071..3de7fe5bb 100644 --- a/tools/io_tracer_parser_tool.cc +++ b/tools/io_tracer_parser_tool.cc @@ -14,6 +14,7 @@ #include #include "port/lang.h" +#include "rocksdb/trace_reader_writer.h" #include "trace_replay/io_tracer.h" #include "util/gflags_compat.h" diff --git a/tools/write_stress.cc b/tools/write_stress.cc index 15b1da881..31161ce1c 100644 --- a/tools/write_stress.cc +++ b/tools/write_stress.cc @@ -69,6 +69,7 @@ int main() { #include "rocksdb/env.h" #include "rocksdb/options.h" #include "rocksdb/slice.h" +#include "rocksdb/system_clock.h" #include "util/gflags_compat.h" using GFLAGS_NAMESPACE::ParseCommandLineFlags; @@ -187,8 +188,8 @@ class WriteStress { void IteratorHoldThread() { while (!stop_.load(std::memory_order_relaxed)) { std::unique_ptr iterator(db_->NewIterator(ReadOptions())); - Env::Default()->SleepForMicroseconds(FLAGS_iterator_hold_sec * 1000 * - 1000LL); + SystemClock::Default()->SleepForMicroseconds(FLAGS_iterator_hold_sec * + 1000 * 1000LL); for (iterator->SeekToFirst(); iterator->Valid(); iterator->Next()) { } if (!iterator->status().ok()) { @@ -204,9 +205,8 @@ class WriteStress { std::uniform_real_distribution dist(0, 1); std::uniform_int_distribution char_dist('a', 'z'); while (!stop_.load(std::memory_order_relaxed)) { - Env::Default()->SleepForMicroseconds(static_cast( - FLAGS_prefix_mutate_period_sec * - 1000 * 1000LL)); + SystemClock::Default()->SleepForMicroseconds( + static_cast(FLAGS_prefix_mutate_period_sec * 1000 * 1000LL)); if (dist(rng) < FLAGS_first_char_mutate_probability) { key_prefix_[0].store(static_cast(char_dist(rng)), std::memory_order_relaxed); } @@ -227,11 +227,12 @@ class WriteStress { if (FLAGS_runtime_sec == -1) { // infinite runtime, until we get killed while (true) { - Env::Default()->SleepForMicroseconds(1000 * 1000); + SystemClock::Default()->SleepForMicroseconds(1000 * 1000); } } - Env::Default()->SleepForMicroseconds(FLAGS_runtime_sec * 1000 * 1000); + SystemClock::Default()->SleepForMicroseconds(FLAGS_runtime_sec * 1000 * + 1000); stop_.store(true, std::memory_order_relaxed); for (auto& t : threads_) { diff --git a/trace_replay/io_tracer.cc b/trace_replay/io_tracer.cc index b665592f9..c9eda4b82 100644 --- a/trace_replay/io_tracer.cc +++ b/trace_replay/io_tracer.cc @@ -12,14 +12,17 @@ #include "db/db_impl/db_impl.h" #include "db/dbformat.h" #include "rocksdb/slice.h" +#include "rocksdb/system_clock.h" +#include "rocksdb/trace_reader_writer.h" #include "util/coding.h" #include "util/hash.h" #include "util/string_util.h" namespace ROCKSDB_NAMESPACE { -IOTraceWriter::IOTraceWriter(Env* env, const TraceOptions& trace_options, +IOTraceWriter::IOTraceWriter(const std::shared_ptr& clock, + const TraceOptions& trace_options, std::unique_ptr&& trace_writer) - : env_(env), + : clock_(clock), trace_options_(trace_options), trace_writer_(std::move(trace_writer)) {} @@ -75,7 +78,7 @@ Status IOTraceWriter::WriteIOOp(const IOTraceRecord& record) { Status IOTraceWriter::WriteHeader() { Trace trace; - trace.ts = env_->NowMicros(); + trace.ts = clock_->NowMicros(); trace.type = TraceType::kTraceBegin; PutLengthPrefixedSlice(&trace.payload, kTraceMagic); PutFixed32(&trace.payload, kMajorVersion); @@ -217,14 +220,16 @@ IOTracer::IOTracer() : tracing_enabled(false) { writer_.store(nullptr); } IOTracer::~IOTracer() { EndIOTrace(); } -Status IOTracer::StartIOTrace(Env* env, const TraceOptions& trace_options, +Status IOTracer::StartIOTrace(const std::shared_ptr& clock, + const TraceOptions& trace_options, std::unique_ptr&& trace_writer) { InstrumentedMutexLock lock_guard(&trace_writer_mutex_); if (writer_.load()) { return Status::Busy(); } trace_options_ = trace_options; - writer_.store(new IOTraceWriter(env, trace_options, std::move(trace_writer))); + writer_.store( + new IOTraceWriter(clock, trace_options, std::move(trace_writer))); tracing_enabled = true; return writer_.load()->WriteHeader(); } diff --git a/trace_replay/io_tracer.h b/trace_replay/io_tracer.h index 8518a230c..bdf346814 100644 --- a/trace_replay/io_tracer.h +++ b/trace_replay/io_tracer.h @@ -9,12 +9,13 @@ #include #include "monitoring/instrumented_mutex.h" -#include "rocksdb/env.h" #include "rocksdb/options.h" -#include "rocksdb/trace_reader_writer.h" #include "trace_replay/trace_replay.h" namespace ROCKSDB_NAMESPACE { +class SystemClock; +class TraceReader; +class TraceWriter; /* In order to log new data in trace record for specified operations, do following: @@ -92,7 +93,8 @@ struct IOTraceHeader { // timestamp and type, followed by the trace payload. class IOTraceWriter { public: - IOTraceWriter(Env* env, const TraceOptions& trace_options, + IOTraceWriter(const std::shared_ptr& clock, + const TraceOptions& trace_options, std::unique_ptr&& trace_writer); ~IOTraceWriter() = default; // No copy and move. @@ -108,7 +110,7 @@ class IOTraceWriter { Status WriteHeader(); private: - Env* env_; + std::shared_ptr clock_; TraceOptions trace_options_; std::unique_ptr trace_writer_; }; @@ -166,7 +168,8 @@ class IOTracer { // Start writing IO operations to the trace_writer. TSAN_SUPPRESSION Status - StartIOTrace(Env* env, const TraceOptions& trace_options, + StartIOTrace(const std::shared_ptr& clock, + const TraceOptions& trace_options, std::unique_ptr&& trace_writer); // Stop writing IO operations to the trace_writer. diff --git a/trace_replay/io_tracer_test.cc b/trace_replay/io_tracer_test.cc index 8994b1acd..78e14969d 100644 --- a/trace_replay/io_tracer_test.cc +++ b/trace_replay/io_tracer_test.cc @@ -7,6 +7,7 @@ #include "rocksdb/env.h" #include "rocksdb/status.h" +#include "rocksdb/trace_reader_writer.h" #include "test_util/testharness.h" #include "test_util/testutil.h" @@ -91,7 +92,8 @@ TEST_F(IOTracerTest, MultipleRecordsWithDifferentIOOpOptions) { ASSERT_OK(NewFileTraceWriter(env_, env_options_, trace_file_path_, &trace_writer)); IOTracer writer; - ASSERT_OK(writer.StartIOTrace(env_, trace_opt, std::move(trace_writer))); + ASSERT_OK(writer.StartIOTrace(env_->GetSystemClock(), trace_opt, + std::move(trace_writer))); // Write general record. IOTraceRecord record0(0, TraceType::kIOTracer, 0 /*io_op_data*/, @@ -200,7 +202,8 @@ TEST_F(IOTracerTest, AtomicWrite) { ASSERT_OK(NewFileTraceWriter(env_, env_options_, trace_file_path_, &trace_writer)); IOTracer writer; - ASSERT_OK(writer.StartIOTrace(env_, trace_opt, std::move(trace_writer))); + ASSERT_OK(writer.StartIOTrace(env_->GetSystemClock(), trace_opt, + std::move(trace_writer))); writer.WriteIOOp(record); ASSERT_OK(env_->FileExists(trace_file_path_)); } @@ -263,7 +266,8 @@ TEST_F(IOTracerTest, AtomicNoWriteAfterEndTrace) { ASSERT_OK(NewFileTraceWriter(env_, env_options_, trace_file_path_, &trace_writer)); IOTracer writer; - ASSERT_OK(writer.StartIOTrace(env_, trace_opt, std::move(trace_writer))); + ASSERT_OK(writer.StartIOTrace(env_->GetSystemClock(), trace_opt, + std::move(trace_writer))); writer.WriteIOOp(record); writer.EndIOTrace(); // Write the record again. This time the record should not be written since @@ -298,7 +302,8 @@ TEST_F(IOTracerTest, AtomicMultipleWrites) { std::unique_ptr trace_writer; ASSERT_OK(NewFileTraceWriter(env_, env_options_, trace_file_path_, &trace_writer)); - IOTraceWriter writer(env_, trace_opt, std::move(trace_writer)); + IOTraceWriter writer(env_->GetSystemClock(), trace_opt, + std::move(trace_writer)); ASSERT_OK(writer.WriteHeader()); // Write 10 records WriteIOOp(&writer, 10); diff --git a/trace_replay/trace_replay.cc b/trace_replay/trace_replay.cc index 949cd450d..24344c75f 100644 --- a/trace_replay/trace_replay.cc +++ b/trace_replay/trace_replay.cc @@ -8,8 +8,13 @@ #include #include #include + #include "db/db_impl/db_impl.h" +#include "rocksdb/env.h" +#include "rocksdb/options.h" #include "rocksdb/slice.h" +#include "rocksdb/system_clock.h" +#include "rocksdb/trace_reader_writer.h" #include "rocksdb/write_batch.h" #include "util/coding.h" #include "util/string_util.h" @@ -56,12 +61,13 @@ Status TracerHelper::DecodeTrace(const std::string& encoded_trace, return Status::OK(); } -Tracer::Tracer(Env* env, const TraceOptions& trace_options, +Tracer::Tracer(const std::shared_ptr& clock, + const TraceOptions& trace_options, std::unique_ptr&& trace_writer) - : env_(env), + : clock_(clock), trace_options_(trace_options), trace_writer_(std::move(trace_writer)), - trace_request_count_ (0) { + trace_request_count_(0) { // TODO: What if this fails? WriteHeader().PermitUncheckedError(); } @@ -74,7 +80,7 @@ Status Tracer::Write(WriteBatch* write_batch) { return Status::OK(); } Trace trace; - trace.ts = env_->NowMicros(); + trace.ts = clock_->NowMicros(); trace.type = trace_type; trace.payload = write_batch->Data(); return WriteTrace(trace); @@ -86,7 +92,7 @@ Status Tracer::Get(ColumnFamilyHandle* column_family, const Slice& key) { return Status::OK(); } Trace trace; - trace.ts = env_->NowMicros(); + trace.ts = clock_->NowMicros(); trace.type = trace_type; EncodeCFAndKey(&trace.payload, column_family->GetID(), key); return WriteTrace(trace); @@ -98,7 +104,7 @@ Status Tracer::IteratorSeek(const uint32_t& cf_id, const Slice& key) { return Status::OK(); } Trace trace; - trace.ts = env_->NowMicros(); + trace.ts = clock_->NowMicros(); trace.type = trace_type; EncodeCFAndKey(&trace.payload, cf_id, key); return WriteTrace(trace); @@ -110,7 +116,7 @@ Status Tracer::IteratorSeekForPrev(const uint32_t& cf_id, const Slice& key) { return Status::OK(); } Trace trace; - trace.ts = env_->NowMicros(); + trace.ts = clock_->NowMicros(); trace.type = trace_type; EncodeCFAndKey(&trace.payload, cf_id, key); return WriteTrace(trace); @@ -148,7 +154,7 @@ Status Tracer::WriteHeader() { std::string header(s.str()); Trace trace; - trace.ts = env_->NowMicros(); + trace.ts = clock_->NowMicros(); trace.type = kTraceBegin; trace.payload = header; return WriteTrace(trace); @@ -156,7 +162,7 @@ Status Tracer::WriteHeader() { Status Tracer::WriteFooter() { Trace trace; - trace.ts = env_->NowMicros(); + trace.ts = clock_->NowMicros(); trace.type = kTraceEnd; trace.payload = ""; return WriteTrace(trace); diff --git a/trace_replay/trace_replay.h b/trace_replay/trace_replay.h index fc1c5cbf0..a7f773a91 100644 --- a/trace_replay/trace_replay.h +++ b/trace_replay/trace_replay.h @@ -9,9 +9,9 @@ #include #include -#include "rocksdb/env.h" #include "rocksdb/options.h" -#include "rocksdb/trace_reader_writer.h" +#include "rocksdb/rocksdb_namespace.h" +#include "rocksdb/status.h" namespace ROCKSDB_NAMESPACE { @@ -22,9 +22,17 @@ class ColumnFamilyHandle; class ColumnFamilyData; class DB; class DBImpl; +class Env; class Slice; +class SystemClock; +class TraceReader; +class TraceWriter; class WriteBatch; +struct ReadOptions; +struct TraceOptions; +struct WriteOptions; + extern const std::string kTraceMagic; const unsigned int kTraceTimestampSize = 8; const unsigned int kTraceTypeSize = 1; @@ -82,7 +90,8 @@ class TracerHelper { // timestamp and type, followed by the trace payload. class Tracer { public: - Tracer(Env* env, const TraceOptions& trace_options, + Tracer(const std::shared_ptr& clock, + const TraceOptions& trace_options, std::unique_ptr&& trace_writer); ~Tracer(); @@ -120,7 +129,7 @@ class Tracer { // Returns true if a trace should be skipped, false otherwise. bool ShouldSkipTrace(const TraceType& type); - Env* env_; + std::shared_ptr clock_; TraceOptions trace_options_; std::unique_ptr trace_writer_; uint64_t trace_request_count_; diff --git a/util/dynamic_bloom_test.cc b/util/dynamic_bloom_test.cc index d8bee265b..0954f7239 100644 --- a/util/dynamic_bloom_test.cc +++ b/util/dynamic_bloom_test.cc @@ -22,6 +22,7 @@ int main() { #include "dynamic_bloom.h" #include "memory/arena.h" #include "port/port.h" +#include "rocksdb/system_clock.h" #include "test_util/testharness.h" #include "test_util/testutil.h" #include "util/gflags_compat.h" @@ -177,7 +178,7 @@ TEST_F(DynamicBloomTest, VaryingLengths) { TEST_F(DynamicBloomTest, perf) { KeyMaker km; - StopWatchNano timer(Env::Default()); + StopWatchNano timer(SystemClock::Default()); uint32_t num_probes = static_cast(FLAGS_num_probes); if (!FLAGS_enable_perf) { @@ -237,7 +238,7 @@ TEST_F(DynamicBloomTest, concurrent_with_perf) { std::function adder([&](size_t t) { KeyMaker km; - StopWatchNano timer(Env::Default()); + StopWatchNano timer(SystemClock::Default()); timer.Start(); for (uint64_t i = 1 + t; i <= num_keys; i += num_threads) { std_bloom.AddConcurrently(km.Seq(i)); @@ -260,7 +261,7 @@ TEST_F(DynamicBloomTest, concurrent_with_perf) { elapsed = 0; std::function hitter([&](size_t t) { KeyMaker km; - StopWatchNano timer(Env::Default()); + StopWatchNano timer(SystemClock::Default()); timer.Start(); for (uint64_t i = 1 + t; i <= num_keys; i += num_threads) { bool f = std_bloom.MayContain(km.Seq(i)); @@ -285,7 +286,7 @@ TEST_F(DynamicBloomTest, concurrent_with_perf) { std::atomic false_positives(0); std::function misser([&](size_t t) { KeyMaker km; - StopWatchNano timer(Env::Default()); + StopWatchNano timer(SystemClock::Default()); timer.Start(); for (uint64_t i = num_keys + 1 + t; i <= 2 * num_keys; i += num_threads) { bool f = std_bloom.MayContain(km.Seq(i)); diff --git a/util/filter_bench.cc b/util/filter_bench.cc index e523aacbe..461b24dbd 100644 --- a/util/filter_bench.cc +++ b/util/filter_bench.cc @@ -19,6 +19,7 @@ int main() { #include "memory/arena.h" #include "port/port.h" #include "port/stack_trace.h" +#include "rocksdb/system_clock.h" #include "table/block_based/filter_policy_internal.h" #include "table/block_based/full_filter_block.h" #include "table/block_based/mock_block_based_table.h" @@ -358,8 +359,8 @@ void FilterBench::Go() { max_mem = static_cast(1024 * 1024 * working_mem_size_mb); } - ROCKSDB_NAMESPACE::StopWatchNano timer(ROCKSDB_NAMESPACE::Env::Default(), - true); + ROCKSDB_NAMESPACE::StopWatchNano timer( + ROCKSDB_NAMESPACE::SystemClock::Default(), true); infos_.clear(); while ((working_mem_size_mb == 0 || total_size < max_mem) && @@ -598,8 +599,8 @@ double FilterBench::RandomQueryTest(uint32_t inside_threshold, bool dry_run, batch_slice_ptrs[i] = &batch_slices[i]; } - ROCKSDB_NAMESPACE::StopWatchNano timer(ROCKSDB_NAMESPACE::Env::Default(), - true); + ROCKSDB_NAMESPACE::StopWatchNano timer( + ROCKSDB_NAMESPACE::SystemClock::Default(), true); for (uint64_t q = 0; q < max_queries; q += batch_size) { bool inside_this_time = random_.Next() <= inside_threshold; diff --git a/util/log_write_bench.cc b/util/log_write_bench.cc index 60798babf..c1637db15 100644 --- a/util/log_write_bench.cc +++ b/util/log_write_bench.cc @@ -14,6 +14,7 @@ int main() { #include "file/writable_file_writer.h" #include "monitoring/histogram.h" #include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "test_util/testharness.h" #include "test_util/testutil.h" #include "util/gflags_compat.h" @@ -34,13 +35,14 @@ void RunBenchmark() { std::string file_name = test::PerThreadDBPath("log_write_benchmark.log"); DBOptions options; Env* env = Env::Default(); + const auto& clock = env->GetSystemClock(); EnvOptions env_options = env->OptimizeForLogWrite(EnvOptions(), options); env_options.bytes_per_sync = FLAGS_bytes_per_sync; std::unique_ptr file; env->NewWritableFile(file_name, &file, env_options); std::unique_ptr writer; writer.reset(new WritableFileWriter(std::move(file), file_name, env_options, - env, nullptr /* stats */, + clock, nullptr /* stats */, options.listeners)); std::string record; @@ -48,24 +50,24 @@ void RunBenchmark() { HistogramImpl hist; - uint64_t start_time = env->NowMicros(); + uint64_t start_time = clock->NowMicros(); for (int i = 0; i < FLAGS_num_records; i++) { - uint64_t start_nanos = env->NowNanos(); + uint64_t start_nanos = clock->NowNanos(); writer->Append(record); writer->Flush(); if (FLAGS_enable_sync) { writer->Sync(false); } - hist.Add(env->NowNanos() - start_nanos); + hist.Add(clock->NowNanos() - start_nanos); if (i % 1000 == 1) { fprintf(stderr, "Wrote %d records...\n", i); } int time_to_sleep = - (i + 1) * FLAGS_record_interval - (env->NowMicros() - start_time); + (i + 1) * FLAGS_record_interval - (clock->NowMicros() - start_time); if (time_to_sleep > 0) { - env->SleepForMicroseconds(time_to_sleep); + clock->SleepForMicroseconds(time_to_sleep); } } diff --git a/util/rate_limiter.cc b/util/rate_limiter.cc index d60434794..600e44f9e 100644 --- a/util/rate_limiter.cc +++ b/util/rate_limiter.cc @@ -8,9 +8,10 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. #include "util/rate_limiter.h" + #include "monitoring/statistics.h" #include "port/port.h" -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "util/aligned_buffer.h" @@ -43,22 +44,22 @@ struct GenericRateLimiter::Req { bool granted; }; -GenericRateLimiter::GenericRateLimiter(int64_t rate_bytes_per_sec, - int64_t refill_period_us, - int32_t fairness, RateLimiter::Mode mode, - Env* env, bool auto_tuned) +GenericRateLimiter::GenericRateLimiter( + int64_t rate_bytes_per_sec, int64_t refill_period_us, int32_t fairness, + RateLimiter::Mode mode, const std::shared_ptr& clock, + bool auto_tuned) : RateLimiter(mode), refill_period_us_(refill_period_us), rate_bytes_per_sec_(auto_tuned ? rate_bytes_per_sec / 2 : rate_bytes_per_sec), refill_bytes_per_period_( CalculateRefillBytesPerPeriod(rate_bytes_per_sec_)), - env_(env), + clock_(clock), stop_(false), exit_cv_(&request_mutex_), requests_to_wait_(0), available_bytes_(0), - next_refill_us_(NowMicrosMonotonic(env_)), + next_refill_us_(NowMicrosMonotonic(clock_)), fairness_(fairness > 100 ? 100 : fairness), rnd_((uint32_t)time(nullptr)), leader_(nullptr), @@ -66,7 +67,7 @@ GenericRateLimiter::GenericRateLimiter(int64_t rate_bytes_per_sec, num_drains_(0), prev_num_drains_(0), max_bytes_per_sec_(rate_bytes_per_sec), - tuned_time_(NowMicrosMonotonic(env_)) { + tuned_time_(NowMicrosMonotonic(clock_)) { total_requests_[0] = 0; total_requests_[1] = 0; total_bytes_through_[0] = 0; @@ -108,7 +109,7 @@ void GenericRateLimiter::Request(int64_t bytes, const Env::IOPriority pri, if (auto_tuned_) { static const int kRefillsPerTune = 100; - std::chrono::microseconds now(NowMicrosMonotonic(env_)); + std::chrono::microseconds now(NowMicrosMonotonic(clock_)); if (now - tuned_time_ >= kRefillsPerTune * std::chrono::microseconds(refill_period_us_)) { Status s = Tune(); @@ -149,12 +150,12 @@ void GenericRateLimiter::Request(int64_t bytes, const Env::IOPriority pri, (!queue_[Env::IO_LOW].empty() && &r == queue_[Env::IO_LOW].front()))) { leader_ = &r; - int64_t delta = next_refill_us_ - NowMicrosMonotonic(env_); + int64_t delta = next_refill_us_ - NowMicrosMonotonic(clock_); delta = delta > 0 ? delta : 0; if (delta == 0) { timedout = true; } else { - int64_t wait_until = env_->NowMicros() + delta; + int64_t wait_until = clock_->NowMicros() + delta; RecordTick(stats, NUMBER_RATE_LIMITER_DRAINS); ++num_drains_; timedout = r.cv.TimedWait(wait_until); @@ -229,7 +230,7 @@ void GenericRateLimiter::Request(int64_t bytes, const Env::IOPriority pri, void GenericRateLimiter::Refill() { TEST_SYNC_POINT("GenericRateLimiter::Refill"); - next_refill_us_ = NowMicrosMonotonic(env_) + refill_period_us_; + next_refill_us_ = NowMicrosMonotonic(clock_) + refill_period_us_; // Carry over the left over quota from the last period auto refill_bytes_per_period = refill_bytes_per_period_.load(std::memory_order_relaxed); @@ -284,7 +285,7 @@ Status GenericRateLimiter::Tune() { const int kAllowedRangeFactor = 20; std::chrono::microseconds prev_tuned_time = tuned_time_; - tuned_time_ = std::chrono::microseconds(NowMicrosMonotonic(env_)); + tuned_time_ = std::chrono::microseconds(NowMicrosMonotonic(clock_)); int64_t elapsed_intervals = (tuned_time_ - prev_tuned_time + std::chrono::microseconds(refill_period_us_) - @@ -334,7 +335,7 @@ RateLimiter* NewGenericRateLimiter( assert(refill_period_us > 0); assert(fairness > 0); return new GenericRateLimiter(rate_bytes_per_sec, refill_period_us, fairness, - mode, Env::Default(), auto_tuned); + mode, SystemClock::Default(), auto_tuned); } } // namespace ROCKSDB_NAMESPACE diff --git a/util/rate_limiter.h b/util/rate_limiter.h index 5f047a567..bfb4f3502 100644 --- a/util/rate_limiter.h +++ b/util/rate_limiter.h @@ -13,9 +13,11 @@ #include #include #include + #include "port/port.h" #include "rocksdb/env.h" #include "rocksdb/rate_limiter.h" +#include "rocksdb/system_clock.h" #include "util/mutexlock.h" #include "util/random.h" @@ -24,7 +26,8 @@ namespace ROCKSDB_NAMESPACE { class GenericRateLimiter : public RateLimiter { public: GenericRateLimiter(int64_t refill_bytes, int64_t refill_period_us, - int32_t fairness, RateLimiter::Mode mode, Env* env, + int32_t fairness, RateLimiter::Mode mode, + const std::shared_ptr& clock, bool auto_tuned); virtual ~GenericRateLimiter(); @@ -71,8 +74,8 @@ class GenericRateLimiter : public RateLimiter { int64_t CalculateRefillBytesPerPeriod(int64_t rate_bytes_per_sec); Status Tune(); - uint64_t NowMicrosMonotonic(Env* env) { - return env->NowNanos() / std::milli::den; + uint64_t NowMicrosMonotonic(const std::shared_ptr& clock) { + return clock->NowNanos() / std::milli::den; } // This mutex guard all internal states @@ -85,7 +88,7 @@ class GenericRateLimiter : public RateLimiter { int64_t rate_bytes_per_sec_; // This variable can be changed dynamically. std::atomic refill_bytes_per_period_; - Env* const env_; + std::shared_ptr clock_; bool stop_; port::CondVar exit_cv_; diff --git a/util/rate_limiter_test.cc b/util/rate_limiter_test.cc index 90b221f84..a979dfd5c 100644 --- a/util/rate_limiter_test.cc +++ b/util/rate_limiter_test.cc @@ -14,7 +14,7 @@ #include #include "db/db_test_util.h" -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" #include "util/random.h" @@ -26,8 +26,8 @@ class RateLimiterTest : public testing::Test {}; TEST_F(RateLimiterTest, OverflowRate) { GenericRateLimiter limiter(port::kMaxInt64, 1000, 10, - RateLimiter::Mode::kWritesOnly, Env::Default(), - false /* auto_tuned */); + RateLimiter::Mode::kWritesOnly, + SystemClock::Default(), false /* auto_tuned */); ASSERT_GT(limiter.GetSingleBurstBytes(), 1000000000ll); } @@ -38,9 +38,10 @@ TEST_F(RateLimiterTest, StartStop) { TEST_F(RateLimiterTest, Modes) { for (auto mode : {RateLimiter::Mode::kWritesOnly, RateLimiter::Mode::kReadsOnly, RateLimiter::Mode::kAllIo}) { - GenericRateLimiter limiter( - 2000 /* rate_bytes_per_sec */, 1000 * 1000 /* refill_period_us */, - 10 /* fairness */, mode, Env::Default(), false /* auto_tuned */); + GenericRateLimiter limiter(2000 /* rate_bytes_per_sec */, + 1000 * 1000 /* refill_period_us */, + 10 /* fairness */, mode, SystemClock::Default(), + false /* auto_tuned */); limiter.Request(1000 /* bytes */, Env::IO_HIGH, nullptr /* stats */, RateLimiter::OpType::kRead); if (mode == RateLimiter::Mode::kWritesOnly) { @@ -72,13 +73,13 @@ TEST_F(RateLimiterTest, Rate) { }; auto writer = [](void* p) { - auto* thread_env = Env::Default(); + const auto& thread_clock = SystemClock::Default(); auto* arg = static_cast(p); // Test for 2 seconds - auto until = thread_env->NowMicros() + 2 * 1000000; - Random r((uint32_t)(thread_env->NowNanos() % + auto until = thread_clock->NowMicros() + 2 * 1000000; + Random r((uint32_t)(thread_clock->NowNanos() % std::numeric_limits::max())); - while (thread_env->NowMicros() < until) { + while (thread_clock->NowMicros() < until) { for (int i = 0; i < static_cast(r.Skewed(arg->burst) + 1); ++i) { arg->limiter->Request(r.Uniform(arg->request_size - 1) + 1, Env::IO_HIGH, nullptr /* stats */, @@ -169,7 +170,7 @@ TEST_F(RateLimiterTest, LimitChangeTest) { std::shared_ptr limiter = std::make_shared( target, refill_period, 10, RateLimiter::Mode::kWritesOnly, - Env::Default(), false /* auto_tuned */); + SystemClock::Default(), false /* auto_tuned */); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency( {{"GenericRateLimiter::Request", "RateLimiterTest::LimitChangeTest:changeLimitStart"}, @@ -204,7 +205,8 @@ TEST_F(RateLimiterTest, AutoTuneIncreaseWhenFull) { std::unique_ptr rate_limiter(new GenericRateLimiter( 1000 /* rate_bytes_per_sec */, std::chrono::microseconds(kTimePerRefill).count(), 10 /* fairness */, - RateLimiter::Mode::kWritesOnly, &special_env, true /* auto_tuned */)); + RateLimiter::Mode::kWritesOnly, special_env.GetSystemClock(), + true /* auto_tuned */)); // Use callback to advance time because we need to advance (1) after Request() // has determined the bytes are not available; and (2) before Refill() diff --git a/util/repeatable_thread.h b/util/repeatable_thread.h index f78b1a8c2..1e12c4840 100644 --- a/util/repeatable_thread.h +++ b/util/repeatable_thread.h @@ -8,11 +8,9 @@ #include #include +#include "monitoring/instrumented_mutex.h" #include "port/port.h" -#include "rocksdb/env.h" -#ifndef NDEBUG -#include "test_util/mock_time_env.h" -#endif // !NDEBUG +#include "rocksdb/system_clock.h" #include "util/mutexlock.h" namespace ROCKSDB_NAMESPACE { @@ -23,14 +21,15 @@ namespace ROCKSDB_NAMESPACE { class RepeatableThread { public: RepeatableThread(std::function function, - const std::string& thread_name, Env* env, uint64_t delay_us, + const std::string& thread_name, + const std::shared_ptr& clock, uint64_t delay_us, uint64_t initial_delay_us = 0) : function_(function), thread_name_("rocksdb:" + thread_name), - env_(env), + clock_(clock), delay_us_(delay_us), initial_delay_us_(initial_delay_us), - mutex_(env), + mutex_(clock), cond_var_(&mutex_), running_(true), #ifndef NDEBUG @@ -59,7 +58,7 @@ class RepeatableThread { #ifndef NDEBUG // Wait until RepeatableThread starting waiting, call the optional callback, // then wait for one run of RepeatableThread. Tests can use provide a - // custom env object to mock time, and use the callback here to bump current + // custom clock object to mock time, and use the callback here to bump current // time and trigger RepeatableThread. See repeatable_thread_test for example. // // Note: only support one caller of this method. @@ -83,14 +82,14 @@ class RepeatableThread { bool wait(uint64_t delay) { InstrumentedMutexLock l(&mutex_); if (running_ && delay > 0) { - uint64_t wait_until = env_->NowMicros() + delay; + uint64_t wait_until = clock_->NowMicros() + delay; #ifndef NDEBUG waiting_ = true; cond_var_.SignalAll(); #endif while (running_) { cond_var_.TimedWait(wait_until); - if (env_->NowMicros() >= wait_until) { + if (clock_->NowMicros() >= wait_until) { break; } } @@ -130,7 +129,8 @@ class RepeatableThread { const std::function function_; const std::string thread_name_; - Env* const env_; + const std::shared_ptr clock_; + ; const uint64_t delay_us_; const uint64_t initial_delay_us_; diff --git a/util/repeatable_thread_test.cc b/util/repeatable_thread_test.cc index a18aa6cd1..5aa6b6c37 100644 --- a/util/repeatable_thread_test.cc +++ b/util/repeatable_thread_test.cc @@ -3,44 +3,46 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). +#include "util/repeatable_thread.h" + #include #include #include "db/db_test_util.h" +#include "test_util/mock_time_env.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" -#include "util/repeatable_thread.h" class RepeatableThreadTest : public testing::Test { public: RepeatableThreadTest() - : mock_env_(new ROCKSDB_NAMESPACE::MockTimeEnv( - ROCKSDB_NAMESPACE::Env::Default())) {} + : mock_clock_(std::make_shared( + ROCKSDB_NAMESPACE::SystemClock::Default())) {} protected: - std::unique_ptr mock_env_; + std::shared_ptr mock_clock_; }; TEST_F(RepeatableThreadTest, TimedTest) { constexpr uint64_t kSecond = 1000000; // 1s = 1000000us constexpr int kIteration = 3; - ROCKSDB_NAMESPACE::Env* env = ROCKSDB_NAMESPACE::Env::Default(); + const auto& clock = ROCKSDB_NAMESPACE::SystemClock::Default(); ROCKSDB_NAMESPACE::port::Mutex mutex; ROCKSDB_NAMESPACE::port::CondVar test_cv(&mutex); int count = 0; - uint64_t prev_time = env->NowMicros(); + uint64_t prev_time = clock->NowMicros(); ROCKSDB_NAMESPACE::RepeatableThread thread( [&] { ROCKSDB_NAMESPACE::MutexLock l(&mutex); count++; - uint64_t now = env->NowMicros(); + uint64_t now = clock->NowMicros(); assert(count == 1 || prev_time + 1 * kSecond <= now); prev_time = now; if (count >= kIteration) { test_cv.SignalAll(); } }, - "rt_test", env, 1 * kSecond); + "rt_test", clock, 1 * kSecond); // Wait for execution finish. { ROCKSDB_NAMESPACE::MutexLock l(&mutex); @@ -56,7 +58,7 @@ TEST_F(RepeatableThreadTest, TimedTest) { TEST_F(RepeatableThreadTest, MockEnvTest) { constexpr uint64_t kSecond = 1000000; // 1s = 1000000us constexpr int kIteration = 3; - mock_env_->set_current_time(0); // in seconds + mock_clock_->SetCurrentTime(0); // in seconds std::atomic count{0}; #if defined(OS_MACOSX) && !defined(NDEBUG) @@ -72,26 +74,27 @@ TEST_F(RepeatableThreadTest, MockEnvTest) { // immediately, the mutex will not be released, and // RepeatableThread::TEST_WaitForRun never has a chance to execute the // callback which, in this case, updates the result returned by - // mock_env->NowMicros. Consequently, RepeatableThread::wait cannot + // mock_clock->NowMicros. Consequently, RepeatableThread::wait cannot // break out of the loop, causing test to hang. The extra 1000 seconds // is a best-effort approach because there seems no reliable and // deterministic way to provide the aforementioned guarantee. By the // time RepeatableThread::wait is called, it is no guarantee that the - // delay + mock_env->NowMicros will be greater than the current real + // delay + mock_clock->NowMicros will be greater than the current real // time. However, 1000 seconds should be sufficient in most cases. uint64_t time_us = *reinterpret_cast(arg); - if (time_us < mock_env_->RealNowMicros()) { - *reinterpret_cast(arg) = mock_env_->RealNowMicros() + 1000; + if (time_us < mock_clock_->RealNowMicros()) { + *reinterpret_cast(arg) = + mock_clock_->RealNowMicros() + 1000; } }); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); #endif // OS_MACOSX && !NDEBUG ROCKSDB_NAMESPACE::RepeatableThread thread( - [&] { count++; }, "rt_test", mock_env_.get(), 1 * kSecond, 1 * kSecond); + [&] { count++; }, "rt_test", mock_clock_, 1 * kSecond, 1 * kSecond); for (int i = 1; i <= kIteration; i++) { // Bump current time - thread.TEST_WaitForRun([&] { mock_env_->set_current_time(i); }); + thread.TEST_WaitForRun([&] { mock_clock_->SetCurrentTime(i); }); } // Test function should be exectued exactly kIteraion times. ASSERT_EQ(kIteration, count.load()); diff --git a/util/ribbon_test.cc b/util/ribbon_test.cc index 9067c9719..f9bcef418 100644 --- a/util/ribbon_test.cc +++ b/util/ribbon_test.cc @@ -5,6 +5,7 @@ #include +#include "rocksdb/system_clock.h" #include "test_util/testharness.h" #include "util/bloom_impl.h" #include "util/coding.h" @@ -496,8 +497,8 @@ TYPED_TEST(RibbonTypeParamTest, CompactnessAndBacktrackAndFpRate) { Index fp_count = 0; cur = other_keys_begin; { - ROCKSDB_NAMESPACE::StopWatchNano timer(ROCKSDB_NAMESPACE::Env::Default(), - true); + ROCKSDB_NAMESPACE::StopWatchNano timer( + ROCKSDB_NAMESPACE::SystemClock::Default(), true); while (cur != other_keys_end) { bool fp = soln.FilterQuery(*cur, hasher); fp_count += fp ? 1 : 0; @@ -523,8 +524,8 @@ TYPED_TEST(RibbonTypeParamTest, CompactnessAndBacktrackAndFpRate) { if (test_interleaved) { Index ifp_count = 0; cur = other_keys_begin; - ROCKSDB_NAMESPACE::StopWatchNano timer(ROCKSDB_NAMESPACE::Env::Default(), - true); + ROCKSDB_NAMESPACE::StopWatchNano timer( + ROCKSDB_NAMESPACE::SystemClock::Default(), true); while (cur != other_keys_end) { ifp_count += isoln.FilterQuery(*cur, hasher) ? 1 : 0; ++cur; @@ -551,8 +552,8 @@ TYPED_TEST(RibbonTypeParamTest, CompactnessAndBacktrackAndFpRate) { if (ibytes >= /* minimum Bloom impl bytes*/ 64) { Index bfp_count = 0; cur = other_keys_begin; - ROCKSDB_NAMESPACE::StopWatchNano timer(ROCKSDB_NAMESPACE::Env::Default(), - true); + ROCKSDB_NAMESPACE::StopWatchNano timer( + ROCKSDB_NAMESPACE::SystemClock::Default(), true); while (cur != other_keys_end) { uint64_t h = hasher.GetHash(*cur); uint32_t h1 = ROCKSDB_NAMESPACE::Lower32of64(h); diff --git a/util/stop_watch.h b/util/stop_watch.h index ad4905960..b6421c631 100644 --- a/util/stop_watch.h +++ b/util/stop_watch.h @@ -5,7 +5,7 @@ // #pragma once #include "monitoring/statistics.h" -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" namespace ROCKSDB_NAMESPACE { // Auto-scoped. @@ -14,10 +14,10 @@ namespace ROCKSDB_NAMESPACE { // and overwrite is true, it will be added to *elapsed if overwrite is false. class StopWatch { public: - StopWatch(Env* const env, Statistics* statistics, const uint32_t hist_type, - uint64_t* elapsed = nullptr, bool overwrite = true, - bool delay_enabled = false) - : env_(env), + StopWatch(const std::shared_ptr& clock, Statistics* statistics, + const uint32_t hist_type, uint64_t* elapsed = nullptr, + bool overwrite = true, bool delay_enabled = false) + : clock_(clock), statistics_(statistics), hist_type_(hist_type), elapsed_(elapsed), @@ -29,15 +29,15 @@ class StopWatch { delay_enabled_(delay_enabled), total_delay_(0), delay_start_time_(0), - start_time_((stats_enabled_ || elapsed != nullptr) ? env->NowMicros() + start_time_((stats_enabled_ || elapsed != nullptr) ? clock->NowMicros() : 0) {} ~StopWatch() { if (elapsed_) { if (overwrite_) { - *elapsed_ = env_->NowMicros() - start_time_; + *elapsed_ = clock_->NowMicros() - start_time_; } else { - *elapsed_ += env_->NowMicros() - start_time_; + *elapsed_ += clock_->NowMicros() - start_time_; } } if (elapsed_ && delay_enabled_) { @@ -47,7 +47,7 @@ class StopWatch { statistics_->reportTimeToHistogram( hist_type_, (elapsed_ != nullptr) ? *elapsed_ - : (env_->NowMicros() - start_time_)); + : (clock_->NowMicros() - start_time_)); } } @@ -55,13 +55,13 @@ class StopWatch { // if delay_start_time_ is not 0, it means we are already tracking delay, // so delay_start_time_ should not be overwritten if (elapsed_ && delay_enabled_ && delay_start_time_ == 0) { - delay_start_time_ = env_->NowMicros(); + delay_start_time_ = clock_->NowMicros(); } } void DelayStop() { if (elapsed_ && delay_enabled_ && delay_start_time_ != 0) { - total_delay_ += env_->NowMicros() - delay_start_time_; + total_delay_ += clock_->NowMicros() - delay_start_time_; } // reset to 0 means currently no delay is being tracked, so two consecutive // calls to DelayStop will not increase total_delay_ @@ -73,7 +73,7 @@ class StopWatch { uint64_t start_time() const { return start_time_; } private: - Env* const env_; + const std::shared_ptr clock_; Statistics* statistics_; const uint32_t hist_type_; uint64_t* elapsed_; @@ -88,17 +88,18 @@ class StopWatch { // a nano second precision stopwatch class StopWatchNano { public: - explicit StopWatchNano(Env* const env, bool auto_start = false) - : env_(env), start_(0) { + explicit StopWatchNano(const std::shared_ptr& clock, + bool auto_start = false) + : clock_(clock), start_(0) { if (auto_start) { Start(); } } - void Start() { start_ = env_->NowNanos(); } + void Start() { start_ = clock_->NowNanos(); } uint64_t ElapsedNanos(bool reset = false) { - auto now = env_->NowNanos(); + auto now = clock_->NowNanos(); auto elapsed = now - start_; if (reset) { start_ = now; @@ -107,11 +108,11 @@ class StopWatchNano { } uint64_t ElapsedNanosSafe(bool reset = false) { - return (env_ != nullptr) ? ElapsedNanos(reset) : 0U; + return (clock_ != nullptr) ? ElapsedNanos(reset) : 0U; } private: - Env* const env_; + const std::shared_ptr clock_; uint64_t start_; }; diff --git a/util/timer.h b/util/timer.h index b6ee42ed0..15520ec2f 100644 --- a/util/timer.h +++ b/util/timer.h @@ -14,7 +14,7 @@ #include #include "monitoring/instrumented_mutex.h" -#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "test_util/sync_point.h" #include "util/mutexlock.h" @@ -38,9 +38,9 @@ namespace ROCKSDB_NAMESPACE { // A map from a function name to the function keeps track of all the functions. class Timer { public: - explicit Timer(Env* env) - : env_(env), - mutex_(env), + explicit Timer(const std::shared_ptr& clock) + : clock_(clock), + mutex_(clock), cond_var_(&mutex_), running_(false), executing_task_(false) {} @@ -60,9 +60,9 @@ class Timer { const std::string& fn_name, uint64_t start_after_us, uint64_t repeat_every_us) { - std::unique_ptr fn_info( - new FunctionInfo(std::move(fn), fn_name, - env_->NowMicros() + start_after_us, repeat_every_us)); + std::unique_ptr fn_info(new FunctionInfo( + std::move(fn), fn_name, clock_->NowMicros() + start_after_us, + repeat_every_us)); { InstrumentedMutexLock l(&mutex_); auto it = map_.find(fn_name); @@ -73,7 +73,7 @@ class Timer { // If it already exists, overriding it. it->second->fn = std::move(fn_info->fn); it->second->valid = true; - it->second->next_run_time_us = env_->NowMicros() + start_after_us; + it->second->next_run_time_us = clock_->NowMicros() + start_after_us; it->second->repeat_every_us = repeat_every_us; } } @@ -151,7 +151,7 @@ class Timer { #ifndef NDEBUG // Wait until Timer starting waiting, call the optional callback, then wait // for Timer waiting again. - // Tests can provide a custom env object to mock time, and use the callback + // Tests can provide a custom Clock object to mock time, and use the callback // here to bump current time and trigger Timer. See timer_test for example. // // Note: only support one caller of this method. @@ -160,18 +160,18 @@ class Timer { // It act as a spin lock while (executing_task_ || (!heap_.empty() && - heap_.top()->next_run_time_us <= env_->NowMicros())) { - cond_var_.TimedWait(env_->NowMicros() + 1000); + heap_.top()->next_run_time_us <= clock_->NowMicros())) { + cond_var_.TimedWait(clock_->NowMicros() + 1000); } if (callback != nullptr) { callback(); } cond_var_.SignalAll(); do { - cond_var_.TimedWait(env_->NowMicros() + 1000); - } while ( - executing_task_ || - (!heap_.empty() && heap_.top()->next_run_time_us <= env_->NowMicros())); + cond_var_.TimedWait(clock_->NowMicros() + 1000); + } while (executing_task_ || + (!heap_.empty() && + heap_.top()->next_run_time_us <= clock_->NowMicros())); } size_t TEST_GetPendingTaskNum() const { @@ -208,7 +208,7 @@ class Timer { continue; } - if (current_fn->next_run_time_us <= env_->NowMicros()) { + if (current_fn->next_run_time_us <= clock_->NowMicros()) { // make a copy of the function so it won't be changed after // mutex_.unlock. std::function fn = current_fn->fn; @@ -229,8 +229,8 @@ class Timer { // current_fn may be cancelled already. if (current_fn->IsValid() && current_fn->repeat_every_us > 0) { assert(running_); - current_fn->next_run_time_us = env_->NowMicros() + - current_fn->repeat_every_us; + current_fn->next_run_time_us = + clock_->NowMicros() + current_fn->repeat_every_us; // Schedule new work into the heap with new time. heap_.push(current_fn); @@ -310,7 +310,7 @@ class Timer { } }; - Env* const env_; + const std::shared_ptr clock_; // This mutex controls both the heap_ and the map_. It needs to be held for // making any changes in them. mutable InstrumentedMutex mutex_; diff --git a/util/timer_test.cc b/util/timer_test.cc index 761143ad3..3158d7516 100644 --- a/util/timer_test.cc +++ b/util/timer_test.cc @@ -6,24 +6,27 @@ #include "util/timer.h" #include "db/db_test_util.h" +#include "test_util/mock_time_env.h" namespace ROCKSDB_NAMESPACE { class TimerTest : public testing::Test { public: - TimerTest() : mock_env_(new MockTimeEnv(Env::Default())) {} + TimerTest() + : mock_clock_(std::make_shared(SystemClock::Default())) { + } protected: - std::unique_ptr mock_env_; + std::shared_ptr mock_clock_; - void SetUp() override { mock_env_->InstallTimedWaitFixCallback(); } + void SetUp() override { mock_clock_->InstallTimedWaitFixCallback(); } const int kUsPerSec = 1000000; }; TEST_F(TimerTest, SingleScheduleOnce) { const int kInitDelayUs = 1 * kUsPerSec; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); int count = 0; timer.Add([&] { count++; }, "fn_sch_test", kInitDelayUs, 0); @@ -33,7 +36,7 @@ TEST_F(TimerTest, SingleScheduleOnce) { ASSERT_EQ(0, count); // Wait for execution to finish timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kInitDelayUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kInitDelayUs); }); ASSERT_EQ(1, count); ASSERT_TRUE(timer.Shutdown()); @@ -42,7 +45,7 @@ TEST_F(TimerTest, SingleScheduleOnce) { TEST_F(TimerTest, MultipleScheduleOnce) { const int kInitDelay1Us = 1 * kUsPerSec; const int kInitDelay2Us = 3 * kUsPerSec; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); int count1 = 0; timer.Add([&] { count1++; }, "fn_sch_test1", kInitDelay1Us, 0); @@ -55,13 +58,13 @@ TEST_F(TimerTest, MultipleScheduleOnce) { ASSERT_EQ(0, count2); timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kInitDelay1Us); }); + [&] { mock_clock_->MockSleepForMicroseconds(kInitDelay1Us); }); ASSERT_EQ(1, count1); ASSERT_EQ(0, count2); timer.TEST_WaitForRun([&] { - mock_env_->MockSleepForMicroseconds(kInitDelay2Us - kInitDelay1Us); + mock_clock_->MockSleepForMicroseconds(kInitDelay2Us - kInitDelay1Us); }); ASSERT_EQ(1, count1); @@ -75,7 +78,7 @@ TEST_F(TimerTest, SingleScheduleRepeatedly) { const int kInitDelayUs = 1 * kUsPerSec; const int kRepeatUs = 1 * kUsPerSec; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); int count = 0; timer.Add([&] { count++; }, "fn_sch_test", kInitDelayUs, kRepeatUs); @@ -83,14 +86,14 @@ TEST_F(TimerTest, SingleScheduleRepeatedly) { ASSERT_EQ(0, count); timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kInitDelayUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kInitDelayUs); }); ASSERT_EQ(1, count); // Wait for execution to finish for (int i = 1; i < kIterations; i++) { timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kRepeatUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kRepeatUs); }); } ASSERT_EQ(kIterations, count); @@ -105,7 +108,7 @@ TEST_F(TimerTest, MultipleScheduleRepeatedly) { const int kRepeatUs = 2 * kUsPerSec; const int kLargeRepeatUs = 100 * kUsPerSec; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); int count1 = 0; timer.Add([&] { count1++; }, "fn_sch_test1", kInitDelay1Us, kRepeatUs); @@ -123,7 +126,7 @@ TEST_F(TimerTest, MultipleScheduleRepeatedly) { // Wait for execution to finish for (int i = 1; i < kIterations * (kRepeatUs / kUsPerSec); i++) { timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(1 * kUsPerSec); }); + [&] { mock_clock_->MockSleepForMicroseconds(1 * kUsPerSec); }); ASSERT_EQ((i + 2) / (kRepeatUs / kUsPerSec), count1); ASSERT_EQ((i + 1) / (kRepeatUs / kUsPerSec), count2); @@ -135,7 +138,7 @@ TEST_F(TimerTest, MultipleScheduleRepeatedly) { // Wait for execution to finish timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(1 * kUsPerSec); }); + [&] { mock_clock_->MockSleepForMicroseconds(1 * kUsPerSec); }); ASSERT_EQ(kIterations, count1); ASSERT_EQ(kIterations, count2); ASSERT_EQ(1, count3); @@ -147,8 +150,8 @@ TEST_F(TimerTest, MultipleScheduleRepeatedly) { // execute the long interval one timer.TEST_WaitForRun([&] { - mock_env_->MockSleepForMicroseconds( - kLargeRepeatUs - static_cast(mock_env_->NowMicros())); + mock_clock_->MockSleepForMicroseconds( + kLargeRepeatUs - static_cast(mock_clock_->NowMicros())); }); ASSERT_EQ(2, count3); @@ -165,7 +168,7 @@ TEST_F(TimerTest, AddAfterStartTest) { {{"Timer::Run::Waiting", "TimerTest:AddAfterStartTest:1"}}); SyncPoint::GetInstance()->EnableProcessing(); - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); ASSERT_TRUE(timer.Start()); @@ -175,12 +178,12 @@ TEST_F(TimerTest, AddAfterStartTest) { ASSERT_EQ(0, count); // Wait for execution to finish timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kInitDelayUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kInitDelayUs); }); ASSERT_EQ(1, count); for (int i = 1; i < kIterations; i++) { timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kRepeatUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kRepeatUs); }); } ASSERT_EQ(kIterations, count); @@ -190,7 +193,7 @@ TEST_F(TimerTest, AddAfterStartTest) { TEST_F(TimerTest, CancelRunningTask) { static constexpr char kTestFuncName[] = "test_func"; const int kRepeatUs = 1 * kUsPerSec; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); ASSERT_TRUE(timer.Start()); int* value = new int; *value = 0; @@ -217,7 +220,7 @@ TEST_F(TimerTest, CancelRunningTask) { delete value; value = nullptr; }); - mock_env_->MockSleepForMicroseconds(kRepeatUs); + mock_clock_->MockSleepForMicroseconds(kRepeatUs); control_thr.join(); ASSERT_TRUE(timer.Shutdown()); } @@ -226,7 +229,7 @@ TEST_F(TimerTest, ShutdownRunningTask) { const int kRepeatUs = 1 * kUsPerSec; constexpr char kTestFunc1Name[] = "test_func1"; constexpr char kTestFunc2Name[] = "test_func2"; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); SyncPoint::GetInstance()->DisableProcessing(); SyncPoint::GetInstance()->LoadDependency({ @@ -255,7 +258,7 @@ TEST_F(TimerTest, ShutdownRunningTask) { TEST_SYNC_POINT("TimerTest::ShutdownRunningTest:BeforeShutdown"); timer.Shutdown(); }); - mock_env_->MockSleepForMicroseconds(kRepeatUs); + mock_clock_->MockSleepForMicroseconds(kRepeatUs); control_thr.join(); delete value; } @@ -265,7 +268,7 @@ TEST_F(TimerTest, AddSameFuncName) { const int kRepeat1Us = 5 * kUsPerSec; const int kRepeat2Us = 4 * kUsPerSec; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); ASSERT_TRUE(timer.Start()); int func_counter1 = 0; @@ -285,14 +288,14 @@ TEST_F(TimerTest, AddSameFuncName) { ASSERT_EQ(0, func_counter2); timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kInitDelayUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kInitDelayUs); }); ASSERT_EQ(0, func_counter1); ASSERT_EQ(1, func2_counter); ASSERT_EQ(1, func_counter2); timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kRepeat1Us); }); + [&] { mock_clock_->MockSleepForMicroseconds(kRepeat1Us); }); ASSERT_EQ(0, func_counter1); ASSERT_EQ(2, func2_counter); @@ -306,22 +309,22 @@ TEST_F(TimerTest, RepeatIntervalWithFuncRunningTime) { const int kRepeatUs = 5 * kUsPerSec; const int kFuncRunningTimeUs = 1 * kUsPerSec; - Timer timer(mock_env_.get()); + Timer timer(mock_clock_); ASSERT_TRUE(timer.Start()); int func_counter = 0; timer.Add( [&] { - mock_env_->MockSleepForMicroseconds(kFuncRunningTimeUs); + mock_clock_->MockSleepForMicroseconds(kFuncRunningTimeUs); func_counter++; }, "func", kInitDelayUs, kRepeatUs); ASSERT_EQ(0, func_counter); timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kInitDelayUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kInitDelayUs); }); ASSERT_EQ(1, func_counter); - ASSERT_EQ(kInitDelayUs + kFuncRunningTimeUs, mock_env_->NowMicros()); + ASSERT_EQ(kInitDelayUs + kFuncRunningTimeUs, mock_clock_->NowMicros()); // After repeat interval time, the function is not executed, as running // the function takes some time (`kFuncRunningTimeSec`). The repeat interval @@ -329,13 +332,13 @@ TEST_F(TimerTest, RepeatIntervalWithFuncRunningTime) { // next call. uint64_t next_abs_interval_time_us = kInitDelayUs + kRepeatUs; timer.TEST_WaitForRun([&] { - mock_env_->set_current_time(next_abs_interval_time_us / kUsPerSec); + mock_clock_->SetCurrentTime(next_abs_interval_time_us / kUsPerSec); }); ASSERT_EQ(1, func_counter); // After the function running time, it's executed again timer.TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kFuncRunningTimeUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kFuncRunningTimeUs); }); ASSERT_EQ(2, func_counter); ASSERT_TRUE(timer.Shutdown()); @@ -345,14 +348,14 @@ TEST_F(TimerTest, DestroyRunningTimer) { const int kInitDelayUs = 1 * kUsPerSec; const int kRepeatUs = 1 * kUsPerSec; - auto timer_ptr = new Timer(mock_env_.get()); + auto timer_ptr = new Timer(mock_clock_); int count = 0; timer_ptr->Add([&] { count++; }, "fn_sch_test", kInitDelayUs, kRepeatUs); ASSERT_TRUE(timer_ptr->Start()); timer_ptr->TEST_WaitForRun( - [&] { mock_env_->MockSleepForMicroseconds(kInitDelayUs); }); + [&] { mock_clock_->MockSleepForMicroseconds(kInitDelayUs); }); // delete a running timer should not cause any exception delete timer_ptr; @@ -360,7 +363,7 @@ TEST_F(TimerTest, DestroyRunningTimer) { TEST_F(TimerTest, DestroyTimerWithRunningFunc) { const int kRepeatUs = 1 * kUsPerSec; - auto timer_ptr = new Timer(mock_env_.get()); + auto timer_ptr = new Timer(mock_clock_); SyncPoint::GetInstance()->DisableProcessing(); SyncPoint::GetInstance()->LoadDependency({ @@ -386,7 +389,7 @@ TEST_F(TimerTest, DestroyTimerWithRunningFunc) { TEST_SYNC_POINT("TimerTest::DestroyTimerWithRunningFunc:BeforeDelete"); delete timer_ptr; }); - mock_env_->MockSleepForMicroseconds(kRepeatUs); + mock_clock_->MockSleepForMicroseconds(kRepeatUs); control_thr.join(); } diff --git a/utilities/blob_db/blob_db_impl.cc b/utilities/blob_db/blob_db_impl.cc index 97f23270a..608cd334b 100644 --- a/utilities/blob_db/blob_db_impl.cc +++ b/utilities/blob_db/blob_db_impl.cc @@ -92,6 +92,7 @@ BlobDBImpl::BlobDBImpl(const std::string& dbname, fifo_eviction_seq_(0), evict_expiration_up_to_(0), debug_level_(0) { + clock_ = env_->GetSystemClock(); blob_dir_ = (bdb_options_.path_relative) ? dbname + "/" + bdb_options_.blob_dir : bdb_options_.blob_dir; @@ -757,7 +758,7 @@ Status BlobDBImpl::CreateWriterLocked(const std::shared_ptr& bfile) { constexpr bool do_flush = true; bfile->log_writer_ = std::make_shared( - std::move(fwriter), env_, statistics_, bfile->file_number_, + std::move(fwriter), clock_, statistics_, bfile->file_number_, db_options_.use_fsync, do_flush, boffset); bfile->log_writer_->last_elem_type_ = et; @@ -994,7 +995,7 @@ class BlobDBImpl::BlobInserter : public WriteBatch::Handler { }; Status BlobDBImpl::Write(const WriteOptions& options, WriteBatch* updates) { - StopWatch write_sw(env_, statistics_, BLOB_DB_WRITE_MICROS); + StopWatch write_sw(clock_, statistics_, BLOB_DB_WRITE_MICROS); RecordTick(statistics_, BLOB_DB_NUM_WRITE); uint32_t default_cf_id = static_cast_with_check(DefaultColumnFamily()) @@ -1029,7 +1030,7 @@ Status BlobDBImpl::PutWithTTL(const WriteOptions& options, Status BlobDBImpl::PutUntil(const WriteOptions& options, const Slice& key, const Slice& value, uint64_t expiration) { - StopWatch write_sw(env_, statistics_, BLOB_DB_WRITE_MICROS); + StopWatch write_sw(clock_, statistics_, BLOB_DB_WRITE_MICROS); RecordTick(statistics_, BLOB_DB_NUM_PUT); Status s; WriteBatch batch; @@ -1136,7 +1137,7 @@ Slice BlobDBImpl::GetCompressedSlice(const Slice& raw, if (bdb_options_.compression == kNoCompression) { return raw; } - StopWatch compression_sw(env_, statistics_, BLOB_DB_COMPRESSION_MICROS); + StopWatch compression_sw(clock_, statistics_, BLOB_DB_COMPRESSION_MICROS); CompressionType type = bdb_options_.compression; CompressionOptions opts; CompressionContext context(type); @@ -1156,7 +1157,8 @@ Status BlobDBImpl::DecompressSlice(const Slice& compressed_value, auto cfh = static_cast(DefaultColumnFamily()); { - StopWatch decompression_sw(env_, statistics_, BLOB_DB_DECOMPRESSION_MICROS); + StopWatch decompression_sw(clock_, statistics_, + BLOB_DB_DECOMPRESSION_MICROS); UncompressionContext context(compression_type); UncompressionInfo info(context, UncompressionDict::GetEmptyDict(), compression_type); @@ -1383,7 +1385,7 @@ Status BlobDBImpl::AppendBlob(const std::shared_ptr& bfile, std::vector BlobDBImpl::MultiGet( const ReadOptions& read_options, const std::vector& keys, std::vector* values) { - StopWatch multiget_sw(env_, statistics_, BLOB_DB_MULTIGET_MICROS); + StopWatch multiget_sw(clock_, statistics_, BLOB_DB_MULTIGET_MICROS); RecordTick(statistics_, BLOB_DB_NUM_MULTIGET); // Get a snapshot to avoid blob file get deleted between we // fetch and index entry and reading from the file. @@ -1535,7 +1537,7 @@ Status BlobDBImpl::GetRawBlobFromFile(const Slice& key, uint64_t file_number, Slice blob_record; { - StopWatch read_sw(env_, statistics_, BLOB_DB_BLOB_FILE_READ_MICROS); + StopWatch read_sw(clock_, statistics_, BLOB_DB_BLOB_FILE_READ_MICROS); if (reader->use_direct_io()) { s = reader->Read(IOOptions(), record_offset, static_cast(record_size), &blob_record, nullptr, @@ -1614,7 +1616,7 @@ Status BlobDBImpl::Get(const ReadOptions& read_options, Status BlobDBImpl::Get(const ReadOptions& read_options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* value, uint64_t* expiration) { - StopWatch get_sw(env_, statistics_, BLOB_DB_GET_MICROS); + StopWatch get_sw(clock_, statistics_, BLOB_DB_GET_MICROS); RecordTick(statistics_, BLOB_DB_NUM_GET); return GetImpl(read_options, column_family, key, value, expiration); } @@ -2043,7 +2045,7 @@ Iterator* BlobDBImpl::NewIterator(const ReadOptions& read_options) { auto* iter = db_impl_->NewIteratorImpl( read_options, cfd, snapshot->GetSequenceNumber(), nullptr /*read_callback*/, true /*expose_blob_index*/); - return new BlobDBIterator(own_snapshot, iter, this, env_, statistics_); + return new BlobDBIterator(own_snapshot, iter, this, clock_, statistics_); } Status DestroyBlobDB(const std::string& dbname, const Options& options, diff --git a/utilities/blob_db/blob_db_impl.h b/utilities/blob_db/blob_db_impl.h index e400fae18..ecc302ff1 100644 --- a/utilities/blob_db/blob_db_impl.h +++ b/utilities/blob_db/blob_db_impl.h @@ -38,6 +38,8 @@ namespace ROCKSDB_NAMESPACE { class DBImpl; class ColumnFamilyHandle; class ColumnFamilyData; +class SystemClock; + struct FlushJobInfo; namespace blob_db { @@ -385,7 +387,7 @@ class BlobDBImpl : public BlobDB { void CopyBlobFiles(std::vector>* bfiles_copy); - uint64_t EpochNow() { return env_->NowMicros() / 1000000; } + uint64_t EpochNow() { return clock_->NowMicros() / 1000000; } // Check if inserting a new blob will make DB grow out of space. // If is_fifo = true, FIFO eviction will be triggered to make room for the @@ -400,7 +402,7 @@ class BlobDBImpl : public BlobDB { // the base DB DBImpl* db_impl_; Env* env_; - + std::shared_ptr clock_; // the options that govern the behavior of Blob Storage BlobDBOptions bdb_options_; DBOptions db_options_; diff --git a/utilities/blob_db/blob_db_iterator.h b/utilities/blob_db/blob_db_iterator.h index af07117eb..8d474f393 100644 --- a/utilities/blob_db/blob_db_iterator.h +++ b/utilities/blob_db/blob_db_iterator.h @@ -7,12 +7,14 @@ #ifndef ROCKSDB_LITE #include "db/arena_wrapped_db_iter.h" -#include "monitoring/statistics.h" #include "rocksdb/iterator.h" #include "util/stop_watch.h" #include "utilities/blob_db/blob_db_impl.h" namespace ROCKSDB_NAMESPACE { +class Statistics; +class SystemClock; + namespace blob_db { using ROCKSDB_NAMESPACE::ManagedSnapshot; @@ -20,11 +22,12 @@ using ROCKSDB_NAMESPACE::ManagedSnapshot; class BlobDBIterator : public Iterator { public: BlobDBIterator(ManagedSnapshot* snapshot, ArenaWrappedDBIter* iter, - BlobDBImpl* blob_db, Env* env, Statistics* statistics) + BlobDBImpl* blob_db, const std::shared_ptr& clock, + Statistics* statistics) : snapshot_(snapshot), iter_(iter), blob_db_(blob_db), - env_(env), + clock_(clock), statistics_(statistics) {} virtual ~BlobDBIterator() = default; @@ -44,7 +47,7 @@ class BlobDBIterator : public Iterator { } void SeekToFirst() override { - StopWatch seek_sw(env_, statistics_, BLOB_DB_SEEK_MICROS); + StopWatch seek_sw(clock_, statistics_, BLOB_DB_SEEK_MICROS); RecordTick(statistics_, BLOB_DB_NUM_SEEK); iter_->SeekToFirst(); while (UpdateBlobValue()) { @@ -53,7 +56,7 @@ class BlobDBIterator : public Iterator { } void SeekToLast() override { - StopWatch seek_sw(env_, statistics_, BLOB_DB_SEEK_MICROS); + StopWatch seek_sw(clock_, statistics_, BLOB_DB_SEEK_MICROS); RecordTick(statistics_, BLOB_DB_NUM_SEEK); iter_->SeekToLast(); while (UpdateBlobValue()) { @@ -62,7 +65,7 @@ class BlobDBIterator : public Iterator { } void Seek(const Slice& target) override { - StopWatch seek_sw(env_, statistics_, BLOB_DB_SEEK_MICROS); + StopWatch seek_sw(clock_, statistics_, BLOB_DB_SEEK_MICROS); RecordTick(statistics_, BLOB_DB_NUM_SEEK); iter_->Seek(target); while (UpdateBlobValue()) { @@ -71,7 +74,7 @@ class BlobDBIterator : public Iterator { } void SeekForPrev(const Slice& target) override { - StopWatch seek_sw(env_, statistics_, BLOB_DB_SEEK_MICROS); + StopWatch seek_sw(clock_, statistics_, BLOB_DB_SEEK_MICROS); RecordTick(statistics_, BLOB_DB_NUM_SEEK); iter_->SeekForPrev(target); while (UpdateBlobValue()) { @@ -81,7 +84,7 @@ class BlobDBIterator : public Iterator { void Next() override { assert(Valid()); - StopWatch next_sw(env_, statistics_, BLOB_DB_NEXT_MICROS); + StopWatch next_sw(clock_, statistics_, BLOB_DB_NEXT_MICROS); RecordTick(statistics_, BLOB_DB_NUM_NEXT); iter_->Next(); while (UpdateBlobValue()) { @@ -91,7 +94,7 @@ class BlobDBIterator : public Iterator { void Prev() override { assert(Valid()); - StopWatch prev_sw(env_, statistics_, BLOB_DB_PREV_MICROS); + StopWatch prev_sw(clock_, statistics_, BLOB_DB_PREV_MICROS); RecordTick(statistics_, BLOB_DB_NUM_PREV); iter_->Prev(); while (UpdateBlobValue()) { @@ -137,7 +140,7 @@ class BlobDBIterator : public Iterator { std::unique_ptr snapshot_; std::unique_ptr iter_; BlobDBImpl* blob_db_; - Env* env_; + std::shared_ptr clock_; Statistics* statistics_; Status status_; PinnableSlice value_; diff --git a/utilities/blob_db/blob_db_test.cc b/utilities/blob_db/blob_db_test.cc index cb2d7d8e3..652b082c8 100644 --- a/utilities/blob_db/blob_db_test.cc +++ b/utilities/blob_db/blob_db_test.cc @@ -24,6 +24,7 @@ #include "file/sst_file_manager_impl.h" #include "port/port.h" #include "rocksdb/utilities/debug.h" +#include "test_util/mock_time_env.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" #include "util/random.h" @@ -58,9 +59,11 @@ class BlobDBTest : public testing::Test { BlobDBTest() : dbname_(test::PerThreadDBPath("blob_db_test")), - mock_env_(new MockTimeEnv(Env::Default())), - fault_injection_env_(new FaultInjectionTestEnv(Env::Default())), blob_db_(nullptr) { + mock_clock_ = std::make_shared(SystemClock::Default()); + mock_env_.reset(new CompositeEnvWrapper(Env::Default(), mock_clock_)); + fault_injection_env_.reset(new FaultInjectionTestEnv(Env::Default())); + Status s = DestroyBlobDB(dbname_, Options(), BlobDBOptions()); assert(s.ok()); } @@ -312,7 +315,8 @@ class BlobDBTest : public testing::Test { } const std::string dbname_; - std::unique_ptr mock_env_; + std::shared_ptr mock_clock_; + std::unique_ptr mock_env_; std::unique_ptr fault_injection_env_; BlobDB *blob_db_; }; // class BlobDBTest @@ -341,13 +345,13 @@ TEST_F(BlobDBTest, PutWithTTL) { bdb_options.disable_background_tasks = true; Open(bdb_options, options); std::map data; - mock_env_->set_current_time(50); + mock_clock_->SetCurrentTime(50); for (size_t i = 0; i < 100; i++) { uint64_t ttl = rnd.Next() % 100; PutRandomWithTTL("key" + ToString(i), ttl, &rnd, (ttl <= 50 ? nullptr : &data)); } - mock_env_->set_current_time(100); + mock_clock_->SetCurrentTime(100); auto *bdb_impl = static_cast(blob_db_); auto blob_files = bdb_impl->TEST_GetBlobFiles(); ASSERT_EQ(1, blob_files.size()); @@ -367,13 +371,13 @@ TEST_F(BlobDBTest, PutUntil) { bdb_options.disable_background_tasks = true; Open(bdb_options, options); std::map data; - mock_env_->set_current_time(50); + mock_clock_->SetCurrentTime(50); for (size_t i = 0; i < 100; i++) { uint64_t expiration = rnd.Next() % 100 + 50; PutRandomUntil("key" + ToString(i), expiration, &rnd, (expiration <= 100 ? nullptr : &data)); } - mock_env_->set_current_time(100); + mock_clock_->SetCurrentTime(100); auto *bdb_impl = static_cast(blob_db_); auto blob_files = bdb_impl->TEST_GetBlobFiles(); ASSERT_EQ(1, blob_files.size()); @@ -410,7 +414,7 @@ TEST_F(BlobDBTest, GetExpiration) { options.env = mock_env_.get(); BlobDBOptions bdb_options; bdb_options.disable_background_tasks = true; - mock_env_->set_current_time(100); + mock_clock_->SetCurrentTime(100); Open(bdb_options, options); ASSERT_OK(Put("key1", "value1")); ASSERT_OK(PutWithTTL("key2", "value2", 200)); @@ -924,7 +928,7 @@ TEST_F(BlobDBTest, SnapshotAndGarbageCollection) { TEST_F(BlobDBTest, ColumnFamilyNotSupported) { Options options; options.env = mock_env_.get(); - mock_env_->set_current_time(0); + mock_clock_->SetCurrentTime(0); Open(BlobDBOptions(), options); ColumnFamilyHandle *default_handle = blob_db_->DefaultColumnFamily(); ColumnFamilyHandle *handle = nullptr; @@ -1256,7 +1260,7 @@ TEST_F(BlobDBTest, InlineSmallValues) { bdb_options.disable_background_tasks = true; Options options; options.env = mock_env_.get(); - mock_env_->set_current_time(0); + mock_clock_->SetCurrentTime(0); Open(bdb_options, options); std::map data; std::map versions; @@ -1490,7 +1494,7 @@ TEST_F(BlobDBTest, FilterExpiredBlobIndex) { constexpr uint64_t kCompactTime = 500; constexpr uint64_t kMinBlobSize = 100; Random rnd(301); - mock_env_->set_current_time(0); + mock_clock_->SetCurrentTime(0); BlobDBOptions bdb_options; bdb_options.min_blob_size = kMinBlobSize; bdb_options.disable_background_tasks = true; @@ -1529,7 +1533,7 @@ TEST_F(BlobDBTest, FilterExpiredBlobIndex) { } VerifyDB(data); - mock_env_->set_current_time(kCompactTime); + mock_clock_->SetCurrentTime(kCompactTime); // Take a snapshot before compaction. Make sure expired blob indexes is // filtered regardless of snapshot. const Snapshot *snapshot = blob_db_->GetSnapshot(); @@ -1615,7 +1619,7 @@ TEST_F(BlobDBTest, FilterForFIFOEviction) { bdb_options.disable_background_tasks = true; Options options; // Use mock env to stop wall clock. - mock_env_->set_current_time(0); + mock_clock_->SetCurrentTime(0); options.env = mock_env_.get(); auto statistics = CreateDBStatistics(); options.statistics = statistics; @@ -1812,7 +1816,7 @@ TEST_F(BlobDBTest, GarbageCollection) { } } - mock_env_->set_current_time(kCompactTime); + mock_clock_->SetCurrentTime(kCompactTime); ASSERT_OK(blob_db_->CompactRange(CompactRangeOptions(), nullptr, nullptr)); @@ -1927,7 +1931,7 @@ TEST_F(BlobDBTest, EvictExpiredFile) { Options options; options.env = mock_env_.get(); Open(bdb_options, options); - mock_env_->set_current_time(50); + mock_clock_->SetCurrentTime(50); std::map data; ASSERT_OK(PutWithTTL("foo", "bar", 100, &data)); auto blob_files = blob_db_impl()->TEST_GetBlobFiles(); @@ -1936,7 +1940,7 @@ TEST_F(BlobDBTest, EvictExpiredFile) { ASSERT_FALSE(blob_file->Immutable()); ASSERT_FALSE(blob_file->Obsolete()); VerifyDB(data); - mock_env_->set_current_time(250); + mock_clock_->SetCurrentTime(250); // The key should expired now. blob_db_impl()->TEST_EvictExpiredFiles(); ASSERT_EQ(1, blob_db_impl()->TEST_GetBlobFiles().size()); @@ -2293,7 +2297,7 @@ TEST_F(BlobDBTest, ShutdownWait) { SyncPoint::GetInstance()->EnableProcessing(); Open(bdb_options, options); - mock_env_->set_current_time(50); + mock_clock_->SetCurrentTime(50); std::map data; ASSERT_OK(PutWithTTL("foo", "bar", 100, &data)); auto blob_files = blob_db_impl()->TEST_GetBlobFiles(); @@ -2304,7 +2308,7 @@ TEST_F(BlobDBTest, ShutdownWait) { VerifyDB(data); TEST_SYNC_POINT("BlobDBTest.ShutdownWait:0"); - mock_env_->set_current_time(250); + mock_clock_->SetCurrentTime(250); // The key should expired now. TEST_SYNC_POINT("BlobDBTest.ShutdownWait:1"); diff --git a/utilities/convenience/info_log_finder.cc b/utilities/convenience/info_log_finder.cc index 980262f22..37f3bceee 100644 --- a/utilities/convenience/info_log_finder.cc +++ b/utilities/convenience/info_log_finder.cc @@ -19,7 +19,7 @@ Status GetInfoLogList(DB* db, std::vector* info_log_list) { } std::string parent_path; const Options& options = db->GetOptions(); - return GetInfoLogFiles(options.env, options.db_log_dir, db->GetName(), - &parent_path, info_log_list); + return GetInfoLogFiles(options.env->GetFileSystem(), options.db_log_dir, + db->GetName(), &parent_path, info_log_list); } } // namespace ROCKSDB_NAMESPACE diff --git a/utilities/persistent_cache/block_cache_tier.cc b/utilities/persistent_cache/block_cache_tier.cc index 658737571..baefa8771 100644 --- a/utilities/persistent_cache/block_cache_tier.cc +++ b/utilities/persistent_cache/block_cache_tier.cc @@ -222,7 +222,7 @@ Status BlockCacheTier::InsertImpl(const Slice& key, const Slice& data) { assert(data.size()); assert(cache_file_); - StopWatchNano timer(opt_.env, /*auto_start=*/ true); + StopWatchNano timer(opt_.clock, /*auto_start=*/true); WriteLock _(&lock_); @@ -265,7 +265,7 @@ Status BlockCacheTier::InsertImpl(const Slice& key, const Slice& data) { Status BlockCacheTier::Lookup(const Slice& key, std::unique_ptr* val, size_t* size) { - StopWatchNano timer(opt_.env, /*auto_start=*/ true); + StopWatchNano timer(opt_.clock, /*auto_start=*/true); LBA lba; bool status; diff --git a/utilities/persistent_cache/block_cache_tier.h b/utilities/persistent_cache/block_cache_tier.h index 13b04f95a..74c3095e7 100644 --- a/utilities/persistent_cache/block_cache_tier.h +++ b/utilities/persistent_cache/block_cache_tier.h @@ -19,21 +19,20 @@ #include #include -#include "rocksdb/cache.h" -#include "rocksdb/comparator.h" -#include "rocksdb/persistent_cache.h" - -#include "utilities/persistent_cache/block_cache_tier_file.h" -#include "utilities/persistent_cache/block_cache_tier_metadata.h" -#include "utilities/persistent_cache/persistent_cache_util.h" - #include "memory/arena.h" #include "memtable/skiplist.h" #include "monitoring/histogram.h" #include "port/port.h" +#include "rocksdb/cache.h" +#include "rocksdb/comparator.h" +#include "rocksdb/persistent_cache.h" +#include "rocksdb/system_clock.h" #include "util/coding.h" #include "util/crc32c.h" #include "util/mutexlock.h" +#include "utilities/persistent_cache/block_cache_tier_file.h" +#include "utilities/persistent_cache/block_cache_tier_metadata.h" +#include "utilities/persistent_cache/persistent_cache_util.h" namespace ROCKSDB_NAMESPACE { @@ -74,7 +73,7 @@ class BlockCacheTier : public PersistentCacheTier { void TEST_Flush() override { while (insert_ops_.Size()) { /* sleep override */ - Env::Default()->SleepForMicroseconds(1000000); + SystemClock::Default()->SleepForMicroseconds(1000000); } } diff --git a/utilities/persistent_cache/block_cache_tier_file.cc b/utilities/persistent_cache/block_cache_tier_file.cc index d2f92a98f..d069acb89 100644 --- a/utilities/persistent_cache/block_cache_tier_file.cc +++ b/utilities/persistent_cache/block_cache_tier_file.cc @@ -16,6 +16,7 @@ #include "env/composite_env_wrapper.h" #include "logging/logging.h" #include "port/port.h" +#include "rocksdb/system_clock.h" #include "util/crc32c.h" namespace ROCKSDB_NAMESPACE { @@ -219,7 +220,7 @@ bool RandomAccessCacheFile::OpenImpl(const bool enable_direct_reads) { return false; } freader_.reset(new RandomAccessFileReader( - NewLegacyRandomAccessFileWrapper(file), Path(), env_)); + NewLegacyRandomAccessFileWrapper(file), Path(), env_->GetSystemClock())); return true; } @@ -579,7 +580,7 @@ void ThreadedWriter::ThreadMain() { // We can fail to reserve space if every file in the system // is being currently accessed /* sleep override */ - Env::Default()->SleepForMicroseconds(1000000); + SystemClock::Default()->SleepForMicroseconds(1000000); } DispatchIO(io); diff --git a/utilities/persistent_cache/persistent_cache_bench.cc b/utilities/persistent_cache/persistent_cache_bench.cc index 359fcdd1d..082010c34 100644 --- a/utilities/persistent_cache/persistent_cache_bench.cc +++ b/utilities/persistent_cache/persistent_cache_bench.cc @@ -15,18 +15,17 @@ int main() { fprintf(stderr, "Please install gflags to run tools\n"); } #include #include -#include "rocksdb/env.h" - -#include "utilities/persistent_cache/block_cache_tier.h" -#include "utilities/persistent_cache/persistent_cache_tier.h" -#include "utilities/persistent_cache/volatile_tier_impl.h" - #include "monitoring/histogram.h" #include "port/port.h" +#include "rocksdb/env.h" +#include "rocksdb/system_clock.h" #include "table/block_based/block_builder.h" #include "util/gflags_compat.h" #include "util/mutexlock.h" #include "util/stop_watch.h" +#include "utilities/persistent_cache/block_cache_tier.h" +#include "utilities/persistent_cache/persistent_cache_tier.h" +#include "utilities/persistent_cache/volatile_tier_impl.h" DEFINE_int32(nsec, 10, "nsec"); DEFINE_int32(nthread_write, 1, "Insert threads"); @@ -128,7 +127,7 @@ class CacheTierBenchmark { std::bind(&CacheTierBenchmark::Read, this)); // Wait till FLAGS_nsec and then signal to quit - StopWatchNano t(Env::Default(), /*auto_start=*/true); + StopWatchNano t(SystemClock::Default(), /*auto_start=*/true); size_t sec = t.ElapsedNanos() / 1000000000ULL; while (!quit_) { sec = t.ElapsedNanos() / 1000000000ULL; @@ -195,7 +194,7 @@ class CacheTierBenchmark { auto block = NewBlock(key); // insert - StopWatchNano timer(Env::Default(), /*auto_start=*/true); + StopWatchNano timer(SystemClock::Default(), /*auto_start=*/true); while (true) { Status status = cache_->Insert(block_key, block.get(), FLAGS_iosize); if (status.ok()) { @@ -227,7 +226,7 @@ class CacheTierBenchmark { Slice key = FillKey(k, val); // Lookup in cache - StopWatchNano timer(Env::Default(), /*auto_start=*/true); + StopWatchNano timer(SystemClock::Default(), /*auto_start=*/true); std::unique_ptr block; size_t size; Status status = cache_->Lookup(key, &block, &size); diff --git a/utilities/persistent_cache/persistent_cache_tier.h b/utilities/persistent_cache/persistent_cache_tier.h index f8fa88c34..5e8b76209 100644 --- a/utilities/persistent_cache/persistent_cache_tier.h +++ b/utilities/persistent_cache/persistent_cache_tier.h @@ -17,6 +17,7 @@ #include "rocksdb/env.h" #include "rocksdb/persistent_cache.h" #include "rocksdb/status.h" +#include "rocksdb/system_clock.h" // Persistent Cache // @@ -86,6 +87,7 @@ struct PersistentCacheConfig { const std::shared_ptr& _log, const uint32_t _write_buffer_size = 1 * 1024 * 1024 /*1MB*/) { env = _env; + clock = (env != nullptr) ? env->GetSystemClock() : SystemClock::Default(); path = _path; log = _log; cache_size = _cache_size; @@ -124,10 +126,10 @@ struct PersistentCacheConfig { } // - // Env abstraction to use for systmer level operations + // Env abstraction to use for system level operations // Env* env; - + std::shared_ptr clock; // // Path for the block cache where blocks are persisted // diff --git a/utilities/write_batch_with_index/write_batch_with_index_internal.cc b/utilities/write_batch_with_index/write_batch_with_index_internal.cc index 5d1979063..f54379924 100644 --- a/utilities/write_batch_with_index/write_batch_with_index_internal.cc +++ b/utilities/write_batch_with_index/write_batch_with_index_internal.cc @@ -213,18 +213,18 @@ Status WriteBatchWithIndexInternal::MergeKey(const Slice& key, return MergeHelper::TimedFullMerge( merge_operator, key, value, merge_context.GetOperands(), result, - logger, statistics, env, result_operand); + logger, statistics, env->GetSystemClock(), result_operand); } else if (db_options_ != nullptr) { Statistics* statistics = db_options_->statistics.get(); Env* env = db_options_->env; Logger* logger = db_options_->info_log.get(); return MergeHelper::TimedFullMerge( merge_operator, key, value, merge_context.GetOperands(), result, - logger, statistics, env, result_operand); + logger, statistics, env->GetSystemClock(), result_operand); } else { return MergeHelper::TimedFullMerge( merge_operator, key, value, merge_context.GetOperands(), result, - nullptr, nullptr, Env::Default(), result_operand); + nullptr, nullptr, SystemClock::Default(), result_operand); } } else { return Status::InvalidArgument("Must provide a column_family");