From 521d234bdabb00bdaf60ebb207f67256deec648d Mon Sep 17 00:00:00 2001 From: Maysam Yabandeh Date: Wed, 1 May 2019 10:04:21 -0700 Subject: [PATCH] Revert snap_refresh_nanos feature (#5269) Summary: Our daily stress tests are failing after this feature. Reverting temporarily until we figure the reason for test failures. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5269 Differential Revision: D15151285 Pulled By: maysamyabandeh fbshipit-source-id: e4002b99690a97df30d4b4b58bf0f61e9591bc6e --- HISTORY.md | 1 - db/c.cc | 5 -- db/compaction_iterator.cc | 60 +++++-------- db/compaction_iterator.h | 60 +------------ db/compaction_job.cc | 5 +- db/compaction_job.h | 31 +++---- db/compaction_job_test.cc | 141 +++--------------------------- db/db_impl.h | 9 +- db/db_impl_compaction_flush.cc | 37 +------- db/snapshot_impl.h | 14 +-- include/rocksdb/c.h | 2 - include/rocksdb/options.h | 11 --- options/cf_options.cc | 2 - options/cf_options.h | 3 - options/options.cc | 4 - options/options_helper.cc | 25 +++--- options/options_settable_test.cc | 1 - options/options_test.cc | 2 - table/mock_table.cc | 14 --- table/mock_table.h | 8 -- util/compaction_job_stats_impl.cc | 3 - 21 files changed, 70 insertions(+), 368 deletions(-) diff --git a/HISTORY.md b/HISTORY.md index 4b08ce9d1..2d3fd87c8 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -8,7 +8,6 @@ * Introduce Periodic Compaction for Level style compaction. Files are re-compacted periodically and put in the same level. * Block-based table index now contains exact highest key in the file, rather than an upper bound. This may improve Get() and iterator Seek() performance in some situations, especially when direct IO is enabled and block cache is disabled. A setting BlockBasedTableOptions::index_shortening is introduced to control this behavior. Set it to kShortenSeparatorsAndSuccessor to get the old behavior. * When reading from option file/string/map, customized envs can be filled according to object registry. -* Add an option `snap_refresh_nanos` (default to 0.5s) to periodically refresh the snapshot list in compaction jobs. Assign to 0 to disable the feature. * Improve range scan performance when using explicit user readahead by not creating new table readers for every iterator. ### Public API Change diff --git a/db/c.cc b/db/c.cc index aac1cf408..9f5995a41 100644 --- a/db/c.cc +++ b/db/c.cc @@ -2226,11 +2226,6 @@ void rocksdb_options_set_max_bytes_for_level_base( opt->rep.max_bytes_for_level_base = n; } -void rocksdb_options_set_snap_refresh_nanos(rocksdb_options_t* opt, - uint64_t n) { - opt->rep.snap_refresh_nanos = n; -} - void rocksdb_options_set_level_compaction_dynamic_level_bytes( rocksdb_options_t* opt, unsigned char v) { opt->rep.level_compaction_dynamic_level_bytes = v; diff --git a/db/compaction_iterator.cc b/db/compaction_iterator.cc index bce0b82db..93c2b5fa9 100644 --- a/db/compaction_iterator.cc +++ b/db/compaction_iterator.cc @@ -38,16 +38,14 @@ CompactionIterator::CompactionIterator( CompactionRangeDelAggregator* range_del_agg, const Compaction* compaction, const CompactionFilter* compaction_filter, const std::atomic* shutting_down, - const SequenceNumber preserve_deletes_seqnum, - SnapshotListFetchCallback* snap_list_callback) + const SequenceNumber preserve_deletes_seqnum) : CompactionIterator( input, cmp, merge_helper, last_sequence, snapshots, earliest_write_conflict_snapshot, snapshot_checker, env, report_detailed_time, expect_valid_internal_key, range_del_agg, std::unique_ptr( compaction ? new CompactionProxy(compaction) : nullptr), - compaction_filter, shutting_down, preserve_deletes_seqnum, - snap_list_callback) {} + compaction_filter, shutting_down, preserve_deletes_seqnum) {} CompactionIterator::CompactionIterator( InternalIterator* input, const Comparator* cmp, MergeHelper* merge_helper, @@ -59,8 +57,7 @@ CompactionIterator::CompactionIterator( std::unique_ptr compaction, const CompactionFilter* compaction_filter, const std::atomic* shutting_down, - const SequenceNumber preserve_deletes_seqnum, - SnapshotListFetchCallback* snap_list_callback) + const SequenceNumber preserve_deletes_seqnum) : input_(input), cmp_(cmp), merge_helper_(merge_helper), @@ -78,8 +75,7 @@ CompactionIterator::CompactionIterator( current_user_key_sequence_(0), current_user_key_snapshot_(0), merge_out_iter_(merge_helper_), - current_key_committed_(false), - snap_list_callback_(snap_list_callback) { + current_key_committed_(false) { assert(compaction_filter_ == nullptr || compaction_ != nullptr); assert(snapshots_ != nullptr); bottommost_level_ = @@ -87,7 +83,24 @@ CompactionIterator::CompactionIterator( if (compaction_ != nullptr) { level_ptrs_ = std::vector(compaction_->number_levels(), 0); } - ProcessSnapshotList(); + if (snapshots_->size() == 0) { + // optimize for fast path if there are no snapshots + visible_at_tip_ = true; + earliest_snapshot_iter_ = snapshots_->end(); + earliest_snapshot_ = kMaxSequenceNumber; + latest_snapshot_ = 0; + } else { + visible_at_tip_ = false; + earliest_snapshot_iter_ = snapshots_->begin(); + earliest_snapshot_ = snapshots_->at(0); + latest_snapshot_ = snapshots_->back(); + } +#ifndef NDEBUG + // findEarliestVisibleSnapshot assumes this ordering. + for (size_t i = 1; i < snapshots_->size(); ++i) { + assert(snapshots_->at(i - 1) < snapshots_->at(i)); + } +#endif input_->SetPinnedItersMgr(&pinned_iters_mgr_); TEST_SYNC_POINT_CALLBACK("CompactionIterator:AfterInit", compaction_.get()); } @@ -209,28 +222,6 @@ void CompactionIterator::InvokeFilterIfNeeded(bool* need_skip, } } -void CompactionIterator::ProcessSnapshotList() { -#ifndef NDEBUG - // findEarliestVisibleSnapshot assumes this ordering. - for (size_t i = 1; i < snapshots_->size(); ++i) { - assert(snapshots_->at(i - 1) < snapshots_->at(i)); - } -#endif - if (snapshots_->size() == 0) { - // optimize for fast path if there are no snapshots - visible_at_tip_ = true; - earliest_snapshot_iter_ = snapshots_->end(); - earliest_snapshot_ = kMaxSequenceNumber; - latest_snapshot_ = 0; - } else { - visible_at_tip_ = false; - earliest_snapshot_iter_ = snapshots_->begin(); - earliest_snapshot_ = snapshots_->at(0); - latest_snapshot_ = snapshots_->back(); - } - released_snapshots_.clear(); -} - void CompactionIterator::NextFromInput() { at_next_ = false; valid_ = false; @@ -278,13 +269,6 @@ void CompactionIterator::NextFromInput() { // compaction filter). ikey_.user_key is pointing to the copy. if (!has_current_user_key_ || !cmp_->Equal(ikey_.user_key, current_user_key_)) { - num_keys_++; - // Use num_keys_ to reduce the overhead of reading current time - if (snap_list_callback_ && snapshots_->size() && - snap_list_callback_->TimeToRefresh(num_keys_)) { - snap_list_callback_->Refresh(snapshots_, latest_snapshot_); - ProcessSnapshotList(); - } // First occurrence of this user key // Copy key for output key_ = current_key_.SetInternalKey(key_, &ikey_); diff --git a/db/compaction_iterator.h b/db/compaction_iterator.h index 6ab43b1be..a9e7a2620 100644 --- a/db/compaction_iterator.h +++ b/db/compaction_iterator.h @@ -21,53 +21,6 @@ namespace rocksdb { -// This callback can be used to refresh the snapshot list from the db. It -// includes logics to exponentially decrease the refresh rate to limit the -// overhead of refresh. -class SnapshotListFetchCallback { - public: - SnapshotListFetchCallback(Env* env, uint64_t snap_refresh_nanos, - size_t every_nth_key = 1024) - : timer_(env, /*auto restart*/ true), - snap_refresh_nanos_(snap_refresh_nanos), - every_nth_key_minus_one_(every_nth_key - 1) { - assert(every_nth_key > 0); - assert((ceil(log2(every_nth_key)) == floor(log2(every_nth_key)))); - } - // Refresh the snapshot list. snapshots will bre replacted with the new list. - // max is the upper bound. Note: this function will acquire the db_mutex_. - virtual void Refresh(std::vector* snapshots, - SequenceNumber max) = 0; - inline bool TimeToRefresh(const size_t key_index) { - // skip the key if key_index % every_nth_key (which is of power 2) is not 0. - if ((key_index & every_nth_key_minus_one_) != 0) { - return false; - } - const uint64_t elapsed = timer_.ElapsedNanos(); - auto ret = elapsed > snap_refresh_nanos_; - // pre-compute the next time threshold - if (ret) { - // inc next refresh period exponentially (by x4) - auto next_refresh_threshold = snap_refresh_nanos_ << 2; - // make sure the shift has not overflown the highest 1 bit - snap_refresh_nanos_ = - std::max(snap_refresh_nanos_, next_refresh_threshold); - } - return ret; - } - static constexpr SnapshotListFetchCallback* kDisabled = nullptr; - - virtual ~SnapshotListFetchCallback() {} - - private: - // Time since the callback was created - StopWatchNano timer_; - // The delay before calling ::Refresh. To be increased exponentially. - uint64_t snap_refresh_nanos_; - // Skip evey nth key. Number n if of power 2. The math will require n-1. - const uint64_t every_nth_key_minus_one_; -}; - class CompactionIterator { public: // A wrapper around Compaction. Has a much smaller interface, only what @@ -116,8 +69,7 @@ class CompactionIterator { const Compaction* compaction = nullptr, const CompactionFilter* compaction_filter = nullptr, const std::atomic* shutting_down = nullptr, - const SequenceNumber preserve_deletes_seqnum = 0, - SnapshotListFetchCallback* snap_list_callback = nullptr); + const SequenceNumber preserve_deletes_seqnum = 0); // Constructor with custom CompactionProxy, used for tests. CompactionIterator(InternalIterator* input, const Comparator* cmp, @@ -130,8 +82,7 @@ class CompactionIterator { std::unique_ptr compaction, const CompactionFilter* compaction_filter = nullptr, const std::atomic* shutting_down = nullptr, - const SequenceNumber preserve_deletes_seqnum = 0, - SnapshotListFetchCallback* snap_list_callback = nullptr); + const SequenceNumber preserve_deletes_seqnum = 0); ~CompactionIterator(); @@ -159,8 +110,6 @@ class CompactionIterator { private: // Processes the input stream to find the next output void NextFromInput(); - // Process snapshots_ and assign related variables - void ProcessSnapshotList(); // Do last preparations before presenting the output to the callee. At this // point this only zeroes out the sequence number if possible for better @@ -195,7 +144,7 @@ class CompactionIterator { InternalIterator* input_; const Comparator* cmp_; MergeHelper* merge_helper_; - std::vector* snapshots_; + const std::vector* snapshots_; // List of snapshots released during compaction. // findEarliestVisibleSnapshot() find them out from return of // snapshot_checker, and make sure they will not be returned as @@ -270,9 +219,6 @@ class CompactionIterator { // Used to avoid purging uncommitted values. The application can specify // uncommitted values by providing a SnapshotChecker object. bool current_key_committed_; - SnapshotListFetchCallback* snap_list_callback_; - // number of distinct keys processed - size_t num_keys_ = 0; bool IsShuttingDown() { // This is a best-effort facility, so memory_order_relaxed is sufficient. diff --git a/db/compaction_job.cc b/db/compaction_job.cc index bc127a4c4..45221a155 100644 --- a/db/compaction_job.cc +++ b/db/compaction_job.cc @@ -315,7 +315,7 @@ CompactionJob::CompactionJob( const SnapshotChecker* snapshot_checker, std::shared_ptr table_cache, EventLogger* event_logger, bool paranoid_file_checks, bool measure_io_stats, const std::string& dbname, CompactionJobStats* compaction_job_stats, - Env::Priority thread_pri, SnapshotListFetchCallback* snap_list_callback) + Env::Priority thread_pri) : job_id_(job_id), compact_(new CompactionState(compaction)), compaction_job_stats_(compaction_job_stats), @@ -336,7 +336,6 @@ CompactionJob::CompactionJob( db_mutex_(db_mutex), db_error_handler_(db_error_handler), existing_snapshots_(std::move(existing_snapshots)), - snap_list_callback_(snap_list_callback), earliest_write_conflict_snapshot_(earliest_write_conflict_snapshot), snapshot_checker_(snapshot_checker), table_cache_(std::move(table_cache)), @@ -893,7 +892,7 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) { &existing_snapshots_, earliest_write_conflict_snapshot_, snapshot_checker_, env_, ShouldReportDetailedTime(env_, stats_), false, &range_del_agg, sub_compact->compaction, compaction_filter, - shutting_down_, preserve_deletes_seqnum_, snap_list_callback_)); + shutting_down_, preserve_deletes_seqnum_)); auto c_iter = sub_compact->c_iter.get(); c_iter->SeekToFirst(); if (c_iter->Valid() && sub_compact->compaction->output_level() != 0) { diff --git a/db/compaction_job.h b/db/compaction_job.h index b3a0f2eb4..9767985f3 100644 --- a/db/compaction_job.h +++ b/db/compaction_job.h @@ -57,20 +57,22 @@ class VersionSet; class CompactionJob { public: - CompactionJob( - int job_id, Compaction* compaction, const ImmutableDBOptions& db_options, - const EnvOptions env_options, VersionSet* versions, - const std::atomic* shutting_down, - const SequenceNumber preserve_deletes_seqnum, LogBuffer* log_buffer, - Directory* db_directory, Directory* output_directory, Statistics* stats, - InstrumentedMutex* db_mutex, ErrorHandler* db_error_handler, - std::vector existing_snapshots, - SequenceNumber earliest_write_conflict_snapshot, - const SnapshotChecker* snapshot_checker, - std::shared_ptr table_cache, EventLogger* event_logger, - bool paranoid_file_checks, bool measure_io_stats, - const std::string& dbname, CompactionJobStats* compaction_job_stats, - Env::Priority thread_pri, SnapshotListFetchCallback* snap_list_callback); + CompactionJob(int job_id, Compaction* compaction, + const ImmutableDBOptions& db_options, + const EnvOptions env_options, VersionSet* versions, + const std::atomic* shutting_down, + const SequenceNumber preserve_deletes_seqnum, + LogBuffer* log_buffer, Directory* db_directory, + Directory* output_directory, Statistics* stats, + InstrumentedMutex* db_mutex, ErrorHandler* db_error_handler, + std::vector existing_snapshots, + SequenceNumber earliest_write_conflict_snapshot, + const SnapshotChecker* snapshot_checker, + std::shared_ptr table_cache, EventLogger* event_logger, + bool paranoid_file_checks, bool measure_io_stats, + const std::string& dbname, + CompactionJobStats* compaction_job_stats, + Env::Priority thread_pri); ~CompactionJob(); @@ -150,7 +152,6 @@ class CompactionJob { // entirely within s1 and s2, then the earlier version of k1 can be safely // deleted because that version is not visible in any snapshot. std::vector existing_snapshots_; - SnapshotListFetchCallback* snap_list_callback_; // This is the earliest snapshot that could be used for write-conflict // checking by a transaction. For any user-key newer than this snapshot, we diff --git a/db/compaction_job_test.cc b/db/compaction_job_test.cc index 60394cc97..f05a8ec2f 100644 --- a/db/compaction_job_test.cc +++ b/db/compaction_job_test.cc @@ -5,13 +5,7 @@ #ifndef ROCKSDB_LITE -#ifndef __STDC_FORMAT_MACROS -#define __STDC_FORMAT_MACROS -#endif - -#include #include -#include #include #include #include @@ -200,13 +194,6 @@ class CompactionJobTest : public testing::Test { } void NewDB() { - DestroyDB(dbname_, Options()); - EXPECT_OK(env_->CreateDirIfMissing(dbname_)); - versions_.reset(new VersionSet(dbname_, &db_options_, env_options_, - table_cache_.get(), &write_buffer_manager_, - &write_controller_)); - compaction_job_stats_.Reset(); - VersionEdit new_db; new_db.SetLogNumber(0); new_db.SetNextFile(2); @@ -243,10 +230,7 @@ class CompactionJobTest : public testing::Test { const std::vector>& input_files, const stl_wrappers::KVMap& expected_results, const std::vector& snapshots = {}, - SequenceNumber earliest_write_conflict_snapshot = kMaxSequenceNumber, - int output_level = 1, bool verify = true, - SnapshotListFetchCallback* snapshot_fetcher = - SnapshotListFetchCallback::kDisabled) { + SequenceNumber earliest_write_conflict_snapshot = kMaxSequenceNumber) { auto cfd = versions_->GetColumnFamilySet()->GetDefault(); size_t num_input_files = 0; @@ -263,7 +247,7 @@ class CompactionJobTest : public testing::Test { Compaction compaction(cfd->current()->storage_info(), *cfd->ioptions(), *cfd->GetLatestMutableCFOptions(), - compaction_input_files, output_level, 1024 * 1024, + compaction_input_files, 1, 1024 * 1024, 10 * 1024 * 1024, 0, kNoCompression, cfd->ioptions()->compression_opts, 0, {}, true); compaction.SetInputVersion(cfd->current()); @@ -279,7 +263,7 @@ class CompactionJobTest : public testing::Test { nullptr, nullptr, &mutex_, &error_handler_, snapshots, earliest_write_conflict_snapshot, snapshot_checker, table_cache_, &event_logger, false, false, dbname_, &compaction_job_stats_, - Env::Priority::USER, snapshot_fetcher); + Env::Priority::USER); VerifyInitializationOfCompactionJobStats(compaction_job_stats_); compaction_job.Prepare(); @@ -291,17 +275,15 @@ class CompactionJobTest : public testing::Test { ASSERT_OK(compaction_job.Install(*cfd->GetLatestMutableCFOptions())); mutex_.Unlock(); - if (verify) { - if (expected_results.size() == 0) { - ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U); - ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files); - ASSERT_EQ(compaction_job_stats_.num_output_files, 0U); - } else { - ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U); - ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files); - ASSERT_EQ(compaction_job_stats_.num_output_files, 1U); - mock_table_factory_->AssertLatestFile(expected_results); - } + if (expected_results.size() == 0) { + ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U); + ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files); + ASSERT_EQ(compaction_job_stats_.num_output_files, 0U); + } else { + ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U); + ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files); + ASSERT_EQ(compaction_job_stats_.num_output_files, 1U); + mock_table_factory_->AssertLatestFile(expected_results); } } @@ -956,105 +938,6 @@ TEST_F(CompactionJobTest, CorruptionAfterDeletion) { RunCompaction({files}, expected_results); } -// Test the snapshot fetcher in compaction -TEST_F(CompactionJobTest, SnapshotRefresh) { - uint64_t time_seed = env_->NowMicros(); - printf("time_seed is %" PRIu64 "\n", time_seed); // would help to reproduce - Random64 rand(time_seed); - std::vector db_snapshots; - class SnapshotListFetchCallbackTest : public SnapshotListFetchCallback { - public: - SnapshotListFetchCallbackTest(Env* env, Random64& rand, - std::vector* snapshots) - : SnapshotListFetchCallback(env, 0 /*no time delay*/, - 1 /*fetch after each key*/), - rand_(rand), - snapshots_(snapshots) {} - virtual void Refresh(std::vector* snapshots, - SequenceNumber) override { - assert(snapshots->size()); - assert(snapshots_->size()); - assert(snapshots_->size() == snapshots->size()); - if (rand_.OneIn(2)) { - uint64_t release_index = rand_.Uniform(snapshots_->size()); - snapshots_->erase(snapshots_->begin() + release_index); - *snapshots = *snapshots_; - } - } - - private: - Random64 rand_; - std::vector* snapshots_; - } snapshot_fetcher(env_, rand, &db_snapshots); - - std::vector> file1_kvs, file2_kvs; - std::array types = {kTypeValue, kTypeDeletion, - kTypeSingleDeletion}; - SequenceNumber last_seq = 0; - for (int i = 1; i < 100; i++) { - SequenceNumber seq = last_seq + 1; - last_seq = seq; - if (rand.OneIn(2)) { - auto type = types[rand.Uniform(types.size())]; - file1_kvs.push_back( - {test::KeyStr("k" + ToString(i), seq, type), "v" + ToString(i)}); - } - } - auto file1 = mock::MakeMockFile(file1_kvs); - for (int i = 1; i < 100; i++) { - SequenceNumber seq = last_seq + 1; - last_seq++; - if (rand.OneIn(2)) { - auto type = types[rand.Uniform(types.size())]; - file2_kvs.push_back( - {test::KeyStr("k" + ToString(i), seq, type), "v" + ToString(i)}); - } - } - auto file2 = mock::MakeMockFile(file2_kvs); - for (SequenceNumber i = 1; i < last_seq + 1; i++) { - if (rand.OneIn(5)) { - db_snapshots.push_back(i); - } - } - - const bool kVerify = true; - const int output_level_0 = 0; - NewDB(); - AddMockFile(file1); - AddMockFile(file2); - SetLastSequence(last_seq); - auto files = cfd_->current()->storage_info()->LevelFiles(0); - // put the output on L0 since it is easier to feed them again to the 2nd - // compaction - RunCompaction({files}, file1, db_snapshots, kMaxSequenceNumber, - output_level_0, !kVerify, &snapshot_fetcher); - - // Now db_snapshots are changed. Run the compaction again without snapshot - // fetcher but with the updated snapshot list. - compaction_job_stats_.Reset(); - files = cfd_->current()->storage_info()->LevelFiles(0); - RunCompaction({files}, file1, db_snapshots, kMaxSequenceNumber, - output_level_0 + 1, !kVerify); - // The result should be what we get if we run compaction without snapshot - // fetcher on the updated list of snapshots - auto expected = mock_table_factory_->output(); - - NewDB(); - AddMockFile(file1); - AddMockFile(file2); - SetLastSequence(last_seq); - files = cfd_->current()->storage_info()->LevelFiles(0); - RunCompaction({files}, expected, db_snapshots, kMaxSequenceNumber, - output_level_0, !kVerify); - // The 2nd compaction above would get rid of useless delete markers. To get - // the output here exactly as what we got above after two compactions, we also - // run the compaction for 2nd time. - compaction_job_stats_.Reset(); - files = cfd_->current()->storage_info()->LevelFiles(0); - RunCompaction({files}, expected, db_snapshots, kMaxSequenceNumber, - output_level_0 + 1, !kVerify); -} - } // namespace rocksdb int main(int argc, char** argv) { diff --git a/db/db_impl.h b/db/db_impl.h index 623f69ba6..9bdb0abdc 100644 --- a/db/db_impl.h +++ b/db/db_impl.h @@ -561,13 +561,6 @@ class DBImpl : public DB { const SnapshotList& snapshots() const { return snapshots_; } - void LoadSnapshots(std::vector* snap_vector, - SequenceNumber* oldest_write_conflict_snapshot, - const SequenceNumber& max_seq) const { - InstrumentedMutexLock l(mutex()); - snapshots().GetAll(snap_vector, oldest_write_conflict_snapshot, max_seq); - } - const ImmutableDBOptions& immutable_db_options() const { return immutable_db_options_; } @@ -746,7 +739,7 @@ class DBImpl : public DB { // Not thread-safe. void SetRecoverableStatePreReleaseCallback(PreReleaseCallback* callback); - InstrumentedMutex* mutex() const { return &mutex_; } + InstrumentedMutex* mutex() { return &mutex_; } Status NewDB(); diff --git a/db/db_impl_compaction_flush.cc b/db/db_impl_compaction_flush.cc index f16c61117..49b6c0fd8 100644 --- a/db/db_impl_compaction_flush.cc +++ b/db/db_impl_compaction_flush.cc @@ -798,29 +798,6 @@ Status DBImpl::CompactRange(const CompactRangeOptions& options, return s; } -class SnapshotListFetchCallbackImpl : public SnapshotListFetchCallback { - public: - SnapshotListFetchCallbackImpl(DBImpl* db_impl, Env* env, - uint64_t snap_refresh_nanos, Logger* info_log) - : SnapshotListFetchCallback(env, snap_refresh_nanos), - db_impl_(db_impl), - info_log_(info_log) {} - virtual void Refresh(std::vector* snapshots, - SequenceNumber max) override { - size_t prev = snapshots->size(); - snapshots->clear(); - db_impl_->LoadSnapshots(snapshots, nullptr, max); - size_t now = snapshots->size(); - ROCKS_LOG_DEBUG(info_log_, - "Compaction snapshot count refreshed from %zu to %zu", prev, - now); - } - - private: - DBImpl* db_impl_; - Logger* info_log_; -}; - Status DBImpl::CompactFiles(const CompactionOptions& compact_options, ColumnFamilyHandle* column_family, const std::vector& input_file_names, @@ -992,9 +969,6 @@ Status DBImpl::CompactFilesImpl( assert(is_snapshot_supported_ || snapshots_.empty()); CompactionJobStats compaction_job_stats; - SnapshotListFetchCallbackImpl fetch_callback( - this, env_, c->mutable_cf_options()->snap_refresh_nanos, - immutable_db_options_.info_log.get()); CompactionJob compaction_job( job_context->job_id, c.get(), immutable_db_options_, env_options_for_compaction_, versions_.get(), &shutting_down_, @@ -1004,9 +978,7 @@ Status DBImpl::CompactFilesImpl( snapshot_checker, table_cache_, &event_logger_, c->mutable_cf_options()->paranoid_file_checks, c->mutable_cf_options()->report_bg_io_stats, dbname_, - &compaction_job_stats, Env::Priority::USER, - immutable_db_options_.max_subcompactions <= 1 ? &fetch_callback - : nullptr); + &compaction_job_stats, Env::Priority::USER); // Creating a compaction influences the compaction score because the score // takes running compactions into account (by skipping files that are already @@ -2650,9 +2622,6 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, GetSnapshotContext(job_context, &snapshot_seqs, &earliest_write_conflict_snapshot, &snapshot_checker); assert(is_snapshot_supported_ || snapshots_.empty()); - SnapshotListFetchCallbackImpl fetch_callback( - this, env_, c->mutable_cf_options()->snap_refresh_nanos, - immutable_db_options_.info_log.get()); CompactionJob compaction_job( job_context->job_id, c.get(), immutable_db_options_, env_options_for_compaction_, versions_.get(), &shutting_down_, @@ -2662,9 +2631,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, earliest_write_conflict_snapshot, snapshot_checker, table_cache_, &event_logger_, c->mutable_cf_options()->paranoid_file_checks, c->mutable_cf_options()->report_bg_io_stats, dbname_, - &compaction_job_stats, thread_pri, - immutable_db_options_.max_subcompactions <= 1 ? &fetch_callback - : nullptr); + &compaction_job_stats, thread_pri); compaction_job.Prepare(); NotifyOnCompactionBegin(c->column_family_data(), c.get(), status, diff --git a/db/snapshot_impl.h b/db/snapshot_impl.h index f1cf6f4b7..f2610fd18 100644 --- a/db/snapshot_impl.h +++ b/db/snapshot_impl.h @@ -91,23 +91,13 @@ class SnapshotList { SequenceNumber* oldest_write_conflict_snapshot = nullptr, const SequenceNumber& max_seq = kMaxSequenceNumber) const { std::vector ret; - GetAll(&ret, oldest_write_conflict_snapshot, max_seq); - return ret; - } - - void GetAll(std::vector* snap_vector, - SequenceNumber* oldest_write_conflict_snapshot = nullptr, - const SequenceNumber& max_seq = kMaxSequenceNumber) const { - std::vector& ret = *snap_vector; - // So far we have no use case that would pass a non-empty vector - assert(ret.size() == 0); if (oldest_write_conflict_snapshot != nullptr) { *oldest_write_conflict_snapshot = kMaxSequenceNumber; } if (empty()) { - return; + return ret; } const SnapshotImpl* s = &list_; while (s->next_ != &list_) { @@ -129,7 +119,7 @@ class SnapshotList { s = s->next_; } - return; + return ret; } // get the sequence number of the most recent snapshot diff --git a/include/rocksdb/c.h b/include/rocksdb/c.h index a0ae7ca77..4b34996a7 100644 --- a/include/rocksdb/c.h +++ b/include/rocksdb/c.h @@ -816,8 +816,6 @@ extern ROCKSDB_LIBRARY_API void rocksdb_options_set_target_file_size_multiplier( rocksdb_options_t*, int); extern ROCKSDB_LIBRARY_API void rocksdb_options_set_max_bytes_for_level_base( rocksdb_options_t*, uint64_t); -extern ROCKSDB_LIBRARY_API void rocksdb_options_set_snap_refresh_nanos( - rocksdb_options_t*, uint64_t); extern ROCKSDB_LIBRARY_API void rocksdb_options_set_level_compaction_dynamic_level_bytes(rocksdb_options_t*, unsigned char); diff --git a/include/rocksdb/options.h b/include/rocksdb/options.h index ab856bee8..4cc2998b2 100644 --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -269,17 +269,6 @@ struct ColumnFamilyOptions : public AdvancedColumnFamilyOptions { // Dynamically changeable through SetOptions() API uint64_t max_bytes_for_level_base = 256 * 1048576; - // If non-zero, compactions will periodically refresh the snapshot list. The - // delay for the first refresh is snap_refresh_nanos nano seconds and - // exponentially increases afterwards. When having many short-lived snapshots, - // this option helps reducing the cpu usage of long-running compactions. The - // feature is disabled when max_subcompactions is greater than one. - // - // Default: 0.5s - // - // Dynamically changeable through SetOptions() API - uint64_t snap_refresh_nanos = 500 * 1000 * 1000; // 0.5s - // Disable automatic compactions. Manual compactions can still // be issued on this column family // diff --git a/options/cf_options.cc b/options/cf_options.cc index f7af3f834..78accaeb9 100644 --- a/options/cf_options.cc +++ b/options/cf_options.cc @@ -169,8 +169,6 @@ void MutableCFOptions::Dump(Logger* log) const { target_file_size_multiplier); ROCKS_LOG_INFO(log, " max_bytes_for_level_base: %" PRIu64, max_bytes_for_level_base); - ROCKS_LOG_INFO(log, " snap_refresh_nanos: %" PRIu64, - snap_refresh_nanos); ROCKS_LOG_INFO(log, " max_bytes_for_level_multiplier: %f", max_bytes_for_level_multiplier); ROCKS_LOG_INFO(log, " ttl: %" PRIu64, diff --git a/options/cf_options.h b/options/cf_options.h index 47fca58fa..d0c4390c3 100644 --- a/options/cf_options.h +++ b/options/cf_options.h @@ -149,7 +149,6 @@ struct MutableCFOptions { target_file_size_base(options.target_file_size_base), target_file_size_multiplier(options.target_file_size_multiplier), max_bytes_for_level_base(options.max_bytes_for_level_base), - snap_refresh_nanos(options.snap_refresh_nanos), max_bytes_for_level_multiplier(options.max_bytes_for_level_multiplier), ttl(options.ttl), periodic_compaction_seconds(options.periodic_compaction_seconds), @@ -186,7 +185,6 @@ struct MutableCFOptions { target_file_size_base(0), target_file_size_multiplier(0), max_bytes_for_level_base(0), - snap_refresh_nanos(0), max_bytes_for_level_multiplier(0), ttl(0), periodic_compaction_seconds(0), @@ -238,7 +236,6 @@ struct MutableCFOptions { uint64_t target_file_size_base; int target_file_size_multiplier; uint64_t max_bytes_for_level_base; - uint64_t snap_refresh_nanos; double max_bytes_for_level_multiplier; uint64_t ttl; uint64_t periodic_compaction_seconds; diff --git a/options/options.cc b/options/options.cc index 900510d01..bfe3e313d 100644 --- a/options/options.cc +++ b/options/options.cc @@ -215,9 +215,6 @@ void ColumnFamilyOptions::Dump(Logger* log) const { ROCKS_LOG_HEADER( log, " Options.max_bytes_for_level_base: %" PRIu64, max_bytes_for_level_base); - ROCKS_LOG_HEADER( - log, " Options.snap_refresh_nanos: %" PRIu64, - snap_refresh_nanos); ROCKS_LOG_HEADER(log, "Options.level_compaction_dynamic_level_bytes: %d", level_compaction_dynamic_level_bytes); ROCKS_LOG_HEADER(log, " Options.max_bytes_for_level_multiplier: %f", @@ -493,7 +490,6 @@ ColumnFamilyOptions* ColumnFamilyOptions::OptimizeForSmallDb( write_buffer_size = 2 << 20; target_file_size_base = 2 * 1048576; max_bytes_for_level_base = 10 * 1048576; - snap_refresh_nanos = 0; soft_pending_compaction_bytes_limit = 256 * 1048576; hard_pending_compaction_bytes_limit = 1073741824ul; diff --git a/options/options_helper.cc b/options/options_helper.cc index a973bbfde..b7781ff6d 100644 --- a/options/options_helper.cc +++ b/options/options_helper.cc @@ -177,7 +177,6 @@ ColumnFamilyOptions BuildColumnFamilyOptions( mutable_cf_options.target_file_size_multiplier; cf_opts.max_bytes_for_level_base = mutable_cf_options.max_bytes_for_level_base; - cf_opts.snap_refresh_nanos = mutable_cf_options.snap_refresh_nanos; cf_opts.max_bytes_for_level_multiplier = mutable_cf_options.max_bytes_for_level_multiplier; cf_opts.ttl = mutable_cf_options.ttl; @@ -527,9 +526,9 @@ bool ParseOptionHelper(char* opt_address, const OptionType& opt_type, opt_address)); case OptionType::kBlockBasedTableIndexShorteningMode: return ParseEnum( - block_base_table_index_shortening_mode_string_map, value, - reinterpret_cast( - opt_address)); + block_base_table_index_shortening_mode_string_map, value, + reinterpret_cast( + opt_address)); case OptionType::kEncodingType: return ParseEnum( encoding_type_string_map, value, @@ -1667,13 +1666,13 @@ std::unordered_map std::unordered_map OptionsHelper::block_base_table_index_shortening_mode_string_map = { - {"kNoShortening", - BlockBasedTableOptions::IndexShorteningMode::kNoShortening}, - {"kShortenSeparators", - BlockBasedTableOptions::IndexShorteningMode::kShortenSeparators}, - {"kShortenSeparatorsAndSuccessor", - BlockBasedTableOptions::IndexShorteningMode:: - kShortenSeparatorsAndSuccessor}}; + {"kNoShortening", + BlockBasedTableOptions::IndexShorteningMode::kNoShortening}, + {"kShortenSeparators", + BlockBasedTableOptions::IndexShorteningMode::kShortenSeparators}, + {"kShortenSeparatorsAndSuccessor", + BlockBasedTableOptions::IndexShorteningMode:: + kShortenSeparatorsAndSuccessor}}; std::unordered_map OptionsHelper::encoding_type_string_map = {{"kPlain", kPlain}, @@ -1911,10 +1910,6 @@ std::unordered_map {offset_of(&ColumnFamilyOptions::max_bytes_for_level_base), OptionType::kUInt64T, OptionVerificationType::kNormal, true, offsetof(struct MutableCFOptions, max_bytes_for_level_base)}}, - {"snap_refresh_nanos", - {offset_of(&ColumnFamilyOptions::snap_refresh_nanos), - OptionType::kUInt64T, OptionVerificationType::kNormal, true, - offsetof(struct MutableCFOptions, snap_refresh_nanos)}}, {"max_bytes_for_level_multiplier", {offset_of(&ColumnFamilyOptions::max_bytes_for_level_multiplier), OptionType::kDouble, OptionVerificationType::kNormal, true, diff --git a/options/options_settable_test.cc b/options/options_settable_test.cc index 2d6cc11c0..005b9d53a 100644 --- a/options/options_settable_test.cc +++ b/options/options_settable_test.cc @@ -415,7 +415,6 @@ TEST_F(OptionsSettableTest, ColumnFamilyOptionsAllFieldsSettable) { "kBZip2Compression:kNoCompression:kZlibCompression:kBZip2Compression:" "kSnappyCompression;" "max_bytes_for_level_base=986;" - "snap_refresh_nanos=1000000000;" "bloom_locality=8016;" "target_file_size_base=4294976376;" "memtable_huge_page_size=2557;" diff --git a/options/options_test.cc b/options/options_test.cc index ded336dd1..fbfee311b 100644 --- a/options/options_test.cc +++ b/options/options_test.cc @@ -74,7 +74,6 @@ TEST_F(OptionsTest, GetOptionsFromMapTest) { {"target_file_size_base", "12"}, {"target_file_size_multiplier", "13"}, {"max_bytes_for_level_base", "14"}, - {"snap_refresh_nanos", "1000000000"}, {"level_compaction_dynamic_level_bytes", "true"}, {"max_bytes_for_level_multiplier", "15.0"}, {"max_bytes_for_level_multiplier_additional", "16:17:18"}, @@ -184,7 +183,6 @@ TEST_F(OptionsTest, GetOptionsFromMapTest) { ASSERT_EQ(new_cf_opt.target_file_size_base, static_cast(12)); ASSERT_EQ(new_cf_opt.target_file_size_multiplier, 13); ASSERT_EQ(new_cf_opt.max_bytes_for_level_base, 14U); - ASSERT_EQ(new_cf_opt.snap_refresh_nanos, 1000000000U); ASSERT_EQ(new_cf_opt.level_compaction_dynamic_level_bytes, true); ASSERT_EQ(new_cf_opt.max_bytes_for_level_multiplier, 15.0); ASSERT_EQ(new_cf_opt.max_bytes_for_level_multiplier_additional.size(), 3U); diff --git a/table/mock_table.cc b/table/mock_table.cc index 9b2506048..65a436169 100644 --- a/table/mock_table.cc +++ b/table/mock_table.cc @@ -21,12 +21,6 @@ const InternalKeyComparator icmp_(BytewiseComparator()); } // namespace -stl_wrappers::KVMap MakeMockFile( - std::vector> l) { - return stl_wrappers::KVMap(l.begin(), l.end(), - stl_wrappers::LessOfComparator(&icmp_)); -} - stl_wrappers::KVMap MakeMockFile( std::initializer_list> l) { return stl_wrappers::KVMap(l, stl_wrappers::LessOfComparator(&icmp_)); @@ -143,14 +137,6 @@ void MockTableFactory::AssertLatestFile( ParseInternalKey(Slice(key), &ikey); std::cout << ikey.DebugString(false) << " -> " << value << std::endl; } - std::cout << "Expected:" << std::endl; - for (const auto& kv : file_contents) { - ParsedInternalKey ikey; - std::string key, value; - std::tie(key, value) = kv; - ParseInternalKey(Slice(key), &ikey); - std::cout << ikey.DebugString(false) << " -> " << value << std::endl; - } FAIL(); } } diff --git a/table/mock_table.h b/table/mock_table.h index 5bca14644..2f123a963 100644 --- a/table/mock_table.h +++ b/table/mock_table.h @@ -28,8 +28,6 @@ namespace mock { stl_wrappers::KVMap MakeMockFile( std::initializer_list> l = {}); -stl_wrappers::KVMap MakeMockFile( - std::vector> l); struct MockTableFileSystem { port::Mutex mutex; @@ -186,12 +184,6 @@ class MockTableFactory : public TableFactory { // contents are equal to file_contents void AssertSingleFile(const stl_wrappers::KVMap& file_contents); void AssertLatestFile(const stl_wrappers::KVMap& file_contents); - stl_wrappers::KVMap output() { - assert(!file_system_.files.empty()); - auto latest = file_system_.files.end(); - --latest; - return latest->second; - } private: uint32_t GetAndWriteNextID(WritableFileWriter* file) const; diff --git a/util/compaction_job_stats_impl.cc b/util/compaction_job_stats_impl.cc index fe9efd1f0..a1ebc8b96 100644 --- a/util/compaction_job_stats_impl.cc +++ b/util/compaction_job_stats_impl.cc @@ -40,9 +40,6 @@ void CompactionJobStats::Reset() { file_fsync_nanos = 0; file_prepare_write_nanos = 0; - smallest_output_key_prefix.clear(); - largest_output_key_prefix.clear(); - num_single_del_fallthru = 0; num_single_del_mismatch = 0; }