Make types of Immutable/Mutable Options fields match that of the underlying Option (#8176)

Summary:
This PR is a first step at attempting to clean up some of the Mutable/Immutable Options code.  With this change, a DBOption and a ColumnFamilyOption can be reconstructed from their Mutable and Immutable equivalents, respectively.

readrandom tests do not show any performance degradation versus master (though both are slightly slower than the current 6.19 release).

There are still fields in the ImmutableCFOptions that are not CF options but DB options.  Eventually, I would like to move those into an ImmutableOptions (= ImmutableDBOptions+ImmutableCFOptions).  But that will be part of a future PR to minimize changes and disruptions.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/8176

Reviewed By: pdillinger

Differential Revision: D27954339

Pulled By: mrambacher

fbshipit-source-id: ec6b805ba9afe6e094bffdbd76246c2d99aa9fad
main
mrambacher 3 years ago committed by Facebook GitHub Bot
parent f0fca2b1d5
commit 01e460d538
  1. 2
      db/blob/blob_file_builder.cc
  2. 2
      db/blob/blob_file_reader.cc
  3. 4
      db/builder.cc
  4. 2
      db/compaction/compaction_job.cc
  5. 4
      db/db_impl/db_impl.cc
  6. 4
      db/db_impl/db_impl_files.cc
  7. 4
      db/db_iter.cc
  8. 2
      db/internal_stats.cc
  9. 4
      db/memtable.cc
  10. 2
      db/table_cache.cc
  11. 4
      db/version_set.cc
  12. 12
      db/wal_manager.cc
  13. 16
      db/wal_manager_test.cc
  14. 88
      options/cf_options.cc
  15. 63
      options/cf_options.h
  16. 14
      options/db_options.cc
  17. 9
      options/db_options.h
  18. 109
      options/options_helper.cc
  19. 3
      options/options_helper.h
  20. 30
      options/options_test.cc
  21. 4
      table/meta_blocks.cc
  22. 4
      table/meta_blocks.h
  23. 2
      table/sst_file_writer.cc
  24. 10
      table/table_reader_bench.cc
  25. 2
      util/filter_bench.cc
  26. 3
      utilities/memory/memory_test.cc
  27. 2
      utilities/write_batch_with_index/write_batch_with_index_internal.cc

@ -190,7 +190,7 @@ Status BlobFileBuilder::OpenBlobFileIfNeeded() {
std::move(file), blob_file_paths_->back(), *file_options_,
immutable_cf_options_->clock, io_tracer_, statistics,
immutable_cf_options_->listeners,
immutable_cf_options_->file_checksum_gen_factory,
immutable_cf_options_->file_checksum_gen_factory.get(),
tmp_set.Contains(FileType::kBlobFile)));
constexpr bool do_flush = false;

