rocksdb/options/cf_options.cc

229 lines
10 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#include "options/cf_options.h"
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <inttypes.h>
#include <cassert>
#include <limits>
#include <string>
#include "options/db_options.h"
#include "port/port.h"
#include "rocksdb/env.h"
#include "rocksdb/options.h"
Concurrent task limiter for compaction thread control (#4332) Summary: The PR is targeting to resolve the issue of: https://github.com/facebook/rocksdb/issues/3972#issue-330771918 We have a rocksdb created with leveled-compaction with multiple column families (CFs), some of CFs are using HDD to store big and less frequently accessed data and others are using SSD. When there are continuously write traffics going on to all CFs, the compaction thread pool is mostly occupied by those slow HDD compactions, which blocks fully utilize SSD bandwidth. Since atomic write and transaction is needed across CFs, so splitting it to multiple rocksdb instance is not an option for us. With the compaction thread control, we got 30%+ HDD write throughput gain, and also a lot smooth SSD write since less write stall happening. ConcurrentTaskLimiter can be shared with multi-CFs across rocksdb instances, so the feature does not only work for multi-CFs scenarios, but also for multi-rocksdbs scenarios, who need disk IO resource control per tenant. The usage is straight forward: e.g.: // // Enable compaction thread limiter thru ColumnFamilyOptions // std::shared_ptr<ConcurrentTaskLimiter> ctl(NewConcurrentTaskLimiter("foo_limiter", 4)); Options options; ColumnFamilyOptions cf_opt(options); cf_opt.compaction_thread_limiter = ctl; ... // // Compaction thread limiter can be tuned or disabled on-the-fly // ctl->SetMaxOutstandingTask(12); // enlarge to 12 tasks ... ctl->ResetMaxOutstandingTask(); // disable (bypass) thread limiter ctl->SetMaxOutstandingTask(-1); // Same as above ... ctl->SetMaxOutstandingTask(0); // full throttle (0 task) // // Sharing compaction thread limiter among CFs (to resolve multiple storage perf issue) // std::shared_ptr<ConcurrentTaskLimiter> ctl_ssd(NewConcurrentTaskLimiter("ssd_limiter", 8)); std::shared_ptr<ConcurrentTaskLimiter> ctl_hdd(NewConcurrentTaskLimiter("hdd_limiter", 4)); Options options; ColumnFamilyOptions cf_opt_ssd1(options); ColumnFamilyOptions cf_opt_ssd2(options); ColumnFamilyOptions cf_opt_hdd1(options); ColumnFamilyOptions cf_opt_hdd2(options); ColumnFamilyOptions cf_opt_hdd3(options); // SSD CFs cf_opt_ssd1.compaction_thread_limiter = ctl_ssd; cf_opt_ssd2.compaction_thread_limiter = ctl_ssd; // HDD CFs cf_opt_hdd1.compaction_thread_limiter = ctl_hdd; cf_opt_hdd2.compaction_thread_limiter = ctl_hdd; cf_opt_hdd3.compaction_thread_limiter = ctl_hdd; ... // // The limiter is disabled by default (or set to nullptr explicitly) // Options options; ColumnFamilyOptions cf_opt(options); cf_opt.compaction_thread_limiter = nullptr; Pull Request resolved: https://github.com/facebook/rocksdb/pull/4332 Differential Revision: D13226590 Pulled By: siying fbshipit-source-id: 14307aec55b8bd59c8223d04aa6db3c03d1b0c1d
2018-12-13 22:16:04 +01:00
#include "rocksdb/concurrent_task_limiter.h"
namespace rocksdb {
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),
compaction_pri(cf_options.compaction_pri),
user_comparator(cf_options.comparator),
internal_comparator(InternalKeyComparator(cf_options.comparator)),
merge_operator(cf_options.merge_operator.get()),
compaction_filter(cf_options.compaction_filter),
compaction_filter_factory(cf_options.compaction_filter_factory.get()),
min_write_buffer_number_to_merge(
cf_options.min_write_buffer_number_to_merge),
max_write_buffer_number_to_maintain(
cf_options.max_write_buffer_number_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_level(db_options.info_log_level),
env(db_options.env),
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()),
table_properties_collector_factories(
cf_options.table_properties_collector_factories),
advise_random_on_open(db_options.advise_random_on_open),
bloom_locality(cf_options.bloom_locality),
purge_redundant_kvs_while_flush(
cf_options.purge_redundant_kvs_while_flush),
use_fsync(db_options.use_fsync),
compression_per_level(cf_options.compression_per_level),
bottommost_compression(cf_options.bottommost_compression),
bottommost_compression_opts(cf_options.bottommost_compression_opts),
compression_opts(cf_options.compression_opts),
level_compaction_dynamic_level_bytes(
cf_options.level_compaction_dynamic_level_bytes),
access_hint_on_compaction_start(
db_options.access_hint_on_compaction_start),
new_table_reader_for_compaction_inputs(
db_options.new_table_reader_for_compaction_inputs),
num_levels(cf_options.num_levels),
optimize_filters_for_hits(cf_options.optimize_filters_for_hits),
force_consistency_checks(cf_options.force_consistency_checks),
allow_ingest_behind(db_options.allow_ingest_behind),
Added support for differential snapshots Summary: The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2). This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages. From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff". This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR. For now, what's done here according to initial discussions: Preserving deletes: - We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion. - I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum. - Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum. Iterator changes: - couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum. TableCache changes: - I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span. What's left: - Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type. Closes https://github.com/facebook/rocksdb/pull/2999 Differential Revision: D6175602 Pulled By: mikhail-antonov fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 02:43:29 +01:00
preserve_deletes(db_options.preserve_deletes),
listeners(db_options.listeners),
row_cache(db_options.row_cache),
max_subcompactions(db_options.max_subcompactions),
memtable_insert_with_hint_prefix_extractor(
cf_options.memtable_insert_with_hint_prefix_extractor.get()),
Concurrent task limiter for compaction thread control (#4332) Summary: The PR is targeting to resolve the issue of: https://github.com/facebook/rocksdb/issues/3972#issue-330771918 We have a rocksdb created with leveled-compaction with multiple column families (CFs), some of CFs are using HDD to store big and less frequently accessed data and others are using SSD. When there are continuously write traffics going on to all CFs, the compaction thread pool is mostly occupied by those slow HDD compactions, which blocks fully utilize SSD bandwidth. Since atomic write and transaction is needed across CFs, so splitting it to multiple rocksdb instance is not an option for us. With the compaction thread control, we got 30%+ HDD write throughput gain, and also a lot smooth SSD write since less write stall happening. ConcurrentTaskLimiter can be shared with multi-CFs across rocksdb instances, so the feature does not only work for multi-CFs scenarios, but also for multi-rocksdbs scenarios, who need disk IO resource control per tenant. The usage is straight forward: e.g.: // // Enable compaction thread limiter thru ColumnFamilyOptions // std::shared_ptr<ConcurrentTaskLimiter> ctl(NewConcurrentTaskLimiter("foo_limiter", 4)); Options options; ColumnFamilyOptions cf_opt(options); cf_opt.compaction_thread_limiter = ctl; ... // // Compaction thread limiter can be tuned or disabled on-the-fly // ctl->SetMaxOutstandingTask(12); // enlarge to 12 tasks ... ctl->ResetMaxOutstandingTask(); // disable (bypass) thread limiter ctl->SetMaxOutstandingTask(-1); // Same as above ... ctl->SetMaxOutstandingTask(0); // full throttle (0 task) // // Sharing compaction thread limiter among CFs (to resolve multiple storage perf issue) // std::shared_ptr<ConcurrentTaskLimiter> ctl_ssd(NewConcurrentTaskLimiter("ssd_limiter", 8)); std::shared_ptr<ConcurrentTaskLimiter> ctl_hdd(NewConcurrentTaskLimiter("hdd_limiter", 4)); Options options; ColumnFamilyOptions cf_opt_ssd1(options); ColumnFamilyOptions cf_opt_ssd2(options); ColumnFamilyOptions cf_opt_hdd1(options); ColumnFamilyOptions cf_opt_hdd2(options); ColumnFamilyOptions cf_opt_hdd3(options); // SSD CFs cf_opt_ssd1.compaction_thread_limiter = ctl_ssd; cf_opt_ssd2.compaction_thread_limiter = ctl_ssd; // HDD CFs cf_opt_hdd1.compaction_thread_limiter = ctl_hdd; cf_opt_hdd2.compaction_thread_limiter = ctl_hdd; cf_opt_hdd3.compaction_thread_limiter = ctl_hdd; ... // // The limiter is disabled by default (or set to nullptr explicitly) // Options options; ColumnFamilyOptions cf_opt(options); cf_opt.compaction_thread_limiter = nullptr; Pull Request resolved: https://github.com/facebook/rocksdb/pull/4332 Differential Revision: D13226590 Pulled By: siying fbshipit-source-id: 14307aec55b8bd59c8223d04aa6db3c03d1b0c1d
2018-12-13 22:16:04 +01:00
cf_paths(cf_options.cf_paths),
compaction_thread_limiter(cf_options.compaction_thread_limiter) {}
// Multiple two operands. If they overflow, return op1.
uint64_t MultiplyCheckOverflow(uint64_t op1, double op2) {
if (op1 == 0 || op2 <= 0) {
return 0;
}
if (port::kMaxUint64 / op1 < op2) {
return op1;
}
return static_cast<uint64_t>(op1 * op2);
}
// when level_compaction_dynamic_level_bytes is true and leveled compaction
// is used, the base level is not always L1, so precomupted max_file_size can
// no longer be used. Recompute file_size_for_level from base level.
uint64_t MaxFileSizeForLevel(const MutableCFOptions& cf_options,
int level, CompactionStyle compaction_style, int base_level,
bool level_compaction_dynamic_level_bytes) {
if (!level_compaction_dynamic_level_bytes || level < base_level ||
compaction_style != kCompactionStyleLevel) {
assert(level >= 0);
assert(level < (int)cf_options.max_file_size.size());
return cf_options.max_file_size[level];
} else {
assert(level >= 0 && base_level >= 0);
assert(level - base_level < (int)cf_options.max_file_size.size());
return cf_options.max_file_size[level - base_level];
}
}
void MutableCFOptions::RefreshDerivedOptions(int num_levels,
CompactionStyle compaction_style) {
max_file_size.resize(num_levels);
for (int i = 0; i < num_levels; ++i) {
if (i == 0 && compaction_style == kCompactionStyleUniversal) {
max_file_size[i] = ULLONG_MAX;
} else if (i > 1) {
max_file_size[i] = MultiplyCheckOverflow(max_file_size[i - 1],
target_file_size_multiplier);
} else {
max_file_size[i] = target_file_size_base;
}
}
}
void MutableCFOptions::Dump(Logger* log) const {
// Memtable related options
ROCKS_LOG_INFO(log,
" write_buffer_size: %" ROCKSDB_PRIszt,
write_buffer_size);
ROCKS_LOG_INFO(log, " max_write_buffer_number: %d",
max_write_buffer_number);
ROCKS_LOG_INFO(log,
" arena_block_size: %" ROCKSDB_PRIszt,
arena_block_size);
ROCKS_LOG_INFO(log, " memtable_prefix_bloom_ratio: %f",
memtable_prefix_bloom_size_ratio);
ROCKS_LOG_INFO(log, " memtable_whole_key_filtering: %d",
memtable_whole_key_filtering);
ROCKS_LOG_INFO(log,
" memtable_huge_page_size: %" ROCKSDB_PRIszt,
memtable_huge_page_size);
ROCKS_LOG_INFO(log,
" max_successive_merges: %" ROCKSDB_PRIszt,
max_successive_merges);
ROCKS_LOG_INFO(log,
" inplace_update_num_locks: %" ROCKSDB_PRIszt,
inplace_update_num_locks);
ROCKS_LOG_INFO(
log, " prefix_extractor: %s",
prefix_extractor == nullptr ? "nullptr" : prefix_extractor->Name());
ROCKS_LOG_INFO(log, " disable_auto_compactions: %d",
disable_auto_compactions);
ROCKS_LOG_INFO(log, " soft_pending_compaction_bytes_limit: %" PRIu64,
soft_pending_compaction_bytes_limit);
ROCKS_LOG_INFO(log, " hard_pending_compaction_bytes_limit: %" PRIu64,
hard_pending_compaction_bytes_limit);
ROCKS_LOG_INFO(log, " level0_file_num_compaction_trigger: %d",
level0_file_num_compaction_trigger);
ROCKS_LOG_INFO(log, " level0_slowdown_writes_trigger: %d",
level0_slowdown_writes_trigger);
ROCKS_LOG_INFO(log, " level0_stop_writes_trigger: %d",
level0_stop_writes_trigger);
ROCKS_LOG_INFO(log, " max_compaction_bytes: %" PRIu64,
max_compaction_bytes);
ROCKS_LOG_INFO(log, " target_file_size_base: %" PRIu64,
target_file_size_base);
ROCKS_LOG_INFO(log, " target_file_size_multiplier: %d",
target_file_size_multiplier);
ROCKS_LOG_INFO(log, " max_bytes_for_level_base: %" PRIu64,
max_bytes_for_level_base);
ROCKS_LOG_INFO(log, " max_bytes_for_level_multiplier: %f",
max_bytes_for_level_multiplier);
ROCKS_LOG_INFO(log, " ttl: %" PRIu64,
ttl);
std::string result;
char buf[10];
for (const auto m : max_bytes_for_level_multiplier_additional) {
snprintf(buf, sizeof(buf), "%d, ", m);
result += buf;
}
if (result.size() >= 2) {
result.resize(result.size() - 2);
} else {
result = "";
}
ROCKS_LOG_INFO(log, "max_bytes_for_level_multiplier_additional: %s",
result.c_str());
ROCKS_LOG_INFO(log, " max_sequential_skip_in_iterations: %" PRIu64,
max_sequential_skip_in_iterations);
ROCKS_LOG_INFO(log, " paranoid_file_checks: %d",
paranoid_file_checks);
ROCKS_LOG_INFO(log, " report_bg_io_stats: %d",
report_bg_io_stats);
ROCKS_LOG_INFO(log, " compression: %d",
static_cast<int>(compression));
// Universal Compaction Options
ROCKS_LOG_INFO(log, "compaction_options_universal.size_ratio : %d",
compaction_options_universal.size_ratio);
ROCKS_LOG_INFO(log, "compaction_options_universal.min_merge_width : %d",
compaction_options_universal.min_merge_width);
ROCKS_LOG_INFO(log, "compaction_options_universal.max_merge_width : %d",
compaction_options_universal.max_merge_width);
ROCKS_LOG_INFO(
log, "compaction_options_universal.max_size_amplification_percent : %d",
compaction_options_universal.max_size_amplification_percent);
ROCKS_LOG_INFO(log,
"compaction_options_universal.compression_size_percent : %d",
compaction_options_universal.compression_size_percent);
ROCKS_LOG_INFO(log, "compaction_options_universal.stop_style : %d",
compaction_options_universal.stop_style);
ROCKS_LOG_INFO(
log, "compaction_options_universal.allow_trivial_move : %d",
static_cast<int>(compaction_options_universal.allow_trivial_move));
// FIFO Compaction Options
ROCKS_LOG_INFO(log, "compaction_options_fifo.max_table_files_size : %" PRIu64,
compaction_options_fifo.max_table_files_size);
ROCKS_LOG_INFO(log, "compaction_options_fifo.allow_compaction : %d",
compaction_options_fifo.allow_compaction);
}
MutableCFOptions::MutableCFOptions(const Options& options)
: MutableCFOptions(ColumnFamilyOptions(options)) {}
} // namespace rocksdb