SetOptions() for memtable related options

Summary: as title

Test Plan:
make all check
I will think a way to set up stress test for this

Reviewers: sdong, yhchiang, igor

Reviewed By: igor

Subscribers: leveldb

Differential Revision: https://reviews.facebook.net/D23055
main
Lei Jin 10 years ago
parent e4eca6a1e5
commit a062e1f2c4
  1. 32
      db/column_family.cc
  2. 25
      db/column_family.h
  3. 31
      db/db_impl.cc
  4. 4
      db/db_impl.h
  5. 18
      db/memtable.cc
  6. 5
      db/memtable.h
  7. 3
      db/repair.cc
  8. 10
      db/version_set.cc
  9. 2
      db/write_batch_test.cc
  10. 8
      include/rocksdb/db.h
  11. 12
      table/table_test.cc
  12. 41
      util/mutable_cf_options.h
  13. 66
      util/options_helper.cc
  14. 18
      util/options_helper.h

@ -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<std::string, std::string>& 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,

@ -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<uint32_t> 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<std::string, std::string>& 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<TableCache> table_cache_;

@ -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<uint64_t>& 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<uint64_t>& 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<std::string, std::string>& options_map) {
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(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(

@ -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<std::string, std::string>& options_map);
using DB::NumberLevels;
virtual int NumberLevels(ColumnFamilyHandle* column_family);
using DB::MaxMemCompactionLevel;

@ -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,

@ -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;

@ -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;

@ -2962,17 +2962,21 @@ void VersionSet::GetObsoleteFiles(std::vector<FileMetaData*>* 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;
}

@ -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);

@ -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<std::string, std::string>& new_options) {
return true;
}
virtual bool SetOptions(
const std::unordered_map<std::string, std::string>& new_options) {
return SetOptions(DefaultColumnFamily(), new_options);
}
// Number of levels used for this DB.
virtual int NumberLevels(ColumnFamilyHandle* column_family) = 0;

@ -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);

@ -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

@ -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 <cassert>
#include "rocksdb/options.h"
#include "util/options_helper.h"
namespace rocksdb {
@ -75,6 +72,49 @@ CompactionStyle ParseCompactionStyle(const std::string& type) {
}
} // anonymouse namespace
template<typename OptionsType>
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<std::string, std::string>& 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<std::string, std::string>& 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") {

@ -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 <string>
#include "util/mutable_cf_options.h"
namespace rocksdb {
bool GetMutableOptionsFromStrings(
const MutableCFOptions& base_options,
const std::unordered_map<std::string, std::string>& options_map,
MutableCFOptions* new_options);
} // namespace rocksdb
Loading…
Cancel
Save