rocksdb/include/rocksdb/advanced_options.h

931 lines
40 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).
// 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.
#pragma once
#include <memory>
#include "rocksdb/compression_type.h"
#include "rocksdb/memtablerep.h"
#include "rocksdb/universal_compaction.h"
namespace ROCKSDB_NAMESPACE {
class Slice;
class SliceTransform;
class TablePropertiesCollectorFactory;
class TableFactory;
struct Options;
enum CompactionStyle : char {
// level based compaction style
kCompactionStyleLevel = 0x0,
// Universal compaction style
// Not supported in ROCKSDB_LITE.
kCompactionStyleUniversal = 0x1,
// FIFO compaction style
// Not supported in ROCKSDB_LITE
kCompactionStyleFIFO = 0x2,
// Disable background compaction. Compaction jobs are submitted
// via CompactFiles().
// Not supported in ROCKSDB_LITE
kCompactionStyleNone = 0x3,
};
// In Level-based compaction, it Determines which file from a level to be
// picked to merge to the next level. We suggest people try
// kMinOverlappingRatio first when you tune your database.
enum CompactionPri : char {
// Slightly prioritize larger files by size compensated by #deletes
kByCompensatedSize = 0x0,
// First compact files whose data's latest update time is oldest.
// Try this if you only update some hot keys in small ranges.
kOldestLargestSeqFirst = 0x1,
// First compact files whose range hasn't been compacted to the next level
// for the longest. If your updates are random across the key space,
// write amplification is slightly better with this option.
kOldestSmallestSeqFirst = 0x2,
// First compact files whose ratio between overlapping size in next level
// and its size is the smallest. It in many cases can optimize write
// amplification.
kMinOverlappingRatio = 0x3,
};
struct CompactionOptionsFIFO {
// once the total sum of table files reaches this, we will delete the oldest
// table file
// Default: 1GB
uint64_t max_table_files_size;
// If true, try to do compaction to compact smaller files into larger ones.
// Minimum files to compact follows options.level0_file_num_compaction_trigger
// and compaction won't trigger if average compact bytes per del file is
// larger than options.write_buffer_size. This is to protect large files
// from being compacted again.
// Default: false;
bool allow_compaction = false;
// When not 0, if the data in the file is older than this threshold, RocksDB
// will soon move the file to warm temperature.
uint64_t age_for_warm = 0;
CompactionOptionsFIFO() : max_table_files_size(1 * 1024 * 1024 * 1024) {}
CompactionOptionsFIFO(uint64_t _max_table_files_size, bool _allow_compaction)
: max_table_files_size(_max_table_files_size),
allow_compaction(_allow_compaction) {}
};
// Compression options for different compression algorithms like Zlib
struct CompressionOptions {
// RocksDB's generic default compression level. Internally it'll be translated
// to the default compression level specific to the library being used (see
// comment above `ColumnFamilyOptions::compression`).
//
// The default value is the max 16-bit int as it'll be written out in OPTIONS
// file, which should be portable.
const static int kDefaultCompressionLevel = 32767;
int window_bits;
int level;
int strategy;
// Maximum size of dictionaries used to prime the compression library.
// Enabling dictionary can improve compression ratios when there are
// repetitions across data blocks.
//
// The dictionary is created by sampling the SST file data. If
// `zstd_max_train_bytes` is nonzero, the samples are passed through zstd's
// dictionary generator. Otherwise, the random samples are used directly as
// the dictionary.
//
// When compression dictionary is disabled, we compress and write each block
// before buffering data for the next one. When compression dictionary is
// enabled, we buffer SST file data in-memory so we can sample it, as data
// can only be compressed and written after the dictionary has been finalized.
//
// The amount of data buffered can be limited by `max_dict_buffer_bytes`. This
// buffered memory is charged to the block cache when there is a block cache.
// If block cache insertion fails with `Status::Incomplete` (i.e., it is
// full), we finalize the dictionary with whatever data we have and then stop
// buffering.
//
// Default: 0.
uint32_t max_dict_bytes;
// Maximum size of training data passed to zstd's dictionary trainer. Using
// zstd's dictionary trainer can achieve even better compression ratio
// improvements than using `max_dict_bytes` alone.
//
// The training data will be used to generate a dictionary of max_dict_bytes.
//
// Default: 0.
uint32_t zstd_max_train_bytes;
// Number of threads for parallel compression.
// Parallel compression is enabled only if threads > 1.
// THE FEATURE IS STILL EXPERIMENTAL
//
// This option is valid only when BlockBasedTable is used.
//
// When parallel compression is enabled, SST size file sizes might be
// more inflated compared to the target size, because more data of unknown
// compressed size is in flight when compression is parallelized. To be
// reasonably accurate, this inflation is also estimated by using historical
// compression ratio and current bytes inflight.
//
// Default: 1.
uint32_t parallel_threads;
// When the compression options are set by the user, it will be set to "true".
// For bottommost_compression_opts, to enable it, user must set enabled=true.
// Otherwise, bottommost compression will use compression_opts as default
// compression options.
//
// For compression_opts, if compression_opts.enabled=false, it is still
// used as compression options for compression process.
//
// Default: false.
bool enabled;
Limit buffering for collecting samples for compression dictionary (#7970) Summary: For dictionary compression, we need to collect some representative samples of the data to be compressed, which we use to either generate or train (when `CompressionOptions::zstd_max_train_bytes > 0`) a dictionary. Previously, the strategy was to buffer all the data blocks during flush, and up to the target file size during compaction. That strategy allowed us to randomly pick samples from as wide a range as possible that'd be guaranteed to land in a single output file. However, some users try to make huge files in memory-constrained environments, where this strategy can cause OOM. This PR introduces an option, `CompressionOptions::max_dict_buffer_bytes`, that limits how much data blocks are buffered before we switch to unbuffered mode (which means creating the per-SST dictionary, writing out the buffered data, and compressing/writing new blocks as soon as they are built). It is not strict as we currently buffer more than just data blocks -- also keys are buffered. But it does make a step towards giving users predictable memory usage. Related changes include: - Changed sampling for dictionary compression to select unique data blocks when there is limited availability of data blocks - Made use of `BlockBuilder::SwapAndReset()` to save an allocation+memcpy when buffering data blocks for building a dictionary - Changed `ParseBoolean()` to accept an input containing characters after the boolean. This is necessary since, with this PR, a value for `CompressionOptions::enabled` is no longer necessarily the final component in the `CompressionOptions` string. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7970 Test Plan: - updated `CompressionOptions` unit tests to verify limit is respected (to the extent expected in the current implementation) in various scenarios of flush/compaction to bottommost/non-bottommost level - looked at jemalloc heap profiles right before and after switching to unbuffered mode during flush/compaction. Verified memory usage in buffering is proportional to the limit set. Reviewed By: pdillinger Differential Revision: D26467994 Pulled By: ajkr fbshipit-source-id: 3da4ef9fba59974e4ef40e40c01611002c861465
2021-02-19 23:06:59 +01:00
// Limit on data buffering when gathering samples to build a dictionary. Zero
// means no limit. When dictionary is disabled (`max_dict_bytes == 0`),
// enabling this limit (`max_dict_buffer_bytes != 0`) has no effect.
//
// In compaction, the buffering is limited to the target file size (see
// `target_file_size_base` and `target_file_size_multiplier`) even if this
// setting permits more buffering. Since we cannot determine where the file
// should be cut until data blocks are compressed with dictionary, buffering
// more than the target file size could lead to selecting samples that belong
// to a later output SST.
//
// Limiting too strictly may harm dictionary effectiveness since it forces
// RocksDB to pick samples from the initial portion of the output SST, which
// may not be representative of the whole file. Configuring this limit below
// `zstd_max_train_bytes` (when enabled) can restrict how many samples we can
// pass to the dictionary trainer. Configuring it below `max_dict_bytes` can
// restrict the size of the final dictionary.
//
// Default: 0 (unlimited)
uint64_t max_dict_buffer_bytes;
CompressionOptions()
: window_bits(-14),
level(kDefaultCompressionLevel),
strategy(0),
max_dict_bytes(0),
zstd_max_train_bytes(0),
parallel_threads(1),
Limit buffering for collecting samples for compression dictionary (#7970) Summary: For dictionary compression, we need to collect some representative samples of the data to be compressed, which we use to either generate or train (when `CompressionOptions::zstd_max_train_bytes > 0`) a dictionary. Previously, the strategy was to buffer all the data blocks during flush, and up to the target file size during compaction. That strategy allowed us to randomly pick samples from as wide a range as possible that'd be guaranteed to land in a single output file. However, some users try to make huge files in memory-constrained environments, where this strategy can cause OOM. This PR introduces an option, `CompressionOptions::max_dict_buffer_bytes`, that limits how much data blocks are buffered before we switch to unbuffered mode (which means creating the per-SST dictionary, writing out the buffered data, and compressing/writing new blocks as soon as they are built). It is not strict as we currently buffer more than just data blocks -- also keys are buffered. But it does make a step towards giving users predictable memory usage. Related changes include: - Changed sampling for dictionary compression to select unique data blocks when there is limited availability of data blocks - Made use of `BlockBuilder::SwapAndReset()` to save an allocation+memcpy when buffering data blocks for building a dictionary - Changed `ParseBoolean()` to accept an input containing characters after the boolean. This is necessary since, with this PR, a value for `CompressionOptions::enabled` is no longer necessarily the final component in the `CompressionOptions` string. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7970 Test Plan: - updated `CompressionOptions` unit tests to verify limit is respected (to the extent expected in the current implementation) in various scenarios of flush/compaction to bottommost/non-bottommost level - looked at jemalloc heap profiles right before and after switching to unbuffered mode during flush/compaction. Verified memory usage in buffering is proportional to the limit set. Reviewed By: pdillinger Differential Revision: D26467994 Pulled By: ajkr fbshipit-source-id: 3da4ef9fba59974e4ef40e40c01611002c861465
2021-02-19 23:06:59 +01:00
enabled(false),
max_dict_buffer_bytes(0) {}
CompressionOptions(int wbits, int _lev, int _strategy,
uint32_t _max_dict_bytes, uint32_t _zstd_max_train_bytes,
uint32_t _parallel_threads, bool _enabled,
uint64_t _max_dict_buffer_bytes)
: window_bits(wbits),
level(_lev),
strategy(_strategy),
max_dict_bytes(_max_dict_bytes),
zstd_max_train_bytes(_zstd_max_train_bytes),
parallel_threads(_parallel_threads),
Limit buffering for collecting samples for compression dictionary (#7970) Summary: For dictionary compression, we need to collect some representative samples of the data to be compressed, which we use to either generate or train (when `CompressionOptions::zstd_max_train_bytes > 0`) a dictionary. Previously, the strategy was to buffer all the data blocks during flush, and up to the target file size during compaction. That strategy allowed us to randomly pick samples from as wide a range as possible that'd be guaranteed to land in a single output file. However, some users try to make huge files in memory-constrained environments, where this strategy can cause OOM. This PR introduces an option, `CompressionOptions::max_dict_buffer_bytes`, that limits how much data blocks are buffered before we switch to unbuffered mode (which means creating the per-SST dictionary, writing out the buffered data, and compressing/writing new blocks as soon as they are built). It is not strict as we currently buffer more than just data blocks -- also keys are buffered. But it does make a step towards giving users predictable memory usage. Related changes include: - Changed sampling for dictionary compression to select unique data blocks when there is limited availability of data blocks - Made use of `BlockBuilder::SwapAndReset()` to save an allocation+memcpy when buffering data blocks for building a dictionary - Changed `ParseBoolean()` to accept an input containing characters after the boolean. This is necessary since, with this PR, a value for `CompressionOptions::enabled` is no longer necessarily the final component in the `CompressionOptions` string. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7970 Test Plan: - updated `CompressionOptions` unit tests to verify limit is respected (to the extent expected in the current implementation) in various scenarios of flush/compaction to bottommost/non-bottommost level - looked at jemalloc heap profiles right before and after switching to unbuffered mode during flush/compaction. Verified memory usage in buffering is proportional to the limit set. Reviewed By: pdillinger Differential Revision: D26467994 Pulled By: ajkr fbshipit-source-id: 3da4ef9fba59974e4ef40e40c01611002c861465
2021-02-19 23:06:59 +01:00
enabled(_enabled),
max_dict_buffer_bytes(_max_dict_buffer_bytes) {}
};
// Temperature of a file. Used to pass to FileSystem for a different
// placement and/or coding.
// Reserve some numbers in the middle, in case we need to insert new tier
// there.
enum class Temperature : uint8_t {
kUnknown = 0,
kHot = 0x04,
kWarm = 0x08,
kCold = 0x0C,
};
// The control option of how the cache tiers will be used. Currently rocksdb
// support block cahe (volatile tier), secondary cache (non-volatile tier).
// In the future, we may add more caching layers.
enum class CacheTier : uint8_t {
kVolatileTier = 0,
kNonVolatileBlockTier = 0x01,
};
enum UpdateStatus { // Return status For inplace update callback
UPDATE_FAILED = 0, // Nothing to update
UPDATED_INPLACE = 1, // Value updated inplace
UPDATED = 2, // No inplace update. Merged value set
};
struct AdvancedColumnFamilyOptions {
// The maximum number of write buffers that are built up in memory.
// The default and the minimum number is 2, so that when 1 write buffer
// is being flushed to storage, new writes can continue to the other
// write buffer.
// If max_write_buffer_number > 3, writing will be slowed down to
// options.delayed_write_rate if we are writing to the last write buffer
// allowed.
//
// Default: 2
//
// Dynamically changeable through SetOptions() API
int max_write_buffer_number = 2;
// The minimum number of write buffers that will be merged together
// before writing to storage. If set to 1, then
// all write buffers are flushed to L0 as individual files and this increases
// read amplification because a get request has to check in all of these
// files. Also, an in-memory merge may result in writing lesser
// data to storage if there are duplicate records in each of these
// individual write buffers. Default: 1
int min_write_buffer_number_to_merge = 1;
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
// DEPRECATED
// The total maximum number of write buffers to maintain in memory including
// copies of buffers that have already been flushed. Unlike
// max_write_buffer_number, this parameter does not affect flushing.
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
// This parameter is being replaced by max_write_buffer_size_to_maintain.
// If both parameters are set to non-zero values, this parameter will be
// ignored.
int max_write_buffer_number_to_maintain = 0;
// The target number of write history bytes to hold in memory. Write history
// comprises the latest write buffers (memtables). To reach the target, write
// buffers that were most recently flushed to SST files may be retained in
// memory.
//
// This controls the target amount of write history that will be available
// in memory for conflict checking when Transactions are used.
//
// This target may be undershot when the CF first opens and has not recovered
// or received enough writes to reach the target. After reaching the target
// once, it is guaranteed to never undershoot again. That guarantee is
// implemented by retaining flushed write buffers in-memory until the oldest
// one can be trimmed without dropping below the target.
//
// Examples with `max_write_buffer_size_to_maintain` set to 32MB:
//
// - One mutable memtable of 64MB, one unflushed immutable memtable of 64MB,
// and zero flushed immutable memtables. Nothing trimmable exists.
// - One mutable memtable of 16MB, zero unflushed immutable memtables, and
// one flushed immutable memtable of 64MB. Trimming is disallowed because
// dropping the earliest (only) flushed immutable memtable would result in
// write history of 16MB < 32MB.
// - One mutable memtable of 24MB, one unflushed immutable memtable of 16MB,
// and one flushed immutable memtable of 16MB. The earliest (only) flushed
// immutable memtable is trimmed because without it we still have
// 16MB + 24MB = 40MB > 32MB of write history.
//
// When using an OptimisticTransactionDB:
// If this value is too low, some transactions may fail at commit time due
// to not being able to determine whether there were any write conflicts.
//
// When using a TransactionDB:
// If Transaction::SetSnapshot is used, TransactionDB will read either
// in-memory write buffers or SST files to do write-conflict checking.
// Increasing this value can reduce the number of reads to SST files
// done for conflict detection.
//
// Setting this value to 0 will cause write buffers to be freed immediately
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
// after they are flushed. If this value is set to -1,
// 'max_write_buffer_number * write_buffer_size' will be used.
//
// Default:
// If using a TransactionDB/OptimisticTransactionDB, the default value will
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
// be set to the value of 'max_write_buffer_number * write_buffer_size'
// if it is not explicitly set by the user. Otherwise, the default is 0.
int64_t max_write_buffer_size_to_maintain = 0;
// Allows thread-safe inplace updates. If this is true, there is no way to
// achieve point-in-time consistency using snapshot or iterator (assuming
// concurrent updates). Hence iterator and multi-get will return results
// which are not consistent as of any point-in-time.
// Backward iteration on memtables will not work either.
// If inplace_callback function is not set,
// Put(key, new_value) will update inplace the existing_value iff
// * key exists in current memtable
// * new sizeof(new_value) <= sizeof(existing_value)
// * existing_value for that key is a put i.e. kTypeValue
// If inplace_callback function is set, check doc for inplace_callback.
// Default: false.
bool inplace_update_support = false;
// Number of locks used for inplace update
// Default: 10000, if inplace_update_support = true, else 0.
//
// Dynamically changeable through SetOptions() API
size_t inplace_update_num_locks = 10000;
// existing_value - pointer to previous value (from both memtable and sst).
// nullptr if key doesn't exist
// existing_value_size - pointer to size of existing_value).
// nullptr if key doesn't exist
// delta_value - Delta value to be merged with the existing_value.
// Stored in transaction logs.
// merged_value - Set when delta is applied on the previous value.
//
// Applicable only when inplace_update_support is true,
// this callback function is called at the time of updating the memtable
// as part of a Put operation, lets say Put(key, delta_value). It allows the
// 'delta_value' specified as part of the Put operation to be merged with
// an 'existing_value' of the key in the database.
//
// If the merged value is smaller in size that the 'existing_value',
// then this function can update the 'existing_value' buffer inplace and
// the corresponding 'existing_value'_size pointer, if it wishes to.
// The callback should return UpdateStatus::UPDATED_INPLACE.
// In this case. (In this case, the snapshot-semantics of the rocksdb
// Iterator is not atomic anymore).
//
// If the merged value is larger in size than the 'existing_value' or the
// application does not wish to modify the 'existing_value' buffer inplace,
// then the merged value should be returned via *merge_value. It is set by
// merging the 'existing_value' and the Put 'delta_value'. The callback should
// return UpdateStatus::UPDATED in this case. This merged value will be added
// to the memtable.
//
// If merging fails or the application does not wish to take any action,
// then the callback should return UpdateStatus::UPDATE_FAILED.
//
// Please remember that the original call from the application is Put(key,
// delta_value). So the transaction log (if enabled) will still contain (key,
// delta_value). The 'merged_value' is not stored in the transaction log.
// Hence the inplace_callback function should be consistent across db reopens.
//
// RocksDB callbacks are NOT exception-safe. A callback completing with an
// exception can lead to undefined behavior in RocksDB, including data loss,
// unreported corruption, deadlocks, and more.
//
// Default: nullptr
UpdateStatus (*inplace_callback)(char* existing_value,
uint32_t* existing_value_size,
Slice delta_value,
std::string* merged_value) = nullptr;
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2022-01-27 23:53:39 +01:00
// Should really be called `memtable_bloom_size_ratio`. Enables a dynamic
// Bloom filter in memtable to optimize many queries that must go beyond
// the memtable. The size in bytes of the filter is
// write_buffer_size * memtable_prefix_bloom_size_ratio.
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2022-01-27 23:53:39 +01:00
// * If prefix_extractor is set, the filter includes prefixes.
// * If memtable_whole_key_filtering, the filter includes whole keys.
// * If both, the filter includes both.
// * If neither, the feature is disabled.
//
// If this value is larger than 0.25, it is sanitized to 0.25.
//
// Default: 0 (disabled)
//
// Dynamically changeable through SetOptions() API
double memtable_prefix_bloom_size_ratio = 0.0;
// Enable whole key bloom filter in memtable. Note this will only take effect
// if memtable_prefix_bloom_size_ratio is not 0. Enabling whole key filtering
// can potentially reduce CPU usage for point-look-ups.
//
// Default: false (disabled)
//
// Dynamically changeable through SetOptions() API
bool memtable_whole_key_filtering = false;
// Page size for huge page for the arena used by the memtable. If <=0, it
// won't allocate from huge page but from malloc.
// Users are responsible to reserve huge pages for it to be allocated. For
// example:
// sysctl -w vm.nr_hugepages=20
// See linux doc Documentation/vm/hugetlbpage.txt
// If there isn't enough free huge page available, it will fall back to
// malloc.
//
// Dynamically changeable through SetOptions() API
size_t memtable_huge_page_size = 0;
// If non-nullptr, memtable will use the specified function to extract
// prefixes for keys, and for each prefix maintain a hint of insert location
// to reduce CPU usage for inserting keys with the prefix. Keys out of
// domain of the prefix extractor will be insert without using hints.
//
// Currently only the default skiplist based memtable implements the feature.
// All other memtable implementation will ignore the option. It incurs ~250
// additional bytes of memory overhead to store a hint for each prefix.
// Also concurrent writes (when allow_concurrent_memtable_write is true) will
// ignore the option.
//
// The option is best suited for workloads where keys will likely to insert
// to a location close the last inserted key with the same prefix.
// One example could be inserting keys of the form (prefix + timestamp),
// and keys of the same prefix always comes in with time order. Another
// example would be updating the same key over and over again, in which case
// the prefix can be the key itself.
//
// Default: nullptr (disabled)
std::shared_ptr<const SliceTransform>
memtable_insert_with_hint_prefix_extractor = nullptr;
// Control locality of bloom filter probes to improve CPU cache hit rate.
// This option now only applies to plaintable prefix bloom. This
// optimization is turned off when set to 0, and positive number to turn
// it on.
// Default: 0
uint32_t bloom_locality = 0;
// size of one block in arena memory allocation.
// If <= 0, a proper value is automatically calculated (usually 1/8 of
// writer_buffer_size, rounded up to a multiple of 4KB, or 1MB which ever is
// smaller).
//
// There are two additional restriction of the specified size:
// (1) size should be in the range of [4096, 2 << 30] and
// (2) be the multiple of the CPU word (which helps with the memory
// alignment).
//
// We'll automatically check and adjust the size number to make sure it
// conforms to the restrictions.
//
// Default: 0
//
// Dynamically changeable through SetOptions() API
size_t arena_block_size = 0;
// Different levels can have different compression policies. There
// are cases where most lower levels would like to use quick compression
// algorithms while the higher levels (which have more data) use
// compression algorithms that have better compression but could
// be slower. This array, if non-empty, should have an entry for
// each level of the database; these override the value specified in
// the previous field 'compression'.
//
// NOTICE if level_compaction_dynamic_level_bytes=true,
// compression_per_level[0] still determines L0, but other elements
// of the array are based on base level (the level L0 files are merged
// to), and may not match the level users see from info log for metadata.
// If L0 files are merged to level-n, then, for i>0, compression_per_level[i]
// determines compaction type for level n+i-1.
// For example, if we have three 5 levels, and we determine to merge L0
// data to L4 (which means L1..L3 will be empty), then the new files go to
// L4 uses compression type compression_per_level[1].
// If now L0 is merged to L2. Data goes to L2 will be compressed
// according to compression_per_level[1], L3 using compression_per_level[2]
// and L4 using compression_per_level[3]. Compaction for each level can
// change when data grows.
//
// NOTE: if the vector size is smaller than the level number, the undefined
// lower level uses the last option in the vector, for example, for 3 level
// LSM tree the following settings are the same:
// {kNoCompression, kSnappyCompression}
// {kNoCompression, kSnappyCompression, kSnappyCompression}
//
// Dynamically changeable through SetOptions() API
std::vector<CompressionType> compression_per_level;
// Number of levels for this database
int num_levels = 7;
// Soft limit on number of level-0 files. We start slowing down writes at this
// point. A value <0 means that no writing slow down will be triggered by
// number of files in level-0.
//
// Default: 20
//
// Dynamically changeable through SetOptions() API
int level0_slowdown_writes_trigger = 20;
// Maximum number of level-0 files. We stop writes at this point.
//
// Default: 36
//
// Dynamically changeable through SetOptions() API
int level0_stop_writes_trigger = 36;
// Target file size for compaction.
// target_file_size_base is per-file size for level-1.
// Target file size for level L can be calculated by
// target_file_size_base * (target_file_size_multiplier ^ (L-1))
// For example, if target_file_size_base is 2MB and
// target_file_size_multiplier is 10, then each file on level-1 will
// be 2MB, and each file on level 2 will be 20MB,
// and each file on level-3 will be 200MB.
//
// Default: 64MB.
//
// Dynamically changeable through SetOptions() API
uint64_t target_file_size_base = 64 * 1048576;
// By default target_file_size_multiplier is 1, which means
// by default files in different levels will have similar size.
//
// Dynamically changeable through SetOptions() API
int target_file_size_multiplier = 1;
// If true, RocksDB will pick target size of each level dynamically.
// We will pick a base level b >= 1. L0 will be directly merged into level b,
// instead of always into level 1. Level 1 to b-1 need to be empty.
// We try to pick b and its target size so that
// 1. target size is in the range of
// (max_bytes_for_level_base / max_bytes_for_level_multiplier,
// max_bytes_for_level_base]
// 2. target size of the last level (level num_levels-1) equals to extra size
// of the level.
// At the same time max_bytes_for_level_multiplier and
// max_bytes_for_level_multiplier_additional are still satisfied.
// (When L0 is too large, we make some adjustment. See below.)
//
// With this option on, from an empty DB, we make last level the base level,
// which means merging L0 data into the last level, until it exceeds
// max_bytes_for_level_base. And then we make the second last level to be
// base level, to start to merge L0 data to second last level, with its
// target size to be 1/max_bytes_for_level_multiplier of the last level's
// extra size. After the data accumulates more so that we need to move the
// base level to the third last one, and so on.
//
// For example, assume max_bytes_for_level_multiplier=10, num_levels=6,
// and max_bytes_for_level_base=10MB.
// Target sizes of level 1 to 5 starts with:
// [- - - - 10MB]
// with base level is level. Target sizes of level 1 to 4 are not applicable
// because they will not be used.
// Until the size of Level 5 grows to more than 10MB, say 11MB, we make
// base target to level 4 and now the targets looks like:
// [- - - 1.1MB 11MB]
// While data are accumulated, size targets are tuned based on actual data
// of level 5. When level 5 has 50MB of data, the target is like:
// [- - - 5MB 50MB]
// Until level 5's actual size is more than 100MB, say 101MB. Now if we keep
// level 4 to be the base level, its target size needs to be 10.1MB, which
// doesn't satisfy the target size range. So now we make level 3 the target
// size and the target sizes of the levels look like:
// [- - 1.01MB 10.1MB 101MB]
// In the same way, while level 5 further grows, all levels' targets grow,
// like
// [- - 5MB 50MB 500MB]
// Until level 5 exceeds 1000MB and becomes 1001MB, we make level 2 the
// base level and make levels' target sizes like this:
// [- 1.001MB 10.01MB 100.1MB 1001MB]
// and go on...
//
// By doing it, we give max_bytes_for_level_multiplier a priority against
// max_bytes_for_level_base, for a more predictable LSM tree shape. It is
// useful to limit worse case space amplification.
//
//
// If the compaction from L0 is lagged behind, a special mode will be turned
// on to prioritize write amplification against max_bytes_for_level_multiplier
// or max_bytes_for_level_base. The L0 compaction is lagged behind by looking
// at number of L0 files and total L0 size. If number of L0 files is at least
// the double of level0_file_num_compaction_trigger, or the total size is
// at least max_bytes_for_level_base, this mode is on. The target of L1 grows
// to the actual data size in L0, and then determine the target for each level
// so that each level will have the same level multiplier.
//
// For example, when L0 size is 100MB, the size of last level is 1600MB,
// max_bytes_for_level_base = 80MB, and max_bytes_for_level_multiplier = 10.
// Since L0 size is larger than max_bytes_for_level_base, this is a L0
// compaction backlogged mode. So that the L1 size is determined to be 100MB.
// Based on max_bytes_for_level_multiplier = 10, at least 3 non-0 levels will
// be needed. The level multiplier will be calculated to be 4 and the three
// levels' target to be [100MB, 400MB, 1600MB].
//
// In this mode, The number of levels will be no more than the normal mode,
// and the level multiplier will be lower. The write amplification will
// likely to be reduced.
//
//
// max_bytes_for_level_multiplier_additional is ignored with this flag on.
//
// Turning this feature on or off for an existing DB can cause unexpected
// LSM tree structure so it's not recommended.
//
// Default: false
bool level_compaction_dynamic_level_bytes = false;
// Default: 10.
//
// Dynamically changeable through SetOptions() API
double max_bytes_for_level_multiplier = 10;
// Different max-size multipliers for different levels.
// These are multiplied by max_bytes_for_level_multiplier to arrive
// at the max-size of each level.
//
// Default: 1
//
// Dynamically changeable through SetOptions() API
std::vector<int> max_bytes_for_level_multiplier_additional =
std::vector<int>(num_levels, 1);
// We try to limit number of bytes in one compaction to be lower than this
// threshold. But it's not guaranteed.
// Value 0 will be sanitized.
//
// Default: target_file_size_base * 25
//
// Dynamically changeable through SetOptions() API
uint64_t max_compaction_bytes = 0;
// All writes will be slowed down to at least delayed_write_rate if estimated
// bytes needed to be compaction exceed this threshold.
//
// Default: 64GB
//
// Dynamically changeable through SetOptions() API
uint64_t soft_pending_compaction_bytes_limit = 64 * 1073741824ull;
// All writes are stopped if estimated bytes needed to be compaction exceed
// this threshold.
//
// Default: 256GB
//
// Dynamically changeable through SetOptions() API
uint64_t hard_pending_compaction_bytes_limit = 256 * 1073741824ull;
// The compaction style. Default: kCompactionStyleLevel
CompactionStyle compaction_style = kCompactionStyleLevel;
// If level compaction_style = kCompactionStyleLevel, for each level,
// which files are prioritized to be picked to compact.
// Default: kMinOverlappingRatio
CompactionPri compaction_pri = kMinOverlappingRatio;
// The options needed to support Universal Style compactions
//
// Dynamically changeable through SetOptions() API
// Dynamic change example:
// SetOptions("compaction_options_universal", "{size_ratio=2;}")
CompactionOptionsUniversal compaction_options_universal;
// The options for FIFO compaction style
//
// Dynamically changeable through SetOptions() API
// Dynamic change example:
// SetOptions("compaction_options_fifo", "{max_table_files_size=100;}")
CompactionOptionsFIFO compaction_options_fifo;
// An iteration->Next() sequentially skips over keys with the same
// user-key unless this option is set. This number specifies the number
// of keys (with the same userkey) that will be sequentially
// skipped before a reseek is issued.
//
// Default: 8
//
// Dynamically changeable through SetOptions() API
uint64_t max_sequential_skip_in_iterations = 8;
// This is a factory that provides MemTableRep objects.
// Default: a factory that provides a skip-list-based implementation of
// MemTableRep.
std::shared_ptr<MemTableRepFactory> memtable_factory =
std::shared_ptr<SkipListFactory>(new SkipListFactory);
// Block-based table related options are moved to BlockBasedTableOptions.
// Related options that were originally here but now moved include:
// no_block_cache
// block_cache
// block_cache_compressed
// block_size
// block_size_deviation
// block_restart_interval
// filter_policy
// whole_key_filtering
// If you'd like to customize some of these options, you will need to
// use NewBlockBasedTableFactory() to construct a new table factory.
// This option allows user to collect their own interested statistics of
// the tables.
// Default: empty vector -- no user-defined statistics collection will be
// performed.
using TablePropertiesCollectorFactories =
std::vector<std::shared_ptr<TablePropertiesCollectorFactory>>;
TablePropertiesCollectorFactories table_properties_collector_factories;
// Maximum number of successive merge operations on a key in the memtable.
//
// When a merge operation is added to the memtable and the maximum number of
// successive merges is reached, the value of the key will be calculated and
// inserted into the memtable instead of the merge operation. This will
// ensure that there are never more than max_successive_merges merge
// operations in the memtable.
//
// Default: 0 (disabled)
//
// Dynamically changeable through SetOptions() API
size_t max_successive_merges = 0;
// This flag specifies that the implementation should optimize the filters
// mainly for cases where keys are found rather than also optimize for keys
// missed. This would be used in cases where the application knows that
// there are very few misses or the performance in the case of misses is not
// important.
//
// For now, this flag allows us to not store filters for the last level i.e
// the largest level which contains data of the LSM store. For keys which
// are hits, the filters in this level are not useful because we will search
// for the data anyway. NOTE: the filters in other levels are still useful
// even for key hit because they tell us whether to look in that level or go
// to the higher level.
//
// Default: false
bool optimize_filters_for_hits = false;
// During flush or compaction, check whether keys inserted to output files
// are in order.
//
// Default: true
//
// Dynamically changeable through SetOptions() API
bool check_flush_compaction_key_order = true;
// After writing every SST file, reopen it and read all the keys.
// Checks the hash of all of the keys and values written versus the
// keys in the file and signals a corruption if they do not match
//
// Default: false
//
// Dynamically changeable through SetOptions() API
bool paranoid_file_checks = false;
// In debug mode, RocksDB runs consistency checks on the LSM every time the
// LSM changes (Flush, Compaction, AddFile). When this option is true, these
// checks are also enabled in release mode. These checks were historically
// disabled in release mode, but are now enabled by default for proactive
// corruption detection. The CPU overhead is negligible for normal mixed
// operations but can slow down saturated writing. See
// Options::DisableExtraChecks().
// Default: true
bool force_consistency_checks = true;
// Measure IO stats in compactions and flushes, if true.
//
// Default: false
//
// Dynamically changeable through SetOptions() API
bool report_bg_io_stats = false;
// Files containing updates older than TTL will go through the compaction
// process. This usually happens in a cascading way so that those entries
// will be compacted to bottommost level/file.
// The feature is used to remove stale entries that have been deleted or
// updated from the file system.
// Pre-req: This needs max_open_files to be set to -1.
// In Level: Non-bottom-level files older than TTL will go through the
// compaction process.
// In FIFO: Files older than TTL will be deleted.
// unit: seconds. Ex: 1 day = 1 * 24 * 60 * 60
// In FIFO, this option will have the same meaning as
// periodic_compaction_seconds. Whichever stricter will be used.
// 0 means disabling.
// UINT64_MAX - 1 (0xfffffffffffffffe) is special flag to allow RocksDB to
// pick default.
//
// Default: 30 days for leveled compaction + block based table. disable
// otherwise.
//
// Dynamically changeable through SetOptions() API
uint64_t ttl = 0xfffffffffffffffe;
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
2019-04-11 04:24:25 +02:00
// Files older than this value will be picked up for compaction, and
// re-written to the same level as they were before.
// One main use of the feature is to make sure a file goes through compaction
// filters periodically. Users can also use the feature to clear up SST
// files using old format.
//
// A file's age is computed by looking at file_creation_time or creation_time
// table properties in order, if they have valid non-zero values; if not, the
// age is based on the file's last modified time (given by the underlying
// Env).
//
// Supported in Level and FIFO compaction.
// In FIFO compaction, this option has the same meaning as TTL and whichever
// stricter will be used.
// Pre-req: max_open_file == -1.
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
2019-04-11 04:24:25 +02:00
// unit: seconds. Ex: 7 days = 7 * 24 * 60 * 60
Auto enable Periodic Compactions if a Compaction Filter is used (#5865) Summary: - Periodic compactions are auto-enabled if a compaction filter or a compaction filter factory is set, in Level Compaction. - The default value of `periodic_compaction_seconds` is changed to UINT64_MAX, which lets RocksDB auto-tune periodic compactions as needed. An explicit value of 0 will still work as before ie. to disable periodic compactions completely. For now, on seeing a compaction filter along with a UINT64_MAX value for `periodic_compaction_seconds`, RocksDB will make SST files older than 30 days to go through periodic copmactions. Some RocksDB users make use of compaction filters to control when their data can be deleted, usually with a custom TTL logic. But it is occasionally possible that the compactions get delayed by considerable time due to factors like low writes to a key range, data reaching bottom level, etc before the TTL expiry. Periodic Compactions feature was originally built to help such cases. Now periodic compactions are auto enabled by default when compaction filters or compaction filter factories are used, as it is generally helpful to all cases to collect garbage. `periodic_compaction_seconds` is set to a large value, 30 days, in `SanitizeOptions` when RocksDB sees that a `compaction_filter` or `compaction_filter_factory` is used. This is done only for Level Compaction style. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5865 Test Plan: - Added a new test `DBCompactionTest.LevelPeriodicCompactionWithCompactionFilters` to make sure that `periodic_compaction_seconds` is set if either `compaction_filter` or `compaction_filter_factory` options are set. - `COMPILE_WITH_ASAN=1 make check` Differential Revision: D17659180 Pulled By: sagar0 fbshipit-source-id: 4887b9cf2e53cf2dc93a7b658c6b15e1181217ee
2019-10-29 23:04:26 +01:00
//
// Values:
// 0: Turn off Periodic compactions.
// UINT64_MAX - 1 (i.e 0xfffffffffffffffe): Let RocksDB control this feature
Auto enable Periodic Compactions if a Compaction Filter is used (#5865) Summary: - Periodic compactions are auto-enabled if a compaction filter or a compaction filter factory is set, in Level Compaction. - The default value of `periodic_compaction_seconds` is changed to UINT64_MAX, which lets RocksDB auto-tune periodic compactions as needed. An explicit value of 0 will still work as before ie. to disable periodic compactions completely. For now, on seeing a compaction filter along with a UINT64_MAX value for `periodic_compaction_seconds`, RocksDB will make SST files older than 30 days to go through periodic copmactions. Some RocksDB users make use of compaction filters to control when their data can be deleted, usually with a custom TTL logic. But it is occasionally possible that the compactions get delayed by considerable time due to factors like low writes to a key range, data reaching bottom level, etc before the TTL expiry. Periodic Compactions feature was originally built to help such cases. Now periodic compactions are auto enabled by default when compaction filters or compaction filter factories are used, as it is generally helpful to all cases to collect garbage. `periodic_compaction_seconds` is set to a large value, 30 days, in `SanitizeOptions` when RocksDB sees that a `compaction_filter` or `compaction_filter_factory` is used. This is done only for Level Compaction style. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5865 Test Plan: - Added a new test `DBCompactionTest.LevelPeriodicCompactionWithCompactionFilters` to make sure that `periodic_compaction_seconds` is set if either `compaction_filter` or `compaction_filter_factory` options are set. - `COMPILE_WITH_ASAN=1 make check` Differential Revision: D17659180 Pulled By: sagar0 fbshipit-source-id: 4887b9cf2e53cf2dc93a7b658c6b15e1181217ee
2019-10-29 23:04:26 +01:00
// as needed. For now, RocksDB will change this value to 30 days
// (i.e 30 * 24 * 60 * 60) so that every file goes through the compaction
// process at least once every 30 days if not compacted sooner.
// In FIFO compaction, since the option has the same meaning as ttl,
// when this value is left default, and ttl is left to 0, 30 days will be
// used. Otherwise, min(ttl, periodic_compaction_seconds) will be used.
Auto enable Periodic Compactions if a Compaction Filter is used (#5865) Summary: - Periodic compactions are auto-enabled if a compaction filter or a compaction filter factory is set, in Level Compaction. - The default value of `periodic_compaction_seconds` is changed to UINT64_MAX, which lets RocksDB auto-tune periodic compactions as needed. An explicit value of 0 will still work as before ie. to disable periodic compactions completely. For now, on seeing a compaction filter along with a UINT64_MAX value for `periodic_compaction_seconds`, RocksDB will make SST files older than 30 days to go through periodic copmactions. Some RocksDB users make use of compaction filters to control when their data can be deleted, usually with a custom TTL logic. But it is occasionally possible that the compactions get delayed by considerable time due to factors like low writes to a key range, data reaching bottom level, etc before the TTL expiry. Periodic Compactions feature was originally built to help such cases. Now periodic compactions are auto enabled by default when compaction filters or compaction filter factories are used, as it is generally helpful to all cases to collect garbage. `periodic_compaction_seconds` is set to a large value, 30 days, in `SanitizeOptions` when RocksDB sees that a `compaction_filter` or `compaction_filter_factory` is used. This is done only for Level Compaction style. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5865 Test Plan: - Added a new test `DBCompactionTest.LevelPeriodicCompactionWithCompactionFilters` to make sure that `periodic_compaction_seconds` is set if either `compaction_filter` or `compaction_filter_factory` options are set. - `COMPILE_WITH_ASAN=1 make check` Differential Revision: D17659180 Pulled By: sagar0 fbshipit-source-id: 4887b9cf2e53cf2dc93a7b658c6b15e1181217ee
2019-10-29 23:04:26 +01:00
//
// Default: UINT64_MAX - 1 (allow RocksDB to auto-tune)
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
2019-04-11 04:24:25 +02:00
//
// Dynamically changeable through SetOptions() API
uint64_t periodic_compaction_seconds = 0xfffffffffffffffe;
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
2019-04-11 04:24:25 +02:00
// If this option is set then 1 in N blocks are compressed
// using a fast (lz4) and slow (zstd) compression algorithm.
// The compressibility is reported as stats and the stored
// data is left uncompressed (unless compression is also requested).
uint64_t sample_for_compression = 0;
// EXPERIMENTAL
// The feature is still in development and is incomplete.
// If this option is set, when creating bottommost files, pass this
// temperature to FileSystem used. Should be no-op for default FileSystem
// and users need to plug in their own FileSystem to take advantage of it.
//
// Dynamically changeable through the SetOptions() API
Temperature bottommost_temperature = Temperature::kUnknown;
// When set, large values (blobs) are written to separate blob files, and
// only pointers to them are stored in SST files. This can reduce write
// amplification for large-value use cases at the cost of introducing a level
// of indirection for reads. See also the options min_blob_size,
// blob_file_size, blob_compression_type, enable_blob_garbage_collection,
// blob_garbage_collection_age_cutoff,
// blob_garbage_collection_force_threshold, and blob_compaction_readahead_size
// below.
//
// Default: false
//
// Dynamically changeable through the SetOptions() API
bool enable_blob_files = false;
// The size of the smallest value to be stored separately in a blob file.
// Values which have an uncompressed size smaller than this threshold are
// stored alongside the keys in SST files in the usual fashion. A value of
// zero for this option means that all values are stored in blob files. Note
// that enable_blob_files has to be set in order for this option to have any
// effect.
//
// Default: 0
//
// Dynamically changeable through the SetOptions() API
uint64_t min_blob_size = 0;
// The size limit for blob files. When writing blob files, a new file is
// opened once this limit is reached. Note that enable_blob_files has to be
// set in order for this option to have any effect.
//
// Default: 256 MB
//
// Dynamically changeable through the SetOptions() API
uint64_t blob_file_size = 1ULL << 28;
// The compression algorithm to use for large values stored in blob files.
// Note that enable_blob_files has to be set in order for this option to have
// any effect.
//
// Default: no compression
//
// Dynamically changeable through the SetOptions() API
CompressionType blob_compression_type = kNoCompression;
// Enables garbage collection of blobs. Blob GC is performed as part of
// compaction. Valid blobs residing in blob files older than a cutoff get
// relocated to new files as they are encountered during compaction, which
// makes it possible to clean up blob files once they contain nothing but
// obsolete/garbage blobs. See also blob_garbage_collection_age_cutoff and
// blob_garbage_collection_force_threshold below.
//
// Default: false
//
// Dynamically changeable through the SetOptions() API
bool enable_blob_garbage_collection = false;
// The cutoff in terms of blob file age for garbage collection. Blobs in
// the oldest N blob files will be relocated when encountered during
// compaction, where N = garbage_collection_cutoff * number_of_blob_files.
// Note that enable_blob_garbage_collection has to be set in order for this
// option to have any effect.
//
// Default: 0.25
//
// Dynamically changeable through the SetOptions() API
double blob_garbage_collection_age_cutoff = 0.25;
Make it possible to force the garbage collection of the oldest blob files (#8994) Summary: The current BlobDB garbage collection logic works by relocating the valid blobs from the oldest blob files as they are encountered during compaction, and cleaning up blob files once they contain nothing but garbage. However, with sufficiently skewed workloads, it is theoretically possible to end up in a situation when few or no compactions get scheduled for the SST files that contain references to the oldest blob files, which can lead to increased space amp due to the lack of GC. In order to efficiently handle such workloads, the patch adds a new BlobDB configuration option called `blob_garbage_collection_force_threshold`, which signals to BlobDB to schedule targeted compactions for the SST files that keep alive the oldest batch of blob files if the overall ratio of garbage in the given blob files meets the threshold *and* all the given blob files are eligible for GC based on `blob_garbage_collection_age_cutoff`. (For example, if the new option is set to 0.9, targeted compactions will get scheduled if the sum of garbage bytes meets or exceeds 90% of the sum of total bytes in the oldest blob files, assuming all affected blob files are below the age-based cutoff.) The net result of these targeted compactions is that the valid blobs in the oldest blob files are relocated and the oldest blob files themselves cleaned up (since *all* SST files that rely on them get compacted away). These targeted compactions are similar to periodic compactions in the sense that they force certain SST files that otherwise would not get picked up to undergo compaction and also in the sense that instead of merging files from multiple levels, they target a single file. (Note: such compactions might still include neighboring files from the same level due to the need of having a "clean cut" boundary but they never include any files from any other level.) This functionality is currently only supported with the leveled compaction style and is inactive by default (since the default value is set to 1.0, i.e. 100%). Pull Request resolved: https://github.com/facebook/rocksdb/pull/8994 Test Plan: Ran `make check` and tested using `db_bench` and the stress/crash tests. Reviewed By: riversand963 Differential Revision: D31489850 Pulled By: ltamasi fbshipit-source-id: 44057d511726a0e2a03c5d9313d7511b3f0c4eab
2021-10-12 03:00:44 +02:00
// If the ratio of garbage in the oldest blob files exceeds this threshold,
// targeted compactions are scheduled in order to force garbage collecting
// the blob files in question, assuming they are all eligible based on the
// value of blob_garbage_collection_age_cutoff above. This option is
// currently only supported with leveled compactions.
// Note that enable_blob_garbage_collection has to be set in order for this
// option to have any effect.
//
// Default: 1.0
//
// Dynamically changeable through the SetOptions() API
double blob_garbage_collection_force_threshold = 1.0;
// Compaction readahead for blob files.
//
// Default: 0
//
// Dynamically changeable through the SetOptions() API
uint64_t blob_compaction_readahead_size = 0;
// Create ColumnFamilyOptions with default values for all fields
AdvancedColumnFamilyOptions();
// Create ColumnFamilyOptions from Options
explicit AdvancedColumnFamilyOptions(const Options& options);
// ---------------- OPTIONS NOT SUPPORTED ANYMORE ----------------
};
} // namespace ROCKSDB_NAMESPACE