]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/rocksdb/options/db_options.cc
import quincy beta 17.1.0
[ceph.git] / ceph / src / rocksdb / options / db_options.cc
index 0dcc8ef79fc1b0d119f8aeb808764dff296c71e0..3733d448c7ac25b8ea47a3acaa29d57a800a3ef0 100644 (file)
 
 #include <cinttypes>
 
-#include "db/version_edit.h"
 #include "logging/logging.h"
+#include "options/configurable_helper.h"
+#include "options/options_helper.h"
+#include "options/options_parser.h"
 #include "port/port.h"
-#include "rocksdb/cache.h"
+#include "rocksdb/configurable.h"
 #include "rocksdb/env.h"
 #include "rocksdb/file_system.h"
+#include "rocksdb/rate_limiter.h"
 #include "rocksdb/sst_file_manager.h"
+#include "rocksdb/utilities/options_type.h"
 #include "rocksdb/wal_filter.h"
+#include "util/string_util.h"
 
 namespace ROCKSDB_NAMESPACE {
+#ifndef ROCKSDB_LITE
+static std::unordered_map<std::string, WALRecoveryMode>
+    wal_recovery_mode_string_map = {
+        {"kTolerateCorruptedTailRecords",
+         WALRecoveryMode::kTolerateCorruptedTailRecords},
+        {"kAbsoluteConsistency", WALRecoveryMode::kAbsoluteConsistency},
+        {"kPointInTimeRecovery", WALRecoveryMode::kPointInTimeRecovery},
+        {"kSkipAnyCorruptedRecords",
+         WALRecoveryMode::kSkipAnyCorruptedRecords}};
+
+static std::unordered_map<std::string, DBOptions::AccessHint>
+    access_hint_string_map = {{"NONE", DBOptions::AccessHint::NONE},
+                              {"NORMAL", DBOptions::AccessHint::NORMAL},
+                              {"SEQUENTIAL", DBOptions::AccessHint::SEQUENTIAL},
+                              {"WILLNEED", DBOptions::AccessHint::WILLNEED}};
+
+static std::unordered_map<std::string, InfoLogLevel> info_log_level_string_map =
+    {{"DEBUG_LEVEL", InfoLogLevel::DEBUG_LEVEL},
+     {"INFO_LEVEL", InfoLogLevel::INFO_LEVEL},
+     {"WARN_LEVEL", InfoLogLevel::WARN_LEVEL},
+     {"ERROR_LEVEL", InfoLogLevel::ERROR_LEVEL},
+     {"FATAL_LEVEL", InfoLogLevel::FATAL_LEVEL},
+     {"HEADER_LEVEL", InfoLogLevel::HEADER_LEVEL}};
+
+static std::unordered_map<std::string, OptionTypeInfo>
+    db_mutable_options_type_info = {
+        {"allow_os_buffer",
+         {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+          OptionTypeFlags::kMutable}},
+        {"max_background_jobs",
+         {offsetof(struct MutableDBOptions, max_background_jobs),
+          OptionType::kInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"max_background_compactions",
+         {offsetof(struct MutableDBOptions, max_background_compactions),
+          OptionType::kInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"base_background_compactions",
+         {offsetof(struct MutableDBOptions, base_background_compactions),
+          OptionType::kInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"max_subcompactions",
+         {offsetof(struct MutableDBOptions, max_subcompactions),
+          OptionType::kUInt32T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"avoid_flush_during_shutdown",
+         {offsetof(struct MutableDBOptions, avoid_flush_during_shutdown),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"writable_file_max_buffer_size",
+         {offsetof(struct MutableDBOptions, writable_file_max_buffer_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"delayed_write_rate",
+         {offsetof(struct MutableDBOptions, delayed_write_rate),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"max_total_wal_size",
+         {offsetof(struct MutableDBOptions, max_total_wal_size),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"delete_obsolete_files_period_micros",
+         {offsetof(struct MutableDBOptions,
+                   delete_obsolete_files_period_micros),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"stats_dump_period_sec",
+         {offsetof(struct MutableDBOptions, stats_dump_period_sec),
+          OptionType::kUInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"stats_persist_period_sec",
+         {offsetof(struct MutableDBOptions, stats_persist_period_sec),
+          OptionType::kUInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"stats_history_buffer_size",
+         {offsetof(struct MutableDBOptions, stats_history_buffer_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"max_open_files",
+         {offsetof(struct MutableDBOptions, max_open_files), OptionType::kInt,
+          OptionVerificationType::kNormal, OptionTypeFlags::kMutable}},
+        {"bytes_per_sync",
+         {offsetof(struct MutableDBOptions, bytes_per_sync),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"wal_bytes_per_sync",
+         {offsetof(struct MutableDBOptions, wal_bytes_per_sync),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"strict_bytes_per_sync",
+         {offsetof(struct MutableDBOptions, strict_bytes_per_sync),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"compaction_readahead_size",
+         {offsetof(struct MutableDBOptions, compaction_readahead_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+        {"max_background_flushes",
+         {offsetof(struct MutableDBOptions, max_background_flushes),
+          OptionType::kInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kMutable}},
+};
+
+static std::unordered_map<std::string, OptionTypeInfo>
+    db_immutable_options_type_info = {
+        /*
+         // not yet supported
+          std::shared_ptr<Cache> row_cache;
+          std::shared_ptr<DeleteScheduler> delete_scheduler;
+          std::shared_ptr<Logger> info_log;
+          std::shared_ptr<RateLimiter> rate_limiter;
+          std::shared_ptr<Statistics> statistics;
+          std::vector<DbPath> db_paths;
+          std::vector<std::shared_ptr<EventListener>> listeners;
+         */
+        {"advise_random_on_open",
+         {offsetof(struct ImmutableDBOptions, advise_random_on_open),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"allow_mmap_reads",
+         {offsetof(struct ImmutableDBOptions, allow_mmap_reads),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"allow_fallocate",
+         {offsetof(struct ImmutableDBOptions, allow_fallocate),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"allow_mmap_writes",
+         {offsetof(struct ImmutableDBOptions, allow_mmap_writes),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"use_direct_reads",
+         {offsetof(struct ImmutableDBOptions, use_direct_reads),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"use_direct_writes",
+         {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+          OptionTypeFlags::kNone}},
+        {"use_direct_io_for_flush_and_compaction",
+         {offsetof(struct ImmutableDBOptions,
+                   use_direct_io_for_flush_and_compaction),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"allow_2pc",
+         {offsetof(struct ImmutableDBOptions, allow_2pc), OptionType::kBoolean,
+          OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+        {"create_if_missing",
+         {offsetof(struct ImmutableDBOptions, create_if_missing),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"create_missing_column_families",
+         {offsetof(struct ImmutableDBOptions, create_missing_column_families),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"disableDataSync",
+         {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+          OptionTypeFlags::kNone}},
+        {"disable_data_sync",  // for compatibility
+         {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+          OptionTypeFlags::kNone}},
+        {"enable_thread_tracking",
+         {offsetof(struct ImmutableDBOptions, enable_thread_tracking),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"error_if_exists",
+         {offsetof(struct ImmutableDBOptions, error_if_exists),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"is_fd_close_on_exec",
+         {offsetof(struct ImmutableDBOptions, is_fd_close_on_exec),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"paranoid_checks",
+         {offsetof(struct ImmutableDBOptions, paranoid_checks),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"track_and_verify_wals_in_manifest",
+         {offsetof(struct ImmutableDBOptions,
+                   track_and_verify_wals_in_manifest),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"skip_log_error_on_recovery",
+         {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+          OptionTypeFlags::kNone}},
+        {"skip_stats_update_on_db_open",
+         {offsetof(struct ImmutableDBOptions, skip_stats_update_on_db_open),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"skip_checking_sst_file_sizes_on_db_open",
+         {offsetof(struct ImmutableDBOptions,
+                   skip_checking_sst_file_sizes_on_db_open),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"new_table_reader_for_compaction_inputs",
+         {offsetof(struct ImmutableDBOptions,
+                   new_table_reader_for_compaction_inputs),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"random_access_max_buffer_size",
+         {offsetof(struct ImmutableDBOptions, random_access_max_buffer_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"use_adaptive_mutex",
+         {offsetof(struct ImmutableDBOptions, use_adaptive_mutex),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"use_fsync",
+         {offsetof(struct ImmutableDBOptions, use_fsync), OptionType::kBoolean,
+          OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+        {"max_file_opening_threads",
+         {offsetof(struct ImmutableDBOptions, max_file_opening_threads),
+          OptionType::kInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"table_cache_numshardbits",
+         {offsetof(struct ImmutableDBOptions, table_cache_numshardbits),
+          OptionType::kInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"db_write_buffer_size",
+         {offsetof(struct ImmutableDBOptions, db_write_buffer_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"keep_log_file_num",
+         {offsetof(struct ImmutableDBOptions, keep_log_file_num),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"recycle_log_file_num",
+         {offsetof(struct ImmutableDBOptions, recycle_log_file_num),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"log_file_time_to_roll",
+         {offsetof(struct ImmutableDBOptions, log_file_time_to_roll),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"manifest_preallocation_size",
+         {offsetof(struct ImmutableDBOptions, manifest_preallocation_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"max_log_file_size",
+         {offsetof(struct ImmutableDBOptions, max_log_file_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"db_log_dir",
+         {offsetof(struct ImmutableDBOptions, db_log_dir), OptionType::kString,
+          OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+        {"wal_dir",
+         {offsetof(struct ImmutableDBOptions, wal_dir), OptionType::kString,
+          OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+        {"WAL_size_limit_MB",
+         {offsetof(struct ImmutableDBOptions, wal_size_limit_mb),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"WAL_ttl_seconds",
+         {offsetof(struct ImmutableDBOptions, wal_ttl_seconds),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"max_manifest_file_size",
+         {offsetof(struct ImmutableDBOptions, max_manifest_file_size),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"persist_stats_to_disk",
+         {offsetof(struct ImmutableDBOptions, persist_stats_to_disk),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"fail_if_options_file_error",
+         {offsetof(struct ImmutableDBOptions, fail_if_options_file_error),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"enable_pipelined_write",
+         {offsetof(struct ImmutableDBOptions, enable_pipelined_write),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"unordered_write",
+         {offsetof(struct ImmutableDBOptions, unordered_write),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"allow_concurrent_memtable_write",
+         {offsetof(struct ImmutableDBOptions, allow_concurrent_memtable_write),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"wal_recovery_mode",
+         OptionTypeInfo::Enum<WALRecoveryMode>(
+             offsetof(struct ImmutableDBOptions, wal_recovery_mode),
+             &wal_recovery_mode_string_map)},
+        {"enable_write_thread_adaptive_yield",
+         {offsetof(struct ImmutableDBOptions,
+                   enable_write_thread_adaptive_yield),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"write_thread_slow_yield_usec",
+         {offsetof(struct ImmutableDBOptions, write_thread_slow_yield_usec),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"max_write_batch_group_size_bytes",
+         {offsetof(struct ImmutableDBOptions, max_write_batch_group_size_bytes),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"write_thread_max_yield_usec",
+         {offsetof(struct ImmutableDBOptions, write_thread_max_yield_usec),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"access_hint_on_compaction_start",
+         OptionTypeInfo::Enum<DBOptions::AccessHint>(
+             offsetof(struct ImmutableDBOptions,
+                      access_hint_on_compaction_start),
+             &access_hint_string_map)},
+        {"info_log_level",
+         OptionTypeInfo::Enum<InfoLogLevel>(
+             offsetof(struct ImmutableDBOptions, info_log_level),
+             &info_log_level_string_map)},
+        {"dump_malloc_stats",
+         {offsetof(struct ImmutableDBOptions, dump_malloc_stats),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"avoid_flush_during_recovery",
+         {offsetof(struct ImmutableDBOptions, avoid_flush_during_recovery),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"allow_ingest_behind",
+         {offsetof(struct ImmutableDBOptions, allow_ingest_behind),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"preserve_deletes",
+         {offsetof(struct ImmutableDBOptions, preserve_deletes),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"concurrent_prepare",  // Deprecated by two_write_queues
+         {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+          OptionTypeFlags::kNone}},
+        {"two_write_queues",
+         {offsetof(struct ImmutableDBOptions, two_write_queues),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"manual_wal_flush",
+         {offsetof(struct ImmutableDBOptions, manual_wal_flush),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"seq_per_batch",
+         {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+          OptionTypeFlags::kNone}},
+        {"atomic_flush",
+         {offsetof(struct ImmutableDBOptions, atomic_flush),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"avoid_unnecessary_blocking_io",
+         {offsetof(struct ImmutableDBOptions, avoid_unnecessary_blocking_io),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"write_dbid_to_manifest",
+         {offsetof(struct ImmutableDBOptions, write_dbid_to_manifest),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"log_readahead_size",
+         {offsetof(struct ImmutableDBOptions, log_readahead_size),
+          OptionType::kSizeT, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"best_efforts_recovery",
+         {offsetof(struct ImmutableDBOptions, best_efforts_recovery),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"max_bgerror_resume_count",
+         {offsetof(struct ImmutableDBOptions, max_bgerror_resume_count),
+          OptionType::kInt, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"bgerror_resume_retry_interval",
+         {offsetof(struct ImmutableDBOptions, bgerror_resume_retry_interval),
+          OptionType::kUInt64T, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+        {"db_host_id",
+         {offsetof(struct ImmutableDBOptions, db_host_id), OptionType::kString,
+          OptionVerificationType::kNormal, OptionTypeFlags::kCompareNever}},
+        // The following properties were handled as special cases in ParseOption
+        // This means that the properties could be read from the options file
+        // but never written to the file or compared to each other.
+        {"rate_limiter_bytes_per_sec",
+         {offsetof(struct ImmutableDBOptions, rate_limiter),
+          OptionType::kUnknown, OptionVerificationType::kNormal,
+          (OptionTypeFlags::kDontSerialize | OptionTypeFlags::kCompareNever),
+          // Parse the input value as a RateLimiter
+          [](const ConfigOptions& /*opts*/, const std::string& /*name*/,
+             const std::string& value, char* addr) {
+            auto limiter =
+                reinterpret_cast<std::shared_ptr<RateLimiter>*>(addr);
+            limiter->reset(NewGenericRateLimiter(
+                static_cast<int64_t>(ParseUint64(value))));
+            return Status::OK();
+          }}},
+        {"env",
+         {offsetof(struct ImmutableDBOptions, env), OptionType::kUnknown,
+          OptionVerificationType::kNormal,
+          (OptionTypeFlags::kDontSerialize | OptionTypeFlags::kCompareNever),
+          // Parse the input value as an Env
+          [](const ConfigOptions& /*opts*/, const std::string& /*name*/,
+             const std::string& value, char* addr) {
+            auto old_env = reinterpret_cast<Env**>(addr);  // Get the old value
+            Env* new_env = *old_env;                       // Set new to old
+            Status s = Env::LoadEnv(value, &new_env);      // Update new value
+            if (s.ok()) {                                  // It worked
+              *old_env = new_env;                          // Update the old one
+            }
+            return s;
+          }}},
+        {"allow_data_in_errors",
+         {offsetof(struct ImmutableDBOptions, allow_data_in_errors),
+          OptionType::kBoolean, OptionVerificationType::kNormal,
+          OptionTypeFlags::kNone}},
+};
+
+const std::string OptionsHelper::kDBOptionsName = "DBOptions";
+
+class MutableDBConfigurable : public Configurable {
+ public:
+  MutableDBConfigurable(const MutableDBOptions& mdb) {
+    mutable_ = mdb;
+    ConfigurableHelper::RegisterOptions(*this, &mutable_,
+                                        &db_mutable_options_type_info);
+  }
+
+ protected:
+  MutableDBOptions mutable_;
+};
+
+class DBOptionsConfigurable : public MutableDBConfigurable {
+ public:
+  DBOptionsConfigurable(const DBOptions& opts)
+      : MutableDBConfigurable(MutableDBOptions(opts)), db_options_(opts) {
+    // The ImmutableDBOptions currently requires the env to be non-null.  Make
+    // sure it is
+    if (opts.env != nullptr) {
+      immutable_ = ImmutableDBOptions(opts);
+    } else {
+      DBOptions copy = opts;
+      copy.env = Env::Default();
+      immutable_ = ImmutableDBOptions(copy);
+    }
+    ConfigurableHelper::RegisterOptions(*this, &immutable_,
+                                        &db_immutable_options_type_info);
+  }
+
+ protected:
+  Status ConfigureOptions(
+      const ConfigOptions& config_options,
+      const std::unordered_map<std::string, std::string>& opts_map,
+      std::unordered_map<std::string, std::string>* unused) override {
+    Status s = ConfigurableHelper::ConfigureOptions(config_options, *this,
+                                                    opts_map, unused);
+    if (s.ok()) {
+      db_options_ = BuildDBOptions(immutable_, mutable_);
+      s = PrepareOptions(config_options);
+    }
+    return s;
+  }
+
+  const void* GetOptionsPtr(const std::string& name) const override {
+    if (name == OptionsHelper::kDBOptionsName) {
+      return &db_options_;
+    } else {
+      return MutableDBConfigurable::GetOptionsPtr(name);
+    }
+  }
+
+ private:
+  ImmutableDBOptions immutable_;
+  DBOptions db_options_;
+};
+
+std::unique_ptr<Configurable> DBOptionsAsConfigurable(
+    const MutableDBOptions& opts) {
+  std::unique_ptr<Configurable> ptr(new MutableDBConfigurable(opts));
+  return ptr;
+}
+std::unique_ptr<Configurable> DBOptionsAsConfigurable(const DBOptions& opts) {
+  std::unique_ptr<Configurable> ptr(new DBOptionsConfigurable(opts));
+  return ptr;
+}
+#endif  // ROCKSDB_LITE
 
 ImmutableDBOptions::ImmutableDBOptions() : ImmutableDBOptions(Options()) {}
 
@@ -25,8 +505,10 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
       create_missing_column_families(options.create_missing_column_families),
       error_if_exists(options.error_if_exists),
       paranoid_checks(options.paranoid_checks),
+      track_and_verify_wals_in_manifest(
+          options.track_and_verify_wals_in_manifest),
       env(options.env),
-      fs(options.file_system),
+      fs(options.env->GetFileSystem()),
       rate_limiter(options.rate_limiter),
       sst_file_manager(options.sst_file_manager),
       info_log(options.info_log),
@@ -37,8 +519,6 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
       db_paths(options.db_paths),
       db_log_dir(options.db_log_dir),
       wal_dir(options.wal_dir),
-      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),
@@ -95,7 +575,12 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
       persist_stats_to_disk(options.persist_stats_to_disk),
       write_dbid_to_manifest(options.write_dbid_to_manifest),
       log_readahead_size(options.log_readahead_size),
-      sst_file_checksum_func(options.sst_file_checksum_func) {
+      file_checksum_gen_factory(options.file_checksum_gen_factory),
+      best_efforts_recovery(options.best_efforts_recovery),
+      max_bgerror_resume_count(options.max_bgerror_resume_count),
+      bgerror_resume_retry_interval(options.bgerror_resume_retry_interval),
+      allow_data_in_errors(options.allow_data_in_errors),
+      db_host_id(options.db_host_id) {
 }
 
 void ImmutableDBOptions::Dump(Logger* log) const {
@@ -105,6 +590,10 @@ void ImmutableDBOptions::Dump(Logger* log) const {
                    create_if_missing);
   ROCKS_LOG_HEADER(log, "                        Options.paranoid_checks: %d",
                    paranoid_checks);
+  ROCKS_LOG_HEADER(log,
+                   "                              "
+                   "Options.track_and_verify_wals_in_manifest: %d",
+                   track_and_verify_wals_in_manifest);
   ROCKS_LOG_HEADER(log, "                                    Options.env: %p",
                    env);
   ROCKS_LOG_HEADER(log, "                                     Options.fs: %s",
@@ -152,11 +641,6 @@ void ImmutableDBOptions::Dump(Logger* log) const {
                    wal_dir.c_str());
   ROCKS_LOG_HEADER(log, "               Options.table_cache_numshardbits: %d",
                    table_cache_numshardbits);
-  ROCKS_LOG_HEADER(log,
-                   "                     Options.max_subcompactions: %" PRIu32,
-                   max_subcompactions);
-  ROCKS_LOG_HEADER(log, "                 Options.max_background_flushes: %d",
-                   max_background_flushes);
   ROCKS_LOG_HEADER(log,
                    "                        Options.WAL_ttl_seconds: %" PRIu64,
                    wal_ttl_seconds);
@@ -246,16 +730,27 @@ void ImmutableDBOptions::Dump(Logger* log) const {
   ROCKS_LOG_HEADER(
       log, "                Options.log_readahead_size: %" ROCKSDB_PRIszt,
       log_readahead_size);
-  ROCKS_LOG_HEADER(log, "                Options.sst_file_checksum_func: %s",
-                   sst_file_checksum_func
-                       ? sst_file_checksum_func->Name()
-                       : kUnknownFileChecksumFuncName.c_str());
+  ROCKS_LOG_HEADER(log, "                Options.file_checksum_gen_factory: %s",
+                   file_checksum_gen_factory ? file_checksum_gen_factory->Name()
+                                             : kUnknownFileChecksumFuncName);
+  ROCKS_LOG_HEADER(log, "                Options.best_efforts_recovery: %d",
+                   static_cast<int>(best_efforts_recovery));
+  ROCKS_LOG_HEADER(log, "               Options.max_bgerror_resume_count: %d",
+                   max_bgerror_resume_count);
+  ROCKS_LOG_HEADER(log,
+                   "           Options.bgerror_resume_retry_interval: %" PRIu64,
+                   bgerror_resume_retry_interval);
+  ROCKS_LOG_HEADER(log, "            Options.allow_data_in_errors: %d",
+                   allow_data_in_errors);
+  ROCKS_LOG_HEADER(log, "            Options.db_host_id: %s",
+                   db_host_id.c_str());
 }
 
 MutableDBOptions::MutableDBOptions()
     : max_background_jobs(2),
       base_background_compactions(-1),
       max_background_compactions(-1),
+      max_subcompactions(0),
       avoid_flush_during_shutdown(false),
       writable_file_max_buffer_size(1024 * 1024),
       delayed_write_rate(2 * 1024U * 1024U),
@@ -268,12 +763,14 @@ MutableDBOptions::MutableDBOptions()
       bytes_per_sync(0),
       wal_bytes_per_sync(0),
       strict_bytes_per_sync(false),
-      compaction_readahead_size(0) {}
+      compaction_readahead_size(0),
+      max_background_flushes(-1) {}
 
 MutableDBOptions::MutableDBOptions(const DBOptions& options)
     : max_background_jobs(options.max_background_jobs),
       base_background_compactions(options.base_background_compactions),
       max_background_compactions(options.max_background_compactions),
+      max_subcompactions(options.max_subcompactions),
       avoid_flush_during_shutdown(options.avoid_flush_during_shutdown),
       writable_file_max_buffer_size(options.writable_file_max_buffer_size),
       delayed_write_rate(options.delayed_write_rate),
@@ -287,13 +784,16 @@ MutableDBOptions::MutableDBOptions(const DBOptions& options)
       bytes_per_sync(options.bytes_per_sync),
       wal_bytes_per_sync(options.wal_bytes_per_sync),
       strict_bytes_per_sync(options.strict_bytes_per_sync),
-      compaction_readahead_size(options.compaction_readahead_size) {}
+      compaction_readahead_size(options.compaction_readahead_size),
+      max_background_flushes(options.max_background_flushes) {}
 
 void MutableDBOptions::Dump(Logger* log) const {
   ROCKS_LOG_HEADER(log, "            Options.max_background_jobs: %d",
                    max_background_jobs);
   ROCKS_LOG_HEADER(log, "            Options.max_background_compactions: %d",
                    max_background_compactions);
+  ROCKS_LOG_HEADER(log, "            Options.max_subcompactions: %" PRIu32,
+                   max_subcompactions);
   ROCKS_LOG_HEADER(log, "            Options.avoid_flush_during_shutdown: %d",
                    avoid_flush_during_shutdown);
   ROCKS_LOG_HEADER(
@@ -328,6 +828,8 @@ void MutableDBOptions::Dump(Logger* log) const {
   ROCKS_LOG_HEADER(log,
                    "      Options.compaction_readahead_size: %" ROCKSDB_PRIszt,
                    compaction_readahead_size);
+  ROCKS_LOG_HEADER(log, "                 Options.max_background_flushes: %d",
+                          max_background_flushes);
 }
 
 }  // namespace ROCKSDB_NAMESPACE