@ -120,7 +120,7 @@ Status BlobFileReader::OpenFile(
file_reader->reset(new RandomAccessFileReader(
std::move(file), blob_file_path, immutable_cf_options.clock, io_tracer,
immutable_cf_options.statistics, BLOB_DB_BLOB_FILE_READ_MICROS,
blob_file_read_hist, immutable_cf_options.rate_limiter,
blob_file_read_hist, immutable_cf_options.rate_limiter.get(),
immutable_cf_options.listeners));
return Status::OK();

@ -156,7 +156,7 @@ Status BuildTable(
file_writer.reset(new WritableFileWriter(
std::move(file), fname, file_options, ioptions.clock, io_tracer,
ioptions.statistics, ioptions.listeners,
ioptions.file_checksum_gen_factory,
ioptions.file_checksum_gen_factory.get(),
tmp_set.Contains(FileType::kTableFile)));
builder = NewTableBuilder(
@ -168,7 +168,7 @@ Status BuildTable(
}
MergeHelper merge(env, internal_comparator.user_comparator(),
ioptions.merge_operator, nullptr, ioptions.info_log,
ioptions.merge_operator.get(), nullptr, ioptions.info_log,
true /* internal key corruption is not ok */,
snapshots.empty() ? 0 : snapshots.back(),
snapshot_checker);

@ -965,7 +965,7 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) {
}
MergeHelper merge(
env_, cfd->user_comparator(), cfd->ioptions()->merge_operator,
env_, cfd->user_comparator(), cfd->ioptions()->merge_operator.get(),
compaction_filter, db_options_.info_log.get(),
false /* internal key corruption is expected */,
existing_snapshots_.empty() ? 0 : existing_snapshots_.back(),

@ -697,8 +697,8 @@ void DBImpl::MaybeIgnoreError(Status* s) const {
}
const Status DBImpl::CreateArchivalDirectory() {
if (immutable_db_options_.wal_ttl_seconds > 0 ||
immutable_db_options_.wal_size_limit_mb > 0) {
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);
}

@ -558,8 +558,8 @@ void DBImpl::PurgeObsoleteFiles(JobContext& state, bool schedule_only) {
}
#ifndef ROCKSDB_LITE
if (type == kWalFile && (immutable_db_options_.wal_ttl_seconds > 0 ||
immutable_db_options_.wal_size_limit_mb > 0)) {
if (type == kWalFile && (immutable_db_options_.WAL_ttl_seconds > 0 ||
immutable_db_options_.WAL_size_limit_MB > 0)) {
wal_manager_.ArchiveWALFile(fname, number);
continue;
}

@ -45,10 +45,10 @@ DBIter::DBIter(Env* _env, const ReadOptions& read_options,
ColumnFamilyData* cfd, bool expose_blob_index)
: prefix_extractor_(mutable_cf_options.prefix_extractor.get()),
env_(_env),
clock_(_env->GetSystemClock().get()),
clock_(cf_options.clock),
logger_(cf_options.info_log),
user_comparator_(cmp),
merge_operator_(cf_options.merge_operator),
merge_operator_(cf_options.merge_operator.get()),
iter_(iter),
version_(version),
read_callback_(read_callback),

@ -961,7 +961,7 @@ bool InternalStats::HandleEstimateOldestKeyTime(uint64_t* value, DBImpl* /*db*/,
bool InternalStats::HandleBlockCacheStat(Cache** block_cache) {
assert(block_cache != nullptr);
auto* table_factory = cfd_->ioptions()->table_factory;
auto* table_factory = cfd_->ioptions()->table_factory.get();
assert(table_factory != nullptr);
*block_cache =
table_factory->GetOptions<Cache>(TableFactory::kBlockCacheOpts());

@ -59,7 +59,7 @@ ImmutableMemTableOptions::ImmutableMemTableOptions(
inplace_callback(ioptions.inplace_callback),
max_successive_merges(mutable_cf_options.max_successive_merges),
statistics(ioptions.statistics),
merge_operator(ioptions.merge_operator),
merge_operator(ioptions.merge_operator.get()),
info_log(ioptions.info_log),
allow_data_in_errors(ioptions.allow_data_in_errors) {}
@ -106,7 +106,7 @@ MemTable::MemTable(const InternalKeyComparator& cmp,
flush_state_(FLUSH_NOT_REQUESTED),
clock_(ioptions.clock),
insert_with_hint_prefix_extractor_(
ioptions.memtable_insert_with_hint_prefix_extractor),
ioptions.memtable_insert_with_hint_prefix_extractor.get()),
oldest_key_time_(std::numeric_limits<uint64_t>::max()),
atomic_flush_seqno_(kMaxSequenceNumber),
approximate_memory_usage_(0) {

@ -130,7 +130,7 @@ Status TableCache::GetTableReader(
new RandomAccessFileReader(
std::move(file), fname, ioptions_.clock, io_tracer_,
record_read_stats ? ioptions_.statistics : nullptr, SST_READ_MICROS,
file_read_hist, ioptions_.rate_limiter, ioptions_.listeners));
file_read_hist, ioptions_.rate_limiter.get(), ioptions_.listeners));
s = ioptions_.table_factory->NewTableReader(
ro,
TableReaderOptions(ioptions_, prefix_extractor, file_options,

@ -1768,8 +1768,8 @@ Version::Version(ColumnFamilyData* column_family_data, VersionSet* vset,
: cfd_->ioptions()->statistics),
table_cache_((cfd_ == nullptr) ? nullptr : cfd_->table_cache()),
blob_file_cache_(cfd_ ? cfd_->blob_file_cache() : nullptr),
merge_operator_((cfd_ == nullptr) ? nullptr
: cfd_->ioptions()->merge_operator),
merge_operator_(
(cfd_ == nullptr) ? nullptr : cfd_->ioptions()->merge_operator.get()),
storage_info_(
(cfd_ == nullptr) ? nullptr : &cfd_->internal_comparator(),
(cfd_ == nullptr) ? nullptr : cfd_->user_comparator(),

@ -134,8 +134,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;
}
@ -150,7 +150,7 @@ void WalManager::PurgeObsoleteWALFiles() {
}
uint64_t const now_seconds = static_cast<uint64_t>(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) {
@ -185,7 +185,7 @@ void WalManager::PurgeObsoleteWALFiles() {
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 = DeleteDBFile(&db_options_, file_path, archival_dir, false,
/*force_fg=*/!wal_in_db_path_);
if (!s.ok()) {
@ -234,8 +234,8 @@ void WalManager::PurgeObsoleteWALFiles() {
return;
}
size_t const files_keep_num =
static_cast<size_t>(db_options_.wal_size_limit_mb * 1024 * 1024 / log_file_size);
size_t const files_keep_num = static_cast<size_t>(
db_options_.WAL_size_limit_MB * 1024 * 1024 / log_file_size);
if (log_files_num <= files_keep_num) {
return;
}

@ -217,8 +217,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.
@ -226,7 +226,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.
@ -238,14 +238,14 @@ TEST_F(WalManagerTest, WALArchivalSizeLimit) {
ListSpecificFiles(env_.get(), archive_dir, kWalFile);
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();
@ -255,7 +255,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.
@ -271,7 +271,7 @@ TEST_F(WalManagerTest, WALArchivalTtl) {
ListSpecificFiles(env_.get(), archive_dir, kWalFile);
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();

@ -38,13 +38,9 @@ namespace ROCKSDB_NAMESPACE {
// http://en.cppreference.com/w/cpp/concept/StandardLayoutType
// https://gist.github.com/graphitemaster/494f21190bb2c63c5516
#ifndef ROCKSDB_LITE
static ColumnFamilyOptions dummy_cf_options;
static ImmutableCFOptions dummy_cf_options;
template <typename T1>
int offset_of(T1 ColumnFamilyOptions::*member) {
return int(size_t(&(dummy_cf_options.*member)) - size_t(&dummy_cf_options));
}
template <typename T1>
int offset_of(T1 AdvancedColumnFamilyOptions::*member) {
int offset_of(T1 ImmutableCFOptions::*member) {
return int(size_t(&(dummy_cf_options.*member)) - size_t(&dummy_cf_options));
}
@ -492,57 +488,58 @@ static std::unordered_map<std::string, OptionTypeInfo>
{0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
OptionTypeFlags::kNone}},
{"inplace_update_support",
{offset_of(&ColumnFamilyOptions::inplace_update_support),
{offset_of(&ImmutableCFOptions::inplace_update_support),
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"level_compaction_dynamic_level_bytes",
{offset_of(&ColumnFamilyOptions::level_compaction_dynamic_level_bytes),
{offset_of(&ImmutableCFOptions::level_compaction_dynamic_level_bytes),
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"optimize_filters_for_hits",
{offset_of(&ColumnFamilyOptions::optimize_filters_for_hits),
{offset_of(&ImmutableCFOptions::optimize_filters_for_hits),
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"force_consistency_checks",
{offset_of(&ColumnFamilyOptions::force_consistency_checks),
{offset_of(&ImmutableCFOptions::force_consistency_checks),
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"purge_redundant_kvs_while_flush",
{offset_of(&ColumnFamilyOptions::purge_redundant_kvs_while_flush),
{offset_of(&ImmutableCFOptions::purge_redundant_kvs_while_flush),
OptionType::kBoolean, OptionVerificationType::kDeprecated,
OptionTypeFlags::kNone}},
{"max_mem_compaction_level",
{0, OptionType::kInt, OptionVerificationType::kDeprecated,
OptionTypeFlags::kNone}},
{"max_write_buffer_number_to_maintain",
{offset_of(&ColumnFamilyOptions::max_write_buffer_number_to_maintain),
{offset_of(&ImmutableCFOptions::max_write_buffer_number_to_maintain),
OptionType::kInt, OptionVerificationType::kNormal,
OptionTypeFlags::kNone, 0}},
{"max_write_buffer_size_to_maintain",
{offset_of(&ColumnFamilyOptions::max_write_buffer_size_to_maintain),
{offset_of(&ImmutableCFOptions::max_write_buffer_size_to_maintain),
OptionType::kInt64T, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"min_write_buffer_number_to_merge",
{offset_of(&ColumnFamilyOptions::min_write_buffer_number_to_merge),
{offset_of(&ImmutableCFOptions::min_write_buffer_number_to_merge),
OptionType::kInt, OptionVerificationType::kNormal,
OptionTypeFlags::kNone, 0}},
{"num_levels",
{offset_of(&ColumnFamilyOptions::num_levels), OptionType::kInt,
{offset_of(&ImmutableCFOptions::num_levels), OptionType::kInt,
OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
{"bloom_locality",
{offset_of(&ColumnFamilyOptions::bloom_locality), OptionType::kUInt32T,
{offset_of(&ImmutableCFOptions::bloom_locality), OptionType::kUInt32T,
OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
{"rate_limit_delay_max_milliseconds",
{0, OptionType::kUInt, OptionVerificationType::kDeprecated,
OptionTypeFlags::kNone}},
{"compression_per_level",
OptionTypeInfo::Vector<CompressionType>(
offset_of(&ColumnFamilyOptions::compression_per_level),
offset_of(&ImmutableCFOptions::compression_per_level),
OptionVerificationType::kNormal, OptionTypeFlags::kNone,
{0, OptionType::kCompressionType})},
{"comparator",
{offset_of(&ColumnFamilyOptions::comparator), OptionType::kComparator,
OptionVerificationType::kByName, OptionTypeFlags::kCompareLoose,
{offset_of(&ImmutableCFOptions::user_comparator),
OptionType::kComparator, OptionVerificationType::kByName,
OptionTypeFlags::kCompareLoose,
// Parses the string and sets the corresponding comparator
[](const ConfigOptions& /*opts*/, const std::string& /*name*/,
const std::string& value, char* addr) {
@ -558,15 +555,15 @@ static std::unordered_map<std::string, OptionTypeInfo>
}}},
{"memtable_insert_with_hint_prefix_extractor",
{offset_of(
&ColumnFamilyOptions::memtable_insert_with_hint_prefix_extractor),
&ImmutableCFOptions::memtable_insert_with_hint_prefix_extractor),
OptionType::kSliceTransform, OptionVerificationType::kByNameAllowNull,
OptionTypeFlags::kNone}},
{"memtable_factory",
{offset_of(&ColumnFamilyOptions::memtable_factory),
{offset_of(&ImmutableCFOptions::memtable_factory),
OptionType::kMemTableRepFactory, OptionVerificationType::kByName,
OptionTypeFlags::kNone}},
{"memtable",
{offset_of(&ColumnFamilyOptions::memtable_factory),
{offset_of(&ImmutableCFOptions::memtable_factory),
OptionType::kMemTableRepFactory, OptionVerificationType::kAlias,
OptionTypeFlags::kNone,
// Parses the value string and updates the memtable_factory
@ -582,13 +579,13 @@ static std::unordered_map<std::string, OptionTypeInfo>
return s;
}}},
{"table_factory", OptionTypeInfo::AsCustomSharedPtr<TableFactory>(
offset_of(&ColumnFamilyOptions::table_factory),
offset_of(&ImmutableCFOptions::table_factory),
OptionVerificationType::kByName,
(OptionTypeFlags::kCompareLoose |
OptionTypeFlags::kStringNameOnly |
OptionTypeFlags::kDontPrepare))},
{"block_based_table_factory",
{offset_of(&ColumnFamilyOptions::table_factory),
{offset_of(&ImmutableCFOptions::table_factory),
OptionType::kCustomizable, OptionVerificationType::kAlias,
OptionTypeFlags::kShared | OptionTypeFlags::kCompareLoose,
// Parses the input value and creates a BlockBasedTableFactory
@ -620,7 +617,7 @@ static std::unordered_map<std::string, OptionTypeInfo>
}
}}},
{"plain_table_factory",
{offset_of(&ColumnFamilyOptions::table_factory),
{offset_of(&ImmutableCFOptions::table_factory),
OptionType::kCustomizable, OptionVerificationType::kAlias,
OptionTypeFlags::kShared | OptionTypeFlags::kCompareLoose,
// Parses the input value and creates a PlainTableFactory
@ -651,15 +648,15 @@ static std::unordered_map<std::string, OptionTypeInfo>
}
}}},
{"compaction_filter",
{offset_of(&ColumnFamilyOptions::compaction_filter),
{offset_of(&ImmutableCFOptions::compaction_filter),
OptionType::kCompactionFilter, OptionVerificationType::kByName,
OptionTypeFlags::kNone}},
{"compaction_filter_factory",
{offset_of(&ColumnFamilyOptions::compaction_filter_factory),
{offset_of(&ImmutableCFOptions::compaction_filter_factory),
OptionType::kCompactionFilterFactory, OptionVerificationType::kByName,
OptionTypeFlags::kNone}},
{"merge_operator",
{offset_of(&ColumnFamilyOptions::merge_operator),
{offset_of(&ImmutableCFOptions::merge_operator),
OptionType::kMergeOperator,
OptionVerificationType::kByNameAllowFromNull,
OptionTypeFlags::kCompareLoose,
@ -677,11 +674,11 @@ static std::unordered_map<std::string, OptionTypeInfo>
return Status::OK();
}}},
{"compaction_style",
{offset_of(&ColumnFamilyOptions::compaction_style),
{offset_of(&ImmutableCFOptions::compaction_style),
OptionType::kCompactionStyle, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"compaction_pri",
{offset_of(&ColumnFamilyOptions::compaction_pri),
{offset_of(&ImmutableCFOptions::compaction_pri),
OptionType::kCompactionPri, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
};
@ -705,11 +702,10 @@ class ConfigurableCFOptions : public ConfigurableMutableCFOptions {
ConfigurableCFOptions(const ColumnFamilyOptions& opts,
const std::unordered_map<std::string, std::string>* map)
: ConfigurableMutableCFOptions(MutableCFOptions(opts)),
immutable_(opts),
immutable_(ImmutableDBOptions(), opts),
cf_options_(opts),
opt_map_(map) {
ConfigurableHelper::RegisterOptions(*this, OptionsHelper::kCFOptionsName,
&immutable_,
ConfigurableHelper::RegisterOptions(*this, &immutable_,
&cf_immutable_options_type_info);
}
@ -774,7 +770,7 @@ class ConfigurableCFOptions : public ConfigurableMutableCFOptions {
}
private:
ColumnFamilyOptions immutable_;
ImmutableCFOptions immutable_;
ColumnFamilyOptions cf_options_;
const std::unordered_map<std::string, std::string>* opt_map_;
};
@ -792,18 +788,22 @@ std::unique_ptr<Configurable> CFOptionsAsConfigurable(
}
#endif // ROCKSDB_LITE
ImmutableCFOptions::ImmutableCFOptions() : ImmutableCFOptions(Options()) {}
ImmutableCFOptions::ImmutableCFOptions(const Options& options)
: ImmutableCFOptions(ImmutableDBOptions(options), options) {}
ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options,
const ColumnFamilyOptions& cf_options)
: compaction_style(cf_options.compaction_style),
: logger(db_options.info_log),
stats(db_options.statistics),
compaction_style(cf_options.compaction_style),
compaction_pri(cf_options.compaction_pri),
user_comparator(cf_options.comparator),
internal_comparator(InternalKeyComparator(cf_options.comparator)),
merge_operator(cf_options.merge_operator.get()),
merge_operator(cf_options.merge_operator),
compaction_filter(cf_options.compaction_filter),
compaction_filter_factory(cf_options.compaction_filter_factory.get()),
compaction_filter_factory(cf_options.compaction_filter_factory),
min_write_buffer_number_to_merge(
cf_options.min_write_buffer_number_to_merge),
max_write_buffer_number_to_maintain(
@ -812,9 +812,9 @@ ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options,
cf_options.max_write_buffer_size_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()),
rate_limiter(db_options.rate_limiter.get()),
info_log(logger.get()),
statistics(stats.get()),
rate_limiter(db_options.rate_limiter),
info_log_level(db_options.info_log_level),
env(db_options.env),
fs(db_options.fs.get()),
@ -822,8 +822,8 @@ ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options,
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()),
memtable_factory(cf_options.memtable_factory),
table_factory(cf_options.table_factory),
table_properties_collector_factories(
cf_options.table_properties_collector_factories),
advise_random_on_open(db_options.advise_random_on_open),
@ -846,10 +846,10 @@ ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options,
listeners(db_options.listeners),
row_cache(db_options.row_cache),
memtable_insert_with_hint_prefix_extractor(
cf_options.memtable_insert_with_hint_prefix_extractor.get()),
cf_options.memtable_insert_with_hint_prefix_extractor),
cf_paths(cf_options.cf_paths),
compaction_thread_limiter(cf_options.compaction_thread_limiter),
file_checksum_gen_factory(db_options.file_checksum_gen_factory.get()),
file_checksum_gen_factory(db_options.file_checksum_gen_factory),
sst_partitioner_factory(cf_options.sst_partitioner_factory),
allow_data_in_errors(db_options.allow_data_in_errors),
db_host_id(db_options.db_host_id),

@ -20,7 +20,12 @@ namespace ROCKSDB_NAMESPACE {
// of DB. Raw pointers defined in this struct do not have ownership to the data
// they point to. Options contains std::shared_ptr to these data.
struct ImmutableCFOptions {
private:
std::shared_ptr<Logger> logger; // ImmutableDBOptions
std::shared_ptr<Statistics> stats; // ImmutableDBOptions
public:
static const char* kName() { return "ImmutableCFOptions"; }
explicit ImmutableCFOptions();
explicit ImmutableCFOptions(const Options& options);
ImmutableCFOptions(const ImmutableDBOptions& db_options,
@ -31,13 +36,13 @@ struct ImmutableCFOptions {
CompactionPri compaction_pri;
const Comparator* user_comparator;
InternalKeyComparator internal_comparator;
InternalKeyComparator internal_comparator; // Only in Immutable
MergeOperator* merge_operator;
std::shared_ptr<MergeOperator> merge_operator;
const CompactionFilter* compaction_filter;
CompactionFilterFactory* compaction_filter_factory;
std::shared_ptr<CompactionFilterFactory> compaction_filter_factory;
int min_write_buffer_number_to_merge;
@ -52,36 +57,36 @@ struct ImmutableCFOptions {
Slice delta_value,
std::string* merged_value);
Logger* info_log;
Logger* info_log; // ImmutableDBOptions
Statistics* statistics;
Statistics* statistics; // ImmutableDBOptions
RateLimiter* rate_limiter;
std::shared_ptr<RateLimiter> rate_limiter; // ImmutableDBOptions
InfoLogLevel info_log_level;
InfoLogLevel info_log_level; // ImmutableDBOptions
Env* env;
Env* env; // ImmutableDBOptions
FileSystem* fs;
FileSystem* fs; // ImmutableDBOptions
SystemClock* clock;
SystemClock* clock; // ImmutableDBOptions
// Allow the OS to mmap file for reading sst tables. Default: false
bool allow_mmap_reads;
bool allow_mmap_reads; // ImmutableDBOptions
// Allow the OS to mmap file for writing. Default: false
bool allow_mmap_writes;
bool allow_mmap_writes; // ImmutableDBOptions
std::vector<DbPath> db_paths;
std::vector<DbPath> db_paths; // ImmutableDBOptions
MemTableRepFactory* memtable_factory;
std::shared_ptr<MemTableRepFactory> memtable_factory;
TableFactory* table_factory;
std::shared_ptr<TableFactory> table_factory;
Options::TablePropertiesCollectorFactories
table_properties_collector_factories;
bool advise_random_on_open;
bool advise_random_on_open; // ImmutableDBOptions
// This options is required by PlainTableReader. May need to move it
// to PlainTableOptions just like bloom_bits_per_key
@ -89,15 +94,15 @@ struct ImmutableCFOptions {
bool purge_redundant_kvs_while_flush;
bool use_fsync;
bool use_fsync; // ImmutableDBOptions
std::vector<CompressionType> compression_per_level;
bool level_compaction_dynamic_level_bytes;
Options::AccessHint access_hint_on_compaction_start;
Options::AccessHint access_hint_on_compaction_start; // ImmutableDBOptions
bool new_table_reader_for_compaction_inputs;
bool new_table_reader_for_compaction_inputs; // ImmutableDBOptions
int num_levels;
@ -105,31 +110,33 @@ struct ImmutableCFOptions {
bool force_consistency_checks;
bool allow_ingest_behind;
bool allow_ingest_behind; // ImmutableDBOptions
bool preserve_deletes;
bool preserve_deletes; // ImmutableDBOptions
// A vector of EventListeners which callback functions will be called
// when specific RocksDB event happens.
std::vector<std::shared_ptr<EventListener>> listeners;
std::vector<std::shared_ptr<EventListener>> listeners; // ImmutableDBOptions
std::shared_ptr<Cache> row_cache;
std::shared_ptr<Cache> row_cache; // ImmutableDBOptions
const SliceTransform* memtable_insert_with_hint_prefix_extractor;
std::shared_ptr<const SliceTransform>
memtable_insert_with_hint_prefix_extractor;
std::vector<DbPath> cf_paths;
std::shared_ptr<ConcurrentTaskLimiter> compaction_thread_limiter;
FileChecksumGenFactory* file_checksum_gen_factory;
std::shared_ptr<FileChecksumGenFactory>
file_checksum_gen_factory; // ImmutableDBOptions
std::shared_ptr<SstPartitionerFactory> sst_partitioner_factory;
bool allow_data_in_errors;
bool allow_data_in_errors; // ImmutableDBOptions
std::string db_host_id;
std::string db_host_id; // ImmutableDBOptions
FileTypeSet checksum_handoff_file_types;
FileTypeSet checksum_handoff_file_types; // ImmutableDBOptions
};
struct MutableCFOptions {

@ -272,11 +272,11 @@ static std::unordered_map<std::string, OptionTypeInfo>
{offsetof(struct ImmutableDBOptions, wal_dir), OptionType::kString,
OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
{"WAL_size_limit_MB",
{offsetof(struct ImmutableDBOptions, 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),
{offsetof(struct ImmutableDBOptions, WAL_ttl_seconds),
OptionType::kUInt64T, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"max_manifest_file_size",
@ -509,7 +509,6 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
track_and_verify_wals_in_manifest(
options.track_and_verify_wals_in_manifest),
env(options.env),
fs(options.env->GetFileSystem()),
rate_limiter(options.rate_limiter),
sst_file_manager(options.sst_file_manager),
info_log(options.info_log),
@ -526,8 +525,8 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
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),
WAL_ttl_seconds(options.WAL_ttl_seconds),
WAL_size_limit_MB(options.WAL_size_limit_MB),
max_write_batch_group_size_bytes(
options.max_write_batch_group_size_bytes),
manifest_preallocation_size(options.manifest_preallocation_size),
@ -583,6 +582,7 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
allow_data_in_errors(options.allow_data_in_errors),
db_host_id(options.db_host_id),
checksum_handoff_file_types(options.checksum_handoff_file_types) {
fs = env->GetFileSystem();
if (env != nullptr) {
clock = env->GetSystemClock().get();
} else {
@ -650,10 +650,10 @@ void ImmutableDBOptions::Dump(Logger* log) const {
table_cache_numshardbits);
ROCKS_LOG_HEADER(log,
" Options.WAL_ttl_seconds: %" PRIu64,
wal_ttl_seconds);
WAL_ttl_seconds);
ROCKS_LOG_HEADER(log,
" Options.WAL_size_limit_MB: %" PRIu64,
wal_size_limit_mb);
WAL_size_limit_MB);
ROCKS_LOG_HEADER(log,
" "
"Options.max_write_batch_group_size_bytes: %" PRIu64,

@ -26,8 +26,6 @@ struct ImmutableDBOptions {
bool paranoid_checks;
bool track_and_verify_wals_in_manifest;
Env* env;
std::shared_ptr<FileSystem> fs;
SystemClock* clock;
std::shared_ptr<RateLimiter> rate_limiter;
std::shared_ptr<SstFileManager> sst_file_manager;
std::shared_ptr<Logger> info_log;
@ -44,8 +42,8 @@ struct ImmutableDBOptions {
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;
uint64_t WAL_ttl_seconds;
uint64_t WAL_size_limit_MB;
uint64_t max_write_batch_group_size_bytes;
size_t manifest_preallocation_size;
bool allow_mmap_reads;
@ -96,6 +94,9 @@ struct ImmutableDBOptions {
bool allow_data_in_errors;
std::string db_host_id;
FileTypeSet checksum_handoff_file_types;
// Convenience/Helper objects that are not part of the base DBOptions
std::shared_ptr<FileSystem> fs;
SystemClock* clock;
};
struct MutableDBOptions {

@ -86,8 +86,8 @@ DBOptions BuildDBOptions(const ImmutableDBOptions& immutable_db_options,
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.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_mmap_reads = immutable_db_options.allow_mmap_reads;
@ -257,6 +257,111 @@ ColumnFamilyOptions BuildColumnFamilyOptions(
return cf_opts;
}
ColumnFamilyOptions BuildColumnFamilyOptions(const ImmutableCFOptions& ioptions,
const MutableCFOptions& moptions) {
ColumnFamilyOptions cf_opts;
// Memtable related options
cf_opts.write_buffer_size = moptions.write_buffer_size;
cf_opts.max_write_buffer_number = moptions.max_write_buffer_number;
cf_opts.arena_block_size = moptions.arena_block_size;
cf_opts.memtable_prefix_bloom_size_ratio =
moptions.memtable_prefix_bloom_size_ratio;
cf_opts.memtable_whole_key_filtering = moptions.memtable_whole_key_filtering;
cf_opts.memtable_huge_page_size = moptions.memtable_huge_page_size;
cf_opts.max_successive_merges = moptions.max_successive_merges;
cf_opts.inplace_update_num_locks = moptions.inplace_update_num_locks;
cf_opts.prefix_extractor = moptions.prefix_extractor;
// Compaction related options
cf_opts.disable_auto_compactions = moptions.disable_auto_compactions;
cf_opts.soft_pending_compaction_bytes_limit =
moptions.soft_pending_compaction_bytes_limit;
cf_opts.hard_pending_compaction_bytes_limit =
moptions.hard_pending_compaction_bytes_limit;
cf_opts.level0_file_num_compaction_trigger =
moptions.level0_file_num_compaction_trigger;
cf_opts.level0_slowdown_writes_trigger =
moptions.level0_slowdown_writes_trigger;
cf_opts.level0_stop_writes_trigger = moptions.level0_stop_writes_trigger;
cf_opts.max_compaction_bytes = moptions.max_compaction_bytes;
cf_opts.target_file_size_base = moptions.target_file_size_base;
cf_opts.target_file_size_multiplier = moptions.target_file_size_multiplier;
cf_opts.max_bytes_for_level_base = moptions.max_bytes_for_level_base;
cf_opts.max_bytes_for_level_multiplier =
moptions.max_bytes_for_level_multiplier;
cf_opts.ttl = moptions.ttl;
cf_opts.periodic_compaction_seconds = moptions.periodic_compaction_seconds;
cf_opts.max_bytes_for_level_multiplier_additional.clear();
for (auto value : moptions.max_bytes_for_level_multiplier_additional) {
cf_opts.max_bytes_for_level_multiplier_additional.emplace_back(value);
}
cf_opts.compaction_options_fifo = moptions.compaction_options_fifo;
cf_opts.compaction_options_universal = moptions.compaction_options_universal;
// Blob file related options
cf_opts.enable_blob_files = moptions.enable_blob_files;
cf_opts.min_blob_size = moptions.min_blob_size;
cf_opts.blob_file_size = moptions.blob_file_size;
cf_opts.blob_compression_type = moptions.blob_compression_type;
cf_opts.enable_blob_garbage_collection =
moptions.enable_blob_garbage_collection;
cf_opts.blob_garbage_collection_age_cutoff =
moptions.blob_garbage_collection_age_cutoff;
// Misc options
cf_opts.max_sequential_skip_in_iterations =
moptions.max_sequential_skip_in_iterations;
cf_opts.check_flush_compaction_key_order =
moptions.check_flush_compaction_key_order;
cf_opts.paranoid_file_checks = moptions.paranoid_file_checks;
cf_opts.report_bg_io_stats = moptions.report_bg_io_stats;
cf_opts.compression = moptions.compression;
cf_opts.compression_opts = moptions.compression_opts;
cf_opts.bottommost_compression = moptions.bottommost_compression;
cf_opts.bottommost_compression_opts = moptions.bottommost_compression_opts;
cf_opts.sample_for_compression = moptions.sample_for_compression;
cf_opts.compaction_style = ioptions.compaction_style;
cf_opts.compaction_pri = ioptions.compaction_pri;
cf_opts.comparator = ioptions.user_comparator;
cf_opts.merge_operator = ioptions.merge_operator;
cf_opts.compaction_filter = ioptions.compaction_filter;
cf_opts.compaction_filter_factory = ioptions.compaction_filter_factory;
cf_opts.min_write_buffer_number_to_merge =
ioptions.min_write_buffer_number_to_merge;
cf_opts.max_write_buffer_number_to_maintain =
ioptions.max_write_buffer_number_to_maintain;
cf_opts.max_write_buffer_size_to_maintain =
ioptions.max_write_buffer_size_to_maintain;
cf_opts.inplace_update_support = ioptions.inplace_update_support;
cf_opts.inplace_callback = ioptions.inplace_callback;
cf_opts.memtable_factory = ioptions.memtable_factory;
cf_opts.table_factory = ioptions.table_factory;
cf_opts.table_properties_collector_factories =
ioptions.table_properties_collector_factories;
cf_opts.bloom_locality = ioptions.bloom_locality;
cf_opts.purge_redundant_kvs_while_flush =
ioptions.purge_redundant_kvs_while_flush;
cf_opts.compression_per_level = ioptions.compression_per_level;
cf_opts.level_compaction_dynamic_level_bytes =
ioptions.level_compaction_dynamic_level_bytes;
cf_opts.num_levels = ioptions.num_levels;
cf_opts.optimize_filters_for_hits = ioptions.optimize_filters_for_hits;
cf_opts.force_consistency_checks = ioptions.force_consistency_checks;
cf_opts.memtable_insert_with_hint_prefix_extractor =
ioptions.memtable_insert_with_hint_prefix_extractor;
cf_opts.cf_paths = ioptions.cf_paths;
cf_opts.compaction_thread_limiter = ioptions.compaction_thread_limiter;
cf_opts.sst_partitioner_factory = ioptions.sst_partitioner_factory;
// TODO(yhchiang): find some way to handle the following derived options
// * max_file_size
return cf_opts;
}
std::map<CompactionStyle, std::string>
OptionsHelper::compaction_style_to_string = {

@ -18,6 +18,7 @@ namespace ROCKSDB_NAMESPACE {
struct ColumnFamilyOptions;
struct ConfigOptions;
struct DBOptions;
struct ImmutableCFOptions;
struct ImmutableDBOptions;
struct MutableDBOptions;
struct MutableCFOptions;
@ -37,6 +38,8 @@ DBOptions BuildDBOptions(const ImmutableDBOptions& immutable_db_options,
ColumnFamilyOptions BuildColumnFamilyOptions(
const ColumnFamilyOptions& ioptions,
const MutableCFOptions& mutable_cf_options);
ColumnFamilyOptions BuildColumnFamilyOptions(const ImmutableCFOptions& ioptions,
const MutableCFOptions& moptions);
#ifndef ROCKSDB_LITE
std::unique_ptr<Configurable> DBOptionsAsConfigurable(

@ -1298,6 +1298,36 @@ TEST_F(OptionsTest, OptionsComposeDecompose) {
delete new_cf_opts.compaction_filter;
}
TEST_F(OptionsTest, DBOptionsComposeImmutable) {
// Build a DBOptions from an Immutable/Mutable one and verify that
// we get same constituent options.
ConfigOptions config_options;
Random rnd(301);
DBOptions base_opts, new_opts;
test::RandomInitDBOptions(&base_opts, &rnd);
MutableDBOptions m_opts(base_opts);
ImmutableDBOptions i_opts(base_opts);
new_opts = BuildDBOptions(i_opts, m_opts);
ASSERT_OK(RocksDBOptionsParser::VerifyDBOptions(config_options, base_opts,
new_opts));
}
TEST_F(OptionsTest, CFOptionsComposeImmutable) {
// Build a DBOptions from an Immutable/Mutable one and verify that
// we get same constituent options.
ConfigOptions config_options;
Random rnd(301);
ColumnFamilyOptions base_opts, new_opts;
DBOptions dummy; // Needed to create ImmutableCFOptions
test::RandomInitCFOptions(&base_opts, dummy, &rnd);
MutableCFOptions m_opts(base_opts);
ImmutableCFOptions i_opts(ImmutableDBOptions(dummy), base_opts);
new_opts = BuildColumnFamilyOptions(i_opts, m_opts);
ASSERT_OK(RocksDBOptionsParser::VerifyCFOptions(config_options, base_opts,
new_opts));
delete new_opts.compaction_filter;
}
TEST_F(OptionsTest, ColumnFamilyOptionsSerialization) {
Options options;
ColumnFamilyOptions base_opt, new_opt;

@ -152,8 +152,8 @@ Slice PropertyBlockBuilder::Finish() {
return properties_block_->Finish();
}
void LogPropertiesCollectionError(
Logger* info_log, const std::string& method, const std::string& name) {
void LogPropertiesCollectionError(Logger* info_log, const std::string& method,
const std::string& name) {
assert(method == "Add" || method == "Finish");
std::string msg =

@ -70,8 +70,8 @@ class PropertyBlockBuilder {
// Were we encounter any error occurs during user-defined statistics collection,
// we'll write the warning message to info log.
void LogPropertiesCollectionError(
Logger* info_log, const std::string& method, const std::string& name);
void LogPropertiesCollectionError(Logger* info_log, const std::string& method,
const std::string& name);
// Utility functions help table builder to trigger batch events for user
// defined property collectors.

@ -260,7 +260,7 @@ Status SstFileWriter::Open(const std::string& file_path) {
r->file_writer.reset(new WritableFileWriter(
std::move(sst_file), file_path, r->env_options, r->ioptions.clock,
nullptr /* io_tracer */, nullptr /* stats */, r->ioptions.listeners,
r->ioptions.file_checksum_gen_factory,
r->ioptions.file_checksum_gen_factory.get(),
tmp_set.Contains(FileType::kTableFile)));
// TODO(tec) : If table_factory is using compressed block cache, we will

@ -175,11 +175,11 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
PinnableSlice value;
MergeContext merge_context;
SequenceNumber max_covering_tombstone_seq = 0;
GetContext get_context(ioptions.user_comparator,
ioptions.merge_operator, ioptions.info_log,
ioptions.statistics, GetContext::kNotFound,
Slice(key), &value, nullptr, &merge_context,
true, &max_covering_tombstone_seq, clock);
GetContext get_context(
ioptions.user_comparator, ioptions.merge_operator.get(),
ioptions.info_log, ioptions.statistics, GetContext::kNotFound,
Slice(key), &value, nullptr, &merge_context, true,
&max_covering_tombstone_seq, clock);
s = table_reader->Get(read_options, key, &get_context, nullptr);
} else {
s = db->Get(read_options, key, &result);

@ -270,8 +270,8 @@ struct FilterBench : public MockBlockBasedTableTester {
Random32 random_;
std::ostringstream fp_rate_report_;
Arena arena_;
StderrLogger stderr_logger_;
double m_queries_;
StderrLogger stderr_logger_;
FilterBench()
: MockBlockBasedTableTester(new BloomFilterPolicy(

@ -68,7 +68,8 @@ class MemoryTest : public testing::Test {
ASSERT_OK(db_impl->TEST_GetAllImmutableCFOptions(&iopts_map));
}
for (auto pair : iopts_map) {
GetCachePointersFromTableFactory(pair.second->table_factory, cache_set);
GetCachePointersFromTableFactory(pair.second->table_factory.get(),
cache_set);
}
}
}

@ -199,7 +199,7 @@ Status WriteBatchWithIndexInternal::MergeKey(const Slice& key,
Slice* result_operand) {
if (column_family_ != nullptr) {
auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(column_family_);
const auto merge_operator = cfh->cfd()->ioptions()->merge_operator;
const auto merge_operator = cfh->cfd()->ioptions()->merge_operator.get();
if (merge_operator == nullptr) {
return Status::InvalidArgument(
"Merge_operator must be set for column_family");

Loading…
Cancel
Save