2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2017-07-16 01:03:42 +02:00
|
|
|
// 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).
|
2014-01-24 23:30:28 +01:00
|
|
|
//
|
|
|
|
// 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.
|
|
|
|
|
2014-01-22 20:44:53 +01:00
|
|
|
#include "db/column_family.h"
|
2014-01-24 23:30:28 +01:00
|
|
|
|
|
|
|
#include <algorithm>
|
2019-09-20 21:00:55 +02:00
|
|
|
#include <cinttypes>
|
2014-05-21 20:43:35 +02:00
|
|
|
#include <limits>
|
2019-09-20 21:00:55 +02:00
|
|
|
#include <string>
|
|
|
|
#include <vector>
|
2014-01-24 23:30:28 +01:00
|
|
|
|
2019-05-31 20:52:59 +02:00
|
|
|
#include "db/compaction/compaction_picker.h"
|
|
|
|
#include "db/compaction/compaction_picker_fifo.h"
|
|
|
|
#include "db/compaction/compaction_picker_level.h"
|
|
|
|
#include "db/compaction/compaction_picker_universal.h"
|
|
|
|
#include "db/db_impl/db_impl.h"
|
2014-02-05 02:45:19 +01:00
|
|
|
#include "db/internal_stats.h"
|
2015-08-19 22:32:09 +02:00
|
|
|
#include "db/job_context.h"
|
2018-12-18 02:26:56 +01:00
|
|
|
#include "db/range_del_aggregator.h"
|
2014-02-05 01:31:18 +01:00
|
|
|
#include "db/table_properties_collector.h"
|
2014-12-02 19:53:39 +01:00
|
|
|
#include "db/version_set.h"
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
#include "db/write_controller.h"
|
2019-05-30 05:44:08 +02:00
|
|
|
#include "file/sst_file_manager_impl.h"
|
2015-10-16 23:10:33 +02:00
|
|
|
#include "memtable/hash_skiplist_rep.h"
|
2017-04-06 04:02:00 +02:00
|
|
|
#include "monitoring/thread_status_util.h"
|
|
|
|
#include "options/options_helper.h"
|
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
|
|
|
#include "port/port.h"
|
2019-05-30 23:47:29 +02:00
|
|
|
#include "table/block_based/block_based_table_factory.h"
|
2018-02-28 02:08:34 +01:00
|
|
|
#include "table/merging_iterator.h"
|
2014-02-07 00:42:16 +01:00
|
|
|
#include "util/autovector.h"
|
2020-07-03 04:24:25 +02:00
|
|
|
#include "util/cast_util.h"
|
2015-06-18 23:55:05 +02:00
|
|
|
#include "util/compression.h"
|
2014-01-22 20:44:53 +01:00
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2014-01-22 20:44:53 +01:00
|
|
|
|
2014-11-06 20:14:28 +01:00
|
|
|
ColumnFamilyHandleImpl::ColumnFamilyHandleImpl(
|
2015-02-05 06:39:45 +01:00
|
|
|
ColumnFamilyData* column_family_data, DBImpl* db, InstrumentedMutex* mutex)
|
2014-11-06 20:14:28 +01:00
|
|
|
: cfd_(column_family_data), db_(db), mutex_(mutex) {
|
2014-02-11 02:04:44 +01:00
|
|
|
if (cfd_ != nullptr) {
|
|
|
|
cfd_->Ref();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyHandleImpl::~ColumnFamilyHandleImpl() {
|
|
|
|
if (cfd_ != nullptr) {
|
2016-09-22 20:56:18 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
for (auto& listener : cfd_->ioptions()->listeners) {
|
|
|
|
listener->OnColumnFamilyHandleDeletionStarted(this);
|
|
|
|
}
|
|
|
|
#endif // ROCKSDB_LITE
|
2015-02-12 18:54:48 +01:00
|
|
|
// Job id == 0 means that this is not our background process, but rather
|
|
|
|
// user thread
|
2018-03-21 01:07:28 +01:00
|
|
|
// Need to hold some shared pointers owned by the initial_cf_options
|
|
|
|
// before final cleaning up finishes.
|
|
|
|
ColumnFamilyOptions initial_cf_options_copy = cfd_->initial_cf_options();
|
2015-02-12 18:54:48 +01:00
|
|
|
JobContext job_context(0);
|
2014-02-11 02:04:44 +01:00
|
|
|
mutex_->Lock();
|
2019-12-13 04:02:51 +01:00
|
|
|
bool dropped = cfd_->IsDropped();
|
|
|
|
if (cfd_->UnrefAndTryDelete()) {
|
2019-04-25 02:07:31 +02:00
|
|
|
if (dropped) {
|
|
|
|
db_->FindObsoleteFiles(&job_context, false, true);
|
|
|
|
}
|
2014-02-11 02:04:44 +01:00
|
|
|
}
|
|
|
|
mutex_->Unlock();
|
2014-10-28 19:54:33 +01:00
|
|
|
if (job_context.HaveSomethingToDelete()) {
|
2019-04-02 02:07:38 +02:00
|
|
|
bool defer_purge =
|
|
|
|
db_->immutable_db_options().avoid_unnecessary_blocking_io;
|
|
|
|
db_->PurgeObsoleteFiles(job_context, defer_purge);
|
|
|
|
if (defer_purge) {
|
|
|
|
mutex_->Lock();
|
|
|
|
db_->SchedulePurge();
|
|
|
|
mutex_->Unlock();
|
|
|
|
}
|
2014-03-11 01:25:10 +01:00
|
|
|
}
|
2015-07-07 21:10:10 +02:00
|
|
|
job_context.Clean();
|
2014-02-11 02:04:44 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-02-26 02:30:54 +01:00
|
|
|
uint32_t ColumnFamilyHandleImpl::GetID() const { return cfd()->GetID(); }
|
|
|
|
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
const std::string& ColumnFamilyHandleImpl::GetName() const {
|
|
|
|
return cfd()->GetName();
|
|
|
|
}
|
|
|
|
|
2016-01-07 03:14:01 +01:00
|
|
|
Status ColumnFamilyHandleImpl::GetDescriptor(ColumnFamilyDescriptor* desc) {
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
// accessing mutable cf-options requires db mutex.
|
|
|
|
InstrumentedMutexLock l(mutex_);
|
2016-09-17 00:09:14 +02:00
|
|
|
*desc = ColumnFamilyDescriptor(cfd()->GetName(), cfd()->GetLatestCFOptions());
|
2016-01-07 03:14:01 +01:00
|
|
|
return Status::OK();
|
|
|
|
#else
|
2018-04-13 02:55:14 +02:00
|
|
|
(void)desc;
|
2016-01-07 03:14:01 +01:00
|
|
|
return Status::NotSupported();
|
|
|
|
#endif // !ROCKSDB_LITE
|
|
|
|
}
|
|
|
|
|
2016-07-26 00:06:11 +02:00
|
|
|
const Comparator* ColumnFamilyHandleImpl::GetComparator() const {
|
2014-09-22 20:37:35 +02:00
|
|
|
return cfd()->user_comparator();
|
|
|
|
}
|
|
|
|
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
2015-04-06 19:04:30 +02:00
|
|
|
void GetIntTblPropCollectorFactory(
|
2016-09-24 01:34:04 +02:00
|
|
|
const ImmutableCFOptions& ioptions,
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
2015-04-06 19:04:30 +02:00
|
|
|
std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
|
|
|
|
int_tbl_prop_collector_factories) {
|
2016-09-24 01:34:04 +02:00
|
|
|
auto& collector_factories = ioptions.table_properties_collector_factories;
|
|
|
|
for (size_t i = 0; i < ioptions.table_properties_collector_factories.size();
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
2015-04-06 19:04:30 +02:00
|
|
|
++i) {
|
|
|
|
assert(collector_factories[i]);
|
|
|
|
int_tbl_prop_collector_factories->emplace_back(
|
|
|
|
new UserKeyTablePropertiesCollectorFactory(collector_factories[i]));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-06-18 23:55:05 +02:00
|
|
|
Status CheckCompressionSupported(const ColumnFamilyOptions& cf_options) {
|
|
|
|
if (!cf_options.compression_per_level.empty()) {
|
|
|
|
for (size_t level = 0; level < cf_options.compression_per_level.size();
|
|
|
|
++level) {
|
|
|
|
if (!CompressionTypeSupported(cf_options.compression_per_level[level])) {
|
|
|
|
return Status::InvalidArgument(
|
|
|
|
"Compression type " +
|
|
|
|
CompressionTypeToString(cf_options.compression_per_level[level]) +
|
|
|
|
" is not linked with the binary.");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
if (!CompressionTypeSupported(cf_options.compression)) {
|
|
|
|
return Status::InvalidArgument(
|
|
|
|
"Compression type " +
|
|
|
|
CompressionTypeToString(cf_options.compression) +
|
|
|
|
" is not linked with the binary.");
|
|
|
|
}
|
|
|
|
}
|
2017-11-03 06:46:13 +01:00
|
|
|
if (cf_options.compression_opts.zstd_max_train_bytes > 0) {
|
2019-01-19 04:10:17 +01:00
|
|
|
if (!ZSTD_TrainDictionarySupported()) {
|
2017-11-03 06:46:13 +01:00
|
|
|
return Status::InvalidArgument(
|
2019-01-19 04:10:17 +01:00
|
|
|
"zstd dictionary trainer cannot be used because ZSTD 1.1.3+ "
|
|
|
|
"is not linked with the binary.");
|
2017-11-03 06:46:13 +01:00
|
|
|
}
|
|
|
|
if (cf_options.compression_opts.max_dict_bytes == 0) {
|
|
|
|
return Status::InvalidArgument(
|
|
|
|
"The dictionary size limit (`CompressionOptions::max_dict_bytes`) "
|
|
|
|
"should be nonzero if we're using zstd's dictionary generator.");
|
|
|
|
}
|
|
|
|
}
|
2015-06-18 23:55:05 +02:00
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
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
2015-08-15 01:59:07 +02:00
|
|
|
Status CheckConcurrentWritesSupported(const ColumnFamilyOptions& cf_options) {
|
|
|
|
if (cf_options.inplace_update_support) {
|
|
|
|
return Status::InvalidArgument(
|
|
|
|
"In-place memtable updates (inplace_update_support) is not compatible "
|
|
|
|
"with concurrent writes (allow_concurrent_memtable_write)");
|
|
|
|
}
|
2016-02-05 22:46:17 +01:00
|
|
|
if (!cf_options.memtable_factory->IsInsertConcurrentlySupported()) {
|
|
|
|
return Status::InvalidArgument(
|
|
|
|
"Memtable doesn't concurrent writes (allow_concurrent_memtable_write)");
|
|
|
|
}
|
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
2015-08-15 01:59:07 +02:00
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2018-04-06 04:49:06 +02:00
|
|
|
Status CheckCFPathsSupported(const DBOptions& db_options,
|
|
|
|
const ColumnFamilyOptions& cf_options) {
|
|
|
|
// More than one cf_paths are supported only in universal
|
|
|
|
// and level compaction styles. This function also checks the case
|
|
|
|
// in which cf_paths is not specified, which results in db_paths
|
|
|
|
// being used.
|
|
|
|
if ((cf_options.compaction_style != kCompactionStyleUniversal) &&
|
|
|
|
(cf_options.compaction_style != kCompactionStyleLevel)) {
|
|
|
|
if (cf_options.cf_paths.size() > 1) {
|
|
|
|
return Status::NotSupported(
|
|
|
|
"More than one CF paths are only supported in "
|
|
|
|
"universal and level compaction styles. ");
|
|
|
|
} else if (cf_options.cf_paths.empty() &&
|
|
|
|
db_options.db_paths.size() > 1) {
|
|
|
|
return Status::NotSupported(
|
|
|
|
"More than one DB paths are only supported in "
|
|
|
|
"universal and level compaction styles. ");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2019-11-26 02:11:26 +01:00
|
|
|
namespace {
|
|
|
|
const uint64_t kDefaultTtl = 0xfffffffffffffffe;
|
|
|
|
const uint64_t kDefaultPeriodicCompSecs = 0xfffffffffffffffe;
|
|
|
|
}; // namespace
|
|
|
|
|
2016-09-24 01:34:04 +02:00
|
|
|
ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
2015-02-05 20:44:17 +01:00
|
|
|
const ColumnFamilyOptions& src) {
|
2014-02-05 01:31:18 +01:00
|
|
|
ColumnFamilyOptions result = src;
|
ColumnFamilyOptions SanitizeOptions is buggy on 32-bit platforms.
Summary:
The pre-existing code is trying to clamp between 65,536 and 0,
resulting in clamping to 65,536, resulting in very small buffers,
resulting in ShouldFlushNow() being true quite easily,
resulting in assertion failing and database performance
being "not what it should be".
https://github.com/facebook/rocksdb/issues/1018
Test Plan: make check
Reviewers: sdong, andrewkr, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D55455
2016-03-14 23:04:40 +01:00
|
|
|
size_t clamp_max = std::conditional<
|
|
|
|
sizeof(size_t) == 4, std::integral_constant<size_t, 0xffffffff>,
|
2016-05-02 19:04:37 +02:00
|
|
|
std::integral_constant<uint64_t, 64ull << 30>>::type::value;
|
ColumnFamilyOptions SanitizeOptions is buggy on 32-bit platforms.
Summary:
The pre-existing code is trying to clamp between 65,536 and 0,
resulting in clamping to 65,536, resulting in very small buffers,
resulting in ShouldFlushNow() being true quite easily,
resulting in assertion failing and database performance
being "not what it should be".
https://github.com/facebook/rocksdb/issues/1018
Test Plan: make check
Reviewers: sdong, andrewkr, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D55455
2016-03-14 23:04:40 +01:00
|
|
|
ClipToRange(&result.write_buffer_size, ((size_t)64) << 10, clamp_max);
|
2014-02-05 01:31:18 +01:00
|
|
|
// if user sets arena_block_size, we trust user to use this value. Otherwise,
|
|
|
|
// calculate a proper value from writer_buffer_size;
|
|
|
|
if (result.arena_block_size <= 0) {
|
2015-09-01 23:43:23 +02:00
|
|
|
result.arena_block_size = result.write_buffer_size / 8;
|
|
|
|
|
|
|
|
// Align up to 4k
|
|
|
|
const size_t align = 4 * 1024;
|
|
|
|
result.arena_block_size =
|
|
|
|
((result.arena_block_size + align - 1) / align) * align;
|
2014-02-05 01:31:18 +01:00
|
|
|
}
|
|
|
|
result.min_write_buffer_number_to_merge =
|
|
|
|
std::min(result.min_write_buffer_number_to_merge,
|
|
|
|
result.max_write_buffer_number - 1);
|
2016-11-15 22:38:24 +01:00
|
|
|
if (result.min_write_buffer_number_to_merge < 1) {
|
|
|
|
result.min_write_buffer_number_to_merge = 1;
|
|
|
|
}
|
|
|
|
|
2015-05-22 20:35:40 +02:00
|
|
|
if (result.num_levels < 1) {
|
|
|
|
result.num_levels = 1;
|
|
|
|
}
|
|
|
|
if (result.compaction_style == kCompactionStyleLevel &&
|
|
|
|
result.num_levels < 2) {
|
|
|
|
result.num_levels = 2;
|
|
|
|
}
|
2017-05-17 20:32:26 +02:00
|
|
|
|
|
|
|
if (result.compaction_style == kCompactionStyleUniversal &&
|
|
|
|
db_options.allow_ingest_behind && result.num_levels < 3) {
|
|
|
|
result.num_levels = 3;
|
|
|
|
}
|
|
|
|
|
2014-06-17 01:26:46 +02:00
|
|
|
if (result.max_write_buffer_number < 2) {
|
|
|
|
result.max_write_buffer_number = 2;
|
|
|
|
}
|
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
|
|
|
// fall back max_write_buffer_number_to_maintain if
|
|
|
|
// max_write_buffer_size_to_maintain is not set
|
|
|
|
if (result.max_write_buffer_size_to_maintain < 0) {
|
|
|
|
result.max_write_buffer_size_to_maintain =
|
|
|
|
result.max_write_buffer_number *
|
|
|
|
static_cast<int64_t>(result.write_buffer_size);
|
|
|
|
} else if (result.max_write_buffer_size_to_maintain == 0 &&
|
|
|
|
result.max_write_buffer_number_to_maintain < 0) {
|
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
2015-05-29 01:34:24 +02:00
|
|
|
result.max_write_buffer_number_to_maintain = result.max_write_buffer_number;
|
|
|
|
}
|
2016-06-04 02:02:10 +02:00
|
|
|
// bloom filter size shouldn't exceed 1/4 of memtable size.
|
|
|
|
if (result.memtable_prefix_bloom_size_ratio > 0.25) {
|
|
|
|
result.memtable_prefix_bloom_size_ratio = 0.25;
|
|
|
|
} else if (result.memtable_prefix_bloom_size_ratio < 0) {
|
|
|
|
result.memtable_prefix_bloom_size_ratio = 0;
|
|
|
|
}
|
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
2015-05-29 01:34:24 +02:00
|
|
|
|
2014-03-11 01:25:10 +01:00
|
|
|
if (!result.prefix_extractor) {
|
|
|
|
assert(result.memtable_factory);
|
|
|
|
Slice name = result.memtable_factory->Name();
|
|
|
|
if (name.compare("HashSkipListRepFactory") == 0 ||
|
|
|
|
name.compare("HashLinkListRepFactory") == 0) {
|
2014-02-05 01:31:18 +01:00
|
|
|
result.memtable_factory = std::make_shared<SkipListFactory>();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-05-21 20:43:35 +02:00
|
|
|
if (result.compaction_style == kCompactionStyleFIFO) {
|
|
|
|
result.num_levels = 1;
|
|
|
|
// since we delete level0 files in FIFO compaction when there are too many
|
|
|
|
// of them, these options don't really mean anything
|
|
|
|
result.level0_slowdown_writes_trigger = std::numeric_limits<int>::max();
|
|
|
|
result.level0_stop_writes_trigger = std::numeric_limits<int>::max();
|
|
|
|
}
|
|
|
|
|
2016-06-17 01:02:52 +02:00
|
|
|
if (result.max_bytes_for_level_multiplier <= 0) {
|
|
|
|
result.max_bytes_for_level_multiplier = 1;
|
|
|
|
}
|
|
|
|
|
2015-11-13 21:01:52 +01:00
|
|
|
if (result.level0_file_num_compaction_trigger == 0) {
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(db_options.info_log.get(),
|
|
|
|
"level0_file_num_compaction_trigger cannot be 0");
|
2015-11-13 21:01:52 +01:00
|
|
|
result.level0_file_num_compaction_trigger = 1;
|
|
|
|
}
|
|
|
|
|
2015-02-24 01:08:27 +01:00
|
|
|
if (result.level0_stop_writes_trigger <
|
|
|
|
result.level0_slowdown_writes_trigger ||
|
|
|
|
result.level0_slowdown_writes_trigger <
|
|
|
|
result.level0_file_num_compaction_trigger) {
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(db_options.info_log.get(),
|
|
|
|
"This condition must be satisfied: "
|
|
|
|
"level0_stop_writes_trigger(%d) >= "
|
|
|
|
"level0_slowdown_writes_trigger(%d) >= "
|
|
|
|
"level0_file_num_compaction_trigger(%d)",
|
|
|
|
result.level0_stop_writes_trigger,
|
|
|
|
result.level0_slowdown_writes_trigger,
|
|
|
|
result.level0_file_num_compaction_trigger);
|
2015-02-24 01:08:27 +01:00
|
|
|
if (result.level0_slowdown_writes_trigger <
|
|
|
|
result.level0_file_num_compaction_trigger) {
|
|
|
|
result.level0_slowdown_writes_trigger =
|
|
|
|
result.level0_file_num_compaction_trigger;
|
|
|
|
}
|
|
|
|
if (result.level0_stop_writes_trigger <
|
|
|
|
result.level0_slowdown_writes_trigger) {
|
|
|
|
result.level0_stop_writes_trigger = result.level0_slowdown_writes_trigger;
|
|
|
|
}
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(db_options.info_log.get(),
|
|
|
|
"Adjust the value to "
|
|
|
|
"level0_stop_writes_trigger(%d)"
|
|
|
|
"level0_slowdown_writes_trigger(%d)"
|
|
|
|
"level0_file_num_compaction_trigger(%d)",
|
|
|
|
result.level0_stop_writes_trigger,
|
|
|
|
result.level0_slowdown_writes_trigger,
|
|
|
|
result.level0_file_num_compaction_trigger);
|
2015-02-24 01:08:27 +01:00
|
|
|
}
|
Add options.base_background_compactions as a number of compaction threads for low compaction debt
Summary:
If options.base_background_compactions is given, we try to schedule number of compactions not existing this number, only when L0 files increase to certain number, or pending compaction bytes more than certain threshold, we schedule compactions based on options.max_background_compactions.
The watermarks are calculated based on slowdown thresholds.
Test Plan:
Add new test cases in column_family_test.
Adding more unit tests.
Reviewers: IslamAbdelRahman, yhchiang, kradhakrishnan, rven, anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D53409
2016-01-28 20:56:16 +01:00
|
|
|
|
|
|
|
if (result.soft_pending_compaction_bytes_limit == 0) {
|
|
|
|
result.soft_pending_compaction_bytes_limit =
|
|
|
|
result.hard_pending_compaction_bytes_limit;
|
|
|
|
} else if (result.hard_pending_compaction_bytes_limit > 0 &&
|
|
|
|
result.soft_pending_compaction_bytes_limit >
|
|
|
|
result.hard_pending_compaction_bytes_limit) {
|
|
|
|
result.soft_pending_compaction_bytes_limit =
|
|
|
|
result.hard_pending_compaction_bytes_limit;
|
|
|
|
}
|
|
|
|
|
2018-04-06 04:49:06 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
// When the DB is stopped, it's possible that there are some .trash files that
|
|
|
|
// were not deleted yet, when we open the DB we will find these .trash files
|
|
|
|
// and schedule them to be deleted (or delete immediately if SstFileManager
|
|
|
|
// was not used)
|
|
|
|
auto sfm = static_cast<SstFileManagerImpl*>(db_options.sst_file_manager.get());
|
|
|
|
for (size_t i = 0; i < result.cf_paths.size(); i++) {
|
|
|
|
DeleteScheduler::CleanupDirectory(db_options.env, sfm, result.cf_paths[i].path);
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
|
|
|
|
if (result.cf_paths.empty()) {
|
|
|
|
result.cf_paths = db_options.db_paths;
|
|
|
|
}
|
|
|
|
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
2015-02-05 20:44:17 +01:00
|
|
|
if (result.level_compaction_dynamic_level_bytes) {
|
|
|
|
if (result.compaction_style != kCompactionStyleLevel ||
|
2018-04-06 04:49:06 +02:00
|
|
|
result.cf_paths.size() > 1U) {
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
2015-02-05 20:44:17 +01:00
|
|
|
// 1. level_compaction_dynamic_level_bytes only makes sense for
|
|
|
|
// level-based compaction.
|
|
|
|
// 2. we don't yet know how to make both of this feature and multiple
|
|
|
|
// DB path work.
|
|
|
|
result.level_compaction_dynamic_level_bytes = false;
|
|
|
|
}
|
|
|
|
}
|
2015-02-24 01:08:27 +01:00
|
|
|
|
2016-06-17 01:02:52 +02:00
|
|
|
if (result.max_compaction_bytes == 0) {
|
|
|
|
result.max_compaction_bytes = result.target_file_size_base * 25;
|
|
|
|
}
|
|
|
|
|
2019-11-26 02:11:26 +01:00
|
|
|
bool is_block_based_table =
|
2020-07-08 20:52:13 +02:00
|
|
|
(result.table_factory->Name() == BlockBasedTableFactory::kName);
|
2019-11-26 02:11:26 +01:00
|
|
|
|
|
|
|
const uint64_t kAdjustedTtl = 30 * 24 * 60 * 60;
|
|
|
|
if (result.ttl == kDefaultTtl) {
|
|
|
|
if (is_block_based_table &&
|
|
|
|
result.compaction_style != kCompactionStyleFIFO) {
|
|
|
|
result.ttl = kAdjustedTtl;
|
|
|
|
} else {
|
|
|
|
result.ttl = 0;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
const uint64_t kAdjustedPeriodicCompSecs = 30 * 24 * 60 * 60;
|
2019-10-31 18:59:13 +01:00
|
|
|
|
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
|
|
|
// Turn on periodic compactions and set them to occur once every 30 days if
|
|
|
|
// compaction filters are used and periodic_compaction_seconds is set to the
|
|
|
|
// default value.
|
2019-11-07 19:56:25 +01:00
|
|
|
if (result.compaction_style != kCompactionStyleFIFO) {
|
|
|
|
if ((result.compaction_filter != nullptr ||
|
|
|
|
result.compaction_filter_factory != nullptr) &&
|
2019-11-26 02:11:26 +01:00
|
|
|
result.periodic_compaction_seconds == kDefaultPeriodicCompSecs &&
|
|
|
|
is_block_based_table) {
|
|
|
|
result.periodic_compaction_seconds = kAdjustedPeriodicCompSecs;
|
2019-11-07 19:56:25 +01:00
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// result.compaction_style == kCompactionStyleFIFO
|
2019-10-31 18:59:13 +01:00
|
|
|
if (result.ttl == 0) {
|
2019-11-26 02:11:26 +01:00
|
|
|
if (is_block_based_table) {
|
|
|
|
if (result.periodic_compaction_seconds == kDefaultPeriodicCompSecs) {
|
|
|
|
result.periodic_compaction_seconds = kAdjustedPeriodicCompSecs;
|
|
|
|
}
|
|
|
|
result.ttl = result.periodic_compaction_seconds;
|
2019-10-31 18:59:13 +01:00
|
|
|
}
|
|
|
|
} else if (result.periodic_compaction_seconds != 0) {
|
|
|
|
result.ttl = std::min(result.ttl, result.periodic_compaction_seconds);
|
|
|
|
}
|
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
|
|
|
}
|
|
|
|
|
2019-11-23 07:12:09 +01:00
|
|
|
// TTL compactions would work similar to Periodic Compactions in Universal in
|
|
|
|
// most of the cases. So, if ttl is set, execute the periodic compaction
|
|
|
|
// codepath.
|
|
|
|
if (result.compaction_style == kCompactionStyleUniversal && result.ttl != 0) {
|
|
|
|
if (result.periodic_compaction_seconds != 0) {
|
|
|
|
result.periodic_compaction_seconds =
|
|
|
|
std::min(result.ttl, result.periodic_compaction_seconds);
|
|
|
|
} else {
|
|
|
|
result.periodic_compaction_seconds = result.ttl;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-11-26 02:11:26 +01:00
|
|
|
if (result.periodic_compaction_seconds == kDefaultPeriodicCompSecs) {
|
|
|
|
result.periodic_compaction_seconds = 0;
|
|
|
|
}
|
|
|
|
|
2014-02-05 01:31:18 +01:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2014-03-08 01:59:47 +01:00
|
|
|
int SuperVersion::dummy = 0;
|
|
|
|
void* const SuperVersion::kSVInUse = &SuperVersion::dummy;
|
|
|
|
void* const SuperVersion::kSVObsolete = nullptr;
|
2014-02-05 01:31:18 +01:00
|
|
|
|
2014-01-24 23:30:28 +01:00
|
|
|
SuperVersion::~SuperVersion() {
|
|
|
|
for (auto td : to_delete) {
|
|
|
|
delete td;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
SuperVersion* SuperVersion::Ref() {
|
|
|
|
refs.fetch_add(1, std::memory_order_relaxed);
|
|
|
|
return this;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool SuperVersion::Unref() {
|
|
|
|
// fetch_sub returns the previous value of ref
|
2015-01-27 22:57:44 +01:00
|
|
|
uint32_t previous_refs = refs.fetch_sub(1);
|
2014-02-04 00:28:03 +01:00
|
|
|
assert(previous_refs > 0);
|
|
|
|
return previous_refs == 1;
|
2014-01-24 23:30:28 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
void SuperVersion::Cleanup() {
|
|
|
|
assert(refs.load(std::memory_order_relaxed) == 0);
|
|
|
|
imm->Unref(&to_delete);
|
|
|
|
MemTable* m = mem->Unref();
|
|
|
|
if (m != nullptr) {
|
2015-08-19 22:32:09 +02:00
|
|
|
auto* memory_usage = current->cfd()->imm()->current_memory_usage();
|
|
|
|
assert(*memory_usage >= m->ApproximateMemoryUsage());
|
|
|
|
*memory_usage -= m->ApproximateMemoryUsage();
|
2014-01-24 23:30:28 +01:00
|
|
|
to_delete.push_back(m);
|
|
|
|
}
|
|
|
|
current->Unref();
|
2019-12-13 04:02:51 +01:00
|
|
|
if (cfd->Unref()) {
|
|
|
|
delete cfd;
|
|
|
|
}
|
2014-01-24 23:30:28 +01:00
|
|
|
}
|
|
|
|
|
2019-12-13 04:02:51 +01:00
|
|
|
void SuperVersion::Init(ColumnFamilyData* new_cfd, MemTable* new_mem,
|
|
|
|
MemTableListVersion* new_imm, Version* new_current) {
|
|
|
|
cfd = new_cfd;
|
2014-01-24 23:30:28 +01:00
|
|
|
mem = new_mem;
|
|
|
|
imm = new_imm;
|
|
|
|
current = new_current;
|
2019-12-13 04:02:51 +01:00
|
|
|
cfd->Ref();
|
2014-01-24 23:30:28 +01:00
|
|
|
mem->Ref();
|
|
|
|
imm->Ref();
|
|
|
|
current->Ref();
|
|
|
|
refs.store(1, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
2014-03-04 02:54:04 +01:00
|
|
|
namespace {
|
|
|
|
void SuperVersionUnrefHandle(void* ptr) {
|
2014-03-08 01:59:47 +01:00
|
|
|
// UnrefHandle is called when a thread exists or a ThreadLocalPtr gets
|
|
|
|
// destroyed. When former happens, the thread shouldn't see kSVInUse.
|
|
|
|
// When latter happens, we are in ~ColumnFamilyData(), no get should happen as
|
|
|
|
// well.
|
2014-03-04 02:54:04 +01:00
|
|
|
SuperVersion* sv = static_cast<SuperVersion*>(ptr);
|
Fix deadlock in ColumnFamilyData::InstallSuperVersion()
Summary:
Deadlock: a memtable flush holds DB::mutex_ and calls ThreadLocalPtr::Scrape(), which locks ThreadLocalPtr mutex; meanwhile, a thread exit handler locks ThreadLocalPtr mutex and calls SuperVersionUnrefHandle, which tries to lock DB::mutex_.
This deadlock is hit all the time on our workload. It blocks our release.
In general, the problem is that ThreadLocalPtr takes an arbitrary callback and calls it while holding a lock on a global mutex. The same global mutex is (at least in some cases) locked by almost all ThreadLocalPtr methods, on any instance of ThreadLocalPtr. So, there'll be a deadlock if the callback tries to do anything to any instance of ThreadLocalPtr, or waits for another thread to do so.
So, probably the only safe way to use ThreadLocalPtr callbacks is to do only do simple and lock-free things in them.
This PR fixes the deadlock by making sure that local_sv_ never holds the last reference to a SuperVersion, and therefore SuperVersionUnrefHandle never has to do any nontrivial cleanup.
I also searched for other uses of ThreadLocalPtr to see if they may have similar bugs. There's only one other use, in transaction_lock_mgr.cc, and it looks fine.
Closes https://github.com/facebook/rocksdb/pull/3510
Reviewed By: sagar0
Differential Revision: D7005346
Pulled By: al13n321
fbshipit-source-id: 37575591b84f07a891d6659e87e784660fde815f
2018-02-16 16:58:18 +01:00
|
|
|
bool was_last_ref __attribute__((__unused__));
|
|
|
|
was_last_ref = sv->Unref();
|
|
|
|
// Thread-local SuperVersions can't outlive ColumnFamilyData::super_version_.
|
|
|
|
// This is important because we can't do SuperVersion cleanup here.
|
|
|
|
// That would require locking DB mutex, which would deadlock because
|
|
|
|
// SuperVersionUnrefHandle is called with locked ThreadLocalPtr mutex.
|
|
|
|
assert(!was_last_ref);
|
2014-03-04 02:54:04 +01:00
|
|
|
}
|
|
|
|
} // anonymous namespace
|
|
|
|
|
2020-03-12 02:36:43 +01:00
|
|
|
std::vector<std::string> ColumnFamilyData::GetDbPaths() const {
|
|
|
|
std::vector<std::string> paths;
|
|
|
|
paths.reserve(ioptions_.cf_paths.size());
|
|
|
|
for (const DbPath& db_path : ioptions_.cf_paths) {
|
|
|
|
paths.emplace_back(db_path.path);
|
|
|
|
}
|
|
|
|
return paths;
|
|
|
|
}
|
|
|
|
|
|
|
|
const uint32_t ColumnFamilyData::kDummyColumnFamilyDataId = port::kMaxUint32;
|
|
|
|
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
ColumnFamilyData::ColumnFamilyData(
|
|
|
|
uint32_t id, const std::string& name, Version* _dummy_versions,
|
2016-06-21 03:01:03 +02:00
|
|
|
Cache* _table_cache, WriteBufferManager* write_buffer_manager,
|
2016-09-24 01:34:04 +02:00
|
|
|
const ColumnFamilyOptions& cf_options, const ImmutableDBOptions& db_options,
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
|
|
|
const FileOptions& file_options, ColumnFamilySet* column_family_set,
|
2019-06-14 00:39:52 +02:00
|
|
|
BlockCacheTracer* const block_cache_tracer)
|
2014-01-29 22:28:50 +01:00
|
|
|
: id_(id),
|
|
|
|
name_(name),
|
2014-11-06 20:14:28 +01:00
|
|
|
dummy_versions_(_dummy_versions),
|
2014-01-29 22:28:50 +01:00
|
|
|
current_(nullptr),
|
2014-02-11 02:04:44 +01:00
|
|
|
refs_(0),
|
2017-06-23 00:45:42 +02:00
|
|
|
initialized_(false),
|
2014-02-11 02:04:44 +01:00
|
|
|
dropped_(false),
|
2014-09-17 21:49:13 +02:00
|
|
|
internal_comparator_(cf_options.comparator),
|
2016-10-21 20:31:42 +02:00
|
|
|
initial_cf_options_(SanitizeOptions(db_options, cf_options)),
|
2016-09-24 01:34:04 +02:00
|
|
|
ioptions_(db_options, initial_cf_options_),
|
|
|
|
mutable_cf_options_(initial_cf_options_),
|
2017-07-13 01:49:56 +02:00
|
|
|
is_delete_range_supported_(
|
|
|
|
cf_options.table_factory->IsDeleteRangeSupported()),
|
2016-06-21 03:01:03 +02:00
|
|
|
write_buffer_manager_(write_buffer_manager),
|
2014-01-29 22:28:50 +01:00
|
|
|
mem_(nullptr),
|
2016-09-24 01:34:04 +02:00
|
|
|
imm_(ioptions_.min_write_buffer_number_to_merge,
|
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
|
|
|
ioptions_.max_write_buffer_number_to_maintain,
|
|
|
|
ioptions_.max_write_buffer_size_to_maintain),
|
2014-01-29 22:28:50 +01:00
|
|
|
super_version_(nullptr),
|
|
|
|
super_version_number_(0),
|
2014-03-04 02:54:04 +01:00
|
|
|
local_sv_(new ThreadLocalPtr(&SuperVersionUnrefHandle)),
|
2014-01-31 01:49:46 +01:00
|
|
|
next_(nullptr),
|
|
|
|
prev_(nullptr),
|
2014-01-31 00:23:13 +01:00
|
|
|
log_number_(0),
|
FlushReason improvement
Summary:
Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full"
> $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304
$ grep 'flush_reason' /dev/shm/dbbench/LOG
...
2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
With the fix:
> 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"}
Closes https://github.com/facebook/rocksdb/pull/3627
Differential Revision: D7328772
Pulled By: miasantreble
fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
|
|
|
flush_reason_(FlushReason::kOthers),
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
column_family_set_(column_family_set),
|
2018-04-27 06:09:53 +02:00
|
|
|
queued_for_flush_(false),
|
2018-04-27 20:11:12 +02:00
|
|
|
queued_for_compaction_(false),
|
2017-01-20 00:21:07 +01:00
|
|
|
prev_compaction_needed_bytes_(0),
|
2018-01-19 02:32:50 +01:00
|
|
|
allow_2pc_(db_options.allow_2pc),
|
2020-03-12 02:36:43 +01:00
|
|
|
last_memtable_id_(0),
|
|
|
|
db_paths_registered_(false) {
|
|
|
|
if (id_ != kDummyColumnFamilyDataId) {
|
|
|
|
// TODO(cc): RegisterDbPaths can be expensive, considering moving it
|
|
|
|
// outside of this constructor which might be called with db mutex held.
|
|
|
|
// TODO(cc): considering using ioptions_.fs, currently some tests rely on
|
|
|
|
// EnvWrapper, that's the main reason why we use env here.
|
|
|
|
Status s = ioptions_.env->RegisterDbPaths(GetDbPaths());
|
|
|
|
if (s.ok()) {
|
|
|
|
db_paths_registered_ = true;
|
|
|
|
} else {
|
|
|
|
ROCKS_LOG_ERROR(
|
|
|
|
ioptions_.info_log,
|
|
|
|
"Failed to register data paths of column family (id: %d, name: %s)",
|
|
|
|
id_, name_.c_str());
|
|
|
|
}
|
|
|
|
}
|
2014-02-11 02:04:44 +01:00
|
|
|
Ref();
|
|
|
|
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
2015-04-06 19:04:30 +02:00
|
|
|
// Convert user defined table properties collector factories to internal ones.
|
2016-09-24 01:34:04 +02:00
|
|
|
GetIntTblPropCollectorFactory(ioptions_, &int_tbl_prop_collector_factories_);
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
2015-04-06 19:04:30 +02:00
|
|
|
|
2014-11-06 20:14:28 +01:00
|
|
|
// if _dummy_versions is nullptr, then this is a dummy column family.
|
|
|
|
if (_dummy_versions != nullptr) {
|
make internal stats independent of statistics
Summary:
also make it aware of column family
output from db_bench
```
** Compaction Stats [default] **
Level Files Size(MB) Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) RW-Amp W-Amp Rd(MB/s) Wr(MB/s) Rn(cnt) Rnp1(cnt) Wnp1(cnt) Wnew(cnt) Comp(sec) Comp(cnt) Avg(sec) Stall(sec) Stall(cnt) Avg(ms)
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
L0 14 956 0.9 0.0 0.0 0.0 2.7 2.7 0.0 0.0 0.0 111.6 0 0 0 0 24 40 0.612 75.20 492387 0.15
L1 21 2001 2.0 5.7 2.0 3.7 5.3 1.6 5.4 2.6 71.2 65.7 31 43 55 12 82 2 41.242 43.72 41183 1.06
L2 217 18974 1.9 16.5 2.0 14.4 15.1 0.7 15.6 7.4 70.1 64.3 17 182 185 3 241 16 15.052 0.00 0 0.00
L3 1641 188245 1.8 9.1 1.1 8.0 8.5 0.5 15.4 7.4 61.3 57.2 9 75 76 1 152 9 16.887 0.00 0 0.00
L4 4447 449025 0.4 13.4 4.8 8.6 9.1 0.5 4.7 1.9 77.8 52.7 38 79 100 21 176 38 4.639 0.00 0 0.00
Sum 6340 659201 0.0 44.7 10.0 34.7 40.6 6.0 32.0 15.2 67.7 61.6 95 379 416 37 676 105 6.439 118.91 533570 0.22
Int 0 0 0.0 1.2 0.4 0.8 1.3 0.5 5.2 2.7 59.1 65.6 3 7 9 2 20 10 2.003 0.00 0 0.00
Stalls(secs): 75.197 level0_slowdown, 0.000 level0_numfiles, 0.000 memtable_compaction, 43.717 leveln_slowdown
Stalls(count): 492387 level0_slowdown, 0 level0_numfiles, 0 memtable_compaction, 41183 leveln_slowdown
** DB Stats **
Uptime(secs): 202.1 total, 13.5 interval
Cumulative writes: 6291456 writes, 6291456 batches, 1.0 writes per batch, 4.90 ingest GB
Cumulative WAL: 6291456 writes, 6291456 syncs, 1.00 writes per sync, 4.90 GB written
Interval writes: 1048576 writes, 1048576 batches, 1.0 writes per batch, 836.0 ingest MB
Interval WAL: 1048576 writes, 1048576 syncs, 1.00 writes per sync, 0.82 MB written
Test Plan: ran it
Reviewers: sdong, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19917
2014-07-21 21:57:29 +02:00
|
|
|
internal_stats_.reset(
|
2016-09-24 01:34:04 +02:00
|
|
|
new InternalStats(ioptions_.num_levels, db_options.env, this));
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
|
|
|
table_cache_.reset(new TableCache(ioptions_, file_options, _table_cache,
|
2019-06-14 00:39:52 +02:00
|
|
|
block_cache_tracer));
|
2014-11-27 00:45:11 +01:00
|
|
|
if (ioptions_.compaction_style == kCompactionStyleLevel) {
|
2014-03-11 22:52:17 +01:00
|
|
|
compaction_picker_.reset(
|
2014-10-02 01:19:16 +02:00
|
|
|
new LevelCompactionPicker(ioptions_, &internal_comparator_));
|
2014-11-27 00:45:11 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
} else if (ioptions_.compaction_style == kCompactionStyleUniversal) {
|
|
|
|
compaction_picker_.reset(
|
|
|
|
new UniversalCompactionPicker(ioptions_, &internal_comparator_));
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
} else if (ioptions_.compaction_style == kCompactionStyleFIFO) {
|
2014-05-21 20:43:35 +02:00
|
|
|
compaction_picker_.reset(
|
2014-10-02 01:19:16 +02:00
|
|
|
new FIFOCompactionPicker(ioptions_, &internal_comparator_));
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
} else if (ioptions_.compaction_style == kCompactionStyleNone) {
|
|
|
|
compaction_picker_.reset(new NullCompactionPicker(
|
|
|
|
ioptions_, &internal_comparator_));
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(ioptions_.info_log,
|
|
|
|
"Column family %s does not use any background compaction. "
|
|
|
|
"Compactions can only be done via CompactFiles\n",
|
|
|
|
GetName().c_str());
|
2014-11-27 00:45:11 +01:00
|
|
|
#endif // !ROCKSDB_LITE
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
} else {
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_ERROR(ioptions_.info_log,
|
|
|
|
"Unable to recognize the specified compaction style %d. "
|
|
|
|
"Column family %s will use kCompactionStyleLevel.\n",
|
|
|
|
ioptions_.compaction_style, GetName().c_str());
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
compaction_picker_.reset(
|
|
|
|
new LevelCompactionPicker(ioptions_, &internal_comparator_));
|
2014-02-05 02:45:19 +01:00
|
|
|
}
|
2014-02-07 06:39:20 +01:00
|
|
|
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
if (column_family_set_->NumberOfColumnFamilies() < 10) {
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_INFO(ioptions_.info_log,
|
|
|
|
"--------------- Options for column family [%s]:\n",
|
|
|
|
name.c_str());
|
2016-09-24 01:34:04 +02:00
|
|
|
initial_cf_options_.Dump(ioptions_.info_log);
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
} else {
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_INFO(ioptions_.info_log, "\t(skipping printing options)\n");
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
}
|
2014-02-01 00:30:27 +01:00
|
|
|
}
|
Cache some conditions for DBImpl::MakeRoomForWrite
Summary:
Task 4580155. Some conditions in DBImpl::MakeRoomForWrite can be cached in
ColumnFamilyData, because theirs value can be changed only during compaction,
adding new memtable and/or add recalculation of compaction score.
These conditions are:
cfd->imm()->size() == cfd->options()->max_write_buffer_number - 1
cfd->current()->NumLevelFiles(0) >= cfd->options()->level0_stop_writes_trigger
cfd->options()->soft_rate_limit > 0.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->soft_rate_limit
cfd->options()->hard_rate_limit > 1.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->hard_rate_limit
P.S.
As it's my first diff, Siying suggested to add everybody as a reviewers
for this diff. Sorry, if I forgot someone or add someone by mistake.
Test Plan: make all check
Reviewers: haobo, xjin, dhruba, yhchiang, zagfox, ljin, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19311
2014-06-27 01:45:27 +02:00
|
|
|
|
2014-10-02 01:19:16 +02:00
|
|
|
RecalculateWriteStallConditions(mutable_cf_options_);
|
2014-02-01 00:30:27 +01:00
|
|
|
}
|
2014-01-22 20:44:53 +01:00
|
|
|
|
2014-02-11 02:04:44 +01:00
|
|
|
// DB mutex held
|
2014-01-22 20:44:53 +01:00
|
|
|
ColumnFamilyData::~ColumnFamilyData() {
|
2015-01-26 20:48:07 +01:00
|
|
|
assert(refs_.load(std::memory_order_relaxed) == 0);
|
2014-02-11 02:04:44 +01:00
|
|
|
// remove from linked list
|
|
|
|
auto prev = prev_;
|
|
|
|
auto next = next_;
|
|
|
|
prev->next_ = next;
|
|
|
|
next->prev_ = prev;
|
|
|
|
|
2015-01-06 21:44:21 +01:00
|
|
|
if (!dropped_ && column_family_set_ != nullptr) {
|
|
|
|
// If it's dropped, it's already removed from column family set
|
|
|
|
// If column_family_set_ == nullptr, this is dummy CFD and not in
|
|
|
|
// ColumnFamilySet
|
2014-03-11 22:52:17 +01:00
|
|
|
column_family_set_->RemoveColumnFamily(this);
|
2014-02-11 02:04:44 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
if (current_ != nullptr) {
|
|
|
|
current_->Unref();
|
|
|
|
}
|
|
|
|
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
// It would be wrong if this ColumnFamilyData is in flush_queue_ or
|
|
|
|
// compaction_queue_ and we destroyed it
|
2018-04-27 06:09:53 +02:00
|
|
|
assert(!queued_for_flush_);
|
2018-04-27 20:11:12 +02:00
|
|
|
assert(!queued_for_compaction_);
|
2019-12-13 04:02:51 +01:00
|
|
|
assert(super_version_ == nullptr);
|
2014-03-04 18:03:56 +01:00
|
|
|
|
2014-01-31 01:49:46 +01:00
|
|
|
if (dummy_versions_ != nullptr) {
|
|
|
|
// List must be empty
|
2014-10-28 18:04:38 +01:00
|
|
|
assert(dummy_versions_->TEST_Next() == dummy_versions_);
|
2018-02-02 21:14:42 +01:00
|
|
|
bool deleted __attribute__((__unused__));
|
2017-10-23 23:20:53 +02:00
|
|
|
deleted = dummy_versions_->Unref();
|
2014-10-28 18:04:38 +01:00
|
|
|
assert(deleted);
|
2014-01-31 01:49:46 +01:00
|
|
|
}
|
2014-01-24 23:30:28 +01:00
|
|
|
|
2014-01-29 22:28:50 +01:00
|
|
|
if (mem_ != nullptr) {
|
|
|
|
delete mem_->Unref();
|
2014-01-24 23:30:28 +01:00
|
|
|
}
|
2014-02-07 00:42:16 +01:00
|
|
|
autovector<MemTable*> to_delete;
|
2014-01-29 22:28:50 +01:00
|
|
|
imm_.current()->Unref(&to_delete);
|
2014-01-24 23:30:28 +01:00
|
|
|
for (MemTable* m : to_delete) {
|
|
|
|
delete m;
|
|
|
|
}
|
2020-03-12 02:36:43 +01:00
|
|
|
|
|
|
|
if (db_paths_registered_) {
|
|
|
|
// TODO(cc): considering using ioptions_.fs, currently some tests rely on
|
|
|
|
// EnvWrapper, that's the main reason why we use env here.
|
|
|
|
Status s = ioptions_.env->UnregisterDbPaths(GetDbPaths());
|
|
|
|
if (!s.ok()) {
|
|
|
|
ROCKS_LOG_ERROR(
|
|
|
|
ioptions_.info_log,
|
|
|
|
"Failed to unregister data paths of column family (id: %d, name: %s)",
|
|
|
|
id_, name_.c_str());
|
|
|
|
}
|
|
|
|
}
|
2014-01-24 23:30:28 +01:00
|
|
|
}
|
|
|
|
|
2019-12-13 04:02:51 +01:00
|
|
|
bool ColumnFamilyData::UnrefAndTryDelete() {
|
|
|
|
int old_refs = refs_.fetch_sub(1);
|
|
|
|
assert(old_refs > 0);
|
|
|
|
|
|
|
|
if (old_refs == 1) {
|
|
|
|
assert(super_version_ == nullptr);
|
|
|
|
delete this;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (old_refs == 2 && super_version_ != nullptr) {
|
|
|
|
// Only the super_version_ holds me
|
|
|
|
SuperVersion* sv = super_version_;
|
|
|
|
super_version_ = nullptr;
|
|
|
|
// Release SuperVersion reference kept in ThreadLocalPtr.
|
|
|
|
// This must be done outside of mutex_ since unref handler can lock mutex.
|
|
|
|
sv->db_mutex->Unlock();
|
|
|
|
local_sv_.reset();
|
|
|
|
sv->db_mutex->Lock();
|
|
|
|
|
|
|
|
if (sv->Unref()) {
|
|
|
|
// May delete this ColumnFamilyData after calling Cleanup()
|
|
|
|
sv->Cleanup();
|
|
|
|
delete sv;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2015-01-06 21:44:21 +01:00
|
|
|
void ColumnFamilyData::SetDropped() {
|
|
|
|
// can't drop default CF
|
|
|
|
assert(id_ != 0);
|
|
|
|
dropped_ = true;
|
|
|
|
write_controller_token_.reset();
|
|
|
|
|
|
|
|
// remove from column_family_set
|
|
|
|
column_family_set_->RemoveColumnFamily(this);
|
|
|
|
}
|
|
|
|
|
2016-09-15 07:10:28 +02:00
|
|
|
ColumnFamilyOptions ColumnFamilyData::GetLatestCFOptions() const {
|
2016-09-24 01:34:04 +02:00
|
|
|
return BuildColumnFamilyOptions(initial_cf_options_, mutable_cf_options_);
|
2016-09-15 07:10:28 +02:00
|
|
|
}
|
|
|
|
|
2017-01-20 00:21:07 +01:00
|
|
|
uint64_t ColumnFamilyData::OldestLogToKeep() {
|
|
|
|
auto current_log = GetLogNumber();
|
|
|
|
|
|
|
|
if (allow_2pc_) {
|
Skip deleted WALs during recovery
Summary:
This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic.
Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction)
This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2.
Closes https://github.com/facebook/rocksdb/pull/3765
Differential Revision: D7747618
Pulled By: siying
fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
2018-05-04 00:35:11 +02:00
|
|
|
autovector<MemTable*> empty_list;
|
|
|
|
auto imm_prep_log =
|
|
|
|
imm()->PrecomputeMinLogContainingPrepSection(empty_list);
|
2017-01-20 00:21:07 +01:00
|
|
|
auto mem_prep_log = mem()->GetMinLogContainingPrepSection();
|
|
|
|
|
|
|
|
if (imm_prep_log > 0 && imm_prep_log < current_log) {
|
|
|
|
current_log = imm_prep_log;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (mem_prep_log > 0 && mem_prep_log < current_log) {
|
|
|
|
current_log = mem_prep_log;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return current_log;
|
|
|
|
}
|
|
|
|
|
2016-11-23 18:19:11 +01:00
|
|
|
const double kIncSlowdownRatio = 0.8;
|
|
|
|
const double kDecSlowdownRatio = 1 / kIncSlowdownRatio;
|
|
|
|
const double kNearStopSlowdownRatio = 0.6;
|
|
|
|
const double kDelayRecoverSlowdownRatio = 1.4;
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
|
|
|
|
namespace {
|
2016-11-23 18:19:11 +01:00
|
|
|
// If penalize_stop is true, we further reduce slowdown rate.
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
std::unique_ptr<WriteControllerToken> SetupDelay(
|
2016-11-23 18:19:11 +01:00
|
|
|
WriteController* write_controller, uint64_t compaction_needed_bytes,
|
|
|
|
uint64_t prev_compaction_need_bytes, bool penalize_stop,
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
bool auto_comapctions_disabled) {
|
2016-11-23 18:19:11 +01:00
|
|
|
const uint64_t kMinWriteRate = 16 * 1024u; // Minimum write rate 16KB/s.
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
|
2016-11-13 00:43:33 +01:00
|
|
|
uint64_t max_write_rate = write_controller->max_delayed_write_rate();
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
uint64_t write_rate = write_controller->delayed_write_rate();
|
|
|
|
|
|
|
|
if (auto_comapctions_disabled) {
|
|
|
|
// When auto compaction is disabled, always use the value user gave.
|
|
|
|
write_rate = max_write_rate;
|
|
|
|
} else if (write_controller->NeedsDelay() && max_write_rate > kMinWriteRate) {
|
|
|
|
// If user gives rate less than kMinWriteRate, don't adjust it.
|
|
|
|
//
|
|
|
|
// If already delayed, need to adjust based on previous compaction debt.
|
|
|
|
// When there are two or more column families require delay, we always
|
|
|
|
// increase or reduce write rate based on information for one single
|
|
|
|
// column family. It is likely to be OK but we can improve if there is a
|
|
|
|
// problem.
|
|
|
|
// Ignore compaction_needed_bytes = 0 case because compaction_needed_bytes
|
|
|
|
// is only available in level-based compaction
|
|
|
|
//
|
|
|
|
// If the compaction debt stays the same as previously, we also further slow
|
|
|
|
// down. It usually means a mem table is full. It's mainly for the case
|
|
|
|
// where both of flush and compaction are much slower than the speed we
|
|
|
|
// insert to mem tables, so we need to actively slow down before we get
|
|
|
|
// feedback signal from compaction and flushes to avoid the full stop
|
|
|
|
// because of hitting the max write buffer number.
|
2016-11-23 18:19:11 +01:00
|
|
|
//
|
|
|
|
// If DB just falled into the stop condition, we need to further reduce
|
|
|
|
// the write rate to avoid the stop condition.
|
|
|
|
if (penalize_stop) {
|
2017-06-05 20:23:31 +02:00
|
|
|
// Penalize the near stop or stop condition by more aggressive slowdown.
|
2016-11-23 18:19:11 +01:00
|
|
|
// This is to provide the long term slowdown increase signal.
|
|
|
|
// The penalty is more than the reward of recovering to the normal
|
|
|
|
// condition.
|
|
|
|
write_rate = static_cast<uint64_t>(static_cast<double>(write_rate) *
|
|
|
|
kNearStopSlowdownRatio);
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
if (write_rate < kMinWriteRate) {
|
|
|
|
write_rate = kMinWriteRate;
|
|
|
|
}
|
2016-11-23 18:19:11 +01:00
|
|
|
} else if (prev_compaction_need_bytes > 0 &&
|
|
|
|
prev_compaction_need_bytes <= compaction_needed_bytes) {
|
|
|
|
write_rate = static_cast<uint64_t>(static_cast<double>(write_rate) *
|
|
|
|
kIncSlowdownRatio);
|
|
|
|
if (write_rate < kMinWriteRate) {
|
|
|
|
write_rate = kMinWriteRate;
|
|
|
|
}
|
|
|
|
} else if (prev_compaction_need_bytes > compaction_needed_bytes) {
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
// We are speeding up by ratio of kSlowdownRatio when we have paid
|
|
|
|
// compaction debt. But we'll never speed up to faster than the write rate
|
|
|
|
// given by users.
|
2015-12-28 23:28:19 +01:00
|
|
|
write_rate = static_cast<uint64_t>(static_cast<double>(write_rate) *
|
2016-11-23 18:19:11 +01:00
|
|
|
kDecSlowdownRatio);
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
if (write_rate > max_write_rate) {
|
|
|
|
write_rate = max_write_rate;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return write_controller->GetDelayToken(write_rate);
|
|
|
|
}
|
Add options.base_background_compactions as a number of compaction threads for low compaction debt
Summary:
If options.base_background_compactions is given, we try to schedule number of compactions not existing this number, only when L0 files increase to certain number, or pending compaction bytes more than certain threshold, we schedule compactions based on options.max_background_compactions.
The watermarks are calculated based on slowdown thresholds.
Test Plan:
Add new test cases in column_family_test.
Adding more unit tests.
Reviewers: IslamAbdelRahman, yhchiang, kradhakrishnan, rven, anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D53409
2016-01-28 20:56:16 +01:00
|
|
|
|
|
|
|
int GetL0ThresholdSpeedupCompaction(int level0_file_num_compaction_trigger,
|
|
|
|
int level0_slowdown_writes_trigger) {
|
|
|
|
// SanitizeOptions() ensures it.
|
|
|
|
assert(level0_file_num_compaction_trigger <= level0_slowdown_writes_trigger);
|
|
|
|
|
2016-10-24 03:43:29 +02:00
|
|
|
if (level0_file_num_compaction_trigger < 0) {
|
|
|
|
return std::numeric_limits<int>::max();
|
|
|
|
}
|
|
|
|
|
2016-11-29 03:25:27 +01:00
|
|
|
const int64_t twice_level0_trigger =
|
|
|
|
static_cast<int64_t>(level0_file_num_compaction_trigger) * 2;
|
2016-10-24 03:43:29 +02:00
|
|
|
|
2016-11-29 03:25:27 +01:00
|
|
|
const int64_t one_fourth_trigger_slowdown =
|
|
|
|
static_cast<int64_t>(level0_file_num_compaction_trigger) +
|
2016-10-24 03:43:29 +02:00
|
|
|
((level0_slowdown_writes_trigger - level0_file_num_compaction_trigger) /
|
|
|
|
4);
|
|
|
|
|
|
|
|
assert(twice_level0_trigger >= 0);
|
|
|
|
assert(one_fourth_trigger_slowdown >= 0);
|
|
|
|
|
2016-11-29 03:25:27 +01:00
|
|
|
// 1/4 of the way between L0 compaction trigger threshold and slowdown
|
|
|
|
// condition.
|
Add options.base_background_compactions as a number of compaction threads for low compaction debt
Summary:
If options.base_background_compactions is given, we try to schedule number of compactions not existing this number, only when L0 files increase to certain number, or pending compaction bytes more than certain threshold, we schedule compactions based on options.max_background_compactions.
The watermarks are calculated based on slowdown thresholds.
Test Plan:
Add new test cases in column_family_test.
Adding more unit tests.
Reviewers: IslamAbdelRahman, yhchiang, kradhakrishnan, rven, anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D53409
2016-01-28 20:56:16 +01:00
|
|
|
// Or twice as compaction trigger, if it is smaller.
|
2016-11-29 03:25:27 +01:00
|
|
|
int64_t res = std::min(twice_level0_trigger, one_fourth_trigger_slowdown);
|
|
|
|
if (res >= port::kMaxInt32) {
|
|
|
|
return port::kMaxInt32;
|
|
|
|
} else {
|
2016-11-29 19:40:31 +01:00
|
|
|
// res fits in int
|
|
|
|
return static_cast<int>(res);
|
2016-11-29 03:25:27 +01:00
|
|
|
}
|
Add options.base_background_compactions as a number of compaction threads for low compaction debt
Summary:
If options.base_background_compactions is given, we try to schedule number of compactions not existing this number, only when L0 files increase to certain number, or pending compaction bytes more than certain threshold, we schedule compactions based on options.max_background_compactions.
The watermarks are calculated based on slowdown thresholds.
Test Plan:
Add new test cases in column_family_test.
Adding more unit tests.
Reviewers: IslamAbdelRahman, yhchiang, kradhakrishnan, rven, anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D53409
2016-01-28 20:56:16 +01:00
|
|
|
}
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
} // namespace
|
|
|
|
|
2018-02-13 00:34:39 +01:00
|
|
|
std::pair<WriteStallCondition, ColumnFamilyData::WriteStallCause>
|
|
|
|
ColumnFamilyData::GetWriteStallConditionAndCause(
|
|
|
|
int num_unflushed_memtables, int num_l0_files,
|
|
|
|
uint64_t num_compaction_needed_bytes,
|
|
|
|
const MutableCFOptions& mutable_cf_options) {
|
|
|
|
if (num_unflushed_memtables >= mutable_cf_options.max_write_buffer_number) {
|
|
|
|
return {WriteStallCondition::kStopped, WriteStallCause::kMemtableLimit};
|
|
|
|
} else if (!mutable_cf_options.disable_auto_compactions &&
|
|
|
|
num_l0_files >= mutable_cf_options.level0_stop_writes_trigger) {
|
|
|
|
return {WriteStallCondition::kStopped, WriteStallCause::kL0FileCountLimit};
|
|
|
|
} else if (!mutable_cf_options.disable_auto_compactions &&
|
|
|
|
mutable_cf_options.hard_pending_compaction_bytes_limit > 0 &&
|
|
|
|
num_compaction_needed_bytes >=
|
|
|
|
mutable_cf_options.hard_pending_compaction_bytes_limit) {
|
|
|
|
return {WriteStallCondition::kStopped,
|
|
|
|
WriteStallCause::kPendingCompactionBytes};
|
|
|
|
} else if (mutable_cf_options.max_write_buffer_number > 3 &&
|
|
|
|
num_unflushed_memtables >=
|
|
|
|
mutable_cf_options.max_write_buffer_number - 1) {
|
|
|
|
return {WriteStallCondition::kDelayed, WriteStallCause::kMemtableLimit};
|
|
|
|
} else if (!mutable_cf_options.disable_auto_compactions &&
|
|
|
|
mutable_cf_options.level0_slowdown_writes_trigger >= 0 &&
|
|
|
|
num_l0_files >=
|
|
|
|
mutable_cf_options.level0_slowdown_writes_trigger) {
|
|
|
|
return {WriteStallCondition::kDelayed, WriteStallCause::kL0FileCountLimit};
|
|
|
|
} else if (!mutable_cf_options.disable_auto_compactions &&
|
|
|
|
mutable_cf_options.soft_pending_compaction_bytes_limit > 0 &&
|
|
|
|
num_compaction_needed_bytes >=
|
|
|
|
mutable_cf_options.soft_pending_compaction_bytes_limit) {
|
|
|
|
return {WriteStallCondition::kDelayed,
|
|
|
|
WriteStallCause::kPendingCompactionBytes};
|
|
|
|
}
|
|
|
|
return {WriteStallCondition::kNormal, WriteStallCause::kNone};
|
|
|
|
}
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions(
|
2014-10-02 01:19:16 +02:00
|
|
|
const MutableCFOptions& mutable_cf_options) {
|
2017-10-06 03:00:38 +02:00
|
|
|
auto write_stall_condition = WriteStallCondition::kNormal;
|
Cache some conditions for DBImpl::MakeRoomForWrite
Summary:
Task 4580155. Some conditions in DBImpl::MakeRoomForWrite can be cached in
ColumnFamilyData, because theirs value can be changed only during compaction,
adding new memtable and/or add recalculation of compaction score.
These conditions are:
cfd->imm()->size() == cfd->options()->max_write_buffer_number - 1
cfd->current()->NumLevelFiles(0) >= cfd->options()->level0_stop_writes_trigger
cfd->options()->soft_rate_limit > 0.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->soft_rate_limit
cfd->options()->hard_rate_limit > 1.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->hard_rate_limit
P.S.
As it's my first diff, Siying suggested to add everybody as a reviewers
for this diff. Sorry, if I forgot someone or add someone by mistake.
Test Plan: make all check
Reviewers: haobo, xjin, dhruba, yhchiang, zagfox, ljin, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19311
2014-06-27 01:45:27 +02:00
|
|
|
if (current_ != nullptr) {
|
2014-10-31 16:48:19 +01:00
|
|
|
auto* vstorage = current_->storage_info();
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
auto write_controller = column_family_set_->write_controller_;
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
uint64_t compaction_needed_bytes =
|
|
|
|
vstorage->estimated_compaction_needed_bytes();
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
|
2018-02-13 00:34:39 +01:00
|
|
|
auto write_stall_condition_and_cause = GetWriteStallConditionAndCause(
|
|
|
|
imm()->NumNotFlushed(), vstorage->l0_delay_trigger_count(),
|
|
|
|
vstorage->estimated_compaction_needed_bytes(), mutable_cf_options);
|
|
|
|
write_stall_condition = write_stall_condition_and_cause.first;
|
|
|
|
auto write_stall_cause = write_stall_condition_and_cause.second;
|
|
|
|
|
2016-11-23 18:19:11 +01:00
|
|
|
bool was_stopped = write_controller->IsStopped();
|
|
|
|
bool needed_delay = write_controller->NeedsDelay();
|
|
|
|
|
2018-02-13 00:34:39 +01:00
|
|
|
if (write_stall_condition == WriteStallCondition::kStopped &&
|
|
|
|
write_stall_cause == WriteStallCause::kMemtableLimit) {
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
write_controller_token_ = write_controller->GetStopToken();
|
2017-09-08 03:13:53 +02:00
|
|
|
internal_stats_->AddCFStats(InternalStats::MEMTABLE_LIMIT_STOPS, 1);
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(
|
|
|
|
ioptions_.info_log,
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
"[%s] Stopping writes because we have %d immutable memtables "
|
2014-10-17 01:57:59 +02:00
|
|
|
"(waiting for flush), max_write_buffer_number is set to %d",
|
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
2015-05-29 01:34:24 +02:00
|
|
|
name_.c_str(), imm()->NumNotFlushed(),
|
2014-10-17 01:57:59 +02:00
|
|
|
mutable_cf_options.max_write_buffer_number);
|
2018-02-13 00:34:39 +01:00
|
|
|
} else if (write_stall_condition == WriteStallCondition::kStopped &&
|
|
|
|
write_stall_cause == WriteStallCause::kL0FileCountLimit) {
|
2014-09-09 00:23:58 +02:00
|
|
|
write_controller_token_ = write_controller->GetStopToken();
|
2017-09-08 03:13:53 +02:00
|
|
|
internal_stats_->AddCFStats(InternalStats::L0_FILE_COUNT_LIMIT_STOPS, 1);
|
2015-09-14 20:03:37 +02:00
|
|
|
if (compaction_picker_->IsLevel0CompactionInProgress()) {
|
|
|
|
internal_stats_->AddCFStats(
|
2017-09-08 03:13:53 +02:00
|
|
|
InternalStats::LOCKED_L0_FILE_COUNT_LIMIT_STOPS, 1);
|
2015-09-14 20:03:37 +02:00
|
|
|
}
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(ioptions_.info_log,
|
|
|
|
"[%s] Stopping writes because we have %d level-0 files",
|
|
|
|
name_.c_str(), vstorage->l0_delay_trigger_count());
|
2018-02-13 00:34:39 +01:00
|
|
|
} else if (write_stall_condition == WriteStallCondition::kStopped &&
|
|
|
|
write_stall_cause == WriteStallCause::kPendingCompactionBytes) {
|
2015-09-11 23:31:23 +02:00
|
|
|
write_controller_token_ = write_controller->GetStopToken();
|
|
|
|
internal_stats_->AddCFStats(
|
2017-09-08 03:13:53 +02:00
|
|
|
InternalStats::PENDING_COMPACTION_BYTES_LIMIT_STOPS, 1);
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(
|
|
|
|
ioptions_.info_log,
|
2015-11-19 03:10:20 +01:00
|
|
|
"[%s] Stopping writes because of estimated pending compaction "
|
|
|
|
"bytes %" PRIu64,
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
name_.c_str(), compaction_needed_bytes);
|
2018-02-13 00:34:39 +01:00
|
|
|
} else if (write_stall_condition == WriteStallCondition::kDelayed &&
|
|
|
|
write_stall_cause == WriteStallCause::kMemtableLimit) {
|
2016-01-28 20:43:28 +01:00
|
|
|
write_controller_token_ =
|
2016-11-13 00:43:33 +01:00
|
|
|
SetupDelay(write_controller, compaction_needed_bytes,
|
2016-11-23 18:19:11 +01:00
|
|
|
prev_compaction_needed_bytes_, was_stopped,
|
2016-01-28 20:43:28 +01:00
|
|
|
mutable_cf_options.disable_auto_compactions);
|
2017-09-08 03:13:53 +02:00
|
|
|
internal_stats_->AddCFStats(InternalStats::MEMTABLE_LIMIT_SLOWDOWNS, 1);
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(
|
|
|
|
ioptions_.info_log,
|
2016-01-28 20:43:28 +01:00
|
|
|
"[%s] Stalling writes because we have %d immutable memtables "
|
|
|
|
"(waiting for flush), max_write_buffer_number is set to %d "
|
|
|
|
"rate %" PRIu64,
|
|
|
|
name_.c_str(), imm()->NumNotFlushed(),
|
|
|
|
mutable_cf_options.max_write_buffer_number,
|
|
|
|
write_controller->delayed_write_rate());
|
2018-02-13 00:34:39 +01:00
|
|
|
} else if (write_stall_condition == WriteStallCondition::kDelayed &&
|
|
|
|
write_stall_cause == WriteStallCause::kL0FileCountLimit) {
|
2016-11-23 18:19:11 +01:00
|
|
|
// L0 is the last two files from stopping.
|
|
|
|
bool near_stop = vstorage->l0_delay_trigger_count() >=
|
|
|
|
mutable_cf_options.level0_stop_writes_trigger - 2;
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
write_controller_token_ =
|
2016-11-13 00:43:33 +01:00
|
|
|
SetupDelay(write_controller, compaction_needed_bytes,
|
2016-11-23 18:19:11 +01:00
|
|
|
prev_compaction_needed_bytes_, was_stopped || near_stop,
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
mutable_cf_options.disable_auto_compactions);
|
2017-09-08 03:13:53 +02:00
|
|
|
internal_stats_->AddCFStats(InternalStats::L0_FILE_COUNT_LIMIT_SLOWDOWNS,
|
|
|
|
1);
|
2015-09-14 20:03:37 +02:00
|
|
|
if (compaction_picker_->IsLevel0CompactionInProgress()) {
|
|
|
|
internal_stats_->AddCFStats(
|
2017-09-08 03:13:53 +02:00
|
|
|
InternalStats::LOCKED_L0_FILE_COUNT_LIMIT_SLOWDOWNS, 1);
|
2015-09-14 20:03:37 +02:00
|
|
|
}
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(ioptions_.info_log,
|
|
|
|
"[%s] Stalling writes because we have %d level-0 files "
|
|
|
|
"rate %" PRIu64,
|
|
|
|
name_.c_str(), vstorage->l0_delay_trigger_count(),
|
|
|
|
write_controller->delayed_write_rate());
|
2018-02-13 00:34:39 +01:00
|
|
|
} else if (write_stall_condition == WriteStallCondition::kDelayed &&
|
|
|
|
write_stall_cause == WriteStallCause::kPendingCompactionBytes) {
|
2016-11-23 18:19:11 +01:00
|
|
|
// If the distance to hard limit is less than 1/4 of the gap between soft
|
|
|
|
// and
|
|
|
|
// hard bytes limit, we think it is near stop and speed up the slowdown.
|
|
|
|
bool near_stop =
|
|
|
|
mutable_cf_options.hard_pending_compaction_bytes_limit > 0 &&
|
|
|
|
(compaction_needed_bytes -
|
|
|
|
mutable_cf_options.soft_pending_compaction_bytes_limit) >
|
|
|
|
3 * (mutable_cf_options.hard_pending_compaction_bytes_limit -
|
|
|
|
mutable_cf_options.soft_pending_compaction_bytes_limit) /
|
|
|
|
4;
|
|
|
|
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
write_controller_token_ =
|
2016-11-13 00:43:33 +01:00
|
|
|
SetupDelay(write_controller, compaction_needed_bytes,
|
2016-11-23 18:19:11 +01:00
|
|
|
prev_compaction_needed_bytes_, was_stopped || near_stop,
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
mutable_cf_options.disable_auto_compactions);
|
2015-11-19 03:10:20 +01:00
|
|
|
internal_stats_->AddCFStats(
|
2017-09-08 03:13:53 +02:00
|
|
|
InternalStats::PENDING_COMPACTION_BYTES_LIMIT_SLOWDOWNS, 1);
|
2017-03-16 03:22:52 +01:00
|
|
|
ROCKS_LOG_WARN(
|
|
|
|
ioptions_.info_log,
|
2015-11-19 03:10:20 +01:00
|
|
|
"[%s] Stalling writes because of estimated pending compaction "
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
"bytes %" PRIu64 " rate %" PRIu64,
|
|
|
|
name_.c_str(), vstorage->estimated_compaction_needed_bytes(),
|
|
|
|
write_controller->delayed_write_rate());
|
2016-11-23 18:19:11 +01:00
|
|
|
} else {
|
2018-02-13 00:34:39 +01:00
|
|
|
assert(write_stall_condition == WriteStallCondition::kNormal);
|
2016-11-23 18:19:11 +01:00
|
|
|
if (vstorage->l0_delay_trigger_count() >=
|
|
|
|
GetL0ThresholdSpeedupCompaction(
|
|
|
|
mutable_cf_options.level0_file_num_compaction_trigger,
|
|
|
|
mutable_cf_options.level0_slowdown_writes_trigger)) {
|
|
|
|
write_controller_token_ =
|
|
|
|
write_controller->GetCompactionPressureToken();
|
2017-05-26 18:50:49 +02:00
|
|
|
ROCKS_LOG_INFO(
|
2017-03-16 03:22:52 +01:00
|
|
|
ioptions_.info_log,
|
2016-11-23 18:19:11 +01:00
|
|
|
"[%s] Increasing compaction threads because we have %d level-0 "
|
|
|
|
"files ",
|
|
|
|
name_.c_str(), vstorage->l0_delay_trigger_count());
|
|
|
|
} else if (vstorage->estimated_compaction_needed_bytes() >=
|
|
|
|
mutable_cf_options.soft_pending_compaction_bytes_limit / 4) {
|
|
|
|
// Increase compaction threads if bytes needed for compaction exceeds
|
|
|
|
// 1/4 of threshold for slowing down.
|
|
|
|
// If soft pending compaction byte limit is not set, always speed up
|
|
|
|
// compaction.
|
|
|
|
write_controller_token_ =
|
|
|
|
write_controller->GetCompactionPressureToken();
|
|
|
|
if (mutable_cf_options.soft_pending_compaction_bytes_limit > 0) {
|
2017-05-26 18:50:49 +02:00
|
|
|
ROCKS_LOG_INFO(
|
2017-03-16 03:22:52 +01:00
|
|
|
ioptions_.info_log,
|
2016-11-23 18:19:11 +01:00
|
|
|
"[%s] Increasing compaction threads because of estimated pending "
|
|
|
|
"compaction "
|
|
|
|
"bytes %" PRIu64,
|
|
|
|
name_.c_str(), vstorage->estimated_compaction_needed_bytes());
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
write_controller_token_.reset();
|
|
|
|
}
|
|
|
|
// If the DB recovers from delay conditions, we reward with reducing
|
|
|
|
// double the slowdown ratio. This is to balance the long term slowdown
|
|
|
|
// increase signal.
|
|
|
|
if (needed_delay) {
|
|
|
|
uint64_t write_rate = write_controller->delayed_write_rate();
|
|
|
|
write_controller->set_delayed_write_rate(static_cast<uint64_t>(
|
|
|
|
static_cast<double>(write_rate) * kDelayRecoverSlowdownRatio));
|
2017-06-05 23:42:34 +02:00
|
|
|
// Set the low pri limit to be 1/4 the delayed write rate.
|
|
|
|
// Note we don't reset this value even after delay condition is relased.
|
|
|
|
// Low-pri rate will continue to apply if there is a compaction
|
|
|
|
// pressure.
|
|
|
|
write_controller->low_pri_rate_limiter()->SetBytesPerSecond(write_rate /
|
|
|
|
4);
|
Add options.base_background_compactions as a number of compaction threads for low compaction debt
Summary:
If options.base_background_compactions is given, we try to schedule number of compactions not existing this number, only when L0 files increase to certain number, or pending compaction bytes more than certain threshold, we schedule compactions based on options.max_background_compactions.
The watermarks are calculated based on slowdown thresholds.
Test Plan:
Add new test cases in column_family_test.
Adding more unit tests.
Reviewers: IslamAbdelRahman, yhchiang, kradhakrishnan, rven, anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D53409
2016-01-28 20:56:16 +01:00
|
|
|
}
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
}
|
When slowdown is triggered, reduce the write rate
Summary: It's usually hard for users to set a value of options.delayed_write_rate. With this diff, after slowdown condition triggers, we greedily reduce write rate if estimated pending compaction bytes increase. If estimated compaction pending bytes drop, we increase the write rate.
Test Plan:
Add a unit test
Test with db_bench setting:
TEST_TMPDIR=/dev/shm/ ./db_bench --benchmarks=fillrandom -num=10000000 --soft_pending_compaction_bytes_limit=1000000000 --hard_pending_compaction_bytes_limit=3000000000 --delayed_write_rate=100000000
and make sure without the commit, write stop will happen, but with the commit, it will not happen.
Reviewers: igor, anthony, rven, yhchiang, kradhakrishnan, IslamAbdelRahman
Reviewed By: IslamAbdelRahman
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D52131
2015-12-18 02:07:44 +01:00
|
|
|
prev_compaction_needed_bytes_ = compaction_needed_bytes;
|
Cache some conditions for DBImpl::MakeRoomForWrite
Summary:
Task 4580155. Some conditions in DBImpl::MakeRoomForWrite can be cached in
ColumnFamilyData, because theirs value can be changed only during compaction,
adding new memtable and/or add recalculation of compaction score.
These conditions are:
cfd->imm()->size() == cfd->options()->max_write_buffer_number - 1
cfd->current()->NumLevelFiles(0) >= cfd->options()->level0_stop_writes_trigger
cfd->options()->soft_rate_limit > 0.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->soft_rate_limit
cfd->options()->hard_rate_limit > 1.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->hard_rate_limit
P.S.
As it's my first diff, Siying suggested to add everybody as a reviewers
for this diff. Sorry, if I forgot someone or add someone by mistake.
Test Plan: make all check
Reviewers: haobo, xjin, dhruba, yhchiang, zagfox, ljin, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19311
2014-06-27 01:45:27 +02:00
|
|
|
}
|
2017-10-06 03:00:38 +02:00
|
|
|
return write_stall_condition;
|
Cache some conditions for DBImpl::MakeRoomForWrite
Summary:
Task 4580155. Some conditions in DBImpl::MakeRoomForWrite can be cached in
ColumnFamilyData, because theirs value can be changed only during compaction,
adding new memtable and/or add recalculation of compaction score.
These conditions are:
cfd->imm()->size() == cfd->options()->max_write_buffer_number - 1
cfd->current()->NumLevelFiles(0) >= cfd->options()->level0_stop_writes_trigger
cfd->options()->soft_rate_limit > 0.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->soft_rate_limit
cfd->options()->hard_rate_limit > 1.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->hard_rate_limit
P.S.
As it's my first diff, Siying suggested to add everybody as a reviewers
for this diff. Sorry, if I forgot someone or add someone by mistake.
Test Plan: make all check
Reviewers: haobo, xjin, dhruba, yhchiang, zagfox, ljin, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19311
2014-06-27 01:45:27 +02:00
|
|
|
}
|
|
|
|
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
|
|
|
const FileOptions* ColumnFamilyData::soptions() const {
|
|
|
|
return &(column_family_set_->file_options_);
|
2014-04-14 19:48:01 +02:00
|
|
|
}
|
|
|
|
|
2014-11-06 20:14:28 +01:00
|
|
|
void ColumnFamilyData::SetCurrent(Version* current_version) {
|
|
|
|
current_ = current_version;
|
|
|
|
}
|
2014-01-31 00:23:13 +01:00
|
|
|
|
2015-02-12 02:10:43 +01:00
|
|
|
uint64_t ColumnFamilyData::GetNumLiveVersions() const {
|
|
|
|
return VersionSet::GetNumLiveVersions(dummy_versions_);
|
|
|
|
}
|
|
|
|
|
2015-08-20 20:47:19 +02:00
|
|
|
uint64_t ColumnFamilyData::GetTotalSstFilesSize() const {
|
|
|
|
return VersionSet::GetTotalSstFilesSize(dummy_versions_);
|
|
|
|
}
|
|
|
|
|
2018-03-02 02:50:54 +01:00
|
|
|
uint64_t ColumnFamilyData::GetLiveSstFilesSize() const {
|
|
|
|
return current_->GetSstFilesSize();
|
|
|
|
}
|
|
|
|
|
2014-12-02 21:09:20 +01:00
|
|
|
MemTable* ColumnFamilyData::ConstructNewMemtable(
|
2015-05-29 23:36:35 +02:00
|
|
|
const MutableCFOptions& mutable_cf_options, SequenceNumber earliest_seq) {
|
|
|
|
return new MemTable(internal_comparator_, ioptions_, mutable_cf_options,
|
2017-06-02 21:08:01 +02:00
|
|
|
write_buffer_manager_, earliest_seq, id_);
|
2014-12-02 21:09:20 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
void ColumnFamilyData::CreateNewMemtable(
|
2015-05-29 23:36:35 +02:00
|
|
|
const MutableCFOptions& mutable_cf_options, SequenceNumber earliest_seq) {
|
2014-01-29 22:28:50 +01:00
|
|
|
if (mem_ != nullptr) {
|
|
|
|
delete mem_->Unref();
|
2014-01-24 23:30:28 +01:00
|
|
|
}
|
2015-05-29 23:36:35 +02:00
|
|
|
SetMemtable(ConstructNewMemtable(mutable_cf_options, earliest_seq));
|
2014-01-29 22:28:50 +01:00
|
|
|
mem_->Ref();
|
|
|
|
}
|
|
|
|
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
bool ColumnFamilyData::NeedsCompaction() const {
|
|
|
|
return compaction_picker_->NeedsCompaction(current_->storage_info());
|
|
|
|
}
|
|
|
|
|
2014-10-02 01:19:16 +02:00
|
|
|
Compaction* ColumnFamilyData::PickCompaction(
|
2020-07-23 03:31:25 +02:00
|
|
|
const MutableCFOptions& mutable_options,
|
|
|
|
const MutableDBOptions& mutable_db_options, LogBuffer* log_buffer) {
|
Fix corruption with intra-L0 on ingested files (#5958)
Summary:
## Problem Description
Our process was abort when it call `CheckConsistency`. And the information in `stderr` show that "`L0 files seqno 3001491972 3004797440 vs. 3002875611 3004524421` ". Here are the causes of the accident I investigated.
* RocksDB will call `CheckConsistency` whenever `MANIFEST` file is update. It will check sequence number interval of every file, except files which were ingested.
* When one file is ingested into RocksDB, it will be assigned the value of global sequence number, and the minimum and maximum seqno of this file are equal, which are both equal to global sequence number.
* `CheckConsistency` determines whether the file is ingested by whether the smallest and largest seqno of an sstable file are equal.
* If IntraL0Compaction picks one sst which was ingested just now and compacted it into another sst, the `smallest_seqno` of this new file will be smaller than his `largest_seqno`.
* If more than one ingested file was ingested before memtable schedule flush, and they all compact into one new sstable file by `IntraL0Compaction`. The sequence interval of this new file will be included in the interval of the memtable. So `CheckConsistency` will return a `Corruption`.
* If a sstable was ingested after the memtable was schedule to flush, which would assign a larger seqno to it than memtable. Then the file was compacted with other files (these files were all flushed before the memtable) in L0 into one file. This compaction start before the flush job of memtable start, but completed after the flush job finish. So this new file produced by the compaction (we call it s1) would have a larger interval of sequence number than the file produced by flush (we call it s2). **But there was still some data in s1 written into RocksDB before the s2, so it's possible that some data in s2 was cover by old data in s1.** Of course, it would also make a `Corruption` because of overlap of seqno. There is the relationship of the files:
> s1.smallest_seqno < s2.smallest_seqno < s2.largest_seqno < s1.largest_seqno
So I skip pick sst file which was ingested in function `FindIntraL0Compaction `
## Reason
Here is my bug report: https://github.com/facebook/rocksdb/issues/5913
There are two situations that can cause the check to fail.
### First situation:
- First we ingest five external sst into Rocksdb, and they happened to be ingested in L0. and there had been some data in memtable, which make the smallest sequence number of memtable is less than which of sst that we ingest.
- If there had been one compaction job which compacted sst from L0 to L1, `LevelCompactionPicker` would trigger a `IntraL0Compaction` which would compact this five sst from L0 to L0. We call this sst A, which was merged from five ingested sst.
- Then some data was put into memtable, and memtable was flushed to L0. We called this sst B.
- RocksDB check consistency , and find the `smallest_seqno` of B is less than that of A and crash. Because A was merged from five sst, the smallest sequence number of it was less than the biggest sequece number of itself, so RocksDB could not tell if A was produce by ingested.
### Secondary situaion
- First we have flushed many sst in L0, we call them [s1, s2, s3].
- There is an immutable memtable request to be flushed, but because flush thread is busy, so it has not been picked. we call it m1. And at the moment, one sst is ingested into L0. We call it s4. Because s4 is ingested after m1 became immutable memtable, so it has a larger log sequence number than m1.
- m1 is flushed in L0. because it is small, this flush job finish quickly. we call it s5.
- [s1, s2, s3, s4] are compacted into one sst to L0, by IntraL0Compaction. We call it s6.
- compacted 4@0 files to L0
- When s6 is added into manifest, the corruption happened. because the largest sequence number of s6 is equal to s4, and they are both larger than that of s5. But because s1 is older than m1, so the smallest sequence number of s6 is smaller than that of s5.
- s6.smallest_seqno < s5.smallest_seqno < s5.largest_seqno < s6.largest_seqno
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5958
Differential Revision: D18601316
fbshipit-source-id: 5fe54b3c9af52a2e1400728f565e895cde1c7267
2019-11-20 00:07:49 +01:00
|
|
|
SequenceNumber earliest_mem_seqno =
|
|
|
|
std::min(mem_->GetEarliestSequenceNumber(),
|
|
|
|
imm_.current()->GetEarliestSequenceNumber(false));
|
2014-10-27 23:49:46 +01:00
|
|
|
auto* result = compaction_picker_->PickCompaction(
|
2020-07-23 03:31:25 +02:00
|
|
|
GetName(), mutable_options, mutable_db_options, current_->storage_info(),
|
|
|
|
log_buffer, earliest_mem_seqno);
|
2014-10-27 23:49:46 +01:00
|
|
|
if (result != nullptr) {
|
|
|
|
result->SetInputVersion(current_);
|
|
|
|
}
|
Cache some conditions for DBImpl::MakeRoomForWrite
Summary:
Task 4580155. Some conditions in DBImpl::MakeRoomForWrite can be cached in
ColumnFamilyData, because theirs value can be changed only during compaction,
adding new memtable and/or add recalculation of compaction score.
These conditions are:
cfd->imm()->size() == cfd->options()->max_write_buffer_number - 1
cfd->current()->NumLevelFiles(0) >= cfd->options()->level0_stop_writes_trigger
cfd->options()->soft_rate_limit > 0.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->soft_rate_limit
cfd->options()->hard_rate_limit > 1.0 &&
(score = cfd->current()->MaxCompactionScore()) > cfd->options()->hard_rate_limit
P.S.
As it's my first diff, Siying suggested to add everybody as a reviewers
for this diff. Sorry, if I forgot someone or add someone by mistake.
Test Plan: make all check
Reviewers: haobo, xjin, dhruba, yhchiang, zagfox, ljin, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19311
2014-06-27 01:45:27 +02:00
|
|
|
return result;
|
2014-02-01 00:30:27 +01:00
|
|
|
}
|
|
|
|
|
2016-10-13 19:49:06 +02:00
|
|
|
bool ColumnFamilyData::RangeOverlapWithCompaction(
|
|
|
|
const Slice& smallest_user_key, const Slice& largest_user_key,
|
|
|
|
int level) const {
|
|
|
|
return compaction_picker_->RangeOverlapWithCompaction(
|
|
|
|
smallest_user_key, largest_user_key, level);
|
|
|
|
}
|
|
|
|
|
2018-02-28 02:08:34 +01:00
|
|
|
Status ColumnFamilyData::RangesOverlapWithMemtables(
|
|
|
|
const autovector<Range>& ranges, SuperVersion* super_version,
|
|
|
|
bool* overlap) {
|
|
|
|
assert(overlap != nullptr);
|
|
|
|
*overlap = false;
|
|
|
|
// Create an InternalIterator over all unflushed memtables
|
|
|
|
Arena arena;
|
|
|
|
ReadOptions read_opts;
|
|
|
|
read_opts.total_order_seek = true;
|
|
|
|
MergeIteratorBuilder merge_iter_builder(&internal_comparator_, &arena);
|
|
|
|
merge_iter_builder.AddIterator(
|
|
|
|
super_version->mem->NewIterator(read_opts, &arena));
|
|
|
|
super_version->imm->AddIterators(read_opts, &merge_iter_builder);
|
|
|
|
ScopedArenaIterator memtable_iter(merge_iter_builder.Finish());
|
|
|
|
|
2018-11-29 00:26:56 +01:00
|
|
|
auto read_seq = super_version->current->version_set()->LastSequence();
|
2018-12-18 02:26:56 +01:00
|
|
|
ReadRangeDelAggregator range_del_agg(&internal_comparator_, read_seq);
|
2018-02-28 02:08:34 +01:00
|
|
|
auto* active_range_del_iter =
|
2018-11-29 00:26:56 +01:00
|
|
|
super_version->mem->NewRangeTombstoneIterator(read_opts, read_seq);
|
|
|
|
range_del_agg.AddTombstones(
|
|
|
|
std::unique_ptr<FragmentedRangeTombstoneIterator>(active_range_del_iter));
|
|
|
|
super_version->imm->AddRangeTombstoneIterators(read_opts, nullptr /* arena */,
|
|
|
|
&range_del_agg);
|
|
|
|
|
2018-11-21 19:53:44 +01:00
|
|
|
Status status;
|
2018-02-28 02:08:34 +01:00
|
|
|
for (size_t i = 0; i < ranges.size() && status.ok() && !*overlap; ++i) {
|
|
|
|
auto* vstorage = super_version->current->storage_info();
|
|
|
|
auto* ucmp = vstorage->InternalComparator()->user_comparator();
|
|
|
|
InternalKey range_start(ranges[i].start, kMaxSequenceNumber,
|
|
|
|
kValueTypeForSeek);
|
|
|
|
memtable_iter->Seek(range_start.Encode());
|
|
|
|
status = memtable_iter->status();
|
|
|
|
ParsedInternalKey seek_result;
|
|
|
|
if (status.ok()) {
|
|
|
|
if (memtable_iter->Valid() &&
|
|
|
|
!ParseInternalKey(memtable_iter->key(), &seek_result)) {
|
|
|
|
status = Status::Corruption("DB have corrupted keys");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (status.ok()) {
|
|
|
|
if (memtable_iter->Valid() &&
|
|
|
|
ucmp->Compare(seek_result.user_key, ranges[i].limit) <= 0) {
|
|
|
|
*overlap = true;
|
|
|
|
} else if (range_del_agg.IsRangeOverlapped(ranges[i].start,
|
|
|
|
ranges[i].limit)) {
|
|
|
|
*overlap = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
2015-04-23 01:55:22 +02:00
|
|
|
const int ColumnFamilyData::kCompactAllLevels = -1;
|
2015-04-15 06:45:20 +02:00
|
|
|
const int ColumnFamilyData::kCompactToBaseLevel = -2;
|
2015-04-23 01:55:22 +02:00
|
|
|
|
2014-10-02 01:19:16 +02:00
|
|
|
Compaction* ColumnFamilyData::CompactRange(
|
2020-07-23 03:31:25 +02:00
|
|
|
const MutableCFOptions& mutable_cf_options,
|
|
|
|
const MutableDBOptions& mutable_db_options, int input_level,
|
2019-04-17 08:29:32 +02:00
|
|
|
int output_level, const CompactRangeOptions& compact_range_options,
|
2018-04-27 20:48:21 +02:00
|
|
|
const InternalKey* begin, const InternalKey* end,
|
2019-04-17 08:29:32 +02:00
|
|
|
InternalKey** compaction_end, bool* conflict,
|
|
|
|
uint64_t max_file_num_to_ignore) {
|
2014-10-27 23:49:46 +01:00
|
|
|
auto* result = compaction_picker_->CompactRange(
|
2020-07-23 03:31:25 +02:00
|
|
|
GetName(), mutable_cf_options, mutable_db_options,
|
|
|
|
current_->storage_info(), input_level, output_level,
|
|
|
|
compact_range_options, begin, end, compaction_end, conflict,
|
2019-04-17 08:29:32 +02:00
|
|
|
max_file_num_to_ignore);
|
2014-10-27 23:49:46 +01:00
|
|
|
if (result != nullptr) {
|
|
|
|
result->SetInputVersion(current_);
|
|
|
|
}
|
|
|
|
return result;
|
2014-02-01 00:30:27 +01:00
|
|
|
}
|
|
|
|
|
2019-12-17 22:20:42 +01:00
|
|
|
SuperVersion* ColumnFamilyData::GetReferencedSuperVersion(DBImpl* db) {
|
|
|
|
SuperVersion* sv = GetThreadLocalSuperVersion(db);
|
2014-09-24 22:12:16 +02:00
|
|
|
sv->Ref();
|
|
|
|
if (!ReturnThreadLocalSuperVersion(sv)) {
|
2017-10-12 00:11:08 +02:00
|
|
|
// This Unref() corresponds to the Ref() in GetThreadLocalSuperVersion()
|
|
|
|
// when the thread-local pointer was populated. So, the Ref() earlier in
|
|
|
|
// this function still prevents the returned SuperVersion* from being
|
|
|
|
// deleted out from under the caller.
|
2014-09-24 22:12:16 +02:00
|
|
|
sv->Unref();
|
2014-04-14 18:34:59 +02:00
|
|
|
}
|
|
|
|
return sv;
|
|
|
|
}
|
|
|
|
|
2019-12-17 22:20:42 +01:00
|
|
|
SuperVersion* ColumnFamilyData::GetThreadLocalSuperVersion(DBImpl* db) {
|
2014-04-14 18:34:59 +02:00
|
|
|
// The SuperVersion is cached in thread local storage to avoid acquiring
|
|
|
|
// mutex when SuperVersion does not change since the last use. When a new
|
|
|
|
// SuperVersion is installed, the compaction or flush thread cleans up
|
|
|
|
// cached SuperVersion in all existing thread local storage. To avoid
|
|
|
|
// acquiring mutex for this operation, we use atomic Swap() on the thread
|
|
|
|
// local pointer to guarantee exclusive access. If the thread local pointer
|
|
|
|
// is being used while a new SuperVersion is installed, the cached
|
|
|
|
// SuperVersion can become stale. In that case, the background thread would
|
|
|
|
// have swapped in kSVObsolete. We re-check the value at when returning
|
|
|
|
// SuperVersion back to thread local, with an atomic compare and swap.
|
|
|
|
// The superversion will need to be released if detected to be stale.
|
|
|
|
void* ptr = local_sv_->Swap(SuperVersion::kSVInUse);
|
|
|
|
// Invariant:
|
|
|
|
// (1) Scrape (always) installs kSVObsolete in ThreadLocal storage
|
|
|
|
// (2) the Swap above (always) installs kSVInUse, ThreadLocal storage
|
|
|
|
// should only keep kSVInUse before ReturnThreadLocalSuperVersion call
|
|
|
|
// (if no Scrape happens).
|
|
|
|
assert(ptr != SuperVersion::kSVInUse);
|
2018-02-16 01:43:23 +01:00
|
|
|
SuperVersion* sv = static_cast<SuperVersion*>(ptr);
|
2014-04-14 18:34:59 +02:00
|
|
|
if (sv == SuperVersion::kSVObsolete ||
|
|
|
|
sv->version_number != super_version_number_.load()) {
|
2014-10-02 01:19:16 +02:00
|
|
|
RecordTick(ioptions_.statistics, NUMBER_SUPERVERSION_ACQUIRES);
|
2014-04-14 18:34:59 +02:00
|
|
|
SuperVersion* sv_to_delete = nullptr;
|
|
|
|
|
|
|
|
if (sv && sv->Unref()) {
|
2014-10-02 01:19:16 +02:00
|
|
|
RecordTick(ioptions_.statistics, NUMBER_SUPERVERSION_CLEANUPS);
|
2019-12-17 22:20:42 +01:00
|
|
|
db->mutex()->Lock();
|
2014-04-14 18:34:59 +02:00
|
|
|
// NOTE: underlying resources held by superversion (sst files) might
|
|
|
|
// not be released until the next background job.
|
|
|
|
sv->Cleanup();
|
2019-12-17 22:20:42 +01:00
|
|
|
if (db->immutable_db_options().avoid_unnecessary_blocking_io) {
|
|
|
|
db->AddSuperVersionsToFreeQueue(sv);
|
|
|
|
db->SchedulePurge();
|
|
|
|
} else {
|
|
|
|
sv_to_delete = sv;
|
|
|
|
}
|
2014-04-14 18:34:59 +02:00
|
|
|
} else {
|
2019-12-17 22:20:42 +01:00
|
|
|
db->mutex()->Lock();
|
2014-04-14 18:34:59 +02:00
|
|
|
}
|
|
|
|
sv = super_version_->Ref();
|
2019-12-17 22:20:42 +01:00
|
|
|
db->mutex()->Unlock();
|
2014-04-14 18:34:59 +02:00
|
|
|
|
|
|
|
delete sv_to_delete;
|
|
|
|
}
|
|
|
|
assert(sv != nullptr);
|
|
|
|
return sv;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool ColumnFamilyData::ReturnThreadLocalSuperVersion(SuperVersion* sv) {
|
|
|
|
assert(sv != nullptr);
|
|
|
|
// Put the SuperVersion back
|
|
|
|
void* expected = SuperVersion::kSVInUse;
|
|
|
|
if (local_sv_->CompareAndSwap(static_cast<void*>(sv), expected)) {
|
|
|
|
// When we see kSVInUse in the ThreadLocal, we are sure ThreadLocal
|
2015-04-25 11:14:27 +02:00
|
|
|
// storage has not been altered and no Scrape has happened. The
|
2014-04-14 18:34:59 +02:00
|
|
|
// SuperVersion is still current.
|
|
|
|
return true;
|
|
|
|
} else {
|
|
|
|
// ThreadLocal scrape happened in the process of this GetImpl call (after
|
|
|
|
// thread local Swap() at the beginning and before CompareAndSwap()).
|
|
|
|
// This means the SuperVersion it holds is obsolete.
|
|
|
|
assert(expected == SuperVersion::kSVObsolete);
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
void ColumnFamilyData::InstallSuperVersion(
|
|
|
|
SuperVersionContext* sv_context, InstrumentedMutex* db_mutex) {
|
2014-09-17 21:49:13 +02:00
|
|
|
db_mutex->AssertHeld();
|
2017-10-06 03:00:38 +02:00
|
|
|
return InstallSuperVersion(sv_context, db_mutex, mutable_cf_options_);
|
2014-09-17 21:49:13 +02:00
|
|
|
}
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
void ColumnFamilyData::InstallSuperVersion(
|
|
|
|
SuperVersionContext* sv_context, InstrumentedMutex* db_mutex,
|
2014-09-17 21:49:13 +02:00
|
|
|
const MutableCFOptions& mutable_cf_options) {
|
2017-10-06 03:00:38 +02:00
|
|
|
SuperVersion* new_superversion = sv_context->new_superversion.release();
|
2014-03-31 21:44:54 +02:00
|
|
|
new_superversion->db_mutex = db_mutex;
|
2014-09-17 21:49:13 +02:00
|
|
|
new_superversion->mutable_cf_options = mutable_cf_options;
|
2019-12-13 04:02:51 +01:00
|
|
|
new_superversion->Init(this, mem_, imm_.current(), current_);
|
2014-01-29 22:28:50 +01:00
|
|
|
SuperVersion* old_superversion = super_version_;
|
|
|
|
super_version_ = new_superversion;
|
|
|
|
++super_version_number_;
|
2014-03-04 02:54:04 +01:00
|
|
|
super_version_->version_number = super_version_number_;
|
2017-10-06 03:00:38 +02:00
|
|
|
super_version_->write_stall_condition =
|
|
|
|
RecalculateWriteStallConditions(mutable_cf_options);
|
|
|
|
|
|
|
|
if (old_superversion != nullptr) {
|
Fix deadlock in ColumnFamilyData::InstallSuperVersion()
Summary:
Deadlock: a memtable flush holds DB::mutex_ and calls ThreadLocalPtr::Scrape(), which locks ThreadLocalPtr mutex; meanwhile, a thread exit handler locks ThreadLocalPtr mutex and calls SuperVersionUnrefHandle, which tries to lock DB::mutex_.
This deadlock is hit all the time on our workload. It blocks our release.
In general, the problem is that ThreadLocalPtr takes an arbitrary callback and calls it while holding a lock on a global mutex. The same global mutex is (at least in some cases) locked by almost all ThreadLocalPtr methods, on any instance of ThreadLocalPtr. So, there'll be a deadlock if the callback tries to do anything to any instance of ThreadLocalPtr, or waits for another thread to do so.
So, probably the only safe way to use ThreadLocalPtr callbacks is to do only do simple and lock-free things in them.
This PR fixes the deadlock by making sure that local_sv_ never holds the last reference to a SuperVersion, and therefore SuperVersionUnrefHandle never has to do any nontrivial cleanup.
I also searched for other uses of ThreadLocalPtr to see if they may have similar bugs. There's only one other use, in transaction_lock_mgr.cc, and it looks fine.
Closes https://github.com/facebook/rocksdb/pull/3510
Reviewed By: sagar0
Differential Revision: D7005346
Pulled By: al13n321
fbshipit-source-id: 37575591b84f07a891d6659e87e784660fde815f
2018-02-16 16:58:18 +01:00
|
|
|
// Reset SuperVersions cached in thread local storage.
|
|
|
|
// This should be done before old_superversion->Unref(). That's to ensure
|
|
|
|
// that local_sv_ never holds the last reference to SuperVersion, since
|
|
|
|
// it has no means to safely do SuperVersion cleanup.
|
|
|
|
ResetThreadLocalSuperVersions();
|
|
|
|
|
2017-11-03 06:16:23 +01:00
|
|
|
if (old_superversion->mutable_cf_options.write_buffer_size !=
|
|
|
|
mutable_cf_options.write_buffer_size) {
|
|
|
|
mem_->UpdateWriteBufferSize(mutable_cf_options.write_buffer_size);
|
|
|
|
}
|
2017-10-06 03:00:38 +02:00
|
|
|
if (old_superversion->write_stall_condition !=
|
|
|
|
new_superversion->write_stall_condition) {
|
|
|
|
sv_context->PushWriteStallNotification(
|
|
|
|
old_superversion->write_stall_condition,
|
|
|
|
new_superversion->write_stall_condition, GetName(), ioptions());
|
|
|
|
}
|
|
|
|
if (old_superversion->Unref()) {
|
|
|
|
old_superversion->Cleanup();
|
|
|
|
sv_context->superversions_to_free.push_back(old_superversion);
|
|
|
|
}
|
|
|
|
}
|
2014-01-22 20:44:53 +01:00
|
|
|
}
|
|
|
|
|
2014-03-04 02:54:04 +01:00
|
|
|
void ColumnFamilyData::ResetThreadLocalSuperVersions() {
|
|
|
|
autovector<void*> sv_ptrs;
|
2014-03-08 01:59:47 +01:00
|
|
|
local_sv_->Scrape(&sv_ptrs, SuperVersion::kSVObsolete);
|
2014-03-04 02:54:04 +01:00
|
|
|
for (auto ptr : sv_ptrs) {
|
|
|
|
assert(ptr);
|
2014-03-08 01:59:47 +01:00
|
|
|
if (ptr == SuperVersion::kSVInUse) {
|
|
|
|
continue;
|
|
|
|
}
|
2014-03-04 02:54:04 +01:00
|
|
|
auto sv = static_cast<SuperVersion*>(ptr);
|
Fix deadlock in ColumnFamilyData::InstallSuperVersion()
Summary:
Deadlock: a memtable flush holds DB::mutex_ and calls ThreadLocalPtr::Scrape(), which locks ThreadLocalPtr mutex; meanwhile, a thread exit handler locks ThreadLocalPtr mutex and calls SuperVersionUnrefHandle, which tries to lock DB::mutex_.
This deadlock is hit all the time on our workload. It blocks our release.
In general, the problem is that ThreadLocalPtr takes an arbitrary callback and calls it while holding a lock on a global mutex. The same global mutex is (at least in some cases) locked by almost all ThreadLocalPtr methods, on any instance of ThreadLocalPtr. So, there'll be a deadlock if the callback tries to do anything to any instance of ThreadLocalPtr, or waits for another thread to do so.
So, probably the only safe way to use ThreadLocalPtr callbacks is to do only do simple and lock-free things in them.
This PR fixes the deadlock by making sure that local_sv_ never holds the last reference to a SuperVersion, and therefore SuperVersionUnrefHandle never has to do any nontrivial cleanup.
I also searched for other uses of ThreadLocalPtr to see if they may have similar bugs. There's only one other use, in transaction_lock_mgr.cc, and it looks fine.
Closes https://github.com/facebook/rocksdb/pull/3510
Reviewed By: sagar0
Differential Revision: D7005346
Pulled By: al13n321
fbshipit-source-id: 37575591b84f07a891d6659e87e784660fde815f
2018-02-16 16:58:18 +01:00
|
|
|
bool was_last_ref __attribute__((__unused__));
|
|
|
|
was_last_ref = sv->Unref();
|
|
|
|
// sv couldn't have been the last reference because
|
|
|
|
// ResetThreadLocalSuperVersions() is called before
|
|
|
|
// unref'ing super_version_.
|
|
|
|
assert(!was_last_ref);
|
2014-03-04 02:54:04 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-06-04 04:47:02 +02:00
|
|
|
Status ColumnFamilyData::ValidateOptions(
|
|
|
|
const DBOptions& db_options, const ColumnFamilyOptions& cf_options) {
|
|
|
|
Status s;
|
|
|
|
s = CheckCompressionSupported(cf_options);
|
|
|
|
if (s.ok() && db_options.allow_concurrent_memtable_write) {
|
|
|
|
s = CheckConcurrentWritesSupported(cf_options);
|
|
|
|
}
|
2020-01-11 01:51:34 +01:00
|
|
|
if (s.ok() && db_options.unordered_write &&
|
|
|
|
cf_options.max_successive_merges != 0) {
|
|
|
|
s = Status::InvalidArgument(
|
|
|
|
"max_successive_merges > 0 is incompatible with unordered_write");
|
|
|
|
}
|
2019-06-04 04:47:02 +02:00
|
|
|
if (s.ok()) {
|
|
|
|
s = CheckCFPathsSupported(db_options, cf_options);
|
|
|
|
}
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2019-11-26 02:11:26 +01:00
|
|
|
if (cf_options.ttl > 0 && cf_options.ttl != kDefaultTtl) {
|
2020-07-08 20:52:13 +02:00
|
|
|
if (cf_options.table_factory->Name() != BlockBasedTableFactory::kName) {
|
2019-06-04 04:47:02 +02:00
|
|
|
return Status::NotSupported(
|
|
|
|
"TTL is only supported in Block-Based Table format. ");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
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
|
|
|
if (cf_options.periodic_compaction_seconds > 0 &&
|
2019-11-26 02:11:26 +01:00
|
|
|
cf_options.periodic_compaction_seconds != kDefaultPeriodicCompSecs) {
|
2020-07-08 20:52:13 +02:00
|
|
|
if (cf_options.table_factory->Name() != BlockBasedTableFactory::kName) {
|
2019-06-04 04:47:02 +02:00
|
|
|
return Status::NotSupported(
|
|
|
|
"Periodic Compaction is only supported in "
|
|
|
|
"Block-Based Table format. ");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2014-11-13 22:45:33 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-11-05 01:23:05 +01:00
|
|
|
Status ColumnFamilyData::SetOptions(
|
2019-06-04 04:47:02 +02:00
|
|
|
const DBOptions& db_options,
|
|
|
|
const std::unordered_map<std::string, std::string>& options_map) {
|
2014-09-17 21:49:13 +02:00
|
|
|
MutableCFOptions new_mutable_cf_options;
|
2018-04-11 03:48:49 +02:00
|
|
|
Status s =
|
|
|
|
GetMutableOptionsFromStrings(mutable_cf_options_, options_map,
|
|
|
|
ioptions_.info_log, &new_mutable_cf_options);
|
2019-06-04 04:47:02 +02:00
|
|
|
if (s.ok()) {
|
|
|
|
ColumnFamilyOptions cf_options =
|
|
|
|
BuildColumnFamilyOptions(initial_cf_options_, new_mutable_cf_options);
|
|
|
|
s = ValidateOptions(db_options, cf_options);
|
|
|
|
}
|
2014-11-05 01:23:05 +01:00
|
|
|
if (s.ok()) {
|
2014-09-17 21:49:13 +02:00
|
|
|
mutable_cf_options_ = new_mutable_cf_options;
|
2014-10-02 01:19:16 +02:00
|
|
|
mutable_cf_options_.RefreshDerivedOptions(ioptions_);
|
2014-09-17 21:49:13 +02:00
|
|
|
}
|
2014-11-05 01:23:05 +01:00
|
|
|
return s;
|
2014-09-17 21:49:13 +02:00
|
|
|
}
|
2014-11-13 22:45:33 +01:00
|
|
|
#endif // ROCKSDB_LITE
|
2014-09-17 21:49:13 +02:00
|
|
|
|
2017-11-10 18:25:26 +01:00
|
|
|
// REQUIRES: DB mutex held
|
|
|
|
Env::WriteLifeTimeHint ColumnFamilyData::CalculateSSTWriteHint(int level) {
|
|
|
|
if (initial_cf_options_.compaction_style != kCompactionStyleLevel) {
|
|
|
|
return Env::WLTH_NOT_SET;
|
|
|
|
}
|
|
|
|
if (level == 0) {
|
|
|
|
return Env::WLTH_MEDIUM;
|
|
|
|
}
|
|
|
|
int base_level = current_->storage_info()->base_level();
|
|
|
|
|
|
|
|
// L1: medium, L2: long, ...
|
|
|
|
if (level - base_level >= 2) {
|
|
|
|
return Env::WLTH_EXTREME;
|
2019-12-19 02:02:39 +01:00
|
|
|
} else if (level < base_level) {
|
|
|
|
// There is no restriction which prevents level passed in to be smaller
|
|
|
|
// than base_level.
|
|
|
|
return Env::WLTH_MEDIUM;
|
2017-11-10 18:25:26 +01:00
|
|
|
}
|
|
|
|
return static_cast<Env::WriteLifeTimeHint>(level - base_level +
|
|
|
|
static_cast<int>(Env::WLTH_MEDIUM));
|
|
|
|
}
|
|
|
|
|
2020-02-03 23:11:50 +01:00
|
|
|
Status ColumnFamilyData::AddDirectories(
|
2020-03-03 01:14:00 +01:00
|
|
|
std::map<std::string, std::shared_ptr<FSDirectory>>* created_dirs) {
|
2018-04-06 04:49:06 +02:00
|
|
|
Status s;
|
2020-02-03 23:11:50 +01:00
|
|
|
assert(created_dirs != nullptr);
|
2018-04-06 04:49:06 +02:00
|
|
|
assert(data_dirs_.empty());
|
|
|
|
for (auto& p : ioptions_.cf_paths) {
|
2020-02-03 23:11:50 +01:00
|
|
|
auto existing_dir = created_dirs->find(p.path);
|
|
|
|
|
|
|
|
if (existing_dir == created_dirs->end()) {
|
2020-03-03 01:14:00 +01:00
|
|
|
std::unique_ptr<FSDirectory> path_directory;
|
|
|
|
s = DBImpl::CreateAndNewDirectory(ioptions_.fs, p.path, &path_directory);
|
2020-02-03 23:11:50 +01:00
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
assert(path_directory != nullptr);
|
|
|
|
data_dirs_.emplace_back(path_directory.release());
|
|
|
|
(*created_dirs)[p.path] = data_dirs_.back();
|
|
|
|
} else {
|
|
|
|
data_dirs_.emplace_back(existing_dir->second);
|
2018-04-06 04:49:06 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
assert(data_dirs_.size() == ioptions_.cf_paths.size());
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2020-03-03 01:14:00 +01:00
|
|
|
FSDirectory* ColumnFamilyData::GetDataDir(size_t path_id) const {
|
2018-04-06 04:49:06 +02:00
|
|
|
if (data_dirs_.empty()) {
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
assert(path_id < data_dirs_.size());
|
|
|
|
return data_dirs_[path_id].get();
|
|
|
|
}
|
|
|
|
|
[CF] Rethink table cache
Summary:
Adapting table cache to column families is interesting. We want table cache to be global LRU, so if some column families are use not as often as others, we want them to be evicted from cache. However, current TableCache object also constructs tables on its own. If table is not found in the cache, TableCache automatically creates new table. We want each column family to be able to specify different table factory.
To solve the problem, we still have a single LRU, but we provide the LRUCache object to TableCache on construction. We have one TableCache per column family, but the underyling cache is shared by all TableCache objects.
This allows us to have a global LRU, but still be able to support different table factories for different column families. Also, in the future it will also be able to support different directories for different column families.
Test Plan: make check
Reviewers: dhruba, haobo, kailiu, sdong
CC: leveldb
Differential Revision: https://reviews.facebook.net/D15915
2014-02-05 18:07:55 +01:00
|
|
|
ColumnFamilySet::ColumnFamilySet(const std::string& dbname,
|
2016-09-24 01:34:04 +02:00
|
|
|
const ImmutableDBOptions* db_options,
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
|
|
|
const FileOptions& file_options,
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
Cache* table_cache,
|
2020-03-21 03:17:54 +01:00
|
|
|
WriteBufferManager* _write_buffer_manager,
|
|
|
|
WriteController* _write_controller,
|
2019-06-14 00:39:52 +02:00
|
|
|
BlockCacheTracer* const block_cache_tracer)
|
2014-01-31 01:49:46 +01:00
|
|
|
: max_column_family_(0),
|
2019-06-14 00:39:52 +02:00
|
|
|
dummy_cfd_(new ColumnFamilyData(
|
2020-03-12 02:36:43 +01:00
|
|
|
ColumnFamilyData::kDummyColumnFamilyDataId, "", nullptr, nullptr,
|
|
|
|
nullptr, ColumnFamilyOptions(), *db_options, file_options, nullptr,
|
|
|
|
block_cache_tracer)),
|
2014-03-11 22:52:17 +01:00
|
|
|
default_cfd_cache_(nullptr),
|
[CF] Rethink table cache
Summary:
Adapting table cache to column families is interesting. We want table cache to be global LRU, so if some column families are use not as often as others, we want them to be evicted from cache. However, current TableCache object also constructs tables on its own. If table is not found in the cache, TableCache automatically creates new table. We want each column family to be able to specify different table factory.
To solve the problem, we still have a single LRU, but we provide the LRUCache object to TableCache on construction. We have one TableCache per column family, but the underyling cache is shared by all TableCache objects.
This allows us to have a global LRU, but still be able to support different table factories for different column families. Also, in the future it will also be able to support different directories for different column families.
Test Plan: make check
Reviewers: dhruba, haobo, kailiu, sdong
CC: leveldb
Differential Revision: https://reviews.facebook.net/D15915
2014-02-05 18:07:55 +01:00
|
|
|
db_name_(dbname),
|
|
|
|
db_options_(db_options),
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
|
|
|
file_options_(file_options),
|
2014-02-06 20:44:50 +01:00
|
|
|
table_cache_(table_cache),
|
2020-03-21 03:17:54 +01:00
|
|
|
write_buffer_manager_(_write_buffer_manager),
|
|
|
|
write_controller_(_write_controller),
|
2019-06-14 00:39:52 +02:00
|
|
|
block_cache_tracer_(block_cache_tracer) {
|
2014-01-31 01:49:46 +01:00
|
|
|
// initialize linked list
|
2014-02-11 02:04:44 +01:00
|
|
|
dummy_cfd_->prev_ = dummy_cfd_;
|
|
|
|
dummy_cfd_->next_ = dummy_cfd_;
|
2014-01-31 01:49:46 +01:00
|
|
|
}
|
2014-01-22 20:44:53 +01:00
|
|
|
|
|
|
|
ColumnFamilySet::~ColumnFamilySet() {
|
2014-02-11 02:04:44 +01:00
|
|
|
while (column_family_data_.size() > 0) {
|
|
|
|
// cfd destructor will delete itself from column_family_data_
|
|
|
|
auto cfd = column_family_data_.begin()->second;
|
2018-01-31 20:50:59 +01:00
|
|
|
bool last_ref __attribute__((__unused__));
|
2019-12-13 04:02:51 +01:00
|
|
|
last_ref = cfd->UnrefAndTryDelete();
|
2018-01-24 00:08:29 +01:00
|
|
|
assert(last_ref);
|
2014-01-22 20:44:53 +01:00
|
|
|
}
|
2018-01-31 20:50:59 +01:00
|
|
|
bool dummy_last_ref __attribute__((__unused__));
|
2019-12-13 04:02:51 +01:00
|
|
|
dummy_last_ref = dummy_cfd_->UnrefAndTryDelete();
|
2018-01-24 00:08:29 +01:00
|
|
|
assert(dummy_last_ref);
|
2014-01-22 20:44:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyData* ColumnFamilySet::GetDefault() const {
|
2014-03-11 22:52:17 +01:00
|
|
|
assert(default_cfd_cache_ != nullptr);
|
|
|
|
return default_cfd_cache_;
|
2014-01-22 20:44:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyData* ColumnFamilySet::GetColumnFamily(uint32_t id) const {
|
|
|
|
auto cfd_iter = column_family_data_.find(id);
|
|
|
|
if (cfd_iter != column_family_data_.end()) {
|
|
|
|
return cfd_iter->second;
|
|
|
|
} else {
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-02-28 23:05:11 +01:00
|
|
|
ColumnFamilyData* ColumnFamilySet::GetColumnFamily(const std::string& name)
|
|
|
|
const {
|
|
|
|
auto cfd_iter = column_families_.find(name);
|
2014-03-11 22:52:17 +01:00
|
|
|
if (cfd_iter != column_families_.end()) {
|
|
|
|
auto cfd = GetColumnFamily(cfd_iter->second);
|
|
|
|
assert(cfd != nullptr);
|
|
|
|
return cfd;
|
|
|
|
} else {
|
2014-02-28 23:05:11 +01:00
|
|
|
return nullptr;
|
|
|
|
}
|
2014-01-22 20:44:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
uint32_t ColumnFamilySet::GetNextColumnFamilyID() {
|
|
|
|
return ++max_column_family_;
|
|
|
|
}
|
|
|
|
|
2014-03-05 21:13:44 +01:00
|
|
|
uint32_t ColumnFamilySet::GetMaxColumnFamily() { return max_column_family_; }
|
|
|
|
|
|
|
|
void ColumnFamilySet::UpdateMaxColumnFamily(uint32_t new_max_column_family) {
|
|
|
|
max_column_family_ = std::max(new_max_column_family, max_column_family_);
|
|
|
|
}
|
|
|
|
|
2014-06-03 00:33:54 +02:00
|
|
|
size_t ColumnFamilySet::NumberOfColumnFamilies() const {
|
|
|
|
return column_families_.size();
|
|
|
|
}
|
|
|
|
|
2015-01-06 21:44:21 +01:00
|
|
|
// under a DB mutex AND write thread
|
2014-01-22 20:44:53 +01:00
|
|
|
ColumnFamilyData* ColumnFamilySet::CreateColumnFamily(
|
|
|
|
const std::string& name, uint32_t id, Version* dummy_versions,
|
|
|
|
const ColumnFamilyOptions& options) {
|
|
|
|
assert(column_families_.find(name) == column_families_.end());
|
2016-06-21 03:01:03 +02:00
|
|
|
ColumnFamilyData* new_cfd = new ColumnFamilyData(
|
|
|
|
id, name, dummy_versions, table_cache_, write_buffer_manager_, options,
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
|
|
|
*db_options_, file_options_, this, block_cache_tracer_);
|
2014-02-11 02:04:44 +01:00
|
|
|
column_families_.insert({name, id});
|
2014-01-22 20:44:53 +01:00
|
|
|
column_family_data_.insert({id, new_cfd});
|
|
|
|
max_column_family_ = std::max(max_column_family_, id);
|
2014-01-31 01:49:46 +01:00
|
|
|
// add to linked list
|
2014-02-11 02:04:44 +01:00
|
|
|
new_cfd->next_ = dummy_cfd_;
|
|
|
|
auto prev = dummy_cfd_->prev_;
|
|
|
|
new_cfd->prev_ = prev;
|
|
|
|
prev->next_ = new_cfd;
|
|
|
|
dummy_cfd_->prev_ = new_cfd;
|
2014-03-11 22:52:17 +01:00
|
|
|
if (id == 0) {
|
|
|
|
default_cfd_cache_ = new_cfd;
|
|
|
|
}
|
2014-01-22 20:44:53 +01:00
|
|
|
return new_cfd;
|
|
|
|
}
|
|
|
|
|
2014-04-07 23:21:25 +02:00
|
|
|
// REQUIRES: DB mutex held
|
|
|
|
void ColumnFamilySet::FreeDeadColumnFamilies() {
|
|
|
|
autovector<ColumnFamilyData*> to_delete;
|
|
|
|
for (auto cfd = dummy_cfd_->next_; cfd != dummy_cfd_; cfd = cfd->next_) {
|
2015-01-26 20:48:07 +01:00
|
|
|
if (cfd->refs_.load(std::memory_order_relaxed) == 0) {
|
2014-04-07 23:21:25 +02:00
|
|
|
to_delete.push_back(cfd);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
for (auto cfd : to_delete) {
|
|
|
|
// this is very rare, so it's not a problem that we do it under a mutex
|
|
|
|
delete cfd;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-01-06 21:44:21 +01:00
|
|
|
// under a DB mutex AND from a write thread
|
2014-03-11 22:52:17 +01:00
|
|
|
void ColumnFamilySet::RemoveColumnFamily(ColumnFamilyData* cfd) {
|
2014-02-11 02:04:44 +01:00
|
|
|
auto cfd_iter = column_family_data_.find(cfd->GetID());
|
2014-01-31 01:49:46 +01:00
|
|
|
assert(cfd_iter != column_family_data_.end());
|
|
|
|
column_family_data_.erase(cfd_iter);
|
2014-02-11 02:04:44 +01:00
|
|
|
column_families_.erase(cfd->GetName());
|
2014-01-22 20:44:53 +01:00
|
|
|
}
|
|
|
|
|
2015-01-06 21:44:21 +01:00
|
|
|
// under a DB mutex OR from a write thread
|
2014-02-06 01:02:48 +01:00
|
|
|
bool ColumnFamilyMemTablesImpl::Seek(uint32_t column_family_id) {
|
2014-03-11 22:52:17 +01:00
|
|
|
if (column_family_id == 0) {
|
|
|
|
// optimization for common case
|
|
|
|
current_ = column_family_set_->GetDefault();
|
|
|
|
} else {
|
|
|
|
current_ = column_family_set_->GetColumnFamily(column_family_id);
|
|
|
|
}
|
2014-02-11 02:04:44 +01:00
|
|
|
handle_.SetCFD(current_);
|
2014-02-06 01:02:48 +01:00
|
|
|
return current_ != nullptr;
|
|
|
|
}
|
2014-01-28 20:05:04 +01:00
|
|
|
|
2014-02-06 01:02:48 +01:00
|
|
|
uint64_t ColumnFamilyMemTablesImpl::GetLogNumber() const {
|
|
|
|
assert(current_ != nullptr);
|
|
|
|
return current_->GetLogNumber();
|
|
|
|
}
|
|
|
|
|
|
|
|
MemTable* ColumnFamilyMemTablesImpl::GetMemTable() const {
|
|
|
|
assert(current_ != nullptr);
|
|
|
|
return current_->mem();
|
|
|
|
}
|
|
|
|
|
2014-02-11 02:04:44 +01:00
|
|
|
ColumnFamilyHandle* ColumnFamilyMemTablesImpl::GetColumnFamilyHandle() {
|
2014-02-06 01:02:48 +01:00
|
|
|
assert(current_ != nullptr);
|
2014-02-11 02:04:44 +01:00
|
|
|
return &handle_;
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
|
|
|
|
2014-08-19 00:19:17 +02:00
|
|
|
uint32_t GetColumnFamilyID(ColumnFamilyHandle* column_family) {
|
|
|
|
uint32_t column_family_id = 0;
|
|
|
|
if (column_family != nullptr) {
|
2020-07-03 04:24:25 +02:00
|
|
|
auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(column_family);
|
2014-08-19 00:19:17 +02:00
|
|
|
column_family_id = cfh->GetID();
|
|
|
|
}
|
|
|
|
return column_family_id;
|
|
|
|
}
|
|
|
|
|
2014-09-22 20:37:35 +02:00
|
|
|
const Comparator* GetColumnFamilyUserComparator(
|
|
|
|
ColumnFamilyHandle* column_family) {
|
|
|
|
if (column_family != nullptr) {
|
2016-07-26 00:06:11 +02:00
|
|
|
return column_family->GetComparator();
|
2014-09-22 20:37:35 +02:00
|
|
|
}
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|