From 9ed928e7a959ea9b3e404e6b325c64d6ee15084f Mon Sep 17 00:00:00 2001 From: Yi Wu Date: Fri, 23 Sep 2016 16:34:04 -0700 Subject: [PATCH] Split DBOptions into ImmutableDBOptions and MutableDBOptions Summary: Use ImmutableDBOptions/MutableDBOptions internally and DBOptions only for user-facing APIs. MutableDBOptions is barely a placeholder for now. I'll start to move options to MutableDBOptions in following diffs. Test Plan: make all check Reviewers: yhchiang, IslamAbdelRahman, sdong Reviewed By: sdong Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D64065 --- CMakeLists.txt | 1 + db/column_family.cc | 36 +- db/column_family.h | 14 +- db/column_family_test.cc | 4 +- db/compacted_db_impl.cc | 4 +- db/compaction_job.cc | 6 +- db/compaction_job.h | 6 +- db/compaction_job_test.cc | 7 +- db/db_filesnapshot.cc | 12 +- db/db_impl.cc | 649 ++++++++++++----------- db/db_impl.h | 11 +- db/db_impl_add_file.cc | 10 +- db/db_impl_experimental.cc | 12 +- db/db_impl_readonly.cc | 5 +- db/db_info_dumper.cc | 4 +- db/db_info_dumper.h | 5 +- db/db_options_test.cc | 3 +- db/db_test.cc | 2 +- db/db_wal_test.cc | 2 +- db/filename.cc | 4 +- db/filename.h | 3 +- db/flush_job.cc | 4 +- db/flush_job.h | 5 +- db/flush_job_test.cc | 5 +- db/memtable_list_test.cc | 6 +- db/repair.cc | 8 +- db/table_properties_collector_test.cc | 5 +- db/transaction_log_impl.cc | 2 +- db/transaction_log_impl.h | 17 +- db/version_set.cc | 11 +- db/version_set.h | 19 +- db/wal_manager.cc | 10 +- db/wal_manager.h | 15 +- db/wal_manager_test.cc | 19 +- include/rocksdb/db.h | 2 +- include/rocksdb/utilities/stackable_db.h | 2 +- src.mk | 63 +-- tools/ldb_cmd.cc | 6 +- util/cf_options.cc | 87 +-- util/cf_options.h | 9 + util/db_options.cc | 233 ++++++++ util/db_options.h | 96 ++++ util/file_util.cc | 5 +- util/file_util.h | 6 +- util/options_helper.cc | 93 ++++ util/options_helper.h | 4 + 46 files changed, 1019 insertions(+), 513 deletions(-) create mode 100644 util/db_options.cc create mode 100644 util/db_options.h diff --git a/CMakeLists.txt b/CMakeLists.txt index 7fe6997ac..e165d58ed 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -200,6 +200,7 @@ set(SOURCES util/comparator.cc util/concurrent_arena.cc util/crc32c.cc + util/db_options.cc util/delete_scheduler.cc util/dynamic_bloom.cc util/env.cc diff --git a/db/column_family.cc b/db/column_family.cc index 619bd3360..7efc5214d 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -88,11 +88,11 @@ const Comparator* ColumnFamilyHandleImpl::GetComparator() const { } void GetIntTblPropCollectorFactory( - const ColumnFamilyOptions& cf_options, + const ImmutableCFOptions& ioptions, std::vector>* int_tbl_prop_collector_factories) { - auto& collector_factories = cf_options.table_properties_collector_factories; - for (size_t i = 0; i < cf_options.table_properties_collector_factories.size(); + auto& collector_factories = ioptions.table_properties_collector_factories; + for (size_t i = 0; i < ioptions.table_properties_collector_factories.size(); ++i) { assert(collector_factories[i]); int_tbl_prop_collector_factories->emplace_back( @@ -138,7 +138,7 @@ Status CheckConcurrentWritesSupported(const ColumnFamilyOptions& cf_options) { return Status::OK(); } -ColumnFamilyOptions SanitizeOptions(const DBOptions& db_options, +ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options, const InternalKeyComparator* icmp, const ColumnFamilyOptions& src) { ColumnFamilyOptions result = src; @@ -340,7 +340,7 @@ void SuperVersionUnrefHandle(void* ptr) { ColumnFamilyData::ColumnFamilyData( uint32_t id, const std::string& name, Version* _dummy_versions, Cache* _table_cache, WriteBufferManager* write_buffer_manager, - const ColumnFamilyOptions& cf_options, const DBOptions* db_options, + const ColumnFamilyOptions& cf_options, const ImmutableDBOptions& db_options, const EnvOptions& env_options, ColumnFamilySet* column_family_set) : id_(id), name_(name), @@ -349,14 +349,14 @@ ColumnFamilyData::ColumnFamilyData( refs_(0), dropped_(false), internal_comparator_(cf_options.comparator), - options_(*db_options, - SanitizeOptions(*db_options, &internal_comparator_, cf_options)), - ioptions_(options_), - mutable_cf_options_(options_), + initial_cf_options_( + SanitizeOptions(db_options, &internal_comparator_, cf_options)), + ioptions_(db_options, initial_cf_options_), + mutable_cf_options_(initial_cf_options_), write_buffer_manager_(write_buffer_manager), mem_(nullptr), - imm_(options_.min_write_buffer_number_to_merge, - options_.max_write_buffer_number_to_maintain), + imm_(ioptions_.min_write_buffer_number_to_merge, + ioptions_.max_write_buffer_number_to_maintain), super_version_(nullptr), super_version_number_(0), local_sv_(new ThreadLocalPtr(&SuperVersionUnrefHandle)), @@ -370,12 +370,12 @@ ColumnFamilyData::ColumnFamilyData( Ref(); // Convert user defined table properties collector factories to internal ones. - GetIntTblPropCollectorFactory(options_, &int_tbl_prop_collector_factories_); + GetIntTblPropCollectorFactory(ioptions_, &int_tbl_prop_collector_factories_); // 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)); + new InternalStats(ioptions_.num_levels, db_options.env, this)); table_cache_.reset(new TableCache(ioptions_, env_options, _table_cache)); if (ioptions_.compaction_style == kCompactionStyleLevel) { compaction_picker_.reset( @@ -407,7 +407,7 @@ ColumnFamilyData::ColumnFamilyData( if (column_family_set_->NumberOfColumnFamilies() < 10) { Log(InfoLogLevel::INFO_LEVEL, ioptions_.info_log, "--------------- Options for column family [%s]:\n", name.c_str()); - options_.DumpCFOptions(ioptions_.info_log); + initial_cf_options_.Dump(ioptions_.info_log); } else { Log(InfoLogLevel::INFO_LEVEL, ioptions_.info_log, "\t(skipping printing options)\n"); @@ -485,7 +485,7 @@ void ColumnFamilyData::SetDropped() { } ColumnFamilyOptions ColumnFamilyData::GetLatestCFOptions() const { - return BuildColumnFamilyOptions(options_, mutable_cf_options_); + return BuildColumnFamilyOptions(initial_cf_options_, mutable_cf_options_); } const double kSlowdownRatio = 1.2; @@ -869,14 +869,14 @@ Status ColumnFamilyData::SetOptions( #endif // ROCKSDB_LITE ColumnFamilySet::ColumnFamilySet(const std::string& dbname, - const DBOptions* db_options, + const ImmutableDBOptions* db_options, const EnvOptions& env_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller) : max_column_family_(0), dummy_cfd_(new ColumnFamilyData(0, "", nullptr, nullptr, nullptr, - ColumnFamilyOptions(), db_options, + ColumnFamilyOptions(), *db_options, env_options, nullptr)), default_cfd_cache_(nullptr), db_name_(dbname), @@ -948,7 +948,7 @@ ColumnFamilyData* ColumnFamilySet::CreateColumnFamily( assert(column_families_.find(name) == column_families_.end()); ColumnFamilyData* new_cfd = new ColumnFamilyData( id, name, dummy_versions, table_cache_, write_buffer_manager_, options, - db_options_, env_options_, this); + *db_options_, env_options_, this); column_families_.insert({name, id}); column_family_data_.insert({id, new_cfd}); max_column_family_ = std::max(max_column_family_, id); diff --git a/db/column_family.h b/db/column_family.h index 7d1348989..3b827a25e 100644 --- a/db/column_family.h +++ b/db/column_family.h @@ -136,14 +136,14 @@ extern Status CheckCompressionSupported(const ColumnFamilyOptions& cf_options); extern Status CheckConcurrentWritesSupported( const ColumnFamilyOptions& cf_options); -extern ColumnFamilyOptions SanitizeOptions(const DBOptions& db_options, +extern ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options, const InternalKeyComparator* icmp, const ColumnFamilyOptions& src); // Wrap user defined table proproties collector factories `from cf_options` // into internal ones in int_tbl_prop_collector_factories. Add a system internal // one too. extern void GetIntTblPropCollectorFactory( - const ColumnFamilyOptions& cf_options, + const ImmutableCFOptions& ioptions, std::vector>* int_tbl_prop_collector_factories); @@ -328,7 +328,8 @@ class ColumnFamilyData { Version* dummy_versions, Cache* table_cache, WriteBufferManager* write_buffer_manager, const ColumnFamilyOptions& options, - const DBOptions* db_options, const EnvOptions& env_options, + const ImmutableDBOptions& db_options, + const EnvOptions& env_options, ColumnFamilySet* column_family_set); uint32_t id_; @@ -343,7 +344,7 @@ class ColumnFamilyData { std::vector> int_tbl_prop_collector_factories_; - const Options options_; + const ColumnFamilyOptions initial_cf_options_; const ImmutableCFOptions ioptions_; MutableCFOptions mutable_cf_options_; @@ -439,7 +440,8 @@ class ColumnFamilySet { ColumnFamilyData* current_; }; - ColumnFamilySet(const std::string& dbname, const DBOptions* db_options, + ColumnFamilySet(const std::string& dbname, + const ImmutableDBOptions* db_options, const EnvOptions& env_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller); @@ -496,7 +498,7 @@ class ColumnFamilySet { ColumnFamilyData* default_cfd_cache_; const std::string db_name_; - const DBOptions* const db_options_; + const ImmutableDBOptions* const db_options_; const EnvOptions env_options_; Cache* table_cache_; WriteBufferManager* write_buffer_manager_; diff --git a/db/column_family_test.cc b/db/column_family_test.cc index d38deb6b6..d0088f386 100644 --- a/db/column_family_test.cc +++ b/db/column_family_test.cc @@ -2178,8 +2178,8 @@ TEST_F(ColumnFamilyTest, SanitizeOptions) { original.write_buffer_size = l * 4 * 1024 * 1024 + i * 1024 * 1024 + j * 1024 + k; - ColumnFamilyOptions result = - SanitizeOptions(db_options, nullptr, original); + ColumnFamilyOptions result = SanitizeOptions( + ImmutableDBOptions(db_options), nullptr, original); ASSERT_TRUE(result.level0_stop_writes_trigger >= result.level0_slowdown_writes_trigger); ASSERT_TRUE(result.level0_slowdown_writes_trigger >= diff --git a/db/compacted_db_impl.cc b/db/compacted_db_impl.cc index e72a4bd59..f59ccd2c9 100644 --- a/db/compacted_db_impl.cc +++ b/db/compacted_db_impl.cc @@ -151,9 +151,9 @@ Status CompactedDBImpl::Open(const Options& options, std::unique_ptr db(new CompactedDBImpl(db_options, dbname)); Status s = db->Init(options); if (s.ok()) { - Log(INFO_LEVEL, db->db_options_.info_log, + Log(INFO_LEVEL, db->immutable_db_options_.info_log, "Opened the db as fully compacted mode"); - LogFlush(db->db_options_.info_log); + LogFlush(db->immutable_db_options_.info_log); *dbptr = db.release(); } return s; diff --git a/db/compaction_job.cc b/db/compaction_job.cc index fee5cdc9a..4d9da9061 100644 --- a/db/compaction_job.cc +++ b/db/compaction_job.cc @@ -262,7 +262,7 @@ void CompactionJob::AggregateStatistics() { } CompactionJob::CompactionJob( - int job_id, Compaction* compaction, const DBOptions& db_options, + int job_id, Compaction* compaction, const ImmutableDBOptions& db_options, const EnvOptions& env_options, VersionSet* versions, std::atomic* shutting_down, LogBuffer* log_buffer, Directory* db_directory, Directory* output_directory, Statistics* stats, @@ -539,7 +539,7 @@ Status CompactionJob::Run() { thread.join(); } - if (output_directory_ && !db_options_.disableDataSync) { + if (output_directory_ && !db_options_.disable_data_sync) { output_directory_->Fsync(); } @@ -963,7 +963,7 @@ Status CompactionJob::FinishCompactionOutputFile( sub_compact->total_bytes += current_bytes; // Finish and check for file errors - if (s.ok() && !db_options_.disableDataSync) { + if (s.ok() && !db_options_.disable_data_sync) { StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS); s = sub_compact->outfile->Sync(db_options_.use_fsync); } diff --git a/db/compaction_job.h b/db/compaction_job.h index b2a592ea9..a8ace56c6 100644 --- a/db/compaction_job.h +++ b/db/compaction_job.h @@ -37,6 +37,7 @@ #include "rocksdb/transaction_log.h" #include "table/scoped_arena_iterator.h" #include "util/autovector.h" +#include "util/db_options.h" #include "util/event_logger.h" #include "util/stop_watch.h" #include "util/thread_local.h" @@ -52,7 +53,8 @@ class Arena; class CompactionJob { public: - CompactionJob(int job_id, Compaction* compaction, const DBOptions& db_options, + CompactionJob(int job_id, Compaction* compaction, + const ImmutableDBOptions& db_options, const EnvOptions& env_options, VersionSet* versions, std::atomic* shutting_down, LogBuffer* log_buffer, Directory* db_directory, Directory* output_directory, @@ -120,7 +122,7 @@ class CompactionJob { // DBImpl state const std::string& dbname_; - const DBOptions& db_options_; + const ImmutableDBOptions& db_options_; const EnvOptions& env_options_; Env* env_; diff --git a/db/compaction_job_test.cc b/db/compaction_job_test.cc index 10236a093..76bf46ceb 100644 --- a/db/compaction_job_test.cc +++ b/db/compaction_job_test.cc @@ -68,7 +68,8 @@ class CompactionJobTest : public testing::Test { CompactionJobTest() : env_(Env::Default()), dbname_(test::TmpDir() + "/compaction_job_test"), - mutable_cf_options_(Options()), + db_options_(), + mutable_cf_options_(cf_options_), table_cache_(NewLRUCache(50000, 16)), write_buffer_manager_(db_options_.db_write_buffer_size), versions_(new VersionSet(dbname_, &db_options_, env_options_, @@ -280,11 +281,11 @@ class CompactionJobTest : public testing::Test { Env* env_; std::string dbname_; EnvOptions env_options_; + ImmutableDBOptions db_options_; + ColumnFamilyOptions cf_options_; MutableCFOptions mutable_cf_options_; std::shared_ptr table_cache_; WriteController write_controller_; - DBOptions db_options_; - ColumnFamilyOptions cf_options_; WriteBufferManager write_buffer_manager_; std::unique_ptr versions_; InstrumentedMutex mutex_; diff --git a/db/db_filesnapshot.cc b/db/db_filesnapshot.cc index 7ed1f1d36..65e8fffb7 100644 --- a/db/db_filesnapshot.cc +++ b/db/db_filesnapshot.cc @@ -35,10 +35,10 @@ Status DBImpl::DisableFileDeletions() { InstrumentedMutexLock l(&mutex_); ++disable_delete_obsolete_files_; if (disable_delete_obsolete_files_ == 1) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "File Deletions Disabled"); } else { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "File Deletions Disabled, but already disabled. Counter: %d", disable_delete_obsolete_files_); } @@ -59,12 +59,12 @@ Status DBImpl::EnableFileDeletions(bool force) { --disable_delete_obsolete_files_; } if (disable_delete_obsolete_files_ == 0) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "File Deletions Enabled"); should_purge_files = true; FindObsoleteFiles(&job_context, true); } else { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "File Deletions Enable, but not really enabled. Counter: %d", disable_delete_obsolete_files_); } @@ -73,7 +73,7 @@ Status DBImpl::EnableFileDeletions(bool force) { PurgeObsoleteFiles(job_context); } job_context.Clean(); - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); return Status::OK(); } @@ -110,7 +110,7 @@ Status DBImpl::GetLiveFiles(std::vector& ret, if (!status.ok()) { mutex_.Unlock(); - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Cannot Flush data %s\n", status.ToString().c_str()); return status; } diff --git a/db/db_impl.cc b/db/db_impl.cc index e26a33d45..5debc9e87 100644 --- a/db/db_impl.cc +++ b/db/db_impl.cc @@ -124,12 +124,14 @@ Options SanitizeOptions(const std::string& dbname, const InternalKeyComparator* icmp, const Options& src) { auto db_options = SanitizeOptions(dbname, DBOptions(src)); - auto cf_options = SanitizeOptions(db_options, icmp, ColumnFamilyOptions(src)); + ImmutableDBOptions immutable_db_options(db_options); + auto cf_options = + SanitizeOptions(immutable_db_options, icmp, ColumnFamilyOptions(src)); return Options(db_options, cf_options); } DBOptions SanitizeOptions(const std::string& dbname, const DBOptions& src) { - DBOptions result = src; + DBOptions result(src); // result.max_open_files means an "infinite" open files. if (result.max_open_files != -1) { @@ -305,10 +307,12 @@ void DumpSupportInfo(Logger* logger) { DBImpl::DBImpl(const DBOptions& options, const std::string& dbname) : env_(options.env), dbname_(dbname), - db_options_(SanitizeOptions(dbname, options)), - stats_(db_options_.statistics.get()), + immutable_db_options_(SanitizeOptions(dbname, options)), + mutable_db_options_(options), + stats_(immutable_db_options_.statistics.get()), db_lock_(nullptr), - mutex_(stats_, env_, DB_MUTEX_WAIT_MICROS, options.use_adaptive_mutex), + mutex_(stats_, env_, DB_MUTEX_WAIT_MICROS, + immutable_db_options_.use_adaptive_mutex), shutting_down_(false), bg_cv_(&mutex_), logfile_number_(0), @@ -319,12 +323,12 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname) total_log_size_(0), max_total_in_memory_state_(0), is_snapshot_supported_(true), - write_buffer_manager_(db_options_.write_buffer_manager.get()), - write_thread_(options.enable_write_thread_adaptive_yield - ? options.write_thread_max_yield_usec + write_buffer_manager_(immutable_db_options_.write_buffer_manager.get()), + write_thread_(immutable_db_options_.enable_write_thread_adaptive_yield + ? immutable_db_options_.write_thread_max_yield_usec : 0, - options.write_thread_slow_yield_usec), - write_controller_(options.delayed_write_rate), + immutable_db_options_.write_thread_slow_yield_usec), + write_controller_(immutable_db_options_.delayed_write_rate), last_batch_group_size_(0), unscheduled_flushes_(0), unscheduled_compactions_(0), @@ -335,16 +339,16 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname) bg_purge_scheduled_(0), disable_delete_obsolete_files_(0), delete_obsolete_files_next_run_( - options.env->NowMicros() + - db_options_.delete_obsolete_files_period_micros), + env_->NowMicros() + + immutable_db_options_.delete_obsolete_files_period_micros), last_stats_dump_time_microsec_(0), next_job_id_(1), has_unpersisted_data_(false), - env_options_(db_options_), + env_options_(BuildDBOptions(immutable_db_options_, mutable_db_options_)), #ifndef ROCKSDB_LITE - wal_manager_(db_options_, env_options_), + wal_manager_(immutable_db_options_, env_options_), #endif // ROCKSDB_LITE - event_logger_(db_options_.info_log.get()), + event_logger_(immutable_db_options_.info_log.get()), bg_work_paused_(0), bg_compaction_paused_(0), refitting_level_(false), @@ -353,21 +357,23 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname) // Reserve ten files or so for other uses and give the rest to TableCache. // Give a large number for setting of "infinite" open files. - const int table_cache_size = (db_options_.max_open_files == -1) ? - 4194304 : db_options_.max_open_files - 10; - table_cache_ = - NewLRUCache(table_cache_size, db_options_.table_cache_numshardbits); + const int table_cache_size = (immutable_db_options_.max_open_files == -1) + ? 4194304 + : immutable_db_options_.max_open_files - 10; + table_cache_ = NewLRUCache(table_cache_size, + immutable_db_options_.table_cache_numshardbits); - versions_.reset(new VersionSet(dbname_, &db_options_, env_options_, + versions_.reset(new VersionSet(dbname_, &immutable_db_options_, env_options_, table_cache_.get(), write_buffer_manager_, &write_controller_)); column_family_memtables_.reset( new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet())); - DumpRocksDBBuildVersion(db_options_.info_log.get()); - DumpDBFileSummary(db_options_, dbname_); - db_options_.Dump(db_options_.info_log.get()); - DumpSupportInfo(db_options_.info_log.get()); + DumpRocksDBBuildVersion(immutable_db_options_.info_log.get()); + DumpDBFileSummary(immutable_db_options_, dbname_); + immutable_db_options_.Dump(immutable_db_options_.info_log.get()); + mutable_db_options_.Dump(immutable_db_options_.info_log.get()); + DumpSupportInfo(immutable_db_options_.info_log.get()); } // Will lock the mutex_, will wait for completion if wait is true @@ -497,7 +503,7 @@ DBImpl::~DBImpl() { env_->UnlockFile(db_lock_); } - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); } Status DBImpl::NewDB() { @@ -508,8 +514,8 @@ Status DBImpl::NewDB() { Status s; - Log(InfoLogLevel::INFO_LEVEL, - db_options_.info_log, "Creating manifest 1 \n"); + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, + "Creating manifest 1 \n"); const std::string manifest = DescriptorFileName(dbname_, 1); { unique_ptr file; @@ -518,7 +524,8 @@ Status DBImpl::NewDB() { if (!s.ok()) { return s; } - file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size); + file->SetPreallocationBlockSize( + immutable_db_options_.manifest_preallocation_size); unique_ptr file_writer( new WritableFileWriter(std::move(file), env_options)); log::Writer log(std::move(file_writer), 0, false); @@ -526,7 +533,7 @@ Status DBImpl::NewDB() { new_db.EncodeTo(&record); s = log.AddRecord(record); if (s.ok()) { - s = SyncManifest(env_, &db_options_, log.file()); + s = SyncManifest(env_, &immutable_db_options_, log.file()); } } if (s.ok()) { @@ -539,29 +546,29 @@ Status DBImpl::NewDB() { } void DBImpl::MaybeIgnoreError(Status* s) const { - if (s->ok() || db_options_.paranoid_checks) { + if (s->ok() || immutable_db_options_.paranoid_checks) { // No change needed } else { - Log(InfoLogLevel::WARN_LEVEL, - db_options_.info_log, "Ignoring error %s", s->ToString().c_str()); + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, + "Ignoring error %s", s->ToString().c_str()); *s = Status::OK(); } } const Status DBImpl::CreateArchivalDirectory() { - if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) { - std::string archivalPath = ArchivalDirectory(db_options_.wal_dir); + if (immutable_db_options_.wal_ttl_seconds > 0 || + immutable_db_options_.wal_size_limit_mb > 0) { + std::string archivalPath = ArchivalDirectory(immutable_db_options_.wal_dir); return env_->CreateDirIfMissing(archivalPath); } return Status::OK(); } void DBImpl::PrintStatistics() { - auto dbstats = db_options_.statistics.get(); + auto dbstats = immutable_db_options_.statistics.get(); if (dbstats) { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, - "STATISTICS:\n %s", - dbstats->ToString().c_str()); + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, + "STATISTICS:\n %s", dbstats->ToString().c_str()); } } @@ -599,13 +606,13 @@ static void DumpMallocStats(std::string* stats) { #endif // !ROCKSDB_LITE void DBImpl::MaybeDumpStats() { - if (db_options_.stats_dump_period_sec == 0) return; + if (immutable_db_options_.stats_dump_period_sec == 0) return; const uint64_t now_micros = env_->NowMicros(); if (last_stats_dump_time_microsec_ + - db_options_.stats_dump_period_sec * 1000000 - <= now_micros) { + immutable_db_options_.stats_dump_period_sec * 1000000 <= + now_micros) { // Multiple threads could race in here simultaneously. // However, the last one will update last_stats_dump_time_microsec_ // atomically. We could see more than one dump during one dump @@ -630,13 +637,13 @@ void DBImpl::MaybeDumpStats() { default_cf_internal_stats_->GetStringProperty( *db_property_info, DB::Properties::kDBStats, &stats); } - if (db_options_.dump_malloc_stats) { + if (immutable_db_options_.dump_malloc_stats) { DumpMallocStats(&stats); } - Log(InfoLogLevel::WARN_LEVEL, - db_options_.info_log, "------- DUMPING STATS -------"); - Log(InfoLogLevel::WARN_LEVEL, - db_options_.info_log, "%s", stats.c_str()); + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, + "------- DUMPING STATS -------"); + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "%s", + stats.c_str()); #endif // !ROCKSDB_LITE PrintStatistics(); @@ -733,7 +740,7 @@ void DBImpl::ScheduleBgLogWriterClose(JobContext* job_context) { // Otherwise, gets obsolete files from VersionSet. // no_full_scan = true -- never do the full scan using GetChildren() // force = false -- don't force the full scan, except every -// db_options_.delete_obsolete_files_period_micros +// immutable_db_options_.delete_obsolete_files_period_micros // force = true -- force the full scan void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, bool no_full_scan) { @@ -749,14 +756,16 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, // logic for figurint out if we're doing the full scan if (no_full_scan) { doing_the_full_scan = false; - } else if (force || db_options_.delete_obsolete_files_period_micros == 0) { + } else if (force || + immutable_db_options_.delete_obsolete_files_period_micros == 0) { doing_the_full_scan = true; } else { const uint64_t now_micros = env_->NowMicros(); if (delete_obsolete_files_next_run_ < now_micros) { doing_the_full_scan = true; delete_obsolete_files_next_run_ = - now_micros + db_options_.delete_obsolete_files_period_micros; + now_micros + + immutable_db_options_.delete_obsolete_files_period_micros; } } @@ -819,8 +828,9 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, // find newly obsoleted log files while (alive_log_files_.begin()->number < min_log_number) { auto& earliest = *alive_log_files_.begin(); - if (db_options_.recycle_log_file_num > log_recycle_files.size()) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + if (immutable_db_options_.recycle_log_file_num > + log_recycle_files.size()) { + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "adding log %" PRIu64 " to recycle list\n", earliest.number); log_recycle_files.push_back(earliest.number); } else { @@ -852,11 +862,12 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, } if (doing_the_full_scan) { - for (size_t path_id = 0; path_id < db_options_.db_paths.size(); path_id++) { + for (size_t path_id = 0; path_id < immutable_db_options_.db_paths.size(); + path_id++) { // set of all files in the directory. We'll exclude files that are still // alive in the subsequent processings. std::vector files; - env_->GetChildren(db_options_.db_paths[path_id].path, + env_->GetChildren(immutable_db_options_.db_paths[path_id].path, &files); // Ignore errors for (std::string file : files) { // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes @@ -866,26 +877,27 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, } //Add log files in wal_dir - if (db_options_.wal_dir != dbname_) { + if (immutable_db_options_.wal_dir != dbname_) { std::vector log_files; - env_->GetChildren(db_options_.wal_dir, &log_files); // Ignore errors - InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_); + env_->GetChildren(immutable_db_options_.wal_dir, + &log_files); // Ignore errors + InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(), + dbname_); for (std::string log_file : log_files) { uint64_t number; FileType type; // Ignore file if we cannot recognize it. if (!ParseFileName(log_file, &number, info_log_prefix.prefix, &type)) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, - "Unrecognized log file %s \n",log_file.c_str()); + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, + "Unrecognized log file %s \n", log_file.c_str()); continue; } // If the log file is already in the log recycle list , don't put // it in the candidate list. if (std::find(log_recycle_files.begin(), log_recycle_files.end(),number) != log_recycle_files.end()) { - - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, - "Log %" PRIu64 " Already added in the recycle list, skipping.\n", + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, + "Log %" PRIu64 " Already added in the recycle list, skipping.\n", number); continue; } @@ -894,10 +906,11 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, } } // Add info log files in db_log_dir - if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) { + if (!immutable_db_options_.db_log_dir.empty() && + immutable_db_options_.db_log_dir != dbname_) { std::vector info_log_files; // Ignore errors - env_->GetChildren(db_options_.db_log_dir, &info_log_files); + env_->GetChildren(immutable_db_options_.db_log_dir, &info_log_files); for (std::string log_file : info_log_files) { job_context->full_scan_candidate_files.emplace_back(log_file, 0); } @@ -928,29 +941,30 @@ void DBImpl::DeleteObsoleteFileImpl(Status file_deletion_status, int job_id, const std::string& fname, FileType type, uint64_t number, uint32_t path_id) { if (type == kTableFile) { - file_deletion_status = DeleteSSTFile(&db_options_, fname, path_id); + file_deletion_status = + DeleteSSTFile(&immutable_db_options_, fname, path_id); } else { file_deletion_status = env_->DeleteFile(fname); } if (file_deletion_status.ok()) { - Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log, + Log(InfoLogLevel::DEBUG_LEVEL, immutable_db_options_.info_log, "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", job_id, fname.c_str(), type, number, file_deletion_status.ToString().c_str()); } else if (env_->FileExists(fname).IsNotFound()) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[JOB %d] Tried to delete a non-existing file %s type=%d #%" PRIu64 " -- %s\n", job_id, fname.c_str(), type, number, file_deletion_status.ToString().c_str()); } else { - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "[JOB %d] Failed to delete %s type=%d #%" PRIu64 " -- %s\n", job_id, fname.c_str(), type, number, file_deletion_status.ToString().c_str()); } if (type == kTableFile) { EventHelpers::LogAndNotifyTableFileDeletion( &event_logger_, job_id, number, fname, file_deletion_status, GetName(), - db_options_.listeners); + immutable_db_options_.listeners); } } @@ -1008,7 +1022,7 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) { candidate_files.end()); if (state.prev_total_log_size > 0) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[JOB %d] Try to delete WAL files size %" PRIu64 ", prev total WAL file size %" PRIu64 ", number of live WAL files %" ROCKSDB_PRIszt ".\n", @@ -1017,7 +1031,8 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) { } std::vector old_info_log_files; - InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_); + InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(), + dbname_); for (const auto& candidate_file : candidate_files) { std::string to_delete = candidate_file.file_name; uint32_t path_id = candidate_file.path_id; @@ -1081,15 +1096,15 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) { if (type == kTableFile) { // evict from cache TableCache::Evict(table_cache_.get(), number); - fname = TableFileName(db_options_.db_paths, number, path_id); + fname = TableFileName(immutable_db_options_.db_paths, number, path_id); } else { - fname = ((type == kLogFile) ? - db_options_.wal_dir : dbname_) + "/" + to_delete; + fname = ((type == kLogFile) ? immutable_db_options_.wal_dir : dbname_) + + "/" + to_delete; } #ifndef ROCKSDB_LITE - if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 || - db_options_.WAL_size_limit_MB > 0)) { + if (type == kLogFile && (immutable_db_options_.wal_ttl_seconds > 0 || + immutable_db_options_.wal_size_limit_mb > 0)) { wal_manager_.ArchiveWALFile(fname, number); continue; } @@ -1108,25 +1123,29 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) { // Delete old info log files. size_t old_info_log_file_count = old_info_log_files.size(); if (old_info_log_file_count != 0 && - old_info_log_file_count >= db_options_.keep_log_file_num) { + old_info_log_file_count >= immutable_db_options_.keep_log_file_num) { std::sort(old_info_log_files.begin(), old_info_log_files.end()); - size_t end = old_info_log_file_count - db_options_.keep_log_file_num; + size_t end = + old_info_log_file_count - immutable_db_options_.keep_log_file_num; for (unsigned int i = 0; i <= end; i++) { std::string& to_delete = old_info_log_files.at(i); - std::string full_path_to_delete = (db_options_.db_log_dir.empty() ? - dbname_ : db_options_.db_log_dir) + "/" + to_delete; - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + std::string full_path_to_delete = + (immutable_db_options_.db_log_dir.empty() + ? dbname_ + : immutable_db_options_.db_log_dir) + + "/" + to_delete; + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[JOB %d] Delete info log file %s\n", state.job_id, full_path_to_delete.c_str()); Status s = env_->DeleteFile(full_path_to_delete); if (!s.ok()) { if (env_->FileExists(full_path_to_delete).IsNotFound()) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[JOB %d] Tried to delete non-existing info log file %s FAILED " "-- %s\n", state.job_id, to_delete.c_str(), s.ToString().c_str()); } else { - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "[JOB %d] Delete info log file %s FAILED -- %s\n", state.job_id, to_delete.c_str(), s.ToString().c_str()); } @@ -1136,7 +1155,7 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) { #ifndef ROCKSDB_LITE wal_manager_.PurgeObsoleteWALFiles(); #endif // ROCKSDB_LITE - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); } void DBImpl::DeleteObsoleteFiles() { @@ -1219,8 +1238,9 @@ Status DBImpl::Recover( bool is_new_db = false; assert(db_lock_ == nullptr); if (!read_only) { - Status s = directories_.SetDirectories(env_, dbname_, db_options_.wal_dir, - db_options_.db_paths); + Status s = directories_.SetDirectories(env_, dbname_, + immutable_db_options_.wal_dir, + immutable_db_options_.db_paths); if (!s.ok()) { return s; } @@ -1232,7 +1252,7 @@ Status DBImpl::Recover( s = env_->FileExists(CurrentFileName(dbname_)); if (s.IsNotFound()) { - if (db_options_.create_if_missing) { + if (immutable_db_options_.create_if_missing) { s = NewDB(); is_new_db = true; if (!s.ok()) { @@ -1243,7 +1263,7 @@ Status DBImpl::Recover( dbname_, "does not exist (create_if_missing is false)"); } } else if (s.ok()) { - if (db_options_.error_if_exists) { + if (immutable_db_options_.error_if_exists) { return Status::InvalidArgument( dbname_, "exists (error_if_exists is true)"); } @@ -1266,7 +1286,7 @@ Status DBImpl::Recover( } Status s = versions_->Recover(column_families, read_only); - if (db_options_.paranoid_checks && s.ok()) { + if (immutable_db_options_.paranoid_checks && s.ok()) { s = CheckConsistency(); } if (s.ok()) { @@ -1285,7 +1305,7 @@ Status DBImpl::Recover( // attention to it in case we are recovering a database // produced by an older version of rocksdb. std::vector filenames; - s = env_->GetChildren(db_options_.wal_dir, &filenames); + s = env_->GetChildren(immutable_db_options_.wal_dir, &filenames); if (!s.ok()) { return s; } @@ -1313,7 +1333,7 @@ Status DBImpl::Recover( "flag but a log file already exists"); } else if (error_if_data_exists_in_logs) { for (auto& log : logs) { - std::string fname = LogFileName(db_options_.wal_dir, log); + std::string fname = LogFileName(immutable_db_options_.wal_dir, log); uint64_t bytes; s = env_->GetFileSize(fname, &bytes); if (s.ok()) { @@ -1360,7 +1380,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, Env* env; Logger* info_log; const char* fname; - Status* status; // nullptr if db_options_.paranoid_checks==false + Status* status; // nullptr if immutable_db_options_.paranoid_checks==false virtual void Corruption(size_t bytes, const Status& s) override { Log(InfoLogLevel::WARN_LEVEL, info_log, "%s%s: dropping %d bytes; %s", @@ -1395,7 +1415,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, } #ifndef ROCKSDB_LITE - if (db_options_.wal_filter != nullptr) { + if (immutable_db_options_.wal_filter != nullptr) { std::map cf_name_id_map; std::map cf_lognumber_map; for (auto cfd : *versions_->GetColumnFamilySet()) { @@ -1405,9 +1425,8 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, std::make_pair(cfd->GetID(), cfd->GetLogNumber())); } - db_options_.wal_filter->ColumnFamilyLogNumberMap( - cf_lognumber_map, - cf_name_id_map); + immutable_db_options_.wal_filter->ColumnFamilyLogNumberMap(cf_lognumber_map, + cf_name_id_map); } #endif @@ -1420,15 +1439,15 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, // update the file number allocation counter in VersionSet. versions_->MarkFileNumberUsedDuringRecovery(log_number); // Open the log file - std::string fname = LogFileName(db_options_.wal_dir, log_number); + std::string fname = LogFileName(immutable_db_options_.wal_dir, log_number); - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "Recovering log #%" PRIu64 " mode %d", log_number, - db_options_.wal_recovery_mode); + immutable_db_options_.wal_recovery_mode); auto logFileDropped = [this, &fname]() { uint64_t bytes; if (env_->GetFileSize(fname, &bytes).ok()) { - auto info_log = db_options_.info_log.get(); + auto info_log = immutable_db_options_.info_log.get(); Log(InfoLogLevel::WARN_LEVEL, info_log, "%s: dropping %d bytes", fname.c_str(), static_cast(bytes)); } @@ -1458,10 +1477,10 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, // Create the log reader. LogReporter reporter; reporter.env = env_; - reporter.info_log = db_options_.info_log.get(); + reporter.info_log = immutable_db_options_.info_log.get(); reporter.fname = fname.c_str(); - if (!db_options_.paranoid_checks || - db_options_.wal_recovery_mode == + if (!immutable_db_options_.paranoid_checks || + immutable_db_options_.wal_recovery_mode == WALRecoveryMode::kSkipAnyCorruptedRecords) { reporter.status = nullptr; } else { @@ -1471,8 +1490,9 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, // paranoid_checks==false so that corruptions cause entire commits // to be skipped instead of propagating bad information (like overly // large sequence numbers). - log::Reader reader(db_options_.info_log, std::move(file_reader), &reporter, - true /*checksum*/, 0 /*initial_offset*/, log_number); + log::Reader reader(immutable_db_options_.info_log, std::move(file_reader), + &reporter, true /*checksum*/, 0 /*initial_offset*/, + log_number); // Determine if we should tolerate incomplete records at the tail end of the // Read all the records and add to a memtable @@ -1480,10 +1500,10 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, Slice record; WriteBatch batch; - while ( - !stop_replay_by_wal_filter && - reader.ReadRecord(&record, &scratch, db_options_.wal_recovery_mode) && - status.ok()) { + while (!stop_replay_by_wal_filter && + reader.ReadRecord(&record, &scratch, + immutable_db_options_.wal_recovery_mode) && + status.ok()) { if (record.size() < WriteBatchInternal::kHeader) { reporter.Corruption(record.size(), Status::Corruption("log record too small")); @@ -1492,7 +1512,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, WriteBatchInternal::SetContents(&batch, record); SequenceNumber sequence = WriteBatchInternal::Sequence(&batch); - if (db_options_.wal_recovery_mode == + if (immutable_db_options_.wal_recovery_mode == WALRecoveryMode::kPointInTimeRecovery) { // In point-in-time recovery mode, if sequence id of log files are // consecutive, we continue recovery despite corruption. This could @@ -1508,7 +1528,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, } bool no_prev_seq = true; - if (!db_options_.allow_2pc) { + if (!immutable_db_options_.allow_2pc) { *next_sequence = sequence; } else { if (*next_sequence == kMaxSequenceNumber) { @@ -1520,13 +1540,13 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, } #ifndef ROCKSDB_LITE - if (db_options_.wal_filter != nullptr) { + if (immutable_db_options_.wal_filter != nullptr) { WriteBatch new_batch; bool batch_changed = false; WalFilter::WalProcessingOption wal_processing_option = - db_options_.wal_filter->LogRecordFound(log_number, fname, batch, - &new_batch, &batch_changed); + immutable_db_options_.wal_filter->LogRecordFound( + log_number, fname, batch, &new_batch, &batch_changed); switch (wal_processing_option) { case WalFilter::WalProcessingOption::kContinueProcessing: @@ -1540,8 +1560,9 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, stop_replay_by_wal_filter = true; continue; case WalFilter::WalProcessingOption::kCorruptedRecord: { - status = Status::Corruption("Corruption reported by Wal Filter ", - db_options_.wal_filter->Name()); + status = + Status::Corruption("Corruption reported by Wal Filter ", + immutable_db_options_.wal_filter->Name()); MaybeIgnoreError(&status); if (!status.ok()) { reporter.Corruption(record.size(), status); @@ -1554,7 +1575,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, status = Status::NotSupported( "Unknown WalProcessingOption returned" " by Wal Filter ", - db_options_.wal_filter->Name()); + immutable_db_options_.wal_filter->Name()); MaybeIgnoreError(&status); if (!status.ok()) { return status; @@ -1571,17 +1592,18 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, int new_count = WriteBatchInternal::Count(&new_batch); int original_count = WriteBatchInternal::Count(&batch); if (new_count > original_count) { - Log(InfoLogLevel::FATAL_LEVEL, db_options_.info_log, + Log(InfoLogLevel::FATAL_LEVEL, immutable_db_options_.info_log, "Recovering log #%" PRIu64 " mode %d log filter %s returned " "more records (%d) than original (%d) which is not allowed. " "Aborting recovery.", - log_number, db_options_.wal_recovery_mode, - db_options_.wal_filter->Name(), new_count, original_count); + log_number, immutable_db_options_.wal_recovery_mode, + immutable_db_options_.wal_filter->Name(), new_count, + original_count); status = Status::NotSupported( "More than original # of records " "returned by Wal Filter ", - db_options_.wal_filter->Name()); + immutable_db_options_.wal_filter->Name()); return status; } // Set the same sequence number in the new_batch @@ -1607,7 +1629,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, // after replaying the file, this file may be a stale file. We ignore // sequence IDs from the file. Otherwise, if a newer stale log file that // has been deleted, the sequenceID may be wrong. - if (db_options_.allow_2pc) { + if (immutable_db_options_.allow_2pc) { if (no_prev_seq && !has_valid_writes) { *next_sequence = kMaxSequenceNumber; } @@ -1648,23 +1670,23 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, } if (!status.ok()) { - if (db_options_.wal_recovery_mode == - WALRecoveryMode::kSkipAnyCorruptedRecords) { + if (immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kSkipAnyCorruptedRecords) { // We should ignore all errors unconditionally status = Status::OK(); - } else if (db_options_.wal_recovery_mode == + } else if (immutable_db_options_.wal_recovery_mode == WALRecoveryMode::kPointInTimeRecovery) { // We should ignore the error but not continue replaying status = Status::OK(); stop_replay_for_corruption = true; - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "Point in time recovered to log #%" PRIu64 " seq #%" PRIu64, log_number, *next_sequence); } else { - assert(db_options_.wal_recovery_mode == - WALRecoveryMode::kTolerateCorruptedTailRecords - || db_options_.wal_recovery_mode == - WALRecoveryMode::kAbsoluteConsistency); + assert(immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kTolerateCorruptedTailRecords || + immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kAbsoluteConsistency); return status; } } @@ -1701,7 +1723,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, // If flush happened in the middle of recovery (e.g. due to memtable // being full), we flush at the end. Otherwise we'll need to record // where we were on last flush, which make the logic complicated. - if (flushed || !db_options_.avoid_flush_during_recovery) { + if (flushed || !immutable_db_options_.avoid_flush_during_recovery) { status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit); if (!status.ok()) { // Recovery failed @@ -1758,7 +1780,7 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, TableProperties table_properties; { ScopedArenaIterator iter(mem->NewIterator(ro, &arena)); - Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log, + Log(InfoLogLevel::DEBUG_LEVEL, immutable_db_options_.info_log, "[%s] [WriteLevel0TableForRecovery]" " Level-0 table #%" PRIu64 ": started", cfd->GetName().c_str(), meta.fd.GetNumber()); @@ -1784,8 +1806,8 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, cfd->ioptions()->compression_opts, paranoid_file_checks, cfd->internal_stats(), TableFileCreationReason::kRecovery, &event_logger_, job_id); - LogFlush(db_options_.info_log); - Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log, + LogFlush(immutable_db_options_.info_log); + Log(InfoLogLevel::DEBUG_LEVEL, immutable_db_options_.info_log, "[%s] [WriteLevel0TableForRecovery]" " Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s", cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(), @@ -1837,10 +1859,10 @@ Status DBImpl::SyncClosedLogs(JobContext* job_context) { mutex_.Unlock(); for (log::Writer* log : logs_to_sync) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[JOB %d] Syncing log #%" PRIu64, job_context->job_id, log->get_log_number()); - s = log->file()->Sync(db_options_.use_fsync); + s = log->file()->Sync(immutable_db_options_.use_fsync); } if (s.ok()) { s = directories_.GetWalDir()->Fsync(); @@ -1871,7 +1893,7 @@ Status DBImpl::FlushMemTableToOutputFile( snapshots_.GetAll(&earliest_write_conflict_snapshot); FlushJob flush_job( - dbname_, cfd, db_options_, mutable_cf_options, env_options_, + dbname_, cfd, immutable_db_options_, mutable_cf_options, env_options_, versions_.get(), &mutex_, &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot, job_context, log_buffer, directories_.GetDbDir(), directories_.GetDataDir(0U), @@ -1885,7 +1907,7 @@ Status DBImpl::FlushMemTableToOutputFile( Status s; if (logfile_number_ > 0 && versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 0 && - !db_options_.disableDataSync) { + !immutable_db_options_.disable_data_sync) { // If there are more than one column families, we need to make sure that // all the log files except the most recent one are synced. Otherwise if // the host crashes after flushing and before WAL is persistent, the @@ -1916,8 +1938,8 @@ Status DBImpl::FlushMemTableToOutputFile( cfd->current()->storage_info()->LevelSummary(&tmp)); } - if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks && - bg_error_.ok()) { + if (!s.ok() && !s.IsShutdownInProgress() && + immutable_db_options_.paranoid_checks && bg_error_.ok()) { // if a bad error happened (not ShutdownInProgress) and paranoid_checks is // true, mark DB read-only bg_error_ = s; @@ -1928,12 +1950,12 @@ Status DBImpl::FlushMemTableToOutputFile( NotifyOnFlushCompleted(cfd, &file_meta, mutable_cf_options, job_context->job_id, flush_job.GetTableProperties()); #endif // ROCKSDB_LITE - auto sfm = - static_cast(db_options_.sst_file_manager.get()); + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); if (sfm) { // Notify sst_file_manager that a new file was added - std::string file_path = MakeTableFileName(db_options_.db_paths[0].path, - file_meta.fd.GetNumber()); + std::string file_path = MakeTableFileName( + immutable_db_options_.db_paths[0].path, file_meta.fd.GetNumber()); sfm->OnAddFile(file_path); if (sfm->IsMaxAllowedSpaceReached() && bg_error_.ok()) { bg_error_ = Status::IOError("Max allowed space was reached"); @@ -1950,7 +1972,7 @@ void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, int job_id, TableProperties prop) { #ifndef ROCKSDB_LITE - if (db_options_.listeners.size() == 0U) { + if (immutable_db_options_.listeners.size() == 0U) { return; } mutex_.AssertHeld(); @@ -1970,7 +1992,7 @@ void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd, info.cf_name = cfd->GetName(); // TODO(yhchiang): make db_paths dynamic in case flush does not // go to L0 in the future. - info.file_path = MakeTableFileName(db_options_.db_paths[0].path, + info.file_path = MakeTableFileName(immutable_db_options_.db_paths[0].path, file_meta->fd.GetNumber()); info.thread_id = env_->GetThreadID(); info.job_id = job_id; @@ -1979,7 +2001,7 @@ void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd, info.smallest_seqno = file_meta->smallest_seqno; info.largest_seqno = file_meta->largest_seqno; info.table_properties = prop; - for (auto listener : db_options_.listeners) { + for (auto listener : immutable_db_options_.listeners) { listener->OnFlushCompleted(this, info); } } @@ -1992,7 +2014,7 @@ void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd, Status DBImpl::CompactRange(const CompactRangeOptions& options, ColumnFamilyHandle* column_family, const Slice* begin, const Slice* end) { - if (options.target_path_id >= db_options_.db_paths.size()) { + if (options.target_path_id >= immutable_db_options_.db_paths.size()) { return Status::InvalidArgument("Invalid target path ID"); } @@ -2002,7 +2024,7 @@ Status DBImpl::CompactRange(const CompactRangeOptions& options, Status s = FlushMemTable(cfd, FlushOptions()); if (!s.ok()) { - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); return s; } @@ -2073,12 +2095,12 @@ Status DBImpl::CompactRange(const CompactRangeOptions& options, } } if (!s.ok()) { - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); return s; } if (options.change_level) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[RefitLevel] waiting for background threads to stop"); s = PauseBackgroundWork(); if (s.ok()) { @@ -2086,7 +2108,7 @@ Status DBImpl::CompactRange(const CompactRangeOptions& options, } ContinueBackgroundWork(); } - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); { InstrumentedMutexLock l(&mutex_); @@ -2117,7 +2139,7 @@ Status DBImpl::CompactFiles( Status s; JobContext job_context(0, true); LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, - db_options_.info_log.get()); + immutable_db_options_.info_log.get()); // Perform CompactFiles SuperVersion* sv = GetAndRefSuperVersion(cfd); @@ -2182,7 +2204,7 @@ Status DBImpl::CompactFilesImpl( version->GetColumnFamilyMetaData(&cf_meta); if (output_path_id < 0) { - if (db_options_.db_paths.size() == 1U) { + if (immutable_db_options_.db_paths.size() == 1U) { output_path_id = 0; } else { return Status::NotSupported( @@ -2237,8 +2259,8 @@ Status DBImpl::CompactFilesImpl( assert(is_snapshot_supported_ || snapshots_.empty()); CompactionJob compaction_job( - job_context->job_id, c.get(), db_options_, env_options_, versions_.get(), - &shutting_down_, log_buffer, directories_.GetDbDir(), + job_context->job_id, c.get(), immutable_db_options_, env_options_, + versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(), directories_.GetDataDir(c->output_path_id()), stats_, &mutex_, &bg_error_, snapshot_seqs, earliest_write_conflict_snapshot, table_cache_, &event_logger_, c->mutable_cf_options()->paranoid_file_checks, @@ -2290,11 +2312,11 @@ Status DBImpl::CompactFilesImpl( } else if (status.IsShutdownInProgress()) { // Ignore compaction errors found during shutting down } else { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "[%s] [JOB %d] Compaction error: %s", c->column_family_data()->GetName().c_str(), job_context->job_id, status.ToString().c_str()); - if (db_options_.paranoid_checks && bg_error_.ok()) { + if (immutable_db_options_.paranoid_checks && bg_error_.ok()) { bg_error_ = status; } } @@ -2342,7 +2364,7 @@ void DBImpl::NotifyOnCompactionCompleted( const CompactionJobStats& compaction_job_stats, const int job_id) { #ifndef ROCKSDB_LITE - if (db_options_.listeners.size() == 0U) { + if (immutable_db_options_.listeners.size() == 0U) { return; } mutex_.AssertHeld(); @@ -2366,8 +2388,8 @@ void DBImpl::NotifyOnCompactionCompleted( info.compression = c->output_compression(); for (size_t i = 0; i < c->num_input_levels(); ++i) { for (const auto fmd : *c->inputs(i)) { - auto fn = TableFileName(db_options_.db_paths, fmd->fd.GetNumber(), - fmd->fd.GetPathId()); + auto fn = TableFileName(immutable_db_options_.db_paths, + fmd->fd.GetNumber(), fmd->fd.GetPathId()); info.input_files.push_back(fn); if (info.table_properties.count(fn) == 0) { std::shared_ptr tp; @@ -2379,12 +2401,11 @@ void DBImpl::NotifyOnCompactionCompleted( } } for (const auto newf : c->edit()->GetNewFiles()) { - info.output_files.push_back( - TableFileName(db_options_.db_paths, - newf.second.fd.GetNumber(), - newf.second.fd.GetPathId())); + info.output_files.push_back(TableFileName(immutable_db_options_.db_paths, + newf.second.fd.GetNumber(), + newf.second.fd.GetPathId())); } - for (auto listener : db_options_.listeners) { + for (auto listener : immutable_db_options_.listeners) { listener->OnCompactionCompleted(this, info); } } @@ -2401,8 +2422,8 @@ Status DBImpl::SetOptions(ColumnFamilyHandle* column_family, #else auto* cfd = reinterpret_cast(column_family)->cfd(); if (options_map.empty()) { - Log(InfoLogLevel::WARN_LEVEL, - db_options_.info_log, "SetOptions() on column family [%s], empty input", + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, + "SetOptions() on column family [%s], empty input", cfd->GetName().c_str()); return Status::InvalidArgument("empty input"); } @@ -2432,33 +2453,31 @@ Status DBImpl::SetOptions(ColumnFamilyHandle* column_family, } } - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, - "SetOptions() on column family [%s], inputs:", - cfd->GetName().c_str()); + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, + "SetOptions() on column family [%s], inputs:", cfd->GetName().c_str()); for (const auto& o : options_map) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, - "%s: %s\n", o.first.c_str(), o.second.c_str()); + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "%s: %s\n", + o.first.c_str(), o.second.c_str()); } if (s.ok()) { - Log(InfoLogLevel::INFO_LEVEL, - db_options_.info_log, "[%s] SetOptions succeeded", - cfd->GetName().c_str()); - new_options.Dump(db_options_.info_log.get()); + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, + "[%s] SetOptions succeeded", cfd->GetName().c_str()); + new_options.Dump(immutable_db_options_.info_log.get()); if (!persist_options_status.ok()) { - if (db_options_.fail_if_options_file_error) { + if (immutable_db_options_.fail_if_options_file_error) { s = Status::IOError( "SetOptions succeeded, but unable to persist options", persist_options_status.ToString()); } - Warn(db_options_.info_log, + Warn(immutable_db_options_.info_log, "Unable to persist options in SetOptions() -- %s", persist_options_status.ToString().c_str()); } } else { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "[%s] SetOptions failed", cfd->GetName().c_str()); } - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); return s; #endif // ROCKSDB_LITE } @@ -2499,7 +2518,7 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { // only allow one thread refitting if (refitting_level_) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[ReFitLevel] another thread is refitting"); return Status::NotSupported("another thread is refitting"); } @@ -2527,7 +2546,7 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { } } if (to_level != level) { - Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log, + Log(InfoLogLevel::DEBUG_LEVEL, immutable_db_options_.info_log, "[%s] Before refitting:\n%s", cfd->GetName().c_str(), cfd->current()->DebugString().data()); @@ -2540,7 +2559,7 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { f->smallest_seqno, f->largest_seqno, f->marked_for_compaction); } - Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log, + Log(InfoLogLevel::DEBUG_LEVEL, immutable_db_options_.info_log, "[%s] Apply version edit:\n%s", cfd->GetName().c_str(), edit.DebugString().data()); @@ -2549,12 +2568,12 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { superversion_to_free.reset(InstallSuperVersionAndScheduleWork( cfd, new_superversion.release(), mutable_cf_options)); - Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log, + Log(InfoLogLevel::DEBUG_LEVEL, immutable_db_options_.info_log, "[%s] LogAndApply: %s\n", cfd->GetName().c_str(), status.ToString().data()); if (status.ok()) { - Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log, + Log(InfoLogLevel::DEBUG_LEVEL, immutable_db_options_.info_log, "[%s] After refitting:\n%s", cfd->GetName().c_str(), cfd->current()->DebugString().data()); } @@ -2608,10 +2627,10 @@ Status DBImpl::SyncWAL() { it != logs_.end() && it->number <= current_log_number; ++it) { if (!it->writer->file()->writable_file()->IsSyncThreadSafe()) { return Status::NotSupported( - "SyncWAL() is not supported for this implementation of WAL file", - db_options_.allow_mmap_writes - ? "try setting Options::allow_mmap_writes to false" - : Slice()); + "SyncWAL() is not supported for this implementation of WAL file", + immutable_db_options_.allow_mmap_writes + ? "try setting Options::allow_mmap_writes to false" + : Slice()); } } for (auto it = logs_.begin(); @@ -2628,7 +2647,7 @@ Status DBImpl::SyncWAL() { RecordTick(stats_, WAL_FILE_SYNCED); Status status; for (log::Writer* log : logs_to_sync) { - status = log->file()->SyncWithoutFlush(db_options_.use_fsync); + status = log->file()->SyncWithoutFlush(immutable_db_options_.use_fsync); if (!status.ok()) { break; } @@ -2736,7 +2755,7 @@ Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level, TEST_SYNC_POINT_CALLBACK("DBImpl::RunManualCompaction:NotScheduled", &mutex_); if (exclusive) { while (bg_compaction_scheduled_ > 0) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[%s] Manual compaction waiting for all other scheduled background " "compactions to finish", cfd->GetName().c_str()); @@ -2744,9 +2763,8 @@ Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level, } } - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, - "[%s] Manual compaction starting", - cfd->GetName().c_str()); + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, + "[%s] Manual compaction starting", cfd->GetName().c_str()); // We don't check bg_error_ here, because if we get the error in compaction, // the compaction will set manual.status to bg_error_ and set manual.done to @@ -2893,7 +2911,7 @@ void DBImpl::MaybeScheduleFlushOrCompaction() { } while (unscheduled_flushes_ > 0 && - bg_flush_scheduled_ < db_options_.max_background_flushes) { + bg_flush_scheduled_ < immutable_db_options_.max_background_flushes) { unscheduled_flushes_--; bg_flush_scheduled_++; env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this); @@ -2903,7 +2921,7 @@ void DBImpl::MaybeScheduleFlushOrCompaction() { // special case -- if max_background_flushes == 0, then schedule flush on a // compaction thread - if (db_options_.max_background_flushes == 0) { + if (immutable_db_options_.max_background_flushes == 0) { while (unscheduled_flushes_ > 0 && bg_flush_scheduled_ + bg_compaction_scheduled_ < bg_compactions_allowed) { @@ -2947,9 +2965,9 @@ void DBImpl::SchedulePurge() { int DBImpl::BGCompactionsAllowed() const { if (write_controller_.NeedSpeedupCompaction()) { - return db_options_.max_background_compactions; + return immutable_db_options_.max_background_compactions; } else { - return db_options_.base_background_compactions; + return immutable_db_options_.base_background_compactions; } } @@ -3117,7 +3135,7 @@ Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context, "Calling FlushMemTableToOutputFile with column " "family [%s], flush slots available %d, compaction slots allowed %d, " "compaction slots scheduled %d", - cfd->GetName().c_str(), db_options_.max_background_flushes, + cfd->GetName().c_str(), immutable_db_options_.max_background_flushes, bg_flush_scheduled_, BGCompactionsAllowed() - bg_compaction_scheduled_); status = FlushMemTableToOutputFile(cfd, mutable_cf_options, made_progress, job_context, log_buffer); @@ -3135,7 +3153,8 @@ void DBImpl::BackgroundCallFlush() { TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:start"); - LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get()); + LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, + immutable_db_options_.info_log.get()); { InstrumentedMutexLock l(&mutex_); num_running_flushes_++; @@ -3153,12 +3172,12 @@ void DBImpl::BackgroundCallFlush() { default_cf_internal_stats_->BumpAndGetBackgroundErrorCount(); bg_cv_.SignalAll(); // In case a waiter can proceed despite the error mutex_.Unlock(); - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Waiting after background flush error: %s" "Accumulated background error counts: %" PRIu64, s.ToString().c_str(), error_cnt); log_buffer.FlushBufferToLog(); - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); env_->SleepForMicroseconds(1000000); mutex_.Lock(); } @@ -3203,7 +3222,8 @@ void DBImpl::BackgroundCallCompaction(void* arg) { JobContext job_context(next_job_id_.fetch_add(1), true); TEST_SYNC_POINT("BackgroundCallCompaction:0"); MaybeDumpStats(); - LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get()); + LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, + immutable_db_options_.info_log.get()); { InstrumentedMutexLock l(&mutex_); num_running_compactions_++; @@ -3225,11 +3245,11 @@ void DBImpl::BackgroundCallCompaction(void* arg) { bg_cv_.SignalAll(); // In case a waiter can proceed despite the error mutex_.Unlock(); log_buffer.FlushBufferToLog(); - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Waiting after background compaction error: %s, " "Accumulated background error counts: %" PRIu64, s.ToString().c_str(), error_cnt); - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); env_->SleepForMicroseconds(1000000); mutex_.Lock(); } @@ -3439,9 +3459,9 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove"); // Instrument for event update // TODO(yhchiang): add op details for showing trivial-move. - ThreadStatusUtil::SetColumnFamily(c->column_family_data(), - c->column_family_data()->ioptions()->env, - db_options_.enable_thread_tracking); + ThreadStatusUtil::SetColumnFamily( + c->column_family_data(), c->column_family_data()->ioptions()->env, + immutable_db_options_.enable_thread_tracking); ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION); compaction_job_stats.num_input_files = c->num_input_files(0); @@ -3508,7 +3528,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, assert(is_snapshot_supported_ || snapshots_.empty()); CompactionJob compaction_job( - job_context->job_id, c.get(), db_options_, env_options_, + job_context->job_id, c.get(), immutable_db_options_, env_options_, versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(), directories_.GetDataDir(c->output_path_id()), stats_, &mutex_, &bg_error_, snapshot_seqs, earliest_write_conflict_snapshot, @@ -3546,9 +3566,9 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, } else if (status.IsShutdownInProgress()) { // Ignore compaction errors found during shutting down } else { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s", - status.ToString().c_str()); - if (db_options_.paranoid_checks && bg_error_.ok()) { + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, + "Compaction error: %s", status.ToString().c_str()); + if (immutable_db_options_.paranoid_checks && bg_error_.ok()) { bg_error_ = status; } } @@ -3684,16 +3704,16 @@ size_t DBImpl::GetWalPreallocateBlockSize(uint64_t write_buffer_size) const { size_t bsize = write_buffer_size / 10 + write_buffer_size; // Some users might set very high write_buffer_size and rely on // max_total_wal_size or other parameters to control the WAL size. - if (db_options_.max_total_wal_size > 0) { - bsize = std::min(bsize, db_options_.max_total_wal_size); + if (immutable_db_options_.max_total_wal_size > 0) { + bsize = std::min(bsize, immutable_db_options_.max_total_wal_size); } - if (db_options_.db_write_buffer_size > 0) { - bsize = std::min(bsize, db_options_.db_write_buffer_size); + if (immutable_db_options_.db_write_buffer_size > 0) { + bsize = std::min(bsize, immutable_db_options_.db_write_buffer_size); } - if (db_options_.write_buffer_manager && - db_options_.write_buffer_manager->enabled()) { - bsize = std::min(bsize, - db_options_.write_buffer_manager->buffer_size()); + if (immutable_db_options_.write_buffer_manager && + immutable_db_options_.write_buffer_manager->enabled()) { + bsize = std::min( + bsize, immutable_db_options_.write_buffer_manager->buffer_size()); } return bsize; @@ -4026,7 +4046,7 @@ Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options, *handle = nullptr; s = CheckCompressionSupported(cf_options); - if (s.ok() && db_options_.allow_concurrent_memtable_write) { + if (s.ok() && immutable_db_options_.allow_concurrent_memtable_write) { s = CheckConcurrentWritesSupported(cf_options); } if (!s.ok()) { @@ -4049,14 +4069,14 @@ Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options, // LogAndApply will both write the creation in MANIFEST and create // ColumnFamilyData object - Options opt(db_options_, cf_options); { // write thread WriteThread::Writer w; write_thread_.EnterUnbatched(&w, &mutex_); // LogAndApply will both write the creation in MANIFEST and create // ColumnFamilyData object - s = versions_->LogAndApply(nullptr, MutableCFOptions(opt), &edit, &mutex_, - directories_.GetDbDir(), false, &cf_options); + s = versions_->LogAndApply(nullptr, MutableCFOptions(cf_options), &edit, + &mutex_, directories_.GetDbDir(), false, + &cf_options); if (s.ok()) { // If the column family was created successfully, we then persist @@ -4078,11 +4098,11 @@ Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options, } *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_); - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, - "Created column family [%s] (ID %u)", - column_family_name.c_str(), (unsigned)cfd->GetID()); + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, + "Created column family [%s] (ID %u)", column_family_name.c_str(), + (unsigned)cfd->GetID()); } else { - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Creating column family [%s] FAILED -- %s", column_family_name.c_str(), s.ToString().c_str()); } @@ -4093,13 +4113,13 @@ Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options, NewThreadStatusCfInfo( reinterpret_cast(*handle)->cfd()); if (!persist_options_status.ok()) { - if (db_options_.fail_if_options_file_error) { + if (immutable_db_options_.fail_if_options_file_error) { s = Status::IOError( "ColumnFamily has been created, but unable to persist" "options in CreateColumnFamily()", persist_options_status.ToString().c_str()); } - Warn(db_options_.info_log, + Warn(immutable_db_options_.info_log, "Unable to persist options in CreateColumnFamily() -- %s", persist_options_status.ToString().c_str()); } @@ -4164,24 +4184,24 @@ Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) { auto* mutable_cf_options = cfd->GetLatestMutableCFOptions(); max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size * mutable_cf_options->max_write_buffer_number; - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "Dropped column family with id %u\n", cfd->GetID()); if (!options_persist_status.ok()) { - if (db_options_.fail_if_options_file_error) { + if (immutable_db_options_.fail_if_options_file_error) { s = Status::IOError( "ColumnFamily has been dropped, but unable to persist " "options in DropColumnFamily()", options_persist_status.ToString().c_str()); } - Warn(db_options_.info_log, + Warn(immutable_db_options_.info_log, "Unable to persist options in DropColumnFamily() -- %s", options_persist_status.ToString().c_str()); } } else { - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, - "Dropping column family with id %u FAILED -- %s\n", - cfd->GetID(), s.ToString().c_str()); + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, + "Dropping column family with id %u FAILED -- %s\n", cfd->GetID(), + s.ToString().c_str()); } return s; @@ -4472,7 +4492,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, bool xfunc_attempted_write = false; XFUNC_TEST("transaction", "transaction_xftest_write_impl", xf_transaction_write1, xf_transaction_write, write_options, - db_options_, my_batch, callback, this, &status, + immutable_db_options_, my_batch, callback, this, &status, &xfunc_attempted_write); if (xfunc_attempted_write) { // Test already did the write @@ -4493,7 +4513,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, RecordTick(stats_, WRITE_WITH_WAL); } - StopWatch write_sw(env_, db_options_.statistics.get(), DB_WRITE); + StopWatch write_sw(env_, immutable_db_options_.statistics.get(), DB_WRITE); write_thread_.JoinBatchGroup(&w); if (w.state == WriteThread::STATE_PARALLEL_FOLLOWER) { @@ -4554,15 +4574,15 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, assert(!single_column_family_mode_ || versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1); - uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0) + uint64_t max_total_wal_size = (immutable_db_options_.max_total_wal_size == 0) ? 4 * max_total_in_memory_state_ - : db_options_.max_total_wal_size; + : immutable_db_options_.max_total_wal_size; if (UNLIKELY(!single_column_family_mode_ && alive_log_files_.begin()->getting_flushed == false && total_log_size_ > max_total_wal_size)) { uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number; alive_log_files_.begin()->getting_flushed = true; - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "Flushing all column families with data in WAL number %" PRIu64 ". Total log size is %" PRIu64 " while max_total_wal_size is %" PRIu64, flush_column_family_if_log_file, total_log_size_, max_total_wal_size); @@ -4588,7 +4608,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, // thread is writing to another DB with the same write buffer, they may also // be flushed. We may end up with flushing much more DBs than needed. It's // suboptimal but still correct. - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "Flushing column family with largest mem table size. Write buffer is " "using %" PRIu64 " bytes out of a total of %" PRIu64 ".", write_buffer_manager_->memory_usage(), @@ -4687,8 +4707,8 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, // assumed to be true. Rule 4 is checked for each batch. We could // relax rules 2 and 3 if we could prevent write batches from referring // more than once to a particular key. - bool parallel = - db_options_.allow_concurrent_memtable_write && write_group.size() > 1; + bool parallel = immutable_db_options_.allow_concurrent_memtable_write && + write_group.size() > 1; int total_count = 0; uint64_t total_byte_size = 0; for (auto writer : write_group) { @@ -4763,7 +4783,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, // - as long as other threads don't modify it, it's safe to read // from std::deque from multiple threads concurrently. for (auto& log : logs_) { - status = log.writer->file()->Sync(db_options_.use_fsync); + status = log.writer->file()->Sync(immutable_db_options_.use_fsync); if (!status.ok()) { break; } @@ -4871,8 +4891,8 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, } PERF_TIMER_START(write_pre_and_post_process_time); - if (db_options_.paranoid_checks && !status.ok() && !w.CallbackFailed() && - !status.IsBusy()) { + if (immutable_db_options_.paranoid_checks && !status.ok() && + !w.CallbackFailed() && !status.IsBusy()) { mutex_.Lock(); if (bg_error_.ok()) { bg_error_ = status; // stop compaction & fail any further writes @@ -4942,14 +4962,14 @@ Status DBImpl::ScheduleFlushes(WriteContext* context) { #ifndef ROCKSDB_LITE void DBImpl::NotifyOnMemTableSealed(ColumnFamilyData* cfd, const MemTableInfo& mem_table_info) { - if (db_options_.listeners.size() == 0U) { + if (immutable_db_options_.listeners.size() == 0U) { return; } if (shutting_down_.load(std::memory_order_acquire)) { return; } - for (auto listener : db_options_.listeners) { + for (auto listener : immutable_db_options_.listeners) { listener->OnMemTableSealed(mem_table_info); } } @@ -4968,7 +4988,7 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) { assert(versions_->prev_log_number() == 0); bool creating_new_log = !log_empty_; uint64_t recycle_log_number = 0; - if (creating_new_log && db_options_.recycle_log_file_num && + if (creating_new_log && immutable_db_options_.recycle_log_file_num && !log_recycle_files.empty()) { recycle_log_number = log_recycle_files.front(); log_recycle_files.pop_front(); @@ -4990,23 +5010,25 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) { // Log this later after lock release. It may be outdated, e.g., if background // flush happens before logging, but that should be ok. int num_imm_unflushed = cfd->imm()->NumNotFlushed(); + DBOptions db_options = + BuildDBOptions(immutable_db_options_, mutable_db_options_); mutex_.Unlock(); Status s; { if (creating_new_log) { EnvOptions opt_env_opt = - env_->OptimizeForLogWrite(env_options_, db_options_); + env_->OptimizeForLogWrite(env_options_, db_options); if (recycle_log_number) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "reusing log %" PRIu64 " from recycle list\n", recycle_log_number); s = env_->ReuseWritableFile( - LogFileName(db_options_.wal_dir, new_log_number), - LogFileName(db_options_.wal_dir, recycle_log_number), &lfile, - opt_env_opt); + LogFileName(immutable_db_options_.wal_dir, new_log_number), + LogFileName(immutable_db_options_.wal_dir, recycle_log_number), + &lfile, opt_env_opt); } else { - s = NewWritableFile(env_, - LogFileName(db_options_.wal_dir, new_log_number), - &lfile, opt_env_opt); + s = NewWritableFile( + env_, LogFileName(immutable_db_options_.wal_dir, new_log_number), + &lfile, opt_env_opt); } if (s.ok()) { // Our final size should be less than write_buffer_size @@ -5015,8 +5037,9 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) { GetWalPreallocateBlockSize(mutable_cf_options.write_buffer_size)); unique_ptr file_writer( new WritableFileWriter(std::move(lfile), opt_env_opt)); - new_log = new log::Writer(std::move(file_writer), new_log_number, - db_options_.recycle_log_file_num > 0); + new_log = + new log::Writer(std::move(file_writer), new_log_number, + immutable_db_options_.recycle_log_file_num > 0); } } @@ -5033,7 +5056,7 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) { NotifyOnMemTableSealed(cfd, memtable_info); #endif //ROCKSDB_LITE } - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "[%s] New memtable created with log file: #%" PRIu64 ". Immutable memtables: %d.\n", cfd->GetName().c_str(), new_log_number, num_imm_unflushed); @@ -5129,10 +5152,14 @@ Env* DBImpl::GetEnv() const { Options DBImpl::GetOptions(ColumnFamilyHandle* column_family) const { InstrumentedMutexLock l(&mutex_); auto cfh = reinterpret_cast(column_family); - return Options(db_options_, cfh->cfd()->GetLatestCFOptions()); + return Options(BuildDBOptions(immutable_db_options_, mutable_db_options_), + cfh->cfd()->GetLatestCFOptions()); } -const DBOptions& DBImpl::GetDBOptions() const { return db_options_; } +DBOptions DBImpl::GetDBOptions() const { + InstrumentedMutexLock l(&mutex_); + return BuildDBOptions(immutable_db_options_, mutable_db_options_); +} bool DBImpl::GetProperty(ColumnFamilyHandle* column_family, const Slice& property, std::string* value) { @@ -5387,7 +5414,7 @@ Status DBImpl::DeleteFile(std::string name) { WalFileType log_type; if (!ParseFileName(name, &number, &type, &log_type) || (type != kTableFile && type != kLogFile)) { - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "DeleteFile %s failed.\n", name.c_str()); return Status::InvalidArgument("Invalid file name"); } @@ -5396,16 +5423,16 @@ Status DBImpl::DeleteFile(std::string name) { if (type == kLogFile) { // Only allow deleting archived log files if (log_type != kArchivedLogFile) { - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, - "DeleteFile %s failed - not archived log.\n", - name.c_str()); + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, + "DeleteFile %s failed - not archived log.\n", name.c_str()); return Status::NotSupported("Delete only supported for archived logs"); } - status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str()); + status = + env_->DeleteFile(immutable_db_options_.wal_dir + "/" + name.c_str()); if (!status.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, - "DeleteFile %s failed -- %s.\n", - name.c_str(), status.ToString().c_str()); + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, + "DeleteFile %s failed -- %s.\n", name.c_str(), + status.ToString().c_str()); } return status; } @@ -5419,7 +5446,7 @@ Status DBImpl::DeleteFile(std::string name) { InstrumentedMutexLock l(&mutex_); status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd); if (!status.ok()) { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "DeleteFile %s failed. File not found\n", name.c_str()); job_context.Clean(); return Status::InvalidArgument("File not found"); @@ -5428,7 +5455,7 @@ Status DBImpl::DeleteFile(std::string name) { // If the file is being compacted no need to delete. if (metadata->being_compacted) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "DeleteFile %s Skipped. File about to be compacted\n", name.c_str()); job_context.Clean(); return Status::OK(); @@ -5440,7 +5467,7 @@ Status DBImpl::DeleteFile(std::string name) { auto* vstoreage = cfd->current()->storage_info(); for (int i = level + 1; i < cfd->NumberLevels(); i++) { if (vstoreage->NumLevelFiles(i) != 0) { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "DeleteFile %s FAILED. File not in last level\n", name.c_str()); job_context.Clean(); return Status::InvalidArgument("File not in last level"); @@ -5449,9 +5476,10 @@ Status DBImpl::DeleteFile(std::string name) { // if level == 0, it has to be the oldest file if (level == 0 && vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "DeleteFile %s failed ---" - " target file in level 0 must be the oldest.", name.c_str()); + " target file in level 0 must be the oldest.", + name.c_str()); job_context.Clean(); return Status::InvalidArgument("File in level 0, but not oldest"); } @@ -5466,7 +5494,7 @@ Status DBImpl::DeleteFile(std::string name) { FindObsoleteFiles(&job_context, false); } // lock released here - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); // remove files outside the db-lock if (job_context.HaveSomethingToDelete()) { // Call PurgeObsoleteFiles() without holding mutex. @@ -5548,7 +5576,7 @@ Status DBImpl::DeleteFilesInRange(ColumnFamilyHandle* column_family, FindObsoleteFiles(&job_context, false); } // lock released here - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); // remove files outside the db-lock if (job_context.HaveSomethingToDelete()) { // Call PurgeObsoleteFiles() without holding mutex. @@ -5730,9 +5758,9 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, } DBImpl* impl = new DBImpl(db_options, dbname); - s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir); + s = impl->env_->CreateDirIfMissing(impl->immutable_db_options_.wal_dir); if (s.ok()) { - for (auto db_path : impl->db_options_.db_paths) { + for (auto db_path : impl->immutable_db_options_.db_paths) { s = impl->env_->CreateDirIfMissing(db_path.path); if (!s.ok()) { break; @@ -5758,10 +5786,13 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, unique_ptr lfile; EnvOptions soptions(db_options); EnvOptions opt_env_options = - impl->db_options_.env->OptimizeForLogWrite(soptions, impl->db_options_); - s = NewWritableFile(impl->db_options_.env, - LogFileName(impl->db_options_.wal_dir, new_log_number), - &lfile, opt_env_options); + impl->immutable_db_options_.env->OptimizeForLogWrite( + soptions, BuildDBOptions(impl->immutable_db_options_, + impl->mutable_db_options_)); + s = NewWritableFile( + impl->immutable_db_options_.env, + LogFileName(impl->immutable_db_options_.wal_dir, new_log_number), + &lfile, opt_env_options); if (s.ok()) { lfile->SetPreallocationBlockSize( impl->GetWalPreallocateBlockSize(max_write_buffer_size)); @@ -5770,8 +5801,9 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, new WritableFileWriter(std::move(lfile), opt_env_options)); impl->logs_.emplace_back( new_log_number, - new log::Writer(std::move(file_writer), new_log_number, - impl->db_options_.recycle_log_file_num > 0)); + new log::Writer( + std::move(file_writer), new_log_number, + impl->immutable_db_options_.recycle_log_file_num > 0)); // set column family handles for (auto cf : column_families) { @@ -5854,13 +5886,13 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, impl->mutex_.Unlock(); auto sfm = static_cast( - impl->db_options_.sst_file_manager.get()); + impl->immutable_db_options_.sst_file_manager.get()); if (s.ok() && sfm) { // Notify SstFileManager about all sst files that already exist in // db_paths[0] when the DB is opened. - auto& db_path = impl->db_options_.db_paths[0]; + auto& db_path = impl->immutable_db_options_.db_paths[0]; std::vector existing_files; - impl->db_options_.env->GetChildren(db_path.path, &existing_files); + impl->immutable_db_options_.env->GetChildren(db_path.path, &existing_files); for (auto& file_name : existing_files) { uint64_t file_number; FileType file_type; @@ -5873,16 +5905,16 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, } if (s.ok()) { - Log(InfoLogLevel::INFO_LEVEL, impl->db_options_.info_log, "DB pointer %p", - impl); - LogFlush(impl->db_options_.info_log); + Log(InfoLogLevel::INFO_LEVEL, impl->immutable_db_options_.info_log, + "DB pointer %p", impl); + LogFlush(impl->immutable_db_options_.info_log); if (!persist_options_status.ok()) { if (db_options.fail_if_options_file_error) { s = Status::IOError( "DB::Open() failed --- Unable to persist Options file", persist_options_status.ToString()); } - Warn(impl->db_options_.info_log, + Warn(impl->immutable_db_options_.info_log, "Unable to persist options in DB::Open() -- %s", persist_options_status.ToString().c_str()); } @@ -5908,8 +5940,7 @@ Snapshot::~Snapshot() { } Status DestroyDB(const std::string& dbname, const Options& options) { - const InternalKeyComparator comparator(options.comparator); - const Options& soptions(SanitizeOptions(dbname, &comparator, options)); + const ImmutableDBOptions soptions(SanitizeOptions(dbname, options)); Env* env = soptions.env; std::vector filenames; @@ -5922,7 +5953,7 @@ Status DestroyDB(const std::string& dbname, const Options& options) { if (result.ok()) { uint64_t number; FileType type; - InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname); + InfoLogPrefix info_log_prefix(!soptions.db_log_dir.empty(), dbname); for (size_t i = 0; i < filenames.size(); i++) { if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) && type != kDBLockFile) { // Lock file will be deleted at end @@ -5931,7 +5962,7 @@ Status DestroyDB(const std::string& dbname, const Options& options) { if (type == kMetaDatabase) { del = DestroyDB(path_to_delete, options); } else if (type == kTableFile) { - del = DeleteSSTFile(&options, path_to_delete, 0); + del = DeleteSSTFile(&soptions, path_to_delete, 0); } else { del = env->DeleteFile(path_to_delete); } @@ -5948,7 +5979,7 @@ Status DestroyDB(const std::string& dbname, const Options& options) { if (ParseFileName(filenames[i], &number, &type) && type == kTableFile) { // Lock file will be deleted at end std::string table_path = db_path.path + "/" + filenames[i]; - Status del = DeleteSSTFile(&options, table_path, + Status del = DeleteSSTFile(&soptions, table_path, static_cast(path_id)); if (result.ok() && !del.ok()) { result = del; @@ -6016,12 +6047,14 @@ Status DBImpl::WriteOptionsFile() { // Unlock during expensive operations. New writes cannot get here // because the single write thread ensures all new writes get queued. + DBOptions db_options = + BuildDBOptions(immutable_db_options_, mutable_db_options_); mutex_.Unlock(); std::string file_name = TempOptionsFileName(GetName(), versions_->NewFileNumber()); - Status s = PersistRocksDBOptions(GetDBOptions(), cf_names, cf_opts, file_name, - GetEnv()); + Status s = + PersistRocksDBOptions(db_options, cf_names, cf_opts, file_name, GetEnv()); if (s.ok()) { s = RenameTempFileToOptionsFile(file_name); @@ -6076,7 +6109,7 @@ Status DBImpl::DeleteObsoleteOptionsFiles() { // Keeps the latest 2 Options file const size_t kNumOptionsFilesKept = 2; DeleteOptionsFilesHelper(options_filenames, kNumOptionsFilesKept, - db_options_.info_log, GetEnv()); + immutable_db_options_.info_log, GetEnv()); return Status::OK(); #else return Status::OK(); @@ -6104,7 +6137,7 @@ Status DBImpl::RenameTempFileToOptionsFile(const std::string& file_name) { void DBImpl::NewThreadStatusCfInfo( ColumnFamilyData* cfd) const { - if (db_options_.enable_thread_tracking) { + if (immutable_db_options_.enable_thread_tracking) { ThreadStatusUtil::NewColumnFamilyInfo(this, cfd, cfd->GetName(), cfd->ioptions()->env); } @@ -6112,13 +6145,13 @@ void DBImpl::NewThreadStatusCfInfo( void DBImpl::EraseThreadStatusCfInfo( ColumnFamilyData* cfd) const { - if (db_options_.enable_thread_tracking) { + if (immutable_db_options_.enable_thread_tracking) { ThreadStatusUtil::EraseColumnFamilyInfo(cfd); } } void DBImpl::EraseThreadStatusDbInfo() const { - if (db_options_.enable_thread_tracking) { + if (immutable_db_options_.enable_thread_tracking) { ThreadStatusUtil::EraseDatabaseInfo(this); } } @@ -6183,7 +6216,7 @@ Status DBImpl::GetLatestSequenceForKey(SuperVersion* sv, const Slice& key, if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { // unexpected error reading memtable. - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Unexpected status returned from MemTable::Get: %s\n", s.ToString().c_str()); @@ -6201,7 +6234,7 @@ Status DBImpl::GetLatestSequenceForKey(SuperVersion* sv, const Slice& key, if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { // unexpected error reading memtable. - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Unexpected status returned from MemTableList::Get: %s\n", s.ToString().c_str()); @@ -6219,7 +6252,7 @@ Status DBImpl::GetLatestSequenceForKey(SuperVersion* sv, const Slice& key, if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { // unexpected error reading memtable. - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Unexpected status returned from MemTableList::GetFromHistory: %s\n", s.ToString().c_str()); @@ -6243,7 +6276,7 @@ Status DBImpl::GetLatestSequenceForKey(SuperVersion* sv, const Slice& key, if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { // unexpected error reading SST files - Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log, + Log(InfoLogLevel::ERROR_LEVEL, immutable_db_options_.info_log, "Unexpected status returned from Version::Get: %s\n", s.ToString().c_str()); diff --git a/db/db_impl.h b/db/db_impl.h index b7dd36b95..b368543c7 100644 --- a/db/db_impl.h +++ b/db/db_impl.h @@ -40,6 +40,7 @@ #include "rocksdb/write_buffer_manager.h" #include "table/scoped_arena_iterator.h" #include "util/autovector.h" +#include "util/db_options.h" #include "util/event_logger.h" #include "util/hash.h" #include "util/instrumented_mutex.h" @@ -167,7 +168,7 @@ class DBImpl : public DB { using DB::GetOptions; virtual Options GetOptions(ColumnFamilyHandle* column_family) const override; using DB::GetDBOptions; - virtual const DBOptions& GetDBOptions() const override; + virtual DBOptions GetDBOptions() const override; using DB::Flush; virtual Status Flush(const FlushOptions& options, ColumnFamilyHandle* column_family) override; @@ -456,7 +457,7 @@ class DBImpl : public DB { ~RecoveredTransaction() { delete batch_; } }; - bool allow_2pc() const { return db_options_.allow_2pc; } + bool allow_2pc() const { return immutable_db_options_.allow_2pc; } std::unordered_map recovered_transactions() { @@ -507,7 +508,8 @@ class DBImpl : public DB { Env* const env_; const std::string dbname_; unique_ptr versions_; - const DBOptions db_options_; + const ImmutableDBOptions immutable_db_options_; + MutableDBOptions mutable_db_options_; Statistics* stats_; std::unordered_map recovered_transactions_; @@ -1071,11 +1073,10 @@ class DBImpl : public DB { size_t GetWalPreallocateBlockSize(uint64_t write_buffer_size) const; }; -// Sanitize db options. The caller should delete result.info_log if -// it is not equal to src.info_log. extern Options SanitizeOptions(const std::string& db, const InternalKeyComparator* icmp, const Options& src); + extern DBOptions SanitizeOptions(const std::string& db, const DBOptions& src); // Fix user-supplied options to be reasonable diff --git a/db/db_impl_add_file.cc b/db/db_impl_add_file.cc index acee4e09d..b31c34aac 100644 --- a/db/db_impl_add_file.cc +++ b/db/db_impl_add_file.cc @@ -209,8 +209,8 @@ Status DBImpl::AddFile(ColumnFamilyHandle* column_family, for (; j < num_files; j++) { StopWatch sw(env_, nullptr, 0, µ_list[j], false); db_fname_list[j] = - TableFileName(db_options_.db_paths, meta_list[j].fd.GetNumber(), - meta_list[j].fd.GetPathId()); + TableFileName(immutable_db_options_.db_paths, + meta_list[j].fd.GetNumber(), meta_list[j].fd.GetPathId()); if (move_file) { status = env_->LinkFile(file_info_list[j].file_path, db_fname_list[j]); if (status.IsNotSupported()) { @@ -226,7 +226,7 @@ Status DBImpl::AddFile(ColumnFamilyHandle* column_family, for (size_t i = 0; i < j; i++) { Status s = env_->DeleteFile(db_fname_list[i]); if (!s.ok()) { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "AddFile() clean up for file %s failed : %s", db_fname_list[i].c_str(), s.ToString().c_str()); } @@ -340,7 +340,7 @@ Status DBImpl::AddFile(ColumnFamilyHandle* column_family, for (size_t i = 0; i < num_files; i++) { Status s = env_->DeleteFile(db_fname_list[i]); if (!s.ok()) { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "AddFile() clean up for file %s failed : %s", db_fname_list[i].c_str(), s.ToString().c_str()); } @@ -350,7 +350,7 @@ Status DBImpl::AddFile(ColumnFamilyHandle* column_family, for (size_t i = 0; i < num_files; i++) { Status s = env_->DeleteFile(file_info_list[i].file_path); if (!s.ok()) { - Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, + Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, "%s was added to DB successfully but failed to remove original " "file " "link : %s", diff --git a/db/db_impl_experimental.cc b/db/db_impl_experimental.cc index 139b0d01d..a708a8b21 100644 --- a/db/db_impl_experimental.cc +++ b/db/db_impl_experimental.cc @@ -61,7 +61,7 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { assert(column_family); if (target_level < 1) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "PromoteL0 FAILED. Invalid target level %d\n", target_level); return Status::InvalidArgument("Invalid target level"); } @@ -75,7 +75,7 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { const auto* vstorage = cfd->current()->storage_info(); if (target_level >= vstorage->num_levels()) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "PromoteL0 FAILED. Target level %d does not exist\n", target_level); job_context.Clean(); return Status::InvalidArgument("Target level does not exist"); @@ -94,7 +94,7 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { for (size_t i = 0; i < l0_files.size(); ++i) { auto f = l0_files[i]; if (f->being_compacted) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "PromoteL0 FAILED. File %" PRIu64 " being compacted\n", f->fd.GetNumber()); job_context.Clean(); @@ -104,7 +104,7 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { if (i == 0) continue; auto prev_f = l0_files[i - 1]; if (icmp->Compare(prev_f->largest, f->smallest) >= 0) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "PromoteL0 FAILED. Files %" PRIu64 " and %" PRIu64 " have overlapping ranges\n", prev_f->fd.GetNumber(), f->fd.GetNumber()); @@ -116,7 +116,7 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { // Check that all levels up to target_level are empty. for (int level = 1; level <= target_level; ++level) { if (vstorage->NumLevelFiles(level) > 0) { - Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log, + Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, "PromoteL0 FAILED. Level %d not empty\n", level); job_context.Clean(); return Status::InvalidArgument( @@ -141,7 +141,7 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { cfd, &job_context, *cfd->GetLatestMutableCFOptions()); } } // lock released here - LogFlush(db_options_.info_log); + LogFlush(immutable_db_options_.info_log); job_context.Clean(); return status; diff --git a/db/db_impl_readonly.cc b/db/db_impl_readonly.cc index 57c14df14..6d2437c17 100644 --- a/db/db_impl_readonly.cc +++ b/db/db_impl_readonly.cc @@ -19,8 +19,9 @@ namespace rocksdb { DBImplReadOnly::DBImplReadOnly(const DBOptions& db_options, const std::string& dbname) : DBImpl(db_options, dbname) { - Log(INFO_LEVEL, db_options_.info_log, "Opening the db in read only mode"); - LogFlush(db_options_.info_log); + Log(INFO_LEVEL, immutable_db_options_.info_log, + "Opening the db in read only mode"); + LogFlush(immutable_db_options_.info_log); } DBImplReadOnly::~DBImplReadOnly() { diff --git a/db/db_info_dumper.cc b/db/db_info_dumper.cc index 56cf3e288..3b1a9b9a0 100644 --- a/db/db_info_dumper.cc +++ b/db/db_info_dumper.cc @@ -16,12 +16,12 @@ #include #include "db/filename.h" -#include "rocksdb/options.h" #include "rocksdb/env.h" namespace rocksdb { -void DumpDBFileSummary(const DBOptions& options, const std::string& dbname) { +void DumpDBFileSummary(const ImmutableDBOptions& options, + const std::string& dbname) { if (options.info_log == nullptr) { return; } diff --git a/db/db_info_dumper.h b/db/db_info_dumper.h index 470b6224f..c142ab233 100644 --- a/db/db_info_dumper.h +++ b/db/db_info_dumper.h @@ -6,8 +6,9 @@ #include -#include "rocksdb/options.h" +#include "util/db_options.h" namespace rocksdb { -void DumpDBFileSummary(const DBOptions& options, const std::string& dbname); +void DumpDBFileSummary(const ImmutableDBOptions& options, + const std::string& dbname); } // namespace rocksdb diff --git a/db/db_options_test.cc b/db/db_options_test.cc index ccc65363e..a164b29b9 100644 --- a/db/db_options_test.cc +++ b/db/db_options_test.cc @@ -45,8 +45,9 @@ class DBOptionsTest : public DBTestBase { std::unordered_map GetRandomizedMutableCFOptionsMap( Random* rnd) { Options options; + ImmutableDBOptions db_options(options); test::RandomInitCFOptions(&options, rnd); - auto sanitized_options = SanitizeOptions(options, nullptr, options); + auto sanitized_options = SanitizeOptions(db_options, nullptr, options); auto opt_map = GetMutableCFOptionsMap(sanitized_options); delete options.compaction_filter; return opt_map; diff --git a/db/db_test.cc b/db/db_test.cc index 32e12bad9..0f7e0adcf 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -2811,7 +2811,7 @@ class ModelDB : public DB { } using DB::GetDBOptions; - virtual const DBOptions& GetDBOptions() const override { return options_; } + virtual DBOptions GetDBOptions() const override { return options_; } using DB::Flush; virtual Status Flush(const rocksdb::FlushOptions& options, diff --git a/db/db_wal_test.cc b/db/db_wal_test.cc index 1d5c046b8..b0ca938a9 100644 --- a/db/db_wal_test.cc +++ b/db/db_wal_test.cc @@ -651,7 +651,7 @@ class RecoveryTestHelper { // Create WAL files with values filled in static void FillData(DBWALTest* test, const Options& options, const size_t wal_count, size_t* count) { - const DBOptions& db_options = options; + const ImmutableDBOptions db_options(options); *count = 0; diff --git a/db/filename.cc b/db/filename.cc index d1f0958bb..04a692cef 100644 --- a/db/filename.cc +++ b/db/filename.cc @@ -386,10 +386,10 @@ Status SetIdentityFile(Env* env, const std::string& dbname) { return s; } -Status SyncManifest(Env* env, const DBOptions* db_options, +Status SyncManifest(Env* env, const ImmutableDBOptions* db_options, WritableFileWriter* file) { TEST_KILL_RANDOM("SyncManifest:0", rocksdb_kill_odds * REDUCE_ODDS2); - if (db_options->disableDataSync) { + if (db_options->disable_data_sync) { return Status::OK(); } else { StopWatch sw(env, db_options->statistics.get(), MANIFEST_FILE_SYNC_MICROS); diff --git a/db/filename.h b/db/filename.h index 9a0a1eee3..5b6f727ad 100644 --- a/db/filename.h +++ b/db/filename.h @@ -20,6 +20,7 @@ #include "rocksdb/slice.h" #include "rocksdb/status.h" #include "rocksdb/transaction_log.h" +#include "util/db_options.h" namespace rocksdb { @@ -159,7 +160,7 @@ extern Status SetCurrentFile(Env* env, const std::string& dbname, extern Status SetIdentityFile(Env* env, const std::string& dbname); // Sync manifest file `file`. -extern Status SyncManifest(Env* env, const DBOptions* db_options, +extern Status SyncManifest(Env* env, const ImmutableDBOptions* db_options, WritableFileWriter* file); } // namespace rocksdb diff --git a/db/flush_job.cc b/db/flush_job.cc index 521e013b9..0014da399 100644 --- a/db/flush_job.cc +++ b/db/flush_job.cc @@ -56,7 +56,7 @@ namespace rocksdb { FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd, - const DBOptions& db_options, + const ImmutableDBOptions& db_options, const MutableCFOptions& mutable_cf_options, const EnvOptions& env_options, VersionSet* versions, InstrumentedMutex* db_mutex, @@ -294,7 +294,7 @@ Status FlushJob::WriteLevel0Table() { meta_.fd.GetFileSize(), s.ToString().c_str(), meta_.marked_for_compaction ? " (needs compaction)" : ""); - if (!db_options_.disableDataSync && output_file_directory_ != nullptr) { + if (!db_options_.disable_data_sync && output_file_directory_ != nullptr) { output_file_directory_->Fsync(); } TEST_SYNC_POINT("FlushJob::WriteLevel0Table"); diff --git a/db/flush_job.h b/db/flush_job.h index 5dc6a9871..5a3229cb6 100644 --- a/db/flush_job.h +++ b/db/flush_job.h @@ -34,6 +34,7 @@ #include "rocksdb/transaction_log.h" #include "table/scoped_arena_iterator.h" #include "util/autovector.h" +#include "util/db_options.h" #include "util/event_logger.h" #include "util/instrumented_mutex.h" #include "util/stop_watch.h" @@ -53,7 +54,7 @@ class FlushJob { // TODO(icanadi) make effort to reduce number of parameters here // IMPORTANT: mutable_cf_options needs to be alive while FlushJob is alive FlushJob(const std::string& dbname, ColumnFamilyData* cfd, - const DBOptions& db_options, + const ImmutableDBOptions& db_options, const MutableCFOptions& mutable_cf_options, const EnvOptions& env_options, VersionSet* versions, InstrumentedMutex* db_mutex, std::atomic* shutting_down, @@ -78,7 +79,7 @@ class FlushJob { Status WriteLevel0Table(); const std::string& dbname_; ColumnFamilyData* cfd_; - const DBOptions& db_options_; + const ImmutableDBOptions& db_options_; const MutableCFOptions& mutable_cf_options_; const EnvOptions& env_options_; VersionSet* versions_; diff --git a/db/flush_job_test.cc b/db/flush_job_test.cc index 9648c01e4..b58f7d51e 100644 --- a/db/flush_job_test.cc +++ b/db/flush_job_test.cc @@ -28,6 +28,8 @@ class FlushJobTest : public testing::Test { FlushJobTest() : env_(Env::Default()), dbname_(test::TmpDir() + "/flush_job_test"), + options_(), + db_options_(options_), table_cache_(NewLRUCache(50000, 16)), write_buffer_manager_(db_options_.db_write_buffer_size), versions_(new VersionSet(dbname_, &db_options_, env_options_, @@ -74,9 +76,10 @@ class FlushJobTest : public testing::Test { Env* env_; std::string dbname_; EnvOptions env_options_; + Options options_; + ImmutableDBOptions db_options_; std::shared_ptr table_cache_; WriteController write_controller_; - DBOptions db_options_; WriteBufferManager write_buffer_manager_; ColumnFamilyOptions cf_options_; std::unique_ptr versions_; diff --git a/db/memtable_list_test.cc b/db/memtable_list_test.cc index f9f73f976..5309ffff9 100644 --- a/db/memtable_list_test.cc +++ b/db/memtable_list_test.cc @@ -57,14 +57,16 @@ class MemTableListTest : public testing::Test { // Create a mock VersionSet DBOptions db_options; + ImmutableDBOptions immutable_db_options(db_options); EnvOptions env_options; shared_ptr table_cache(NewLRUCache(50000, 16)); WriteBufferManager write_buffer_manager(db_options.db_write_buffer_size); WriteController write_controller(10000000u); CreateDB(); - VersionSet versions(dbname, &db_options, env_options, table_cache.get(), - &write_buffer_manager, &write_controller); + VersionSet versions(dbname, &immutable_db_options, env_options, + table_cache.get(), &write_buffer_manager, + &write_controller); // Create mock default ColumnFamilyData ColumnFamilyOptions cf_options; diff --git a/db/repair.cc b/db/repair.cc index ad5839e85..b7bd38465 100644 --- a/db/repair.cc +++ b/db/repair.cc @@ -99,10 +99,11 @@ class Repairer { env_(db_options.env), env_options_(), db_options_(SanitizeOptions(dbname_, db_options)), + immutable_db_options_(db_options_), icmp_(default_cf_opts.comparator), default_cf_opts_(default_cf_opts), default_cf_iopts_( - ImmutableCFOptions(Options(db_options_, default_cf_opts))), + ImmutableCFOptions(immutable_db_options_, default_cf_opts)), unknown_cf_opts_(unknown_cf_opts), create_unknown_cfs_(create_unknown_cfs), raw_table_cache_( @@ -113,8 +114,8 @@ class Repairer { raw_table_cache_.get())), wb_(db_options_.db_write_buffer_size), wc_(db_options_.delayed_write_rate), - vset_(dbname_, &db_options_, env_options_, raw_table_cache_.get(), &wb_, - &wc_), + vset_(dbname_, &immutable_db_options_, env_options_, + raw_table_cache_.get(), &wb_, &wc_), next_file_number_(1) { for (const auto& cfd : column_families) { cf_name_to_opts_[cfd.name] = cfd.options; @@ -222,6 +223,7 @@ class Repairer { Env* const env_; const EnvOptions env_options_; const DBOptions db_options_; + const ImmutableDBOptions immutable_db_options_; const InternalKeyComparator icmp_; const ColumnFamilyOptions default_cf_opts_; const ImmutableCFOptions default_cf_iopts_; // table_cache_ holds reference diff --git a/db/table_properties_collector_test.cc b/db/table_properties_collector_test.cc index c5c43760c..7d96d6d80 100644 --- a/db/table_properties_collector_test.cc +++ b/db/table_properties_collector_test.cc @@ -256,7 +256,7 @@ void TestCustomizedTablePropertiesCollector( int_tbl_prop_collector_factories.emplace_back( new RegularKeysStartWithAFactory(backward_mode)); } else { - GetIntTblPropCollectorFactory(options, &int_tbl_prop_collector_factories); + GetIntTblPropCollectorFactory(ioptions, &int_tbl_prop_collector_factories); } MakeBuilder(options, ioptions, internal_comparator, &int_tbl_prop_collector_factories, &writer, &builder); @@ -393,7 +393,8 @@ void TestInternalKeyPropertiesCollector( options = SanitizeOptions("db", // just a place holder &pikc, options); - GetIntTblPropCollectorFactory(options, &int_tbl_prop_collector_factories); + ImmutableCFOptions ioptions(options); + GetIntTblPropCollectorFactory(ioptions, &int_tbl_prop_collector_factories); options.comparator = comparator; } else { int_tbl_prop_collector_factories.emplace_back( diff --git a/db/transaction_log_impl.cc b/db/transaction_log_impl.cc index 91ed05412..75582c163 100644 --- a/db/transaction_log_impl.cc +++ b/db/transaction_log_impl.cc @@ -16,7 +16,7 @@ namespace rocksdb { TransactionLogIteratorImpl::TransactionLogIteratorImpl( - const std::string& dir, const DBOptions* options, + const std::string& dir, const ImmutableDBOptions* options, const TransactionLogIterator::ReadOptions& read_options, const EnvOptions& soptions, const SequenceNumber seq, std::unique_ptr files, VersionSet const* const versions) diff --git a/db/transaction_log_impl.h b/db/transaction_log_impl.h index d4a2468e7..3114b2120 100644 --- a/db/transaction_log_impl.h +++ b/db/transaction_log_impl.h @@ -2,19 +2,20 @@ // This source code is licensed under the BSD-style license found in the // LICENSE file in the root directory of this source tree. An additional grant // of patent rights can be found in the PATENTS file in the same directory. +#pragma once #ifndef ROCKSDB_LITE -#pragma once #include +#include "db/filename.h" +#include "db/log_reader.h" +#include "db/version_set.h" +#include "port/port.h" #include "rocksdb/env.h" #include "rocksdb/options.h" -#include "rocksdb/types.h" #include "rocksdb/transaction_log.h" -#include "db/version_set.h" -#include "db/log_reader.h" -#include "db/filename.h" -#include "port/port.h" +#include "rocksdb/types.h" +#include "util/db_options.h" namespace rocksdb { @@ -58,7 +59,7 @@ class LogFileImpl : public LogFile { class TransactionLogIteratorImpl : public TransactionLogIterator { public: TransactionLogIteratorImpl( - const std::string& dir, const DBOptions* options, + const std::string& dir, const ImmutableDBOptions* options, const TransactionLogIterator::ReadOptions& read_options, const EnvOptions& soptions, const SequenceNumber seqNum, std::unique_ptr files, VersionSet const* const versions); @@ -73,7 +74,7 @@ class TransactionLogIteratorImpl : public TransactionLogIterator { private: const std::string& dir_; - const DBOptions* options_; + const ImmutableDBOptions* options_; const TransactionLogIterator::ReadOptions read_options_; const EnvOptions& soptions_; SequenceNumber startingSequenceNumber_; diff --git a/db/version_set.cc b/db/version_set.cc index 6d5d4d5df..0f668ca82 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -2093,7 +2093,8 @@ struct VersionSet::ManifestWriter { : done(false), cv(mu), cfd(_cfd), edit_list(e) {} }; -VersionSet::VersionSet(const std::string& dbname, const DBOptions* db_options, +VersionSet::VersionSet(const std::string& dbname, + const ImmutableDBOptions* db_options, const EnvOptions& storage_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller) @@ -2335,8 +2336,9 @@ Status VersionSet::LogAndApply(ColumnFamilyData* column_family_data, // If we just created a new descriptor file, install it by writing a // new CURRENT file that points to it. if (s.ok() && new_descriptor_log) { - s = SetCurrentFile(env_, dbname_, pending_manifest_file_number_, - db_options_->disableDataSync ? nullptr : db_directory); + s = SetCurrentFile( + env_, dbname_, pending_manifest_file_number_, + db_options_->disable_data_sync ? nullptr : db_directory); } if (s.ok()) { @@ -2847,12 +2849,13 @@ Status VersionSet::ReduceNumberOfLevels(const std::string& dbname, "Number of levels needs to be bigger than 1"); } + ImmutableDBOptions db_options(*options); ColumnFamilyOptions cf_options(*options); std::shared_ptr tc(NewLRUCache(options->max_open_files - 10, options->table_cache_numshardbits)); WriteController wc(options->delayed_write_rate); WriteBufferManager wb(options->db_write_buffer_size); - VersionSet versions(dbname, options, env_options, tc.get(), &wb, &wc); + VersionSet versions(dbname, &db_options, env_options, tc.get(), &wb, &wc); Status status; std::vector dummy; diff --git a/db/version_set.h b/db/version_set.h index 41f0c4e42..a2dd3afc3 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -28,18 +28,19 @@ #include #include -#include "db/dbformat.h" -#include "db/version_builder.h" -#include "db/version_edit.h" -#include "port/port.h" -#include "db/table_cache.h" +#include "db/column_family.h" #include "db/compaction.h" #include "db/compaction_picker.h" -#include "db/column_family.h" -#include "db/log_reader.h" +#include "db/dbformat.h" #include "db/file_indexer.h" +#include "db/log_reader.h" +#include "db/table_cache.h" +#include "db/version_builder.h" +#include "db/version_edit.h" #include "db/write_controller.h" +#include "port/port.h" #include "rocksdb/env.h" +#include "util/db_options.h" #include "util/instrumented_mutex.h" namespace rocksdb { @@ -574,7 +575,7 @@ class Version { class VersionSet { public: - VersionSet(const std::string& dbname, const DBOptions* db_options, + VersionSet(const std::string& dbname, const ImmutableDBOptions* db_options, const EnvOptions& env_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller); @@ -756,7 +757,7 @@ class VersionSet { Env* const env_; const std::string dbname_; - const DBOptions* const db_options_; + const ImmutableDBOptions* const db_options_; std::atomic next_file_number_; uint64_t manifest_file_number_; uint64_t options_file_number_; diff --git a/db/wal_manager.cc b/db/wal_manager.cc index e57c12040..2f19c0196 100644 --- a/db/wal_manager.cc +++ b/db/wal_manager.cc @@ -128,8 +128,8 @@ Status WalManager::GetUpdatesSince( // b. get sorted non-empty archived logs // c. delete what should be deleted void WalManager::PurgeObsoleteWALFiles() { - bool const ttl_enabled = db_options_.WAL_ttl_seconds > 0; - bool const size_limit_enabled = db_options_.WAL_size_limit_MB > 0; + bool const ttl_enabled = db_options_.wal_ttl_seconds > 0; + bool const size_limit_enabled = db_options_.wal_size_limit_mb > 0; if (!ttl_enabled && !size_limit_enabled) { return; } @@ -144,7 +144,7 @@ void WalManager::PurgeObsoleteWALFiles() { } uint64_t const now_seconds = static_cast(current_time); uint64_t const time_to_check = (ttl_enabled && !size_limit_enabled) - ? db_options_.WAL_ttl_seconds / 2 + ? db_options_.wal_ttl_seconds / 2 : kDefaultIntervalToDeleteObsoleteWAL; if (purge_wal_files_last_run_ + time_to_check > now_seconds) { @@ -180,7 +180,7 @@ void WalManager::PurgeObsoleteWALFiles() { file_path.c_str(), s.ToString().c_str()); continue; } - if (now_seconds - file_m_time > db_options_.WAL_ttl_seconds) { + if (now_seconds - file_m_time > db_options_.wal_ttl_seconds) { s = env_->DeleteFile(file_path); if (!s.ok()) { Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, @@ -229,7 +229,7 @@ void WalManager::PurgeObsoleteWALFiles() { } size_t const files_keep_num = - db_options_.WAL_size_limit_MB * 1024 * 1024 / log_file_size; + db_options_.wal_size_limit_mb * 1024 * 1024 / log_file_size; if (log_files_num <= files_keep_num) { return; } diff --git a/db/wal_manager.h b/db/wal_manager.h index d27985b9d..532772aec 100644 --- a/db/wal_manager.h +++ b/db/wal_manager.h @@ -17,22 +17,21 @@ #include #include +#include "db/version_set.h" #include "port/port.h" - #include "rocksdb/env.h" -#include "rocksdb/options.h" -#include "rocksdb/types.h" -#include "rocksdb/transaction_log.h" #include "rocksdb/status.h" - -#include "db/version_set.h" +#include "rocksdb/transaction_log.h" +#include "rocksdb/types.h" +#include "util/db_options.h" namespace rocksdb { #ifndef ROCKSDB_LITE class WalManager { public: - WalManager(const DBOptions& db_options, const EnvOptions& env_options) + WalManager(const ImmutableDBOptions& db_options, + const EnvOptions& env_options) : db_options_(db_options), env_options_(env_options), env_(db_options.env), @@ -75,7 +74,7 @@ class WalManager { SequenceNumber* sequence); // ------- state from DBImpl ------ - const DBOptions& db_options_; + const ImmutableDBOptions& db_options_; const EnvOptions& env_options_; Env* env_; diff --git a/db/wal_manager_test.cc b/db/wal_manager_test.cc index 9c7ac50ba..33435c10b 100644 --- a/db/wal_manager_test.cc +++ b/db/wal_manager_test.cc @@ -33,6 +33,7 @@ class WalManagerTest : public testing::Test { WalManagerTest() : env_(new MockEnv(Env::Default())), dbname_(test::TmpDir() + "/wal_manager_test"), + db_options_(), table_cache_(NewLRUCache(50000, 16)), write_buffer_manager_(db_options_.db_write_buffer_size), current_log_number_(0) { @@ -100,10 +101,10 @@ class WalManagerTest : public testing::Test { std::unique_ptr env_; std::string dbname_; + ImmutableDBOptions db_options_; WriteController write_controller_; EnvOptions env_options_; std::shared_ptr table_cache_; - DBOptions db_options_; WriteBufferManager write_buffer_manager_; std::unique_ptr versions_; std::unique_ptr wal_manager_; @@ -205,8 +206,8 @@ int CountRecords(TransactionLogIterator* iter) { } // namespace TEST_F(WalManagerTest, WALArchivalSizeLimit) { - db_options_.WAL_ttl_seconds = 0; - db_options_.WAL_size_limit_MB = 1000; + db_options_.wal_ttl_seconds = 0; + db_options_.wal_size_limit_mb = 1000; Init(); // TEST : Create WalManager with huge size limit and no ttl. @@ -214,7 +215,7 @@ TEST_F(WalManagerTest, WALArchivalSizeLimit) { // Count the archived log files that survived. // Assert that all of them did. // Change size limit. Re-open WalManager. - // Assert that archive is not greater than WAL_size_limit_MB after + // Assert that archive is not greater than wal_size_limit_mb after // PurgeObsoleteWALFiles() // Set ttl and time_to_check_ to small values. Re-open db. // Assert that there are no archived logs left. @@ -226,14 +227,14 @@ TEST_F(WalManagerTest, WALArchivalSizeLimit) { ListSpecificFiles(env_.get(), archive_dir, kLogFile); ASSERT_EQ(log_files.size(), 20U); - db_options_.WAL_size_limit_MB = 8; + db_options_.wal_size_limit_mb = 8; Reopen(); wal_manager_->PurgeObsoleteWALFiles(); uint64_t archive_size = GetLogDirSize(archive_dir, env_.get()); - ASSERT_TRUE(archive_size <= db_options_.WAL_size_limit_MB * 1024 * 1024); + ASSERT_TRUE(archive_size <= db_options_.wal_size_limit_mb * 1024 * 1024); - db_options_.WAL_ttl_seconds = 1; + db_options_.wal_ttl_seconds = 1; env_->FakeSleepForMicroseconds(2 * 1000 * 1000); Reopen(); wal_manager_->PurgeObsoleteWALFiles(); @@ -243,7 +244,7 @@ TEST_F(WalManagerTest, WALArchivalSizeLimit) { } TEST_F(WalManagerTest, WALArchivalTtl) { - db_options_.WAL_ttl_seconds = 1000; + db_options_.wal_ttl_seconds = 1000; Init(); // TEST : Create WalManager with a ttl and no size limit. @@ -259,7 +260,7 @@ TEST_F(WalManagerTest, WALArchivalTtl) { ListSpecificFiles(env_.get(), archive_dir, kLogFile); ASSERT_GT(log_files.size(), 0U); - db_options_.WAL_ttl_seconds = 1; + db_options_.wal_ttl_seconds = 1; env_->FakeSleepForMicroseconds(3 * 1000 * 1000); Reopen(); wal_manager_->PurgeObsoleteWALFiles(); diff --git a/include/rocksdb/db.h b/include/rocksdb/db.h index 534c120cd..7b6ce0aad 100644 --- a/include/rocksdb/db.h +++ b/include/rocksdb/db.h @@ -701,7 +701,7 @@ class DB { return GetOptions(DefaultColumnFamily()); } - virtual const DBOptions& GetDBOptions() const = 0; + virtual DBOptions GetDBOptions() const = 0; // Flush all mem-table data. virtual Status Flush(const FlushOptions& options, diff --git a/include/rocksdb/utilities/stackable_db.h b/include/rocksdb/utilities/stackable_db.h index 68a729d23..c91ac8ff6 100644 --- a/include/rocksdb/utilities/stackable_db.h +++ b/include/rocksdb/utilities/stackable_db.h @@ -224,7 +224,7 @@ class StackableDB : public DB { } using DB::GetDBOptions; - virtual const DBOptions& GetDBOptions() const override { + virtual DBOptions GetDBOptions() const override { return db_->GetDBOptions(); } diff --git a/src.mk b/src.mk index 1b2be29b9..9cbc8b438 100644 --- a/src.mk +++ b/src.mk @@ -88,18 +88,21 @@ LIB_SOURCES = \ util/arena.cc \ util/bloom.cc \ util/build_version.cc \ + util/cf_options.cc \ util/clock_cache.cc \ util/coding.cc \ util/comparator.cc \ util/compaction_job_stats_impl.cc \ util/concurrent_arena.cc \ util/crc32c.cc \ + util/db_options.cc \ util/delete_scheduler.cc \ util/dynamic_bloom.cc \ util/env.cc \ util/env_chroot.cc \ util/env_hdfs.cc \ util/env_posix.cc \ + util/event_logger.cc \ util/file_util.cc \ util/file_reader_writer.cc \ util/filter_policy.cc \ @@ -109,13 +112,39 @@ LIB_SOURCES = \ util/instrumented_mutex.cc \ util/iostats_context.cc \ util/io_posix.cc \ + util/log_buffer.cc \ + util/logging.cc \ util/lru_cache.cc \ - util/threadpool_imp.cc \ - util/transaction_test_util.cc \ + util/memenv.cc \ + util/murmurhash.cc \ + util/options.cc \ + util/options_helper.cc \ + util/options_parser.cc \ + util/options_sanity_check.cc \ + util/perf_context.cc \ + util/perf_level.cc \ + util/random.cc \ + util/rate_limiter.cc \ util/sharded_cache.cc \ + util/slice.cc \ util/sst_file_manager_impl.cc \ + util/statistics.cc \ + util/status.cc \ + util/status_message.cc \ + util/string_util.cc \ + util/sync_point.cc \ + util/thread_local.cc \ + util/thread_status_impl.cc \ + util/thread_status_updater.cc \ + util/thread_status_updater_debug.cc \ + util/thread_status_util.cc \ + util/thread_status_util_debug.cc \ + util/threadpool_imp.cc \ + util/transaction_test_util.cc \ + util/xfunc.cc \ + util/xxhash.cc \ utilities/backupable/backupable_db.cc \ - utilities/blob_db/blob_db.cc \ + utilities/blob_db/blob_db.cc \ utilities/convenience/info_log_finder.cc \ utilities/checkpoint/checkpoint.cc \ utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc \ @@ -156,34 +185,6 @@ LIB_SOURCES = \ utilities/date_tiered/date_tiered_db_impl.cc \ utilities/write_batch_with_index/write_batch_with_index.cc \ utilities/write_batch_with_index/write_batch_with_index_internal.cc \ - util/cf_options.cc \ - util/event_logger.cc \ - util/log_buffer.cc \ - util/logging.cc \ - util/memenv.cc \ - util/murmurhash.cc \ - util/options.cc \ - util/options_helper.cc \ - util/options_parser.cc \ - util/options_sanity_check.cc \ - util/perf_context.cc \ - util/perf_level.cc \ - util/random.cc \ - util/rate_limiter.cc \ - util/slice.cc \ - util/statistics.cc \ - util/status.cc \ - util/status_message.cc \ - util/string_util.cc \ - util/sync_point.cc \ - util/thread_local.cc \ - util/thread_status_impl.cc \ - util/thread_status_updater.cc \ - util/thread_status_updater_debug.cc \ - util/thread_status_util.cc \ - util/thread_status_util_debug.cc \ - util/xfunc.cc \ - util/xxhash.cc \ TOOL_LIB_SOURCES = \ tools/ldb_cmd.cc \ diff --git a/tools/ldb_cmd.cc b/tools/ldb_cmd.cc index ccd1972ff..d7e65bbf8 100644 --- a/tools/ldb_cmd.cc +++ b/tools/ldb_cmd.cc @@ -881,7 +881,8 @@ void DumpManifestFile(std::string file, bool verbose, bool hex, bool json) { options.num_levels = 64; WriteController wc(options.delayed_write_rate); WriteBufferManager wb(options.db_write_buffer_size); - VersionSet versions(dbname, &options, sopt, tc.get(), &wb, &wc); + ImmutableDBOptions immutable_db_options(options); + VersionSet versions(dbname, &immutable_db_options, sopt, tc.get(), &wb, &wc); Status s = versions.DumpManifest(options, file, verbose, hex, json); if (!s.ok()) { printf("Error in processing file %s %s\n", file.c_str(), @@ -1585,13 +1586,14 @@ Options ReduceDBLevelsCommand::PrepareOptionsForOpenDB() { Status ReduceDBLevelsCommand::GetOldNumOfLevels(Options& opt, int* levels) { + ImmutableDBOptions db_options(opt); EnvOptions soptions; std::shared_ptr tc( NewLRUCache(opt.max_open_files - 10, opt.table_cache_numshardbits)); const InternalKeyComparator cmp(opt.comparator); WriteController wc(opt.delayed_write_rate); WriteBufferManager wb(opt.db_write_buffer_size); - VersionSet versions(db_path_, &opt, soptions, tc.get(), &wb, &wc); + VersionSet versions(db_path_, &db_options, soptions, tc.get(), &wb, &wc); std::vector dummy; ColumnFamilyDescriptor dummy_descriptor(kDefaultColumnFamilyName, ColumnFamilyOptions(opt)); diff --git a/util/cf_options.cc b/util/cf_options.cc index ebe65be17..25399a88d 100644 --- a/util/cf_options.cc +++ b/util/cf_options.cc @@ -16,51 +16,62 @@ #include "port/port.h" #include "rocksdb/env.h" #include "rocksdb/options.h" +#include "util/db_options.h" namespace rocksdb { ImmutableCFOptions::ImmutableCFOptions(const Options& options) - : compaction_style(options.compaction_style), - compaction_pri(options.compaction_pri), - compaction_options_universal(options.compaction_options_universal), - compaction_options_fifo(options.compaction_options_fifo), - prefix_extractor(options.prefix_extractor.get()), - comparator(options.comparator), - merge_operator(options.merge_operator.get()), - compaction_filter(options.compaction_filter), - compaction_filter_factory(options.compaction_filter_factory.get()), - inplace_update_support(options.inplace_update_support), - inplace_callback(options.inplace_callback), - info_log(options.info_log.get()), - statistics(options.statistics.get()), - env(options.env), - delayed_write_rate(options.delayed_write_rate), - allow_mmap_reads(options.allow_mmap_reads), - allow_mmap_writes(options.allow_mmap_writes), - db_paths(options.db_paths), - memtable_factory(options.memtable_factory.get()), - table_factory(options.table_factory.get()), + : ImmutableCFOptions(ImmutableDBOptions(options), options) {} + +ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options, + const ColumnFamilyOptions& cf_options) + : compaction_style(cf_options.compaction_style), + compaction_pri(cf_options.compaction_pri), + compaction_options_universal(cf_options.compaction_options_universal), + compaction_options_fifo(cf_options.compaction_options_fifo), + prefix_extractor(cf_options.prefix_extractor.get()), + comparator(cf_options.comparator), + merge_operator(cf_options.merge_operator.get()), + compaction_filter(cf_options.compaction_filter), + compaction_filter_factory(cf_options.compaction_filter_factory.get()), + min_write_buffer_number_to_merge( + cf_options.min_write_buffer_number_to_merge), + max_write_buffer_number_to_maintain( + cf_options.max_write_buffer_number_to_maintain), + inplace_update_support(cf_options.inplace_update_support), + inplace_callback(cf_options.inplace_callback), + info_log(db_options.info_log.get()), + statistics(db_options.statistics.get()), + env(db_options.env), + delayed_write_rate(db_options.delayed_write_rate), + allow_mmap_reads(db_options.allow_mmap_reads), + allow_mmap_writes(db_options.allow_mmap_writes), + db_paths(db_options.db_paths), + memtable_factory(cf_options.memtable_factory.get()), + table_factory(cf_options.table_factory.get()), table_properties_collector_factories( - options.table_properties_collector_factories), - advise_random_on_open(options.advise_random_on_open), - bloom_locality(options.bloom_locality), - purge_redundant_kvs_while_flush(options.purge_redundant_kvs_while_flush), - disable_data_sync(options.disableDataSync), - use_fsync(options.use_fsync), - compression_per_level(options.compression_per_level), - bottommost_compression(options.bottommost_compression), - compression_opts(options.compression_opts), + cf_options.table_properties_collector_factories), + advise_random_on_open(db_options.advise_random_on_open), + bloom_locality(cf_options.bloom_locality), + purge_redundant_kvs_while_flush( + cf_options.purge_redundant_kvs_while_flush), + disable_data_sync(db_options.disable_data_sync), + use_fsync(db_options.use_fsync), + compression_per_level(cf_options.compression_per_level), + bottommost_compression(cf_options.bottommost_compression), + compression_opts(cf_options.compression_opts), level_compaction_dynamic_level_bytes( - options.level_compaction_dynamic_level_bytes), - access_hint_on_compaction_start(options.access_hint_on_compaction_start), + cf_options.level_compaction_dynamic_level_bytes), + access_hint_on_compaction_start( + db_options.access_hint_on_compaction_start), new_table_reader_for_compaction_inputs( - options.new_table_reader_for_compaction_inputs), - compaction_readahead_size(options.compaction_readahead_size), - num_levels(options.num_levels), - optimize_filters_for_hits(options.optimize_filters_for_hits), - listeners(options.listeners), - row_cache(options.row_cache), - max_subcompactions(options.max_subcompactions) {} + db_options.new_table_reader_for_compaction_inputs), + compaction_readahead_size(db_options.compaction_readahead_size), + num_levels(cf_options.num_levels), + optimize_filters_for_hits(cf_options.optimize_filters_for_hits), + listeners(db_options.listeners), + row_cache(db_options.row_cache), + max_subcompactions(db_options.max_subcompactions) {} // Multiple two operands. If they overflow, return op1. uint64_t MultiplyCheckOverflow(uint64_t op1, int op2) { diff --git a/util/cf_options.h b/util/cf_options.h index c08534635..0329167a9 100644 --- a/util/cf_options.h +++ b/util/cf_options.h @@ -10,6 +10,7 @@ #include "rocksdb/options.h" #include "util/compression.h" +#include "util/db_options.h" namespace rocksdb { @@ -18,8 +19,12 @@ namespace rocksdb { // of DB. Raw pointers defined in this struct do not have ownership to the data // they point to. Options contains shared_ptr to these data. struct ImmutableCFOptions { + ImmutableCFOptions(); explicit ImmutableCFOptions(const Options& options); + ImmutableCFOptions(const ImmutableDBOptions& db_options, + const ColumnFamilyOptions& cf_options); + CompactionStyle compaction_style; CompactionPri compaction_pri; @@ -37,6 +42,10 @@ struct ImmutableCFOptions { CompactionFilterFactory* compaction_filter_factory; + int min_write_buffer_number_to_merge; + + int max_write_buffer_number_to_maintain; + bool inplace_update_support; UpdateStatus (*inplace_callback)(char* existing_value, diff --git a/util/db_options.cc b/util/db_options.cc new file mode 100644 index 000000000..19a263ea2 --- /dev/null +++ b/util/db_options.cc @@ -0,0 +1,233 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + +#include "util/db_options.h" + +#ifndef __STDC_FORMAT_MACROS +#define __STDC_FORMAT_MACROS +#endif + +#include + +#include "port/port.h" +#include "rocksdb/cache.h" +#include "rocksdb/env.h" +#include "rocksdb/sst_file_manager.h" +#include "rocksdb/wal_filter.h" + +namespace rocksdb { + +ImmutableDBOptions::ImmutableDBOptions() : ImmutableDBOptions(Options()) {} + +ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options) + : create_if_missing(options.create_if_missing), + create_missing_column_families(options.create_missing_column_families), + error_if_exists(options.error_if_exists), + paranoid_checks(options.paranoid_checks), + env(options.env), + rate_limiter(options.rate_limiter), + sst_file_manager(options.sst_file_manager), + info_log(options.info_log), + info_log_level(options.info_log_level), + max_open_files(options.max_open_files), + max_file_opening_threads(options.max_file_opening_threads), + max_total_wal_size(options.max_total_wal_size), + statistics(options.statistics), + disable_data_sync(options.disableDataSync), + use_fsync(options.use_fsync), + db_paths(options.db_paths), + db_log_dir(options.db_log_dir), + wal_dir(options.wal_dir), + delete_obsolete_files_period_micros( + options.delete_obsolete_files_period_micros), + base_background_compactions(options.base_background_compactions), + max_background_compactions(options.max_background_compactions), + max_subcompactions(options.max_subcompactions), + max_background_flushes(options.max_background_flushes), + max_log_file_size(options.max_log_file_size), + log_file_time_to_roll(options.log_file_time_to_roll), + keep_log_file_num(options.keep_log_file_num), + recycle_log_file_num(options.recycle_log_file_num), + max_manifest_file_size(options.max_manifest_file_size), + table_cache_numshardbits(options.table_cache_numshardbits), + wal_ttl_seconds(options.WAL_ttl_seconds), + wal_size_limit_mb(options.WAL_size_limit_MB), + manifest_preallocation_size(options.manifest_preallocation_size), + allow_os_buffer(options.allow_os_buffer), + allow_mmap_reads(options.allow_mmap_reads), + allow_mmap_writes(options.allow_mmap_writes), + allow_fallocate(options.allow_fallocate), + is_fd_close_on_exec(options.is_fd_close_on_exec), + stats_dump_period_sec(options.stats_dump_period_sec), + advise_random_on_open(options.advise_random_on_open), + db_write_buffer_size(options.db_write_buffer_size), + write_buffer_manager(options.write_buffer_manager), + access_hint_on_compaction_start(options.access_hint_on_compaction_start), + new_table_reader_for_compaction_inputs( + options.new_table_reader_for_compaction_inputs), + compaction_readahead_size(options.compaction_readahead_size), + random_access_max_buffer_size(options.random_access_max_buffer_size), + writable_file_max_buffer_size(options.writable_file_max_buffer_size), + use_adaptive_mutex(options.use_adaptive_mutex), + bytes_per_sync(options.bytes_per_sync), + wal_bytes_per_sync(options.wal_bytes_per_sync), + listeners(options.listeners), + enable_thread_tracking(options.enable_thread_tracking), + delayed_write_rate(options.delayed_write_rate), + allow_concurrent_memtable_write(options.allow_concurrent_memtable_write), + enable_write_thread_adaptive_yield( + options.enable_write_thread_adaptive_yield), + write_thread_max_yield_usec(options.write_thread_max_yield_usec), + write_thread_slow_yield_usec(options.write_thread_slow_yield_usec), + skip_stats_update_on_db_open(options.skip_stats_update_on_db_open), + wal_recovery_mode(options.wal_recovery_mode), + allow_2pc(options.allow_2pc), + row_cache(options.row_cache), +#ifndef ROCKSDB_LITE + wal_filter(options.wal_filter), +#endif // ROCKSDB_LITE + fail_if_options_file_error(options.fail_if_options_file_error), + dump_malloc_stats(options.dump_malloc_stats), + avoid_flush_during_recovery(options.avoid_flush_during_recovery) { +} + +void ImmutableDBOptions::Dump(Logger* log) const { + Header(log, " Options.error_if_exists: %d", + error_if_exists); + Header(log, " Options.create_if_missing: %d", + create_if_missing); + Header(log, " Options.paranoid_checks: %d", + paranoid_checks); + Header(log, " Options.env: %p", env); + Header(log, " Options.info_log: %p", + info_log.get()); + Header(log, " Options.max_open_files: %d", + max_open_files); + Header(log, " Options.max_file_opening_threads: %d", + max_file_opening_threads); + Header(log, " Options.max_total_wal_size: %" PRIu64, + max_total_wal_size); + Header(log, " Options.disableDataSync: %d", + disable_data_sync); + Header(log, " Options.use_fsync: %d", use_fsync); + Header(log, + " Options.max_log_file_size: %" ROCKSDB_PRIszt, + max_log_file_size); + Header(log, " Options.max_manifest_file_size: %" PRIu64, + max_manifest_file_size); + Header(log, + " Options.log_file_time_to_roll: %" ROCKSDB_PRIszt, + log_file_time_to_roll); + Header(log, + " Options.keep_log_file_num: %" ROCKSDB_PRIszt, + keep_log_file_num); + Header(log, + " Options.recycle_log_file_num: %" ROCKSDB_PRIszt, + recycle_log_file_num); + Header(log, " Options.allow_os_buffer: %d", + allow_os_buffer); + Header(log, " Options.allow_mmap_reads: %d", + allow_mmap_reads); + Header(log, " Options.allow_fallocate: %d", + allow_fallocate); + Header(log, " Options.allow_mmap_writes: %d", + allow_mmap_writes); + Header(log, " Options.create_missing_column_families: %d", + create_missing_column_families); + Header(log, " Options.db_log_dir: %s", + db_log_dir.c_str()); + Header(log, " Options.wal_dir: %s", + wal_dir.c_str()); + Header(log, " Options.table_cache_numshardbits: %d", + table_cache_numshardbits); + Header(log, " Options.delete_obsolete_files_period_micros: %" PRIu64, + delete_obsolete_files_period_micros); + Header(log, " Options.base_background_compactions: %d", + base_background_compactions); + Header(log, " Options.max_background_compactions: %d", + max_background_compactions); + Header(log, " Options.max_subcompactions: %" PRIu32, + max_subcompactions); + Header(log, " Options.max_background_flushes: %d", + max_background_flushes); + Header(log, " Options.WAL_ttl_seconds: %" PRIu64, + wal_ttl_seconds); + Header(log, " Options.WAL_size_limit_MB: %" PRIu64, + wal_size_limit_mb); + Header(log, + " Options.manifest_preallocation_size: %" ROCKSDB_PRIszt, + manifest_preallocation_size); + Header(log, " Options.allow_os_buffer: %d", + allow_os_buffer); + Header(log, " Options.allow_mmap_reads: %d", + allow_mmap_reads); + Header(log, " Options.allow_mmap_writes: %d", + allow_mmap_writes); + Header(log, " Options.is_fd_close_on_exec: %d", + is_fd_close_on_exec); + Header(log, " Options.stats_dump_period_sec: %u", + stats_dump_period_sec); + Header(log, " Options.advise_random_on_open: %d", + advise_random_on_open); + Header(log, + " Options.db_write_buffer_size: %" ROCKSDB_PRIszt, + db_write_buffer_size); + Header(log, " Options.access_hint_on_compaction_start: %d", + static_cast(access_hint_on_compaction_start)); + Header(log, " Options.new_table_reader_for_compaction_inputs: %d", + new_table_reader_for_compaction_inputs); + Header(log, + " Options.compaction_readahead_size: %" ROCKSDB_PRIszt, + compaction_readahead_size); + Header(log, + " Options.random_access_max_buffer_size: %" ROCKSDB_PRIszt, + random_access_max_buffer_size); + Header(log, + " Options.writable_file_max_buffer_size: %" ROCKSDB_PRIszt, + writable_file_max_buffer_size); + Header(log, " Options.use_adaptive_mutex: %d", + use_adaptive_mutex); + Header(log, " Options.rate_limiter: %p", + rate_limiter.get()); + Header( + log, " Options.sst_file_manager.rate_bytes_per_sec: %" PRIi64, + sst_file_manager ? sst_file_manager->GetDeleteRateBytesPerSecond() : 0); + Header(log, " Options.bytes_per_sync: %" PRIu64, + bytes_per_sync); + Header(log, " Options.wal_bytes_per_sync: %" PRIu64, + wal_bytes_per_sync); + Header(log, " Options.wal_recovery_mode: %d", + wal_recovery_mode); + Header(log, " Options.enable_thread_tracking: %d", + enable_thread_tracking); + Log(log, " Options.delayed_write_rate : %" PRIu64, + delayed_write_rate); + Header(log, " Options.allow_concurrent_memtable_write: %d", + allow_concurrent_memtable_write); + Header(log, " Options.enable_write_thread_adaptive_yield: %d", + enable_write_thread_adaptive_yield); + Header(log, " Options.write_thread_max_yield_usec: %" PRIu64, + write_thread_max_yield_usec); + Header(log, " Options.write_thread_slow_yield_usec: %" PRIu64, + write_thread_slow_yield_usec); + if (row_cache) { + Header(log, " Options.row_cache: %" PRIu64, + row_cache->GetCapacity()); + } else { + Header(log, " Options.row_cache: None"); + } +#ifndef ROCKSDB_LITE + Header(log, " Options.wal_filter: %s", + wal_filter ? wal_filter->Name() : "None"); +#endif // ROCKDB_LITE + Header(log, " Options.avoid_flush_during_recovery: %d", + avoid_flush_during_recovery); +} + +MutableDBOptions::MutableDBOptions(const DBOptions& options) {} + +void MutableDBOptions::Dump(Logger* log) const {} + +} // namespace rocksdb diff --git a/util/db_options.h b/util/db_options.h new file mode 100644 index 000000000..dbe17b485 --- /dev/null +++ b/util/db_options.h @@ -0,0 +1,96 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + +#pragma once + +#include +#include + +#include "rocksdb/options.h" + +namespace rocksdb { + +struct ImmutableDBOptions { + ImmutableDBOptions(); + explicit ImmutableDBOptions(const DBOptions& options); + + void Dump(Logger* log) const; + + bool create_if_missing; + bool create_missing_column_families; + bool error_if_exists; + bool paranoid_checks; + Env* env; + std::shared_ptr rate_limiter; + std::shared_ptr sst_file_manager; + std::shared_ptr info_log; + InfoLogLevel info_log_level; + int max_open_files; + int max_file_opening_threads; + uint64_t max_total_wal_size; + std::shared_ptr statistics; + bool disable_data_sync; + bool use_fsync; + std::vector db_paths; + std::string db_log_dir; + std::string wal_dir; + uint64_t delete_obsolete_files_period_micros; + int base_background_compactions; + int max_background_compactions; + uint32_t max_subcompactions; + int max_background_flushes; + size_t max_log_file_size; + size_t log_file_time_to_roll; + size_t keep_log_file_num; + size_t recycle_log_file_num; + uint64_t max_manifest_file_size; + int table_cache_numshardbits; + uint64_t wal_ttl_seconds; + uint64_t wal_size_limit_mb; + size_t manifest_preallocation_size; + bool allow_os_buffer; + bool allow_mmap_reads; + bool allow_mmap_writes; + bool allow_fallocate; + bool is_fd_close_on_exec; + unsigned int stats_dump_period_sec; + bool advise_random_on_open; + size_t db_write_buffer_size; + std::shared_ptr write_buffer_manager; + DBOptions::AccessHint access_hint_on_compaction_start; + bool new_table_reader_for_compaction_inputs; + size_t compaction_readahead_size; + size_t random_access_max_buffer_size; + size_t writable_file_max_buffer_size; + bool use_adaptive_mutex; + uint64_t bytes_per_sync; + uint64_t wal_bytes_per_sync; + std::vector> listeners; + bool enable_thread_tracking; + uint64_t delayed_write_rate; + bool allow_concurrent_memtable_write; + bool enable_write_thread_adaptive_yield; + uint64_t write_thread_max_yield_usec; + uint64_t write_thread_slow_yield_usec; + bool skip_stats_update_on_db_open; + WALRecoveryMode wal_recovery_mode; + bool allow_2pc; + std::shared_ptr row_cache; +#ifndef ROCKSDB_LITE + WalFilter* wal_filter; +#endif // ROCKSDB_LITE + bool fail_if_options_file_error; + bool dump_malloc_stats; + bool avoid_flush_during_recovery; +}; + +struct MutableDBOptions { + explicit MutableDBOptions(const MutableDBOptions& options) = default; + explicit MutableDBOptions(const DBOptions& options); + + void Dump(Logger* log) const; +}; + +} // namespace rocksdb diff --git a/util/file_util.cc b/util/file_util.cc index c14309da2..29a84555a 100644 --- a/util/file_util.cc +++ b/util/file_util.cc @@ -9,7 +9,6 @@ #include #include "rocksdb/env.h" -#include "rocksdb/options.h" #include "util/sst_file_manager_impl.h" #include "util/file_reader_writer.h" @@ -82,8 +81,8 @@ Status CreateFile(Env* env, const std::string& destination, return dest_writer->Append(Slice(contents)); } -Status DeleteSSTFile(const DBOptions* db_options, const std::string& fname, - uint32_t path_id) { +Status DeleteSSTFile(const ImmutableDBOptions* db_options, + const std::string& fname, uint32_t path_id) { // TODO(tec): support sst_file_manager for multiple path_ids auto sfm = static_cast(db_options->sst_file_manager.get()); diff --git a/util/file_util.h b/util/file_util.h index 5b2320e33..f2c5c329f 100644 --- a/util/file_util.h +++ b/util/file_util.h @@ -6,10 +6,10 @@ #pragma once #include +#include "rocksdb/env.h" #include "rocksdb/status.h" #include "rocksdb/types.h" -#include "rocksdb/env.h" -#include "rocksdb/options.h" +#include "util/db_options.h" namespace rocksdb { @@ -19,7 +19,7 @@ extern Status CopyFile(Env* env, const std::string& source, extern Status CreateFile(Env* env, const std::string& destination, const std::string& contents); -extern Status DeleteSSTFile(const DBOptions* db_options, +extern Status DeleteSSTFile(const ImmutableDBOptions* db_options, const std::string& fname, uint32_t path_id); } // namespace rocksdb diff --git a/util/options_helper.cc b/util/options_helper.cc index 8ec3e1fbb..4240e319b 100644 --- a/util/options_helper.cc +++ b/util/options_helper.cc @@ -26,6 +26,99 @@ namespace rocksdb { +DBOptions BuildDBOptions(const ImmutableDBOptions& immutable_db_options, + const MutableDBOptions& mutable_db_options) { + DBOptions options; + + options.create_if_missing = immutable_db_options.create_if_missing; + options.create_missing_column_families = + immutable_db_options.create_missing_column_families; + options.error_if_exists = immutable_db_options.error_if_exists; + options.paranoid_checks = immutable_db_options.paranoid_checks; + options.env = immutable_db_options.env; + options.rate_limiter = immutable_db_options.rate_limiter; + options.sst_file_manager = immutable_db_options.sst_file_manager; + options.info_log = immutable_db_options.info_log; + options.info_log_level = immutable_db_options.info_log_level; + options.max_open_files = immutable_db_options.max_open_files; + options.max_file_opening_threads = + immutable_db_options.max_file_opening_threads; + options.max_total_wal_size = immutable_db_options.max_total_wal_size; + options.statistics = immutable_db_options.statistics; + options.disableDataSync = immutable_db_options.disable_data_sync; + options.use_fsync = immutable_db_options.use_fsync; + options.db_paths = immutable_db_options.db_paths; + options.db_log_dir = immutable_db_options.db_log_dir; + options.wal_dir = immutable_db_options.wal_dir; + options.delete_obsolete_files_period_micros = + immutable_db_options.delete_obsolete_files_period_micros; + options.base_background_compactions = + immutable_db_options.base_background_compactions; + options.max_background_compactions = + immutable_db_options.max_background_compactions; + options.max_subcompactions = immutable_db_options.max_subcompactions; + options.max_background_flushes = immutable_db_options.max_background_flushes; + options.max_log_file_size = immutable_db_options.max_log_file_size; + options.log_file_time_to_roll = immutable_db_options.log_file_time_to_roll; + options.keep_log_file_num = immutable_db_options.keep_log_file_num; + options.recycle_log_file_num = immutable_db_options.recycle_log_file_num; + options.max_manifest_file_size = immutable_db_options.max_manifest_file_size; + options.table_cache_numshardbits = + immutable_db_options.table_cache_numshardbits; + options.WAL_ttl_seconds = immutable_db_options.wal_ttl_seconds; + options.WAL_size_limit_MB = immutable_db_options.wal_size_limit_mb; + options.manifest_preallocation_size = + immutable_db_options.manifest_preallocation_size; + options.allow_os_buffer = immutable_db_options.allow_os_buffer; + options.allow_mmap_reads = immutable_db_options.allow_mmap_reads; + options.allow_mmap_writes = immutable_db_options.allow_mmap_writes; + options.allow_fallocate = immutable_db_options.allow_fallocate; + options.is_fd_close_on_exec = immutable_db_options.is_fd_close_on_exec; + options.stats_dump_period_sec = immutable_db_options.stats_dump_period_sec; + options.advise_random_on_open = immutable_db_options.advise_random_on_open; + options.db_write_buffer_size = immutable_db_options.db_write_buffer_size; + options.write_buffer_manager = immutable_db_options.write_buffer_manager; + options.access_hint_on_compaction_start = + immutable_db_options.access_hint_on_compaction_start; + options.new_table_reader_for_compaction_inputs = + immutable_db_options.new_table_reader_for_compaction_inputs; + options.compaction_readahead_size = + immutable_db_options.compaction_readahead_size; + options.random_access_max_buffer_size = + immutable_db_options.random_access_max_buffer_size; + options.writable_file_max_buffer_size = + immutable_db_options.writable_file_max_buffer_size; + options.use_adaptive_mutex = immutable_db_options.use_adaptive_mutex; + options.bytes_per_sync = immutable_db_options.bytes_per_sync; + options.wal_bytes_per_sync = immutable_db_options.wal_bytes_per_sync; + options.listeners = immutable_db_options.listeners; + options.enable_thread_tracking = immutable_db_options.enable_thread_tracking; + options.delayed_write_rate = immutable_db_options.delayed_write_rate; + options.allow_concurrent_memtable_write = + immutable_db_options.allow_concurrent_memtable_write; + options.enable_write_thread_adaptive_yield = + immutable_db_options.enable_write_thread_adaptive_yield; + options.write_thread_max_yield_usec = + immutable_db_options.write_thread_max_yield_usec; + options.write_thread_slow_yield_usec = + immutable_db_options.write_thread_slow_yield_usec; + options.skip_stats_update_on_db_open = + immutable_db_options.skip_stats_update_on_db_open; + options.wal_recovery_mode = immutable_db_options.wal_recovery_mode; + options.allow_2pc = immutable_db_options.allow_2pc; + options.row_cache = immutable_db_options.row_cache; +#ifndef ROCKSDB_LITE + options.wal_filter = immutable_db_options.wal_filter; +#endif // ROCKSDB_LITE + options.fail_if_options_file_error = + immutable_db_options.fail_if_options_file_error; + options.dump_malloc_stats = immutable_db_options.dump_malloc_stats; + options.avoid_flush_during_recovery = + immutable_db_options.avoid_flush_during_recovery; + + return options; +} + ColumnFamilyOptions BuildColumnFamilyOptions( const ColumnFamilyOptions& options, const MutableCFOptions& mutable_cf_options) { diff --git a/util/options_helper.h b/util/options_helper.h index f62ad8ef8..2a8c96108 100644 --- a/util/options_helper.h +++ b/util/options_helper.h @@ -13,9 +13,13 @@ #include "rocksdb/status.h" #include "rocksdb/table.h" #include "util/cf_options.h" +#include "util/db_options.h" namespace rocksdb { +DBOptions BuildDBOptions(const ImmutableDBOptions& immutable_db_options, + const MutableDBOptions& mutable_db_options); + ColumnFamilyOptions BuildColumnFamilyOptions( const ColumnFamilyOptions& ioptions, const MutableCFOptions& mutable_cf_options);