You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/options/options.cc

621 lines
26 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
// This source code is also licensed under the GPLv2 license found in the
// COPYING file in the root directory of this source tree.
//
// 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 "rocksdb/options.h"
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <inttypes.h>
#include <limits>
#include "monitoring/statistics.h"
#include "options/db_options.h"
#include "options/options_helper.h"
#include "rocksdb/cache.h"
#include "rocksdb/compaction_filter.h"
#include "rocksdb/comparator.h"
#include "rocksdb/env.h"
#include "rocksdb/memtablerep.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "rocksdb/sst_file_manager.h"
#include "rocksdb/table.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/wal_filter.h"
#include "table/block_based_table_factory.h"
#include "util/compression.h"
namespace rocksdb {
AdvancedColumnFamilyOptions::AdvancedColumnFamilyOptions() {
assert(memtable_factory.get() != nullptr);
}
AdvancedColumnFamilyOptions::AdvancedColumnFamilyOptions(const Options& options)
: max_write_buffer_number(options.max_write_buffer_number),
min_write_buffer_number_to_merge(
options.min_write_buffer_number_to_merge),
Support saving history in memtable_list Summary: For transactions, we are using the memtables to validate that there are no write conflicts. But after flushing, we don't have any memtables, and transactions could fail to commit. So we want to someone keep around some extra history to use for conflict checking. In addition, we want to provide a way to increase the size of this history if too many transactions fail to commit. After chatting with people, it seems like everyone prefers just using Memtables to store this history (instead of a separate history structure). It seems like the best place for this is abstracted inside the memtable_list. I decide to create a separate list in MemtableListVersion as using the same list complicated the flush/installalflushresults logic too much. This diff adds a new parameter to control how much memtable history to keep around after flushing. However, it sounds like people aren't too fond of adding new parameters. So I am making the default size of flushed+not-flushed memtables be set to max_write_buffers. This should not change the maximum amount of memory used, but make it more likely we're using closer the the limit. (We are now postponing deleting flushed memtables until the max_write_buffer limit is reached). So while we might use more memory on average, we are still obeying the limit set (and you could argue it's better to go ahead and use up memory now instead of waiting for a write stall to happen to test this limit). However, if people are opposed to this default behavior, we can easily set it to 0 and require this parameter be set in order to use transactions. Test Plan: Added a xfunc test to play around with setting different values of this parameter in all tests. Added testing in memtablelist_test and planning on adding more testing here. Reviewers: sdong, rven, igor Reviewed By: igor Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D37443
9 years ago
max_write_buffer_number_to_maintain(
options.max_write_buffer_number_to_maintain),
inplace_update_support(options.inplace_update_support),
inplace_update_num_locks(options.inplace_update_num_locks),
inplace_callback(options.inplace_callback),
memtable_prefix_bloom_size_ratio(
options.memtable_prefix_bloom_size_ratio),
memtable_huge_page_size(options.memtable_huge_page_size),
memtable_insert_with_hint_prefix_extractor(
options.memtable_insert_with_hint_prefix_extractor),
bloom_locality(options.bloom_locality),
arena_block_size(options.arena_block_size),
compression_per_level(options.compression_per_level),
num_levels(options.num_levels),
level0_slowdown_writes_trigger(options.level0_slowdown_writes_trigger),
level0_stop_writes_trigger(options.level0_stop_writes_trigger),
target_file_size_base(options.target_file_size_base),
target_file_size_multiplier(options.target_file_size_multiplier),
level_compaction_dynamic_level_bytes(
options.level_compaction_dynamic_level_bytes),
max_bytes_for_level_multiplier(options.max_bytes_for_level_multiplier),
max_bytes_for_level_multiplier_additional(
options.max_bytes_for_level_multiplier_additional),
max_compaction_bytes(options.max_compaction_bytes),
soft_pending_compaction_bytes_limit(
options.soft_pending_compaction_bytes_limit),
hard_pending_compaction_bytes_limit(
options.hard_pending_compaction_bytes_limit),
compaction_style(options.compaction_style),
compaction_pri(options.compaction_pri),
compaction_options_universal(options.compaction_options_universal),
compaction_options_fifo(options.compaction_options_fifo),
max_sequential_skip_in_iterations(
options.max_sequential_skip_in_iterations),
memtable_factory(options.memtable_factory),
TablePropertiesCollectorFactory Summary: This diff addresses task #4296714 and rethinks how users provide us with TablePropertiesCollectors as part of Options. Here's description of task #4296714: I'm debugging #4295529 and noticed that our count of user properties kDeletedKeys is wrong. We're sharing one single InternalKeyPropertiesCollector with all Table Builders. In LOG Files, we're outputting number of kDeletedKeys as connected with a single table, while it's actually the total count of deleted keys since creation of the DB. For example, this table has 3155 entries and 1391828 deleted keys. The problem with current approach that we call methods on a single TablePropertiesCollector for all the tables we create. Even worse, we could do it from multiple threads at the same time and TablePropertiesCollector has no way of knowing which table we're calling it for. Good part: Looks like nobody inside Facebook is using Options::table_properties_collectors. This means we should be able to painfully change the API. In this change, I introduce TablePropertiesCollectorFactory. For every table we create, we call `CreateTablePropertiesCollector`, which creates a TablePropertiesCollector for a single table. We then use it sequentially from a single thread, which means it doesn't have to be thread-safe. Test Plan: Added a test in table_properties_collector_test that fails on master (build two tables, assert that kDeletedKeys count is correct for the second one). Also, all other tests Reviewers: sdong, dhruba, haobo, kailiu Reviewed By: kailiu CC: leveldb Differential Revision: https://reviews.facebook.net/D18579
10 years ago
table_properties_collector_factories(
options.table_properties_collector_factories),
max_successive_merges(options.max_successive_merges),
optimize_filters_for_hits(options.optimize_filters_for_hits),
paranoid_file_checks(options.paranoid_file_checks),
force_consistency_checks(options.force_consistency_checks),
report_bg_io_stats(options.report_bg_io_stats) {
assert(memtable_factory.get() != nullptr);
if (max_bytes_for_level_multiplier_additional.size() <
static_cast<unsigned int>(num_levels)) {
max_bytes_for_level_multiplier_additional.resize(num_levels, 1);
}
}
ColumnFamilyOptions::ColumnFamilyOptions()
: compression(Snappy_Supported() ? kSnappyCompression : kNoCompression),
table_factory(
std::shared_ptr<TableFactory>(new BlockBasedTableFactory())) {}
ColumnFamilyOptions::ColumnFamilyOptions(const Options& options)
: AdvancedColumnFamilyOptions(options),
comparator(options.comparator),
merge_operator(options.merge_operator),
compaction_filter(options.compaction_filter),
compaction_filter_factory(options.compaction_filter_factory),
write_buffer_size(options.write_buffer_size),
compression(options.compression),
bottommost_compression(options.bottommost_compression),
compression_opts(options.compression_opts),
level0_file_num_compaction_trigger(
options.level0_file_num_compaction_trigger),
prefix_extractor(options.prefix_extractor),
max_bytes_for_level_base(options.max_bytes_for_level_base),
disable_auto_compactions(options.disable_auto_compactions),
table_factory(options.table_factory) {}
DBOptions::DBOptions() {}
DBOptions::DBOptions(const Options& options)
: create_if_missing(options.create_if_missing),
create_missing_column_families(options.create_missing_column_families),
error_if_exists(options.error_if_exists),
paranoid_checks(options.paranoid_checks),
env(options.env),
rate_limiter(options.rate_limiter),
sst_file_manager(options.sst_file_manager),
info_log(options.info_log),
info_log_level(options.info_log_level),
max_open_files(options.max_open_files),
max_file_opening_threads(options.max_file_opening_threads),
max_total_wal_size(options.max_total_wal_size),
statistics(options.statistics),
use_fsync(options.use_fsync),
db_paths(options.db_paths),
db_log_dir(options.db_log_dir),
wal_dir(options.wal_dir),
delete_obsolete_files_period_micros(
options.delete_obsolete_files_period_micros),
base_background_compactions(options.base_background_compactions),
max_background_compactions(options.max_background_compactions),
max_subcompactions(options.max_subcompactions),
max_background_flushes(options.max_background_flushes),
max_log_file_size(options.max_log_file_size),
log_file_time_to_roll(options.log_file_time_to_roll),
keep_log_file_num(options.keep_log_file_num),
recycle_log_file_num(options.recycle_log_file_num),
max_manifest_file_size(options.max_manifest_file_size),
table_cache_numshardbits(options.table_cache_numshardbits),
WAL_ttl_seconds(options.WAL_ttl_seconds),
WAL_size_limit_MB(options.WAL_size_limit_MB),
manifest_preallocation_size(options.manifest_preallocation_size),
allow_mmap_reads(options.allow_mmap_reads),
allow_mmap_writes(options.allow_mmap_writes),
use_direct_reads(options.use_direct_reads),
use_direct_io_for_flush_and_compaction(
options.use_direct_io_for_flush_and_compaction),
allow_fallocate(options.allow_fallocate),
is_fd_close_on_exec(options.is_fd_close_on_exec),
skip_log_error_on_recovery(options.skip_log_error_on_recovery),
stats_dump_period_sec(options.stats_dump_period_sec),
advise_random_on_open(options.advise_random_on_open),
db_write_buffer_size(options.db_write_buffer_size),
write_buffer_manager(options.write_buffer_manager),
access_hint_on_compaction_start(options.access_hint_on_compaction_start),
new_table_reader_for_compaction_inputs(
options.new_table_reader_for_compaction_inputs),
compaction_readahead_size(options.compaction_readahead_size),
random_access_max_buffer_size(options.random_access_max_buffer_size),
writable_file_max_buffer_size(options.writable_file_max_buffer_size),
use_adaptive_mutex(options.use_adaptive_mutex),
bytes_per_sync(options.bytes_per_sync),
wal_bytes_per_sync(options.wal_bytes_per_sync),
listeners(options.listeners),
enable_thread_tracking(options.enable_thread_tracking),
delayed_write_rate(options.delayed_write_rate),
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
allow_concurrent_memtable_write(options.allow_concurrent_memtable_write),
enable_write_thread_adaptive_yield(
options.enable_write_thread_adaptive_yield),
write_thread_max_yield_usec(options.write_thread_max_yield_usec),
write_thread_slow_yield_usec(options.write_thread_slow_yield_usec),
skip_stats_update_on_db_open(options.skip_stats_update_on_db_open),
wal_recovery_mode(options.wal_recovery_mode),
row_cache(options.row_cache),
#ifndef ROCKSDB_LITE
wal_filter(options.wal_filter),
#endif // ROCKSDB_LITE
fail_if_options_file_error(options.fail_if_options_file_error),
dump_malloc_stats(options.dump_malloc_stats),
avoid_flush_during_recovery(options.avoid_flush_during_recovery),
avoid_flush_during_shutdown(options.avoid_flush_during_shutdown) {
}
void DBOptions::Dump(Logger* log) const {
ImmutableDBOptions(*this).Dump(log);
MutableDBOptions(*this).Dump(log);
} // DBOptions::Dump
void ColumnFamilyOptions::Dump(Logger* log) const {
ROCKS_LOG_HEADER(log, " Options.comparator: %s",
comparator->Name());
ROCKS_LOG_HEADER(log, " Options.merge_operator: %s",
merge_operator ? merge_operator->Name() : "None");
ROCKS_LOG_HEADER(log, " Options.compaction_filter: %s",
compaction_filter ? compaction_filter->Name() : "None");
ROCKS_LOG_HEADER(
log, " Options.compaction_filter_factory: %s",
compaction_filter_factory ? compaction_filter_factory->Name() : "None");
ROCKS_LOG_HEADER(log, " Options.memtable_factory: %s",
memtable_factory->Name());
ROCKS_LOG_HEADER(log, " Options.table_factory: %s",
table_factory->Name());
ROCKS_LOG_HEADER(log, " table_factory options: %s",
table_factory->GetPrintableTableOptions().c_str());
ROCKS_LOG_HEADER(log, " Options.write_buffer_size: %" ROCKSDB_PRIszt,
write_buffer_size);
ROCKS_LOG_HEADER(log, " Options.max_write_buffer_number: %d",
max_write_buffer_number);
if (!compression_per_level.empty()) {
for (unsigned int i = 0; i < compression_per_level.size(); i++) {
ROCKS_LOG_HEADER(
log, " Options.compression[%d]: %s", i,
CompressionTypeToString(compression_per_level[i]).c_str());
}
Allow having different compression algorithms on different levels. Summary: The leveldb API is enhanced to support different compression algorithms at different levels. This adds the option min_level_to_compress to db_bench that specifies the minimum level for which compression should be done when compression is enabled. This can be used to disable compression for levels 0 and 1 which are likely to suffer from stalls because of the CPU load for memtable flushes and (L0,L1) compaction. Level 0 is special as it gets frequent memtable flushes. Level 1 is special as it frequently gets all:all file compactions between it and level 0. But all other levels could be the same. For any level N where N > 1, the rate of sequential IO for that level should be the same. The last level is the exception because it might not be full and because files from it are not read to compact with the next larger level. The same amount of time will be spent doing compaction at any level N excluding N=0, 1 or the last level. By this standard all of those levels should use the same compression. The difference is that the loss (using more disk space) from a faster compression algorithm is less significant for N=2 than for N=3. So we might be willing to trade disk space for faster write rates with no compression for L0 and L1, snappy for L2, zlib for L3. Using a faster compression algorithm for the mid levels also allows us to reclaim some cpu without trading off much loss in disk space overhead. Also note that little is to be gained by compressing levels 0 and 1. For a 4-level tree they account for 10% of the data. For a 5-level tree they account for 1% of the data. With compression enabled: * memtable flush rate is ~18MB/second * (L0,L1) compaction rate is ~30MB/second With compression enabled but min_level_to_compress=2 * memtable flush rate is ~320MB/second * (L0,L1) compaction rate is ~560MB/second This practicaly takes the same code from https://reviews.facebook.net/D6225 but makes the leveldb api more general purpose with a few additional lines of code. Test Plan: make check Differential Revision: https://reviews.facebook.net/D6261
12 years ago
} else {
ROCKS_LOG_HEADER(log, " Options.compression: %s",
CompressionTypeToString(compression).c_str());
Allow having different compression algorithms on different levels. Summary: The leveldb API is enhanced to support different compression algorithms at different levels. This adds the option min_level_to_compress to db_bench that specifies the minimum level for which compression should be done when compression is enabled. This can be used to disable compression for levels 0 and 1 which are likely to suffer from stalls because of the CPU load for memtable flushes and (L0,L1) compaction. Level 0 is special as it gets frequent memtable flushes. Level 1 is special as it frequently gets all:all file compactions between it and level 0. But all other levels could be the same. For any level N where N > 1, the rate of sequential IO for that level should be the same. The last level is the exception because it might not be full and because files from it are not read to compact with the next larger level. The same amount of time will be spent doing compaction at any level N excluding N=0, 1 or the last level. By this standard all of those levels should use the same compression. The difference is that the loss (using more disk space) from a faster compression algorithm is less significant for N=2 than for N=3. So we might be willing to trade disk space for faster write rates with no compression for L0 and L1, snappy for L2, zlib for L3. Using a faster compression algorithm for the mid levels also allows us to reclaim some cpu without trading off much loss in disk space overhead. Also note that little is to be gained by compressing levels 0 and 1. For a 4-level tree they account for 10% of the data. For a 5-level tree they account for 1% of the data. With compression enabled: * memtable flush rate is ~18MB/second * (L0,L1) compaction rate is ~30MB/second With compression enabled but min_level_to_compress=2 * memtable flush rate is ~320MB/second * (L0,L1) compaction rate is ~560MB/second This practicaly takes the same code from https://reviews.facebook.net/D6225 but makes the leveldb api more general purpose with a few additional lines of code. Test Plan: make check Differential Revision: https://reviews.facebook.net/D6261
12 years ago
}
ROCKS_LOG_HEADER(
log, " Options.bottommost_compression: %s",
bottommost_compression == kDisableCompressionOption
? "Disabled"
: CompressionTypeToString(bottommost_compression).c_str());
ROCKS_LOG_HEADER(
log, " Options.prefix_extractor: %s",
prefix_extractor == nullptr ? "nullptr" : prefix_extractor->Name());
ROCKS_LOG_HEADER(log,
" Options.memtable_insert_with_hint_prefix_extractor: %s",
memtable_insert_with_hint_prefix_extractor == nullptr
? "nullptr"
: memtable_insert_with_hint_prefix_extractor->Name());
ROCKS_LOG_HEADER(log, " Options.num_levels: %d", num_levels);
ROCKS_LOG_HEADER(log, " Options.min_write_buffer_number_to_merge: %d",
min_write_buffer_number_to_merge);
ROCKS_LOG_HEADER(log, " Options.max_write_buffer_number_to_maintain: %d",
max_write_buffer_number_to_maintain);
ROCKS_LOG_HEADER(log, " Options.compression_opts.window_bits: %d",
compression_opts.window_bits);
ROCKS_LOG_HEADER(log, " Options.compression_opts.level: %d",
compression_opts.level);
ROCKS_LOG_HEADER(log, " Options.compression_opts.strategy: %d",
compression_opts.strategy);
ROCKS_LOG_HEADER(
log,
" Options.compression_opts.max_dict_bytes: %" ROCKSDB_PRIszt,
compression_opts.max_dict_bytes);
ROCKS_LOG_HEADER(log, " Options.level0_file_num_compaction_trigger: %d",
level0_file_num_compaction_trigger);
ROCKS_LOG_HEADER(log, " Options.level0_slowdown_writes_trigger: %d",
level0_slowdown_writes_trigger);
ROCKS_LOG_HEADER(log, " Options.level0_stop_writes_trigger: %d",
level0_stop_writes_trigger);
ROCKS_LOG_HEADER(
log, " Options.target_file_size_base: %" PRIu64,
target_file_size_base);
ROCKS_LOG_HEADER(log, " Options.target_file_size_multiplier: %d",
target_file_size_multiplier);
ROCKS_LOG_HEADER(
log, " Options.max_bytes_for_level_base: %" PRIu64,
max_bytes_for_level_base);
ROCKS_LOG_HEADER(log, "Options.level_compaction_dynamic_level_bytes: %d",
level_compaction_dynamic_level_bytes);
ROCKS_LOG_HEADER(log, " Options.max_bytes_for_level_multiplier: %f",
max_bytes_for_level_multiplier);
for (size_t i = 0; i < max_bytes_for_level_multiplier_additional.size();
i++) {
ROCKS_LOG_HEADER(
log, "Options.max_bytes_for_level_multiplier_addtl[%" ROCKSDB_PRIszt
"]: %d",
i, max_bytes_for_level_multiplier_additional[i]);
}
ROCKS_LOG_HEADER(
log, " Options.max_sequential_skip_in_iterations: %" PRIu64,
max_sequential_skip_in_iterations);
ROCKS_LOG_HEADER(
log, " Options.max_compaction_bytes: %" PRIu64,
max_compaction_bytes);
ROCKS_LOG_HEADER(
log,
" Options.arena_block_size: %" ROCKSDB_PRIszt,
arena_block_size);
ROCKS_LOG_HEADER(log,
" Options.soft_pending_compaction_bytes_limit: %" PRIu64,
soft_pending_compaction_bytes_limit);
ROCKS_LOG_HEADER(log,
" Options.hard_pending_compaction_bytes_limit: %" PRIu64,
hard_pending_compaction_bytes_limit);
ROCKS_LOG_HEADER(log, " Options.rate_limit_delay_max_milliseconds: %u",
rate_limit_delay_max_milliseconds);
ROCKS_LOG_HEADER(log, " Options.disable_auto_compactions: %d",
disable_auto_compactions);
const auto& it_compaction_style =
compaction_style_to_string.find(compaction_style);
std::string str_compaction_style;
if (it_compaction_style == compaction_style_to_string.end()) {
assert(false);
str_compaction_style = "unknown_" + std::to_string(compaction_style);
} else {
str_compaction_style = it_compaction_style->second;
}
ROCKS_LOG_HEADER(log,
" Options.compaction_style: %s",
str_compaction_style.c_str());
const auto& it_compaction_pri =
compaction_pri_to_string.find(compaction_pri);
std::string str_compaction_pri;
if (it_compaction_pri == compaction_pri_to_string.end()) {
assert(false);
str_compaction_pri = "unknown_" + std::to_string(compaction_pri);
} else {
str_compaction_pri = it_compaction_pri->second;
}
ROCKS_LOG_HEADER(log,
" Options.compaction_pri: %s",
str_compaction_pri.c_str());
ROCKS_LOG_HEADER(log,
" Options.compaction_options_universal.size_ratio: %u",
compaction_options_universal.size_ratio);
ROCKS_LOG_HEADER(log,
"Options.compaction_options_universal.min_merge_width: %u",
compaction_options_universal.min_merge_width);
ROCKS_LOG_HEADER(log,
"Options.compaction_options_universal.max_merge_width: %u",
compaction_options_universal.max_merge_width);
ROCKS_LOG_HEADER(
log,
"Options.compaction_options_universal."
"max_size_amplification_percent: %u",
compaction_options_universal.max_size_amplification_percent);
ROCKS_LOG_HEADER(
log,
"Options.compaction_options_universal.compression_size_percent: %d",
compaction_options_universal.compression_size_percent);
ROCKS_LOG_HEADER(
log, "Options.compaction_options_fifo.max_table_files_size: %" PRIu64,
compaction_options_fifo.max_table_files_size);
ROCKS_LOG_HEADER(log,
"Options.compaction_options_fifo.allow_compaction: %d",
compaction_options_fifo.allow_compaction);
std::string collector_names;
TablePropertiesCollectorFactory Summary: This diff addresses task #4296714 and rethinks how users provide us with TablePropertiesCollectors as part of Options. Here's description of task #4296714: I'm debugging #4295529 and noticed that our count of user properties kDeletedKeys is wrong. We're sharing one single InternalKeyPropertiesCollector with all Table Builders. In LOG Files, we're outputting number of kDeletedKeys as connected with a single table, while it's actually the total count of deleted keys since creation of the DB. For example, this table has 3155 entries and 1391828 deleted keys. The problem with current approach that we call methods on a single TablePropertiesCollector for all the tables we create. Even worse, we could do it from multiple threads at the same time and TablePropertiesCollector has no way of knowing which table we're calling it for. Good part: Looks like nobody inside Facebook is using Options::table_properties_collectors. This means we should be able to painfully change the API. In this change, I introduce TablePropertiesCollectorFactory. For every table we create, we call `CreateTablePropertiesCollector`, which creates a TablePropertiesCollector for a single table. We then use it sequentially from a single thread, which means it doesn't have to be thread-safe. Test Plan: Added a test in table_properties_collector_test that fails on master (build two tables, assert that kDeletedKeys count is correct for the second one). Also, all other tests Reviewers: sdong, dhruba, haobo, kailiu Reviewed By: kailiu CC: leveldb Differential Revision: https://reviews.facebook.net/D18579
10 years ago
for (const auto& collector_factory : table_properties_collector_factories) {
collector_names.append(collector_factory->Name());
collector_names.append("; ");
}
ROCKS_LOG_HEADER(
log, " Options.table_properties_collectors: %s",
collector_names.c_str());
ROCKS_LOG_HEADER(log,
" Options.inplace_update_support: %d",
inplace_update_support);
ROCKS_LOG_HEADER(
log,
" Options.inplace_update_num_locks: %" ROCKSDB_PRIszt,
inplace_update_num_locks);
// TODO: easier config for bloom (maybe based on avg key/value size)
ROCKS_LOG_HEADER(
log, " Options.memtable_prefix_bloom_size_ratio: %f",
memtable_prefix_bloom_size_ratio);
ROCKS_LOG_HEADER(log, " Options.memtable_huge_page_size: %" ROCKSDB_PRIszt,
memtable_huge_page_size);
ROCKS_LOG_HEADER(log,
" Options.bloom_locality: %d",
bloom_locality);
ROCKS_LOG_HEADER(
log,
" Options.max_successive_merges: %" ROCKSDB_PRIszt,
max_successive_merges);
ROCKS_LOG_HEADER(log,
" Options.optimize_filters_for_hits: %d",
optimize_filters_for_hits);
ROCKS_LOG_HEADER(log, " Options.paranoid_file_checks: %d",
paranoid_file_checks);
ROCKS_LOG_HEADER(log, " Options.force_consistency_checks: %d",
force_consistency_checks);
ROCKS_LOG_HEADER(log, " Options.report_bg_io_stats: %d",
report_bg_io_stats);
} // ColumnFamilyOptions::Dump
void Options::Dump(Logger* log) const {
DBOptions::Dump(log);
ColumnFamilyOptions::Dump(log);
} // Options::Dump
void Options::DumpCFOptions(Logger* log) const {
ColumnFamilyOptions::Dump(log);
} // Options::DumpCFOptions
//
// The goal of this method is to create a configuration that
// allows an application to write all files into L0 and
// then do a single compaction to output all files into L1.
Options*
Options::PrepareForBulkLoad()
{
// never slowdown ingest.
level0_file_num_compaction_trigger = (1<<30);
level0_slowdown_writes_trigger = (1<<30);
level0_stop_writes_trigger = (1<<30);
soft_pending_compaction_bytes_limit = 0;
hard_pending_compaction_bytes_limit = 0;
// no auto compactions please. The application should issue a
// manual compaction after all data is loaded into L0.
disable_auto_compactions = true;
// A manual compaction run should pick all files in L0 in
// a single compaction run.
max_compaction_bytes = (static_cast<uint64_t>(1) << 60);
// It is better to have only 2 levels, otherwise a manual
// compaction would compact at every possible level, thereby
// increasing the total time needed for compactions.
num_levels = 2;
// Need to allow more write buffers to allow more parallism
// of flushes.
max_write_buffer_number = 6;
min_write_buffer_number_to_merge = 1;
// When compaction is disabled, more parallel flush threads can
// help with write throughput.
max_background_flushes = 4;
// Prevent a memtable flush to automatically promote files
// to L1. This is helpful so that all files that are
// input to the manual compaction are all at L0.
max_background_compactions = 2;
base_background_compactions = 2;
// The compaction would create large files in L1.
target_file_size_base = 256 * 1024 * 1024;
return this;
}
Options* Options::OptimizeForSmallDb() {
ColumnFamilyOptions::OptimizeForSmallDb();
DBOptions::OptimizeForSmallDb();
return this;
}
Options* Options::OldDefaults(int rocksdb_major_version,
int rocksdb_minor_version) {
ColumnFamilyOptions::OldDefaults(rocksdb_major_version,
rocksdb_minor_version);
DBOptions::OldDefaults(rocksdb_major_version, rocksdb_minor_version);
return this;
}
DBOptions* DBOptions::OldDefaults(int rocksdb_major_version,
int rocksdb_minor_version) {
if (rocksdb_major_version < 4 ||
(rocksdb_major_version == 4 && rocksdb_minor_version < 7)) {
max_file_opening_threads = 1;
table_cache_numshardbits = 4;
}
if (rocksdb_major_version < 5 ||
(rocksdb_major_version == 5 && rocksdb_minor_version < 2)) {
delayed_write_rate = 2 * 1024U * 1024U;
}
max_open_files = 5000;
base_background_compactions = -1;
wal_recovery_mode = WALRecoveryMode::kTolerateCorruptedTailRecords;
return this;
}
ColumnFamilyOptions* ColumnFamilyOptions::OldDefaults(
int rocksdb_major_version, int rocksdb_minor_version) {
if (rocksdb_major_version < 4 ||
(rocksdb_major_version == 4 && rocksdb_minor_version < 7)) {
write_buffer_size = 4 << 20;
target_file_size_base = 2 * 1048576;
max_bytes_for_level_base = 10 * 1048576;
soft_pending_compaction_bytes_limit = 0;
hard_pending_compaction_bytes_limit = 0;
}
if (rocksdb_major_version < 5) {
level0_stop_writes_trigger = 24;
} else if (rocksdb_major_version == 5 && rocksdb_minor_version < 2) {
level0_stop_writes_trigger = 30;
}
compaction_pri = CompactionPri::kByCompensatedSize;
return this;
}
// Optimization functions
DBOptions* DBOptions::OptimizeForSmallDb() {
max_file_opening_threads = 1;
max_open_files = 5000;
return this;
}
ColumnFamilyOptions* ColumnFamilyOptions::OptimizeForSmallDb() {
write_buffer_size = 2 << 20;
target_file_size_base = 2 * 1048576;
max_bytes_for_level_base = 10 * 1048576;
soft_pending_compaction_bytes_limit = 256 * 1048576;
hard_pending_compaction_bytes_limit = 1073741824ul;
return this;
}
#ifndef ROCKSDB_LITE
ColumnFamilyOptions* ColumnFamilyOptions::OptimizeForPointLookup(
uint64_t block_cache_size_mb) {
prefix_extractor.reset(NewNoopTransform());
BlockBasedTableOptions block_based_options;
block_based_options.index_type = BlockBasedTableOptions::kHashSearch;
block_based_options.filter_policy.reset(NewBloomFilterPolicy(10));
block_based_options.block_cache =
NewLRUCache(static_cast<size_t>(block_cache_size_mb * 1024 * 1024));
table_factory.reset(new BlockBasedTableFactory(block_based_options));
memtable_prefix_bloom_size_ratio = 0.02;
return this;
}
ColumnFamilyOptions* ColumnFamilyOptions::OptimizeLevelStyleCompaction(
uint64_t memtable_memory_budget) {
write_buffer_size = static_cast<size_t>(memtable_memory_budget / 4);
// merge two memtables when flushing to L0
min_write_buffer_number_to_merge = 2;
// this means we'll use 50% extra memory in the worst case, but will reduce
// write stalls.
max_write_buffer_number = 6;
// start flushing L0->L1 as soon as possible. each file on level0 is
// (memtable_memory_budget / 2). This will flush level 0 when it's bigger than
// memtable_memory_budget.
level0_file_num_compaction_trigger = 2;
// doesn't really matter much, but we don't want to create too many files
target_file_size_base = memtable_memory_budget / 8;
// make Level1 size equal to Level0 size, so that L0->L1 compactions are fast
max_bytes_for_level_base = memtable_memory_budget;
// level style compaction
compaction_style = kCompactionStyleLevel;
// only compress levels >= 2
compression_per_level.resize(num_levels);
for (int i = 0; i < num_levels; ++i) {
if (i < 2) {
compression_per_level[i] = kNoCompression;
} else {
compression_per_level[i] = kSnappyCompression;
}
}
return this;
}
ColumnFamilyOptions* ColumnFamilyOptions::OptimizeUniversalStyleCompaction(
uint64_t memtable_memory_budget) {
write_buffer_size = static_cast<size_t>(memtable_memory_budget / 4);
// merge two memtables when flushing to L0
min_write_buffer_number_to_merge = 2;
// this means we'll use 50% extra memory in the worst case, but will reduce
// write stalls.
max_write_buffer_number = 6;
// universal style compaction
compaction_style = kCompactionStyleUniversal;
compaction_options_universal.compression_size_percent = 80;
return this;
}
DBOptions* DBOptions::IncreaseParallelism(int total_threads) {
max_background_compactions = total_threads - 1;
max_background_flushes = 1;
env->SetBackgroundThreads(total_threads, Env::LOW);
env->SetBackgroundThreads(1, Env::HIGH);
return this;
}
#endif // !ROCKSDB_LITE
ReadOptions::ReadOptions()
: verify_checksums(true),
fill_cache(true),
snapshot(nullptr),
iterate_upper_bound(nullptr),
read_tier(kReadAllTier),
tailing(false),
managed(false),
total_order_seek(false),
Introduce ReadOptions::pin_data (support zero copy for keys) Summary: This patch update the Iterator API to introduce new functions that allow users to keep the Slices returned by key() valid as long as the Iterator is not deleted ReadOptions::pin_data : If true keep loaded blocks in memory as long as the iterator is not deleted Iterator::IsKeyPinned() : If true, this mean that the Slice returned by key() is valid as long as the iterator is not deleted Also add a new option BlockBasedTableOptions::use_delta_encoding to allow users to disable delta_encoding if needed. Benchmark results (using https://phabricator.fb.com/P20083553) ``` // $ du -h /home/tec/local/normal.4K.Snappy/db10077 // 6.1G /home/tec/local/normal.4K.Snappy/db10077 // $ du -h /home/tec/local/zero.8K.LZ4/db10077 // 6.4G /home/tec/local/zero.8K.LZ4/db10077 // Benchmarks for shard db10077 // _build/opt/rocks/benchmark/rocks_copy_benchmark \ // --normal_db_path="/home/tec/local/normal.4K.Snappy/db10077" \ // --zero_db_path="/home/tec/local/zero.8K.LZ4/db10077" // First run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 1.73s 576.97m // BM_StringPiece 103.74% 1.67s 598.55m // ============================================================================ // Match rate : 1000000 / 1000000 // Second run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 611.99ms 1.63 // BM_StringPiece 203.76% 300.35ms 3.33 // ============================================================================ // Match rate : 1000000 / 1000000 ``` Test Plan: Unit tests Reviewers: sdong, igor, anthony, yhchiang, rven Reviewed By: rven Subscribers: dhruba, lovro, adsharma Differential Revision: https://reviews.facebook.net/D48999
9 years ago
prefix_same_as_start(false),
pin_data(false),
background_purge_on_iterator_cleanup(false),
readahead_size(0),
ignore_range_deletions(false),
max_skippable_internal_keys(0) {}
ReadOptions::ReadOptions(bool cksum, bool cache)
: verify_checksums(cksum),
fill_cache(cache),
snapshot(nullptr),
iterate_upper_bound(nullptr),
read_tier(kReadAllTier),
tailing(false),
managed(false),
total_order_seek(false),
Introduce ReadOptions::pin_data (support zero copy for keys) Summary: This patch update the Iterator API to introduce new functions that allow users to keep the Slices returned by key() valid as long as the Iterator is not deleted ReadOptions::pin_data : If true keep loaded blocks in memory as long as the iterator is not deleted Iterator::IsKeyPinned() : If true, this mean that the Slice returned by key() is valid as long as the iterator is not deleted Also add a new option BlockBasedTableOptions::use_delta_encoding to allow users to disable delta_encoding if needed. Benchmark results (using https://phabricator.fb.com/P20083553) ``` // $ du -h /home/tec/local/normal.4K.Snappy/db10077 // 6.1G /home/tec/local/normal.4K.Snappy/db10077 // $ du -h /home/tec/local/zero.8K.LZ4/db10077 // 6.4G /home/tec/local/zero.8K.LZ4/db10077 // Benchmarks for shard db10077 // _build/opt/rocks/benchmark/rocks_copy_benchmark \ // --normal_db_path="/home/tec/local/normal.4K.Snappy/db10077" \ // --zero_db_path="/home/tec/local/zero.8K.LZ4/db10077" // First run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 1.73s 576.97m // BM_StringPiece 103.74% 1.67s 598.55m // ============================================================================ // Match rate : 1000000 / 1000000 // Second run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 611.99ms 1.63 // BM_StringPiece 203.76% 300.35ms 3.33 // ============================================================================ // Match rate : 1000000 / 1000000 ``` Test Plan: Unit tests Reviewers: sdong, igor, anthony, yhchiang, rven Reviewed By: rven Subscribers: dhruba, lovro, adsharma Differential Revision: https://reviews.facebook.net/D48999
9 years ago
prefix_same_as_start(false),
pin_data(false),
background_purge_on_iterator_cleanup(false),
readahead_size(0),
ignore_range_deletions(false),
max_skippable_internal_keys(0) {}
} // namespace rocksdb