diff --git a/db/column_family.cc b/db/column_family.cc index c8ea7accf..ff6b8fe6c 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -27,6 +27,7 @@ #include "db/write_controller.h" #include "util/autovector.h" #include "util/hash_skiplist_rep.h" +#include "util/options_helper.h" namespace rocksdb { @@ -212,7 +213,7 @@ void SuperVersionUnrefHandle(void* ptr) { ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name, Version* dummy_versions, Cache* table_cache, - const ColumnFamilyOptions& options, + const ColumnFamilyOptions& cf_options, const DBOptions* db_options, const EnvOptions& env_options, ColumnFamilySet* column_family_set) @@ -222,9 +223,10 @@ ColumnFamilyData::ColumnFamilyData(uint32_t id, const std::string& name, current_(nullptr), refs_(0), dropped_(false), - internal_comparator_(options.comparator), - options_(*db_options, SanitizeOptions(&internal_comparator_, options)), + internal_comparator_(cf_options.comparator), + options_(*db_options, SanitizeOptions(&internal_comparator_, cf_options)), ioptions_(options_), + mutable_cf_options_(options_), mem_(nullptr), imm_(options_.min_write_buffer_number_to_merge), super_version_(nullptr), @@ -378,13 +380,12 @@ const EnvOptions* ColumnFamilyData::soptions() const { void ColumnFamilyData::SetCurrent(Version* current) { current_ = current; } -void ColumnFamilyData::CreateNewMemtable() { +void ColumnFamilyData::CreateNewMemtable(const MemTableOptions& moptions) { assert(current_ != nullptr); if (mem_ != nullptr) { delete mem_->Unref(); } - mem_ = new MemTable(internal_comparator_, ioptions_, - MemTableOptions(options_)); + mem_ = new MemTable(internal_comparator_, ioptions_, moptions); mem_->Ref(); } @@ -486,7 +487,15 @@ bool ColumnFamilyData::ReturnThreadLocalSuperVersion(SuperVersion* sv) { SuperVersion* ColumnFamilyData::InstallSuperVersion( SuperVersion* new_superversion, port::Mutex* db_mutex) { + db_mutex->AssertHeld(); + return InstallSuperVersion(new_superversion, db_mutex, mutable_cf_options_); +} + +SuperVersion* ColumnFamilyData::InstallSuperVersion( + SuperVersion* new_superversion, port::Mutex* db_mutex, + const MutableCFOptions& mutable_cf_options) { new_superversion->db_mutex = db_mutex; + new_superversion->mutable_cf_options = mutable_cf_options; new_superversion->Init(mem_, imm_.current(), current_); SuperVersion* old_superversion = super_version_; super_version_ = new_superversion; @@ -522,6 +531,17 @@ void ColumnFamilyData::ResetThreadLocalSuperVersions() { } } +bool ColumnFamilyData::SetOptions( + const std::unordered_map& options_map) { + MutableCFOptions new_mutable_cf_options; + if (GetMutableOptionsFromStrings(mutable_cf_options_, options_map, + &new_mutable_cf_options)) { + mutable_cf_options_ = new_mutable_cf_options; + return true; + } + return false; +} + ColumnFamilySet::ColumnFamilySet(const std::string& dbname, const DBOptions* db_options, const EnvOptions& env_options, diff --git a/db/column_family.h b/db/column_family.h index e7b21036f..f1ef13cf1 100644 --- a/db/column_family.h +++ b/db/column_family.h @@ -23,6 +23,7 @@ #include "db/table_cache.h" #include "util/thread_local.h" #include "db/flush_scheduler.h" +#include "util/mutable_cf_options.h" namespace rocksdb { @@ -80,6 +81,7 @@ struct SuperVersion { MemTable* mem; MemTableListVersion* imm; Version* current; + MutableCFOptions mutable_cf_options; std::atomic refs; // We need to_delete because during Cleanup(), imm->Unref() returns // all memtables that we need to free through this vector. We then @@ -168,11 +170,24 @@ class ColumnFamilyData { void SetLogNumber(uint64_t log_number) { log_number_ = log_number; } uint64_t GetLogNumber() const { return log_number_; } - // TODO(ljin): make this API thread-safe once we allow updating options_ - const Options* options() const { return &options_; } // thread-safe + const Options* options() const { return &options_; } const EnvOptions* soptions() const; const ImmutableCFOptions* ioptions() const { return &ioptions_; } + // REQUIRES: DB mutex held + // This returns the MutableCFOptions used by current SuperVersion + // You shoul use this API to reference MutableCFOptions most of the time. + const MutableCFOptions* mutable_cf_options() const { + return &(super_version_->mutable_cf_options); + } + // REQUIRES: DB mutex held + // This returns the latest MutableCFOptions, which may be not in effect yet. + const MutableCFOptions* GetLatestMutableCFOptions() const { + return &mutable_cf_options_; + } + // REQUIRES: DB mutex held + bool SetOptions( + const std::unordered_map& options_map); InternalStats* internal_stats() { return internal_stats_.get(); } @@ -182,7 +197,7 @@ class ColumnFamilyData { Version* dummy_versions() { return dummy_versions_; } void SetMemtable(MemTable* new_mem) { mem_ = new_mem; } void SetCurrent(Version* current); - void CreateNewMemtable(); + void CreateNewMemtable(const MemTableOptions& moptions); TableCache* table_cache() const { return table_cache_.get(); } @@ -223,6 +238,9 @@ class ColumnFamilyData { // if its reference count is zero and needs deletion or nullptr if not // As argument takes a pointer to allocated SuperVersion to enable // the clients to allocate SuperVersion outside of mutex. + SuperVersion* InstallSuperVersion(SuperVersion* new_superversion, + port::Mutex* db_mutex, + const MutableCFOptions& mutable_cf_options); SuperVersion* InstallSuperVersion(SuperVersion* new_superversion, port::Mutex* db_mutex); @@ -255,6 +273,7 @@ class ColumnFamilyData { const Options options_; const ImmutableCFOptions ioptions_; + MutableCFOptions mutable_cf_options_; std::unique_ptr table_cache_; diff --git a/db/db_impl.cc b/db/db_impl.cc index 0b332b72f..addce91c2 100644 --- a/db/db_impl.cc +++ b/db/db_impl.cc @@ -1228,7 +1228,8 @@ Status DBImpl::Recover( if (!s.ok()) { // Clear memtables if recovery failed for (auto cfd : *versions_->GetColumnFamilySet()) { - cfd->CreateNewMemtable(); + cfd->CreateNewMemtable(MemTableOptions( + *cfd->GetLatestMutableCFOptions(), *cfd->options())); } } } @@ -1356,7 +1357,8 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, // file-systems cause the DB::Open() to fail. return status; } - cfd->CreateNewMemtable(); + cfd->CreateNewMemtable(MemTableOptions( + *cfd->GetLatestMutableCFOptions(), *cfd->options())); } } } @@ -1393,7 +1395,8 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, // Recovery failed break; } - cfd->CreateNewMemtable(); + cfd->CreateNewMemtable(MemTableOptions( + *cfd->GetLatestMutableCFOptions(), *cfd->options())); } // write MANIFEST with update @@ -1623,6 +1626,7 @@ Status DBImpl::FlushMemTableToOutputFile(ColumnFamilyData* cfd, } if (s.ok()) { + // Use latest MutableCFOptions InstallSuperVersion(cfd, deletion_state); if (madeProgress) { *madeProgress = 1; @@ -1714,6 +1718,13 @@ Status DBImpl::CompactRange(ColumnFamilyHandle* column_family, return s; } +bool DBImpl::SetOptions(ColumnFamilyHandle* column_family, + const std::unordered_map& options_map) { + auto cfh = reinterpret_cast(column_family); + MutexLock l(&mutex_); + return cfh->cfd()->SetOptions(options_map); +} + // return the same level if it cannot be moved int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd, int level) { mutex_.AssertHeld(); @@ -1784,6 +1795,7 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { cfd->GetName().c_str(), edit.DebugString().data()); status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get()); + // Use latest MutableCFOptions superversion_to_free = cfd->InstallSuperVersion(new_superversion, &mutex_); new_superversion = nullptr; @@ -2322,6 +2334,7 @@ Status DBImpl::BackgroundCompaction(bool* madeProgress, } status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_, db_directory_.get()); + // Use latest MutableCFOptions InstallSuperVersion(c->column_family_data(), deletion_state); LogToBuffer(log_buffer, "[%s] Deleted %d files\n", c->column_family_data()->GetName().c_str(), @@ -2338,6 +2351,7 @@ Status DBImpl::BackgroundCompaction(bool* madeProgress, f->smallest_seqno, f->largest_seqno); status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_, db_directory_.get()); + // Use latest MutableCFOptions InstallSuperVersion(c->column_family_data(), deletion_state); Version::LevelSummaryStorage tmp; @@ -3322,6 +3336,7 @@ Status DBImpl::DoCompactionWork(CompactionState* compact, if (status.ok()) { status = InstallCompactionResults(compact, log_buffer); + // Use latest MutableCFOptions InstallSuperVersion(cfd, deletion_state); } Version::LevelSummaryStorage tmp; @@ -3426,6 +3441,7 @@ void DBImpl::InstallSuperVersion(ColumnFamilyData* cfd, SuperVersion* new_superversion = (deletion_state.new_superversion != nullptr) ? deletion_state.new_superversion : new SuperVersion(); + // Use latest MutableCFOptions SuperVersion* old_superversion = cfd->InstallSuperVersion(new_superversion, &mutex_); deletion_state.new_superversion = nullptr; @@ -3618,6 +3634,7 @@ Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& options, auto cfd = versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name); assert(cfd != nullptr); + // Use latest MutableCFOptions delete cfd->InstallSuperVersion(new SuperVersion(), &mutex_); *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_); Log(db_options_.info_log, "Created column family [%s] (ID %u)", @@ -4138,6 +4155,7 @@ Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd, uint64_t new_log_number = creating_new_log ? versions_->NewFileNumber() : logfile_number_; SuperVersion* new_superversion = nullptr; + const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions(); mutex_.Unlock(); Status s; { @@ -4156,8 +4174,8 @@ Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd, if (s.ok()) { new_mem = new MemTable(cfd->internal_comparator(), - *cfd->ioptions(), - MemTableOptions(*cfd->options())); + *cfd->ioptions(), MemTableOptions(mutable_cf_options, + *cfd->options())); new_superversion = new SuperVersion(); } } @@ -4197,7 +4215,7 @@ Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd, "[%s] New memtable created with log file: #%" PRIu64 "\n", cfd->GetName().c_str(), logfile_number_); context->superversions_to_free_.push_back( - cfd->InstallSuperVersion(new_superversion, &mutex_)); + cfd->InstallSuperVersion(new_superversion, &mutex_, mutable_cf_options)); return s; } @@ -4672,6 +4690,7 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, } if (s.ok()) { for (auto cfd : *impl->versions_->GetColumnFamilySet()) { + // Use latest MutableCFOptions delete cfd->InstallSuperVersion(new SuperVersion(), &impl->mutex_); } impl->alive_log_files_.push_back( diff --git a/db/db_impl.h b/db/db_impl.h index d2b0dfc94..0bc2018b4 100644 --- a/db/db_impl.h +++ b/db/db_impl.h @@ -112,6 +112,10 @@ class DBImpl : public DB { bool reduce_level = false, int target_level = -1, uint32_t target_path_id = 0); + using DB::SetOptions; + bool SetOptions(ColumnFamilyHandle* column_family, + const std::unordered_map& options_map); + using DB::NumberLevels; virtual int NumberLevels(ColumnFamilyHandle* column_family); using DB::MaxMemCompactionLevel; diff --git a/db/memtable.cc b/db/memtable.cc index 804404bb8..bdfbc805f 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -31,18 +31,20 @@ namespace rocksdb { -MemTableOptions::MemTableOptions(const Options& options) - : write_buffer_size(options.write_buffer_size), - arena_block_size(options.arena_block_size), - memtable_prefix_bloom_bits(options.memtable_prefix_bloom_bits), - memtable_prefix_bloom_probes(options.memtable_prefix_bloom_probes), +MemTableOptions::MemTableOptions( + const MutableCFOptions& mutable_cf_options, const Options& options) + : write_buffer_size(mutable_cf_options.write_buffer_size), + arena_block_size(mutable_cf_options.arena_block_size), + memtable_prefix_bloom_bits(mutable_cf_options.memtable_prefix_bloom_bits), + memtable_prefix_bloom_probes( + mutable_cf_options.memtable_prefix_bloom_probes), memtable_prefix_bloom_huge_page_tlb_size( - options.memtable_prefix_bloom_huge_page_tlb_size), + mutable_cf_options.memtable_prefix_bloom_huge_page_tlb_size), inplace_update_support(options.inplace_update_support), inplace_update_num_locks(options.inplace_update_num_locks), inplace_callback(options.inplace_callback), - max_successive_merges(options.max_successive_merges), - filter_deletes(options.filter_deletes) {} + max_successive_merges(mutable_cf_options.max_successive_merges), + filter_deletes(mutable_cf_options.filter_deletes) {} MemTable::MemTable(const InternalKeyComparator& cmp, const ImmutableCFOptions& ioptions, diff --git a/db/memtable.h b/db/memtable.h index fa6db6fe1..ce6cce7f6 100644 --- a/db/memtable.h +++ b/db/memtable.h @@ -21,6 +21,7 @@ #include "rocksdb/immutable_options.h" #include "util/arena.h" #include "util/dynamic_bloom.h" +#include "util/mutable_cf_options.h" namespace rocksdb { @@ -30,7 +31,9 @@ class MemTableIterator; class MergeContext; struct MemTableOptions { - explicit MemTableOptions(const Options& options); + explicit MemTableOptions( + const MutableCFOptions& mutable_cf_options, + const Options& options); size_t write_buffer_size; size_t arena_block_size; uint32_t memtable_prefix_bloom_bits; diff --git a/db/repair.cc b/db/repair.cc index bff81991e..2773d4c71 100644 --- a/db/repair.cc +++ b/db/repair.cc @@ -219,7 +219,8 @@ class Repairer { std::string scratch; Slice record; WriteBatch batch; - MemTable* mem = new MemTable(icmp_, ioptions_, MemTableOptions(options_)); + MemTable* mem = new MemTable(icmp_, ioptions_, + MemTableOptions(MutableCFOptions(options_), options_)); auto cf_mems_default = new ColumnFamilyMemTablesDefault(mem, &options_); mem->Ref(); int counter = 0; diff --git a/db/version_set.cc b/db/version_set.cc index dbf055ce2..7edfaa788 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -2962,17 +2962,21 @@ void VersionSet::GetObsoleteFiles(std::vector* files) { } ColumnFamilyData* VersionSet::CreateColumnFamily( - const ColumnFamilyOptions& options, VersionEdit* edit) { + const ColumnFamilyOptions& cf_options, VersionEdit* edit) { assert(edit->is_column_family_add_); Version* dummy_versions = new Version(nullptr, this); auto new_cfd = column_family_set_->CreateColumnFamily( - edit->column_family_name_, edit->column_family_, dummy_versions, options); + edit->column_family_name_, edit->column_family_, dummy_versions, + cf_options); Version* v = new Version(new_cfd, this, current_version_number_++); AppendVersion(new_cfd, v); - new_cfd->CreateNewMemtable(); + // GetLatestMutableCFOptions() is safe here without mutex since the + // cfd is not available to client + new_cfd->CreateNewMemtable(MemTableOptions( + *new_cfd->GetLatestMutableCFOptions(), *new_cfd->options())); new_cfd->SetLogNumber(edit->log_number_); return new_cfd; } diff --git a/db/write_batch_test.cc b/db/write_batch_test.cc index 0c69b6af9..d8fa52d40 100644 --- a/db/write_batch_test.cc +++ b/db/write_batch_test.cc @@ -28,7 +28,7 @@ static std::string PrintContents(WriteBatch* b) { Options options; options.memtable_factory = factory; MemTable* mem = new MemTable(cmp, ImmutableCFOptions(options), - MemTableOptions(options)); + MemTableOptions(MutableCFOptions(options), options)); mem->Ref(); std::string state; ColumnFamilyMemTablesDefault cf_mems_default(mem, &options); diff --git a/include/rocksdb/db.h b/include/rocksdb/db.h index 47372c42b..0653a8386 100644 --- a/include/rocksdb/db.h +++ b/include/rocksdb/db.h @@ -359,6 +359,14 @@ class DB { return CompactRange(DefaultColumnFamily(), begin, end, reduce_level, target_level, target_path_id); } + virtual bool SetOptions(ColumnFamilyHandle* column_family, + const std::unordered_map& new_options) { + return true; + } + virtual bool SetOptions( + const std::unordered_map& new_options) { + return SetOptions(DefaultColumnFamily(), new_options); + } // Number of levels used for this DB. virtual int NumberLevels(ColumnFamilyHandle* column_family) = 0; diff --git a/table/table_test.cc b/table/table_test.cc index 118291daa..ed8123352 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -437,9 +437,8 @@ class MemTableConstructor: public Constructor { table_factory_(new SkipListFactory) { Options options; options.memtable_factory = table_factory_; - memtable_ = new MemTable(internal_comparator_, - ImmutableCFOptions(options), - MemTableOptions(options)); + memtable_ = new MemTable(internal_comparator_, ImmutableCFOptions(options), + MemTableOptions(MutableCFOptions(options), options)); memtable_->Ref(); } ~MemTableConstructor() { @@ -453,9 +452,8 @@ class MemTableConstructor: public Constructor { delete memtable_->Unref(); Options options; options.memtable_factory = table_factory_; - memtable_ = new MemTable(internal_comparator_, - ImmutableCFOptions(options), - MemTableOptions(options)); + memtable_ = new MemTable(internal_comparator_, ImmutableCFOptions(options), + MemTableOptions(MutableCFOptions(options), options)); memtable_->Ref(); int seq = 1; for (KVMap::const_iterator it = data.begin(); @@ -1864,7 +1862,7 @@ TEST(MemTableTest, Simple) { Options options; options.memtable_factory = table_factory; MemTable* memtable = new MemTable(cmp, ImmutableCFOptions(options), - MemTableOptions(options)); + MemTableOptions(MutableCFOptions(options), options)); memtable->Ref(); WriteBatch batch; WriteBatchInternal::SetSequence(&batch, 100); diff --git a/util/mutable_cf_options.h b/util/mutable_cf_options.h new file mode 100644 index 000000000..39ebe2d85 --- /dev/null +++ b/util/mutable_cf_options.h @@ -0,0 +1,41 @@ +// Copyright (c) 2014, 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 "rocksdb/options.h" + +namespace rocksdb { + +struct MutableCFOptions { + explicit MutableCFOptions(const Options& options) + : write_buffer_size(options.write_buffer_size), + arena_block_size(options.arena_block_size), + memtable_prefix_bloom_bits(options.memtable_prefix_bloom_bits), + memtable_prefix_bloom_probes(options.memtable_prefix_bloom_probes), + memtable_prefix_bloom_huge_page_tlb_size( + options.memtable_prefix_bloom_huge_page_tlb_size), + max_successive_merges(options.max_successive_merges), + filter_deletes(options.filter_deletes) { + } + MutableCFOptions() + : write_buffer_size(0), + arena_block_size(0), + memtable_prefix_bloom_bits(0), + memtable_prefix_bloom_probes(0), + memtable_prefix_bloom_huge_page_tlb_size(0), + max_successive_merges(0), + filter_deletes(false) {} + + size_t write_buffer_size; + size_t arena_block_size; + uint32_t memtable_prefix_bloom_bits; + uint32_t memtable_prefix_bloom_probes; + size_t memtable_prefix_bloom_huge_page_tlb_size; + size_t max_successive_merges; + bool filter_deletes; +}; + +} // namespace rocksdb diff --git a/util/options_helper.cc b/util/options_helper.cc index a4d46ccb0..db066f747 100644 --- a/util/options_helper.cc +++ b/util/options_helper.cc @@ -1,14 +1,11 @@ -// Copyright (c) 2013, Facebook, Inc. All rights reserved. +// Copyright (c) 2014, 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. -// -// Copyright (c) 2011 The LevelDB Authors. All rights reserved. -// Use of this source code is governed by a BSD-style license that can be -// found in the LICENSE file. See the AUTHORS file for names of contributors. #include #include "rocksdb/options.h" +#include "util/options_helper.h" namespace rocksdb { @@ -75,6 +72,49 @@ CompactionStyle ParseCompactionStyle(const std::string& type) { } } // anonymouse namespace +template +bool ParseMemtableOption(const std::string& name, const std::string& value, + OptionsType* new_options) { + if (name == "write_buffer_size") { + new_options->write_buffer_size = ParseInt64(value); + } else if (name == "arena_block_size") { + new_options->arena_block_size = ParseInt64(value); + } else if (name == "memtable_prefix_bloom_bits") { + new_options->memtable_prefix_bloom_bits = stoul(value); + } else if (name == "memtable_prefix_bloom_probes") { + new_options->memtable_prefix_bloom_probes = stoul(value); + } else if (name == "memtable_prefix_bloom_huge_page_tlb_size") { + new_options->memtable_prefix_bloom_huge_page_tlb_size = + ParseInt64(value); + } else if (name == "max_successive_merges") { + new_options->max_successive_merges = ParseInt64(value); + } else if (name == "filter_deletes") { + new_options->filter_deletes = ParseBoolean(name, value); + } else { + return false; + } + return true; +} + +bool GetMutableOptionsFromStrings( + const MutableCFOptions& base_options, + const std::unordered_map& options_map, + MutableCFOptions* new_options) { + assert(new_options); + *new_options = base_options; + try { + for (const auto& o : options_map) { + if (ParseMemtableOption(o.first, o.second, new_options)) { + } else { + return false; + } + } + } catch (std::exception) { + return false; + } + return true; +} + bool GetOptionsFromStrings( const Options& base_options, const std::unordered_map& options_map, @@ -83,8 +123,7 @@ bool GetOptionsFromStrings( *new_options = base_options; for (const auto& o : options_map) { try { - if (o.first == "write_buffer_size") { - new_options->write_buffer_size = ParseInt64(o.second); + if (ParseMemtableOption(o.first, o.second, new_options)) { } else if (o.first == "max_write_buffer_number") { new_options->max_write_buffer_number = ParseInt(o.second); } else if (o.first == "min_write_buffer_number_to_merge") { @@ -170,8 +209,6 @@ bool GetOptionsFromStrings( new_options->soft_rate_limit = ParseDouble(o.second); } else if (o.first == "hard_rate_limit") { new_options->hard_rate_limit = ParseDouble(o.second); - } else if (o.first == "arena_block_size") { - new_options->arena_block_size = ParseInt64(o.second); } else if (o.first == "disable_auto_compactions") { new_options->disable_auto_compactions = ParseBoolean(o.first, o.second); } else if (o.first == "purge_redundant_kvs_while_flush") { @@ -188,25 +225,14 @@ bool GetOptionsFromStrings( } else if (o.first == "compaction_options_fifo") { new_options->compaction_options_fifo.max_table_files_size = ParseUint64(o.second); - } else if (o.first == "filter_deletes") { - new_options->filter_deletes = ParseBoolean(o.first, o.second); } else if (o.first == "max_sequential_skip_in_iterations") { new_options->max_sequential_skip_in_iterations = ParseUint64(o.second); } else if (o.first == "inplace_update_support") { new_options->inplace_update_support = ParseBoolean(o.first, o.second); } else if (o.first == "inplace_update_num_locks") { new_options->inplace_update_num_locks = ParseInt64(o.second); - } else if (o.first == "memtable_prefix_bloom_bits") { - new_options->memtable_prefix_bloom_bits = stoul(o.second); - } else if (o.first == "memtable_prefix_bloom_probes") { - new_options->memtable_prefix_bloom_probes = stoul(o.second); - } else if (o.first == "memtable_prefix_bloom_huge_page_tlb_size") { - new_options->memtable_prefix_bloom_huge_page_tlb_size = - ParseInt64(o.second); } else if (o.first == "bloom_locality") { new_options->bloom_locality = ParseUint32(o.second); - } else if (o.first == "max_successive_merges") { - new_options->max_successive_merges = ParseInt64(o.second); } else if (o.first == "min_partial_merge_operands") { new_options->min_partial_merge_operands = ParseUint32(o.second); } else if (o.first == "create_if_missing") { diff --git a/util/options_helper.h b/util/options_helper.h new file mode 100644 index 000000000..c04d2a5d7 --- /dev/null +++ b/util/options_helper.h @@ -0,0 +1,18 @@ +// Copyright (c) 2014, 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 "util/mutable_cf_options.h" + +namespace rocksdb { + +bool GetMutableOptionsFromStrings( + const MutableCFOptions& base_options, + const std::unordered_map& options_map, + MutableCFOptions* new_options); + +} // namespace rocksdb