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

846 lines
34 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#pragma once
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
2 years ago
#include <atomic>
#include <string>
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
2 years ago
#include <unordered_map>
#include <vector>
Account memory of FileMetaData in global memory limit (#9924) Summary: **Context/Summary:** As revealed by heap profiling, allocation of `FileMetaData` for [newly created file added to a Version](https://github.com/facebook/rocksdb/pull/9924/files#diff-a6aa385940793f95a2c5b39cc670bd440c4547fa54fd44622f756382d5e47e43R774) can consume significant heap memory. This PR is to account that toward our global memory limit based on block cache capacity. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9924 Test Plan: - Previous `make check` verified there are only 2 places where the memory of the allocated `FileMetaData` can be released - New unit test `TEST_P(ChargeFileMetadataTestWithParam, Basic)` - db bench (CPU cost of `charge_file_metadata` in write and compact) - **write micros/op: -0.24%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 (remove this option for pre-PR) -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` - **compact micros/op -0.87%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 -numdistinct=1000 && ./db_bench -benchmarks=compact -db=$TEST_TMPDIR -use_existing_db=1 -charge_file_metadata=1 -disable_auto_compactions=1 | egrep 'compact'` table 1 - write #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | -0.3633711465 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | 0.5289363078 80 | 3.87828 | 0.119007 | 3.86791 | 0.115674 | **-0.2673865734** 160 | 3.87677 | 0.162231 | 3.86739 | 0.16663 | **-0.2419539978** table 2 - compact #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 2,399,650.00 | 96,375.80 | 2,359,537.00 | 53,243.60 | -1.67 20 | 2,410,480.00 | 89,988.00 | 2,433,580.00 | 91,121.20 | 0.96 40 | 2.41E+06 | 121811 | 2.39E+06 | 131525 | **-0.96** 80 | 2.40E+06 | 134503 | 2.39E+06 | 108799 | **-0.78** - stress test: `python3 tools/db_crashtest.py blackbox --charge_file_metadata=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36055583 Pulled By: hx235 fbshipit-source-id: b60eab94707103cb1322cf815f05810ef0232625
2 years ago
#include "cache/cache_reservation_manager.h"
#include "db/memtable_list.h"
#include "db/table_cache.h"
#include "db/table_properties_collector.h"
#include "db/write_batch_internal.h"
#include "db/write_controller.h"
#include "options/cf_options.h"
#include "rocksdb/compaction_job_stats.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/options.h"
#include "trace_replay/block_cache_tracer.h"
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
2 years ago
#include "util/hash_containers.h"
#include "util/thread_local.h"
namespace ROCKSDB_NAMESPACE {
class Version;
class VersionSet;
class VersionStorageInfo;
class MemTable;
class MemTableListVersion;
class CompactionPicker;
class Compaction;
class InternalKey;
class InternalStats;
class ColumnFamilyData;
class DBImpl;
class LogBuffer;
class InstrumentedMutex;
class InstrumentedMutexLock;
struct SuperVersionContext;
class BlobFileCache;
class BlobSource;
extern const double kIncSlowdownRatio;
// This file contains a list of data structures for managing column family
// level metadata.
//
// The basic relationships among classes declared here are illustrated as
// following:
//
// +----------------------+ +----------------------+ +--------+
// +---+ ColumnFamilyHandle 1 | +--+ ColumnFamilyHandle 2 | | DBImpl |
// | +----------------------+ | +----------------------+ +----+---+
// | +--------------------------+ |
// | | +-----------------------------+
// | | |
// | | +-----------------------------v-------------------------------+
// | | | |
// | | | ColumnFamilySet |
// | | | |
// | | +-------------+--------------------------+----------------+---+
// | | | | |
// | +-------------------------------------+ | |
// | | | | v
// | +-------------v-------------+ +-----v----v---------+
// | | | | |
// | | ColumnFamilyData 1 | | ColumnFamilyData 2 | ......
// | | | | |
// +---> | | |
// | +---------+ | |
// | | MemTable| | |
// | | List | | |
// +--------+---+--+-+----+----+ +--------------------++
// | | | |
// | | | |
// | | | +-----------------------+
// | | +-----------+ |
// v +--------+ | |
// +--------+--------+ | | |
// | | | | +----------v----------+
// +---> |SuperVersion 1.a +-----------------> |
// | +------+ | | MemTableListVersion |
// +---+-------------+ | | | | |
// | | | | +----+------------+---+
// | current | | | | |
// | +-------------+ | |mem | |
// | | | | | |
// +-v---v-------+ +---v--v---+ +-----v----+ +----v-----+
// | | | | | | | |
// | Version 1.a | | memtable | | memtable | | memtable |
// | | | 1.a | | 1.b | | 1.c |
// +-------------+ | | | | | |
// +----------+ +----------+ +----------+
//
// DBImpl keeps a ColumnFamilySet, which references to all column families by
// pointing to respective ColumnFamilyData object of each column family.
// This is how DBImpl can list and operate on all the column families.
// ColumnFamilyHandle also points to ColumnFamilyData directly, so that
// when a user executes a query, it can directly find memtables and Version
// as well as SuperVersion to the column family, without going through
// ColumnFamilySet.
//
// ColumnFamilySet points to the latest view of the LSM-tree (list of memtables
// and SST files) indirectly, while ongoing operations may hold references
// to a current or an out-of-date SuperVersion, which in turn points to a
// point-in-time view of the LSM-tree. This guarantees the memtables and SST
// files being operated on will not go away, until the SuperVersion is
// unreferenced to 0 and destoryed.
//
// The following graph illustrates a possible referencing relationships:
//
// Column +--------------+ current +-----------+
// Family +---->+ +------------------->+ |
// Data | SuperVersion +----------+ | Version A |
// | 3 | imm | | |
// Iter2 +----->+ | +-------v------+ +-----------+
// +-----+--------+ | MemtableList +----------------> Empty
// | | Version r | +-----------+
// | +--------------+ | |
// +------------------+ current| Version B |
// +--------------+ | +----->+ |
// | | | | +-----+-----+
// Compaction +>+ SuperVersion +-------------+ ^
// Job | 2 +------+ | |current
// | +----+ | | mem | +------------+
// +--------------+ | | +---------------------> |
// | +------------------------> MemTable a |
// | mem | | |
// +--------------+ | | +------------+
// | +--------------------------+
// Iter1 +-----> SuperVersion | | +------------+
// | 1 +------------------------------>+ |
// | +-+ | mem | MemTable b |
// +--------------+ | | | |
// | | +--------------+ +-----^------+
// | |imm | MemtableList | |
// | +--->+ Version s +------------+
// | +--------------+
// | +--------------+
// | | MemtableList |
// +------>+ Version t +--------> Empty
// imm +--------------+
//
// In this example, even if the current LSM-tree consists of Version A and
// memtable a, which is also referenced by SuperVersion, two older SuperVersion
// SuperVersion2 and Superversion1 still exist, and are referenced by a
// compaction job and an old iterator Iter1, respectively. SuperVersion2
// contains Version B, memtable a and memtable b; SuperVersion1 contains
// Version B and memtable b (mutable). As a result, Version B and memtable b
// are prevented from being destroyed or deleted.
// ColumnFamilyHandleImpl is the class that clients use to access different
// column families. It has non-trivial destructor, which gets called when client
// is done using the column family
class ColumnFamilyHandleImpl : public ColumnFamilyHandle {
public:
// create while holding the mutex
ColumnFamilyHandleImpl(
ColumnFamilyData* cfd, DBImpl* db, InstrumentedMutex* mutex);
// destroy without mutex
virtual ~ColumnFamilyHandleImpl();
virtual ColumnFamilyData* cfd() const { return cfd_; }
virtual uint32_t GetID() const override;
virtual const std::string& GetName() const override;
virtual Status GetDescriptor(ColumnFamilyDescriptor* desc) override;
virtual const Comparator* GetComparator() const override;
private:
ColumnFamilyData* cfd_;
DBImpl* db_;
InstrumentedMutex* mutex_;
};
// Does not ref-count ColumnFamilyData
// We use this dummy ColumnFamilyHandleImpl because sometimes MemTableInserter
// calls DBImpl methods. When this happens, MemTableInserter need access to
// ColumnFamilyHandle (same as the client would need). In that case, we feed
// MemTableInserter dummy ColumnFamilyHandle and enable it to call DBImpl
// methods
class ColumnFamilyHandleInternal : public ColumnFamilyHandleImpl {
public:
ColumnFamilyHandleInternal()
Fix coverity issues column_family, compaction_db/iterator Summary: db/column_family.h : 79 ColumnFamilyHandleInternal() CID 1322806 (#1 of 1): Uninitialized pointer field (UNINIT_CTOR) 2. uninit_member: Non-static class member internal_cfd_ is not initialized in this constructor nor in any functions that it calls. 80 : ColumnFamilyHandleImpl(nullptr, nullptr, nullptr) {} db/compacted_db_impl.cc: 18CompactedDBImpl::CompactedDBImpl( 19 const DBOptions& options, const std::string& dbname) 20 : DBImpl(options, dbname) { 2. uninit_member: Non-static class member cfd_ is not initialized in this constructor nor in any functions that it calls. 4. uninit_member: Non-static class member version_ is not initialized in this constructor nor in any functions that it calls. CID 1396120 (#1 of 1): Uninitialized pointer field (UNINIT_CTOR) 6. uninit_member: Non-static class member user_comparator_ is not initialized in this constructor nor in any functions that it calls. 21} db/compaction_iterator.cc: 9. uninit_member: Non-static class member current_user_key_sequence_ is not initialized in this constructor nor in any functions that it calls. 11. uninit_member: Non-static class member current_user_key_snapshot_ is not initialized in this constructor nor in any functions that it calls. CID 1419855 (#1 of 1): Uninitialized scalar field (UNINIT_CTOR) 13. uninit_member: Non-static class member current_key_committed_ is not initialized in this constructor nor in any functions that it calls. Closes https://github.com/facebook/rocksdb/pull/3084 Differential Revision: D6172999 Pulled By: sagar0 fbshipit-source-id: 084d73393faf8022c01359cfb445807b6a782460
7 years ago
: ColumnFamilyHandleImpl(nullptr, nullptr, nullptr), internal_cfd_(nullptr) {}
void SetCFD(ColumnFamilyData* _cfd) { internal_cfd_ = _cfd; }
virtual ColumnFamilyData* cfd() const override { return internal_cfd_; }
private:
ColumnFamilyData* internal_cfd_;
};
// holds references to memtable, all immutable memtables and version
struct SuperVersion {
// Accessing members of this class is not thread-safe and requires external
// synchronization (ie db mutex held or on write thread).
ColumnFamilyData* cfd;
MemTable* mem;
MemTableListVersion* imm;
Version* current;
MutableCFOptions mutable_cf_options;
// Version number of the current SuperVersion
uint64_t version_number;
WriteStallCondition write_stall_condition;
// should be called outside the mutex
SuperVersion() = default;
~SuperVersion();
SuperVersion* Ref();
// If Unref() returns true, Cleanup() should be called with mutex held
// before deleting this SuperVersion.
bool Unref();
// call these two methods with db mutex held
// Cleanup unrefs mem, imm and current. Also, it stores all memtables
// that needs to be deleted in to_delete vector. Unrefing those
// objects needs to be done in the mutex
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
3 years ago
void Cleanup();
void Init(ColumnFamilyData* new_cfd, MemTable* new_mem,
MemTableListVersion* new_imm, Version* new_current);
// The value of dummy is not actually used. kSVInUse takes its address as a
// mark in the thread local storage to indicate the SuperVersion is in use
// by thread. This way, the value of kSVInUse is guaranteed to have no
// conflict with SuperVersion object address and portable on different
// platform.
static int dummy;
static void* const kSVInUse;
static void* const kSVObsolete;
private:
std::atomic<uint32_t> refs;
// We need to_delete because during Cleanup(), imm->Unref() returns
// all memtables that we need to free through this vector. We then
// delete all those memtables outside of mutex, during destruction
autovector<MemTable*> to_delete;
};
extern Status CheckCompressionSupported(const ColumnFamilyOptions& cf_options);
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
extern Status CheckConcurrentWritesSupported(
const ColumnFamilyOptions& cf_options);
extern Status CheckCFPathsSupported(const DBOptions& db_options,
const ColumnFamilyOptions& cf_options);
extern ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
const ColumnFamilyOptions& src);
// Wrap user defined table properties collector factories `from cf_options`
// into internal ones in int_tbl_prop_collector_factories. Add a system internal
// one too.
extern void GetIntTblPropCollectorFactory(
const ImmutableCFOptions& ioptions,
IntTblPropCollectorFactories* int_tbl_prop_collector_factories);
class ColumnFamilySet;
// This class keeps all the data that a column family needs.
// Most methods require DB mutex held, unless otherwise noted
class ColumnFamilyData {
public:
~ColumnFamilyData();
// thread-safe
uint32_t GetID() const { return id_; }
// thread-safe
const std::string& GetName() const { return name_; }
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// Ref() can only be called from a context where the caller can guarantee
// that ColumnFamilyData is alive (while holding a non-zero ref already,
// holding a DB mutex, or as the leader in a write batch group).
Bump up memory order of ref counting of ColumnFamilyData (#5723) Summary: We see this TSAN warning: WARNING: ThreadSanitizer: data race (pid=282806) Write of size 8 at 0x7b6c00000e38 by thread T16 (mutexes: write M1023578822185846136): #0 operator delete(void*) <null> (libtsan.so.0+0x0000000795f8) https://github.com/facebook/rocksdb/issues/1 rocksdb::DBImpl::BackgroundFlush(bool*, rocksdb::JobContext*, rocksdb::LogBuffer*, rocksdb::FlushReason*, rocksdb::Env::Priority) db/db_impl/db_impl_compaction_flush.cc:2202 (db_flush_test+0x00000060b462) https://github.com/facebook/rocksdb/issues/2 rocksdb::DBImpl::BackgroundCallFlush(rocksdb::Env::Priority) db/db_impl/db_impl_compaction_flush.cc:2226 (db_flush_test+0x00000060cbd8) https://github.com/facebook/rocksdb/issues/3 rocksdb::DBImpl::BGWorkFlush(void*) db/db_impl/db_impl_compaction_flush.cc:2073 (db_flush_test+0x00000060d5ac) ...... Previous atomic write of size 4 at 0x7b6c00000e38 by main thread: #0 __tsan_atomic32_fetch_sub <null> (libtsan.so.0+0x00000006d721) https://github.com/facebook/rocksdb/issues/1 std::__atomic_base<int>::fetch_sub(int, std::memory_order) /mnt/gvfs/third-party2/libgcc/c67031f0f739ac61575a061518d6ef5038f99f90/7.x/platform007/5620abc/include/c++/7.3.0/bits/atomic_base.h:524 (db_flush_test+0x0000005f9e38) https://github.com/facebook/rocksdb/issues/2 rocksdb::ColumnFamilyData::Unref() db/column_family.h:286 (db_flush_test+0x0000005f9e38) https://github.com/facebook/rocksdb/issues/3 rocksdb::DBImpl::FlushMemTable(rocksdb::ColumnFamilyData*, rocksdb::FlushOptions const&, rocksdb::FlushReason, bool) db/db_impl/db_impl_compaction_flush.cc:1624 (db_flush_test+0x0000005f9e38) https://github.com/facebook/rocksdb/issues/4 rocksdb::DBImpl::TEST_FlushMemTable(rocksdb::ColumnFamilyData*, rocksdb::FlushOptions const&) db/db_impl/db_impl_debug.cc:127 (db_flush_test+0x00000061ace9) https://github.com/facebook/rocksdb/issues/5 rocksdb::DBFlushTest_CFDropRaceWithWaitForFlushMemTables_Test::TestBody() db/db_flush_test.cc:320 (db_flush_test+0x0000004b44e5) https://github.com/facebook/rocksdb/issues/6 void testing::internal::HandleSehExceptionsInMethodIfSupported<testing::Test, void>(testing::Test*, void (testing::Test::*)(), char const*) third-party/gtest-1.7.0/fused-src/gtest/gtest-all.cc:3824 (db_flush_test+0x000000be2988) ...... It's still very clear the cause of the warning is because that TSAN treats results from relaxed atomic::fetch_sub() as non-atomic with the operation itself. We can make it more explicit by bumping up the order to CS. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5723 Test Plan: Run all existing test. Differential Revision: D16908250 fbshipit-source-id: bf17d39ed19058372bdf97f6440a743f88153021
5 years ago
void Ref() { refs_.fetch_add(1); }
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// UnrefAndTryDelete() decreases the reference count and do free if needed,
// return true if this is freed else false, UnrefAndTryDelete() can only
// be called while holding a DB mutex, or during single-threaded recovery.
Fix a race in ColumnFamilyData::UnrefAndTryDelete (#8605) Summary: The `ColumnFamilyData::UnrefAndTryDelete` code currently on the trunk unlocks the DB mutex before destroying the `ThreadLocalPtr` holding the per-thread `SuperVersion` pointers when the only remaining reference is the back reference from `super_version_`. The idea behind this was to break the circular dependency between `ColumnFamilyData` and `SuperVersion`: when the penultimate reference goes away, `ColumnFamilyData` can clean up the `SuperVersion`, which can in turn clean up `ColumnFamilyData`. (Assuming there is a `SuperVersion` and it is not referenced by anything else.) However, unlocking the mutex throws a wrench in this plan by making it possible for another thread to jump in and take another reference to the `ColumnFamilyData`, keeping the object alive in a zombie `ThreadLocalPtr`-less state. This can cause issues like https://github.com/facebook/rocksdb/issues/8440 , https://github.com/facebook/rocksdb/issues/8382 , and might also explain the `was_last_ref` assertion failures from the `ColumnFamilySet` destructor we sometimes observe during close in our stress tests. Digging through the archives, this unlocking goes way back to 2014 (or earlier). The original rationale was that `SuperVersionUnrefHandle` used to lock the mutex so it can call `SuperVersion::Cleanup`; however, this logic turned out to be deadlock-prone. https://github.com/facebook/rocksdb/pull/3510 fixed the deadlock but left the unlocking in place. https://github.com/facebook/rocksdb/pull/6147 then introduced the circular dependency and associated cleanup logic described above (in order to enable iterators to keep the `ColumnFamilyData` for dropped column families alive), and moved the unlocking-relocking snippet to its present location in `UnrefAndTryDelete`. Finally, https://github.com/facebook/rocksdb/pull/7749 fixed a memory leak but apparently exacerbated the race by (otherwise correctly) switching to `UnrefAndTryDelete` in `SuperVersion::Cleanup`. The patch simply eliminates the unlocking and relocking, which has been unnecessary ever since https://github.com/facebook/rocksdb/issues/3510 made `SuperVersionUnrefHandle` lock-free. This closes the window during which another thread could increase the reference count, and hopefully fixes the issues above. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8605 Test Plan: Ran `make check` and stress tests locally. Reviewed By: pdillinger Differential Revision: D30051035 Pulled By: ltamasi fbshipit-source-id: 8fe559e4b4ad69fc142579f8bc393ef525918528
3 years ago
bool UnrefAndTryDelete();
// SetDropped() can only be called under following conditions:
// 1) Holding a DB mutex,
// 2) from single-threaded write thread, AND
// 3) from single-threaded VersionSet::LogAndApply()
// After dropping column family no other operation on that column family
// will be executed. All the files and memory will be, however, kept around
// until client drops the column family handle. That way, client can still
// access data from dropped column family.
// Column family can be dropped and still alive. In that state:
// *) Compaction and flush is not executed on the dropped column family.
// *) Client can continue reading from column family. Writes will fail unless
// WriteOptions::ignore_missing_column_families is true
// When the dropped column family is unreferenced, then we:
// *) Remove column family from the linked list maintained by ColumnFamilySet
// *) delete all memory associated with that column family
// *) delete all the files associated with that column family
void SetDropped();
bool IsDropped() const { return dropped_.load(std::memory_order_relaxed); }
// thread-safe
int NumberLevels() const { return ioptions_.num_levels; }
void SetLogNumber(uint64_t log_number) { log_number_ = log_number; }
uint64_t GetLogNumber() const { return log_number_; }
void SetFlushReason(FlushReason flush_reason) {
flush_reason_ = flush_reason;
}
FlushReason GetFlushReason() const { return flush_reason_; }
// thread-safe
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
5 years ago
const FileOptions* soptions() const;
const ImmutableOptions* ioptions() const { return &ioptions_; }
// REQUIRES: DB mutex held
// This returns the MutableCFOptions used by current SuperVersion
// You should use this API to reference MutableCFOptions most of the time.
const MutableCFOptions* GetCurrentMutableCFOptions() const {
return &(super_version_->mutable_cf_options);
}
// REQUIRES: DB mutex held
// This returns the latest MutableCFOptions, which may be not in effect yet.
const MutableCFOptions* GetLatestMutableCFOptions() const {
return &mutable_cf_options_;
}
// REQUIRES: DB mutex held
// Build ColumnFamiliesOptions with immutable options and latest mutable
// options.
ColumnFamilyOptions GetLatestCFOptions() const;
bool is_delete_range_supported() { return is_delete_range_supported_; }
// Validate CF options against DB options
static Status ValidateOptions(const DBOptions& db_options,
const ColumnFamilyOptions& cf_options);
#ifndef ROCKSDB_LITE
// REQUIRES: DB mutex held
Status SetOptions(
const DBOptions& db_options,
const std::unordered_map<std::string, std::string>& options_map);
#endif // ROCKSDB_LITE
InternalStats* internal_stats() { return internal_stats_.get(); }
MemTableList* imm() { return &imm_; }
MemTable* mem() { return mem_; }
Track WAL obsoletion when updating empty CF's log number (#7781) Summary: In the write path, there is an optimization: when a new WAL is created during SwitchMemtable, we update the internal log number of the empty column families to the new WAL. `FindObsoleteFiles` marks a WAL as obsolete if the WAL's log number is less than `VersionSet::MinLogNumberWithUnflushedData`. After updating the empty column families' internal log number, `VersionSet::MinLogNumberWithUnflushedData` might change, so some WALs might become obsolete to be purged from disk. For example, consider there are 3 column families: 0, 1, 2: 1. initially, all the column families' log number is 1; 2. write some data to cf0, and flush cf0, but the flush is pending; 3. now a new WAL 2 is created; 4. write data to cf1 and WAL 2, now cf0's log number is 1, cf1's log number is 2, cf2's log number is 2 (because cf1 and cf2 are empty, so their log numbers will be set to the highest log number); 5. now cf0's flush hasn't finished, flush cf1, a new WAL 3 is created, and cf1's flush finishes, now cf0's log number is 1, cf1's log number is 3, cf2's log number is 3, since WAL 1 still contains data for the unflushed cf0, no WAL can be deleted from disk; 6. now cf0's flush finishes, cf0's log number is 2 (because when cf0 was switching memtable, WAL 3 does not exist yet), cf1's log number is 3, cf2's log number is 3, so WAL 1 can be purged from disk now, but WAL 2 still cannot because `MinLogNumberToKeep()` is 2; 7. write data to cf2 and WAL 3, because cf0 is empty, its log number is updated to 3, so now cf0's log number is 3, cf1's log number is 3, cf2's log number is 3; 8. now if the background threads want to purge obsolete files from disk, WAL 2 can be purged because `MinLogNumberToKeep()` is 3. But there are only two flush results written to MANIFEST: the first is for flushing cf1, and the `MinLogNumberToKeep` is 1, the second is for flushing cf0, and the `MinLogNumberToKeep` is 2. So without this PR, if the DB crashes at this point and try to recover, `WalSet` will still expect WAL 2 to exist. When WAL tracking is enabled, we assume WALs will only become obsolete after a flush result is written to MANIFEST in `MemtableList::TryInstallMemtableFlushResults` (or its atomic flush counterpart). The above situation breaks this assumption. This PR tracks WAL obsoletion if necessary before updating the empty column families' log numbers. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7781 Test Plan: watch existing tests and stress tests to pass. `make -j48 blackbox_crash_test` on devserver Reviewed By: ltamasi Differential Revision: D25631695 Pulled By: cheng-chang fbshipit-source-id: ca7fff967bdb42204b84226063d909893bc0a4ec
4 years ago
bool IsEmpty() {
return mem()->GetFirstSequenceNumber() == 0 && imm()->NumNotFlushed() == 0;
}
Version* current() { return current_; }
Version* dummy_versions() { return dummy_versions_; }
void SetCurrent(Version* _current);
uint64_t GetNumLiveVersions() const; // REQUIRE: DB mutex held
uint64_t GetTotalSstFilesSize() const; // REQUIRE: DB mutex held
uint64_t GetLiveSstFilesSize() const; // REQUIRE: DB mutex held
Add DB properties for BlobDB (#8734) Summary: RocksDB exposes certain internal statistics via the DB property interface. However, there are currently no properties related to BlobDB. For starters, we would like to add the following BlobDB properties: `rocksdb.num-blob-files`: number of blob files in the current Version (kind of like `num-files-at-level` but note this is not per level, since blob files are not part of the LSM tree). `rocksdb.blob-stats`: this could return the total number and size of all blob files, and potentially also the total amount of garbage (in bytes) in the blob files in the current Version. `rocksdb.total-blob-file-size`: the total size of all blob files (as a blob counterpart for `total-sst-file-size`) of all Versions. `rocksdb.live-blob-file-size`: the total size of all blob files in the current Version. `rocksdb.estimate-live-data-size`: this is actually an existing property that we can extend so it considers blob files as well. When it comes to blobs, we actually have an exact value for live bytes. Namely, live bytes can be computed simply as total bytes minus garbage bytes, summed over the entire set of blob files in the Version. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8734 Test Plan: ``` ➜ rocksdb git:(new_feature_blobDB_properties) ./db_blob_basic_test [==========] Running 16 tests from 2 test cases. [----------] Global test environment set-up. [----------] 10 tests from DBBlobBasicTest [ RUN ] DBBlobBasicTest.GetBlob [ OK ] DBBlobBasicTest.GetBlob (12 ms) [ RUN ] DBBlobBasicTest.MultiGetBlobs [ OK ] DBBlobBasicTest.MultiGetBlobs (11 ms) [ RUN ] DBBlobBasicTest.GetBlob_CorruptIndex [ OK ] DBBlobBasicTest.GetBlob_CorruptIndex (10 ms) [ RUN ] DBBlobBasicTest.GetBlob_InlinedTTLIndex [ OK ] DBBlobBasicTest.GetBlob_InlinedTTLIndex (12 ms) [ RUN ] DBBlobBasicTest.GetBlob_IndexWithInvalidFileNumber [ OK ] DBBlobBasicTest.GetBlob_IndexWithInvalidFileNumber (9 ms) [ RUN ] DBBlobBasicTest.GenerateIOTracing [ OK ] DBBlobBasicTest.GenerateIOTracing (11 ms) [ RUN ] DBBlobBasicTest.BestEffortsRecovery_MissingNewestBlobFile [ OK ] DBBlobBasicTest.BestEffortsRecovery_MissingNewestBlobFile (13 ms) [ RUN ] DBBlobBasicTest.GetMergeBlobWithPut [ OK ] DBBlobBasicTest.GetMergeBlobWithPut (11 ms) [ RUN ] DBBlobBasicTest.MultiGetMergeBlobWithPut [ OK ] DBBlobBasicTest.MultiGetMergeBlobWithPut (14 ms) [ RUN ] DBBlobBasicTest.BlobDBProperties [ OK ] DBBlobBasicTest.BlobDBProperties (21 ms) [----------] 10 tests from DBBlobBasicTest (124 ms total) [----------] 6 tests from DBBlobBasicTest/DBBlobBasicIOErrorTest [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/0 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/0 (12 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/1 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/1 (10 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/0 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/0 (10 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/1 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/1 (10 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/0 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/0 (1011 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/1 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/1 (1013 ms) [----------] 6 tests from DBBlobBasicTest/DBBlobBasicIOErrorTest (2066 ms total) [----------] Global test environment tear-down [==========] 16 tests from 2 test cases ran. (2190 ms total) [ PASSED ] 16 tests. ``` Reviewed By: ltamasi Differential Revision: D30690849 Pulled By: Zhiyi-Zhang fbshipit-source-id: a7567319487ad76bd1a2e24bf143afdbbd9e4346
3 years ago
uint64_t GetTotalBlobFileSize() const; // REQUIRE: DB mutex held
void SetMemtable(MemTable* new_mem) {
uint64_t memtable_id = last_memtable_id_.fetch_add(1) + 1;
new_mem->SetID(memtable_id);
mem_ = new_mem;
}
// calculate the oldest log needed for the durability of this column family
uint64_t OldestLogToKeep();
// See Memtable constructor for explanation of earliest_seq param.
MemTable* ConstructNewMemtable(const MutableCFOptions& mutable_cf_options,
SequenceNumber earliest_seq);
void CreateNewMemtable(const MutableCFOptions& mutable_cf_options,
SequenceNumber earliest_seq);
TableCache* table_cache() const { return table_cache_.get(); }
BlobSource* blob_source() const { return blob_source_.get(); }
// See documentation in compaction_picker.h
// REQUIRES: DB mutex held
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
10 years ago
bool NeedsCompaction() const;
// REQUIRES: DB mutex held
Compaction* PickCompaction(const MutableCFOptions& mutable_options,
const MutableDBOptions& mutable_db_options,
LogBuffer* log_buffer);
// Check if the passed range overlap with any running compactions.
// REQUIRES: DB mutex held
bool RangeOverlapWithCompaction(const Slice& smallest_user_key,
const Slice& largest_user_key,
int level) const;
// Check if the passed ranges overlap with any unflushed memtables
// (immutable or mutable).
//
// @param super_version A referenced SuperVersion that will be held for the
// duration of this function.
//
// Thread-safe
Status RangesOverlapWithMemtables(const autovector<Range>& ranges,
SuperVersion* super_version,
bool allow_data_in_errors, bool* overlap);
// A flag to tell a manual compaction is to compact all levels together
// instead of a specific level.
static const int kCompactAllLevels;
// A flag to tell a manual compaction's output is base level.
static const int kCompactToBaseLevel;
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
10 years ago
// REQUIRES: DB mutex held
Compaction* CompactRange(const MutableCFOptions& mutable_cf_options,
const MutableDBOptions& mutable_db_options,
int input_level, int output_level,
const CompactRangeOptions& compact_range_options,
const InternalKey* begin, const InternalKey* end,
InternalKey** compaction_end, bool* manual_conflict,
uint64_t max_file_num_to_ignore,
const std::string& trim_ts);
CompactionPicker* compaction_picker() { return compaction_picker_.get(); }
// thread-safe
const Comparator* user_comparator() const {
return internal_comparator_.user_comparator();
}
// thread-safe
const InternalKeyComparator& internal_comparator() const {
return internal_comparator_;
}
const IntTblPropCollectorFactories* int_tbl_prop_collector_factories() const {
return &int_tbl_prop_collector_factories_;
}
SuperVersion* GetSuperVersion() { return super_version_; }
// thread-safe
// Return a already referenced SuperVersion to be used safely.
SuperVersion* GetReferencedSuperVersion(DBImpl* db);
// thread-safe
// Get SuperVersion stored in thread local storage. If it does not exist,
// get a reference from a current SuperVersion.
SuperVersion* GetThreadLocalSuperVersion(DBImpl* db);
// Try to return SuperVersion back to thread local storage. Return true on
// success and false on failure. It fails when the thread local storage
// contains anything other than SuperVersion::kSVInUse flag.
bool ReturnThreadLocalSuperVersion(SuperVersion* sv);
// thread-safe
uint64_t GetSuperVersionNumber() const {
return super_version_number_.load();
}
// will return a pointer to SuperVersion* if previous SuperVersion
// if its reference count is zero and needs deletion or nullptr if not
// As argument takes a pointer to allocated SuperVersion to enable
// the clients to allocate SuperVersion outside of mutex.
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
10 years ago
// IMPORTANT: Only call this from DBImpl::InstallSuperVersion()
void InstallSuperVersion(SuperVersionContext* sv_context,
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
3 years ago
const MutableCFOptions& mutable_cf_options);
void InstallSuperVersion(SuperVersionContext* sv_context,
InstrumentedMutex* db_mutex);
void ResetThreadLocalSuperVersions();
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
10 years ago
// Protected by DB mutex
void set_queued_for_flush(bool value) { queued_for_flush_ = value; }
void set_queued_for_compaction(bool value) { queued_for_compaction_ = value; }
bool queued_for_flush() { return queued_for_flush_; }
bool queued_for_compaction() { return queued_for_compaction_; }
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
10 years ago
enum class WriteStallCause {
kNone,
kMemtableLimit,
kL0FileCountLimit,
kPendingCompactionBytes,
};
static std::pair<WriteStallCondition, WriteStallCause>
Fix checkpoint stuck (#7921) Summary: ## 1. Bug description: When RocksDB Checkpoint, it may be stuck in `WaitUntilFlushWouldNotStallWrites` method. ## 2. Simple analysis of the reasons: ### 2.1 Configuration parameters: ```yaml Compaction Style : Universal max_write_buffer_number : 4 min_write_buffer_number_to_merge : 3 ``` Checkpoint is usually very fast. When the Checkpoint is executed, `WaitUntilFlushWouldNotStallWrites` is called. If there are 2 Immutable MemTables, which are less than `min_write_buffer_number_to_merge`, they will not be flushed. But will enter this code. ```c++ // method: GetWriteStallConditionAndCause 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}; } ``` code link: https://github.com/facebook/rocksdb/blob/fbed72f03c3d9e4fdca3e5993587ef2559ba6ab9/db/column_family.cc#L847 Checkpoint thought there was a FlushJob, but it didn't. So will always wait. ### 2.2 solution: Increase the restriction: the `number of Immutable MemTable` >= `min_write_buffer_number_to_merge will wait`. If there are other better solutions, you can correct me. ### 2.3 Code that can reproduce the problem: https://github.com/1996fanrui/fanrui-learning/blob/flink-1.12/module-java/src/main/java/com/dream/rocksdb/RocksDBCheckpointStuck.java ## 3. Interesting point This bug will be triggered only when `the number of sorted runs >= level0_file_num_compaction_trigger`. Because there is a break in WaitUntilFlushWouldNotStallWrites. ```c++ if (cfd->imm()->NumNotFlushed() < cfd->ioptions()->min_write_buffer_number_to_merge && vstorage->l0_delay_trigger_count() < mutable_cf_options.level0_file_num_compaction_trigger) { break; } ``` code link: https://github.com/facebook/rocksdb/blob/fbed72f03c3d9e4fdca3e5993587ef2559ba6ab9/db/db_impl/db_impl_compaction_flush.cc#L1974 Universal may have `l0_delay_trigger_count() >= level0_file_num_compaction_trigger`, so this bug is triggered. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7921 Reviewed By: jay-zhuang Differential Revision: D26900559 Pulled By: ajkr fbshipit-source-id: 133c1252dad7393753f04a47590b68c7d8e670df
3 years ago
GetWriteStallConditionAndCause(
int num_unflushed_memtables, int num_l0_files,
uint64_t num_compaction_needed_bytes,
const MutableCFOptions& mutable_cf_options,
const ImmutableCFOptions& immutable_cf_options);
// Recalculate some stall conditions, which are changed only during
// compaction, adding new memtable and/or recalculation of compaction score.
WriteStallCondition RecalculateWriteStallConditions(
const MutableCFOptions& mutable_cf_options);
void set_initialized() { initialized_.store(true); }
bool initialized() const { return initialized_.load(); }
const ColumnFamilyOptions& initial_cf_options() {
return initial_cf_options_;
}
Env::WriteLifeTimeHint CalculateSSTWriteHint(int level);
// created_dirs remembers directory created, so that we don't need to call
// the same data creation operation again.
Status AddDirectories(
std::map<std::string, std::shared_ptr<FSDirectory>>* created_dirs);
FSDirectory* GetDataDir(size_t path_id) const;
// full_history_ts_low_ can only increase.
void SetFullHistoryTsLow(std::string ts_low) {
assert(!ts_low.empty());
const Comparator* ucmp = user_comparator();
assert(ucmp);
if (full_history_ts_low_.empty() ||
ucmp->CompareTimestamp(ts_low, full_history_ts_low_) > 0) {
full_history_ts_low_ = std::move(ts_low);
}
}
const std::string& GetFullHistoryTsLow() const {
return full_history_ts_low_;
}
ThreadLocalPtr* TEST_GetLocalSV() { return local_sv_.get(); }
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
3 years ago
WriteBufferManager* write_buffer_mgr() { return write_buffer_manager_; }
Account memory of FileMetaData in global memory limit (#9924) Summary: **Context/Summary:** As revealed by heap profiling, allocation of `FileMetaData` for [newly created file added to a Version](https://github.com/facebook/rocksdb/pull/9924/files#diff-a6aa385940793f95a2c5b39cc670bd440c4547fa54fd44622f756382d5e47e43R774) can consume significant heap memory. This PR is to account that toward our global memory limit based on block cache capacity. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9924 Test Plan: - Previous `make check` verified there are only 2 places where the memory of the allocated `FileMetaData` can be released - New unit test `TEST_P(ChargeFileMetadataTestWithParam, Basic)` - db bench (CPU cost of `charge_file_metadata` in write and compact) - **write micros/op: -0.24%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 (remove this option for pre-PR) -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` - **compact micros/op -0.87%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 -numdistinct=1000 && ./db_bench -benchmarks=compact -db=$TEST_TMPDIR -use_existing_db=1 -charge_file_metadata=1 -disable_auto_compactions=1 | egrep 'compact'` table 1 - write #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | -0.3633711465 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | 0.5289363078 80 | 3.87828 | 0.119007 | 3.86791 | 0.115674 | **-0.2673865734** 160 | 3.87677 | 0.162231 | 3.86739 | 0.16663 | **-0.2419539978** table 2 - compact #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 2,399,650.00 | 96,375.80 | 2,359,537.00 | 53,243.60 | -1.67 20 | 2,410,480.00 | 89,988.00 | 2,433,580.00 | 91,121.20 | 0.96 40 | 2.41E+06 | 121811 | 2.39E+06 | 131525 | **-0.96** 80 | 2.40E+06 | 134503 | 2.39E+06 | 108799 | **-0.78** - stress test: `python3 tools/db_crashtest.py blackbox --charge_file_metadata=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36055583 Pulled By: hx235 fbshipit-source-id: b60eab94707103cb1322cf815f05810ef0232625
2 years ago
std::shared_ptr<CacheReservationManager>
GetFileMetadataCacheReservationManager() {
return file_metadata_cache_res_mgr_;
}
SequenceNumber GetFirstMemtableSequenceNumber() const;
static const uint32_t kDummyColumnFamilyDataId;
Dynamically changeable `MemPurge` option (#10011) Summary: **Summary** Make the mempurge option flag a Mutable Column Family option flag. Therefore, the mempurge feature can be dynamically toggled. **Motivation** RocksDB users prefer having the ability to switch features on and off without having to close and reopen the DB. This is particularly important if the feature causes issues and needs to be turned off. Dynamically changing a DB option flag does not seem currently possible. Moreover, with this new change, the MemPurge feature can be toggled on or off independently between column families, which we see as a major improvement. **Content of this PR** This PR includes removal of the `experimental_mempurge_threshold` flag as a DB option flag, and its re-introduction as a `MutableCFOption` flag. I updated the code to handle dynamic changes of the flag (in particular inside the `FlushJob` file). Additionally, this PR includes a new test to demonstrate the capacity of the code to toggle the MemPurge feature on and off, as well as the addition in the `db_stress` module of 2 different mempurge threshold values (0.0 and 1.0) that can be randomly changed with the `set_option_one_in` flag. This is useful to stress test the dynamic changes. **Benchmarking** I will add numbers to prove that there is no performance impact within the next 12 hours. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10011 Reviewed By: pdillinger Differential Revision: D36462357 Pulled By: bjlemaire fbshipit-source-id: 5e3d63bdadf085c0572ecc2349e7dd9729ce1802
2 years ago
// Keep track of whether the mempurge feature was ever used.
void SetMempurgeUsed() { mempurge_used_ = true; }
bool GetMempurgeUsed() { return mempurge_used_; }
private:
friend class ColumnFamilySet;
ColumnFamilyData(uint32_t id, const std::string& name,
Version* dummy_versions, Cache* table_cache,
WriteBufferManager* write_buffer_manager,
const ColumnFamilyOptions& options,
const ImmutableDBOptions& db_options,
const FileOptions* file_options,
ColumnFamilySet* column_family_set,
BlockCacheTracer* const block_cache_tracer,
const std::shared_ptr<IOTracer>& io_tracer,
const std::string& db_id, const std::string& db_session_id);
std::vector<std::string> GetDbPaths() const;
uint32_t id_;
const std::string name_;
Version* dummy_versions_; // Head of circular doubly-linked list of versions.
Version* current_; // == dummy_versions->prev_
std::atomic<int> refs_; // outstanding references to ColumnFamilyData
std::atomic<bool> initialized_;
std::atomic<bool> dropped_; // true if client dropped it
const InternalKeyComparator internal_comparator_;
IntTblPropCollectorFactories int_tbl_prop_collector_factories_;
const ColumnFamilyOptions initial_cf_options_;
const ImmutableOptions ioptions_;
MutableCFOptions mutable_cf_options_;
const bool is_delete_range_supported_;
std::unique_ptr<TableCache> table_cache_;
std::unique_ptr<BlobFileCache> blob_file_cache_;
std::unique_ptr<BlobSource> blob_source_;
std::unique_ptr<InternalStats> internal_stats_;
WriteBufferManager* write_buffer_manager_;
MemTable* mem_;
MemTableList imm_;
SuperVersion* super_version_;
// An ordinal representing the current SuperVersion. Updated by
// InstallSuperVersion(), i.e. incremented every time super_version_
// changes.
std::atomic<uint64_t> super_version_number_;
// Thread's local copy of SuperVersion pointer
// This needs to be destructed before mutex_
std::unique_ptr<ThreadLocalPtr> local_sv_;
// pointers for a circular linked list. we use it to support iterations over
// all column families that are alive (note: dropped column families can also
// be alive as long as client holds a reference)
ColumnFamilyData* next_;
ColumnFamilyData* prev_;
// This is the earliest log file number that contains data from this
// Column Family. All earlier log files must be ignored and not
// recovered from
uint64_t log_number_;
std::atomic<FlushReason> flush_reason_;
// An object that keeps all the compaction stats
// and picks the next compaction
std::unique_ptr<CompactionPicker> compaction_picker_;
ColumnFamilySet* column_family_set_;
std::unique_ptr<WriteControllerToken> write_controller_token_;
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
10 years ago
// If true --> this ColumnFamily is currently present in DBImpl::flush_queue_
bool queued_for_flush_;
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
10 years ago
// If true --> this ColumnFamily is currently present in
// DBImpl::compaction_queue_
bool queued_for_compaction_;
uint64_t prev_compaction_needed_bytes_;
// if the database was opened with 2pc enabled
bool allow_2pc_;
// Memtable id to track flush.
std::atomic<uint64_t> last_memtable_id_;
// Directories corresponding to cf_paths.
std::vector<std::shared_ptr<FSDirectory>> data_dirs_;
bool db_paths_registered_;
std::string full_history_ts_low_;
Account memory of FileMetaData in global memory limit (#9924) Summary: **Context/Summary:** As revealed by heap profiling, allocation of `FileMetaData` for [newly created file added to a Version](https://github.com/facebook/rocksdb/pull/9924/files#diff-a6aa385940793f95a2c5b39cc670bd440c4547fa54fd44622f756382d5e47e43R774) can consume significant heap memory. This PR is to account that toward our global memory limit based on block cache capacity. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9924 Test Plan: - Previous `make check` verified there are only 2 places where the memory of the allocated `FileMetaData` can be released - New unit test `TEST_P(ChargeFileMetadataTestWithParam, Basic)` - db bench (CPU cost of `charge_file_metadata` in write and compact) - **write micros/op: -0.24%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 (remove this option for pre-PR) -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` - **compact micros/op -0.87%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 -numdistinct=1000 && ./db_bench -benchmarks=compact -db=$TEST_TMPDIR -use_existing_db=1 -charge_file_metadata=1 -disable_auto_compactions=1 | egrep 'compact'` table 1 - write #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | -0.3633711465 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | 0.5289363078 80 | 3.87828 | 0.119007 | 3.86791 | 0.115674 | **-0.2673865734** 160 | 3.87677 | 0.162231 | 3.86739 | 0.16663 | **-0.2419539978** table 2 - compact #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 2,399,650.00 | 96,375.80 | 2,359,537.00 | 53,243.60 | -1.67 20 | 2,410,480.00 | 89,988.00 | 2,433,580.00 | 91,121.20 | 0.96 40 | 2.41E+06 | 121811 | 2.39E+06 | 131525 | **-0.96** 80 | 2.40E+06 | 134503 | 2.39E+06 | 108799 | **-0.78** - stress test: `python3 tools/db_crashtest.py blackbox --charge_file_metadata=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36055583 Pulled By: hx235 fbshipit-source-id: b60eab94707103cb1322cf815f05810ef0232625
2 years ago
// For charging memory usage of file metadata created for newly added files to
// a Version associated with this CFD
std::shared_ptr<CacheReservationManager> file_metadata_cache_res_mgr_;
Dynamically changeable `MemPurge` option (#10011) Summary: **Summary** Make the mempurge option flag a Mutable Column Family option flag. Therefore, the mempurge feature can be dynamically toggled. **Motivation** RocksDB users prefer having the ability to switch features on and off without having to close and reopen the DB. This is particularly important if the feature causes issues and needs to be turned off. Dynamically changing a DB option flag does not seem currently possible. Moreover, with this new change, the MemPurge feature can be toggled on or off independently between column families, which we see as a major improvement. **Content of this PR** This PR includes removal of the `experimental_mempurge_threshold` flag as a DB option flag, and its re-introduction as a `MutableCFOption` flag. I updated the code to handle dynamic changes of the flag (in particular inside the `FlushJob` file). Additionally, this PR includes a new test to demonstrate the capacity of the code to toggle the MemPurge feature on and off, as well as the addition in the `db_stress` module of 2 different mempurge threshold values (0.0 and 1.0) that can be randomly changed with the `set_option_one_in` flag. This is useful to stress test the dynamic changes. **Benchmarking** I will add numbers to prove that there is no performance impact within the next 12 hours. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10011 Reviewed By: pdillinger Differential Revision: D36462357 Pulled By: bjlemaire fbshipit-source-id: 5e3d63bdadf085c0572ecc2349e7dd9729ce1802
2 years ago
bool mempurge_used_;
};
// ColumnFamilySet has interesting thread-safety requirements
// * CreateColumnFamily() or RemoveColumnFamily() -- need to be protected by DB
// mutex AND executed in the write thread.
// CreateColumnFamily() should ONLY be called from VersionSet::LogAndApply() AND
// single-threaded write thread. It is also called during Recovery and in
// DumpManifest().
// RemoveColumnFamily() is only called from SetDropped(). DB mutex needs to be
// held and it needs to be executed from the write thread. SetDropped() also
// guarantees that it will be called only from single-threaded LogAndApply(),
// but this condition is not that important.
// * Iteration -- hold DB mutex. If you want to release the DB mutex in the
// body of the iteration, wrap in a RefedColumnFamilySet.
// * GetDefault() -- thread safe
// * GetColumnFamily() -- either inside of DB mutex or from a write thread
// * GetNextColumnFamilyID(), GetMaxColumnFamily(), UpdateMaxColumnFamily(),
// NumberOfColumnFamilies -- inside of DB mutex
class ColumnFamilySet {
public:
// ColumnFamilySet supports iteration
class iterator {
public:
explicit iterator(ColumnFamilyData* cfd)
: current_(cfd) {}
// NOTE: minimum operators for for-loop iteration
iterator& operator++() {
current_ = current_->next_;
return *this;
}
bool operator!=(const iterator& other) const {
return this->current_ != other.current_;
}
ColumnFamilyData* operator*() { return current_; }
private:
ColumnFamilyData* current_;
};
ColumnFamilySet(const std::string& dbname,
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
5 years ago
const FileOptions& file_options, Cache* table_cache,
WriteBufferManager* _write_buffer_manager,
WriteController* _write_controller,
BlockCacheTracer* const block_cache_tracer,
const std::shared_ptr<IOTracer>& io_tracer,
const std::string& db_id, const std::string& db_session_id);
~ColumnFamilySet();
ColumnFamilyData* GetDefault() const;
// GetColumnFamily() calls return nullptr if column family is not found
ColumnFamilyData* GetColumnFamily(uint32_t id) const;
ColumnFamilyData* GetColumnFamily(const std::string& name) const;
// this call will return the next available column family ID. it guarantees
// that there is no column family with id greater than or equal to the
// returned value in the current running instance or anytime in RocksDB
// instance history.
uint32_t GetNextColumnFamilyID();
uint32_t GetMaxColumnFamily();
void UpdateMaxColumnFamily(uint32_t new_max_column_family);
size_t NumberOfColumnFamilies() const;
ColumnFamilyData* CreateColumnFamily(const std::string& name, uint32_t id,
Version* dummy_version,
const ColumnFamilyOptions& options);
iterator begin() { return iterator(dummy_cfd_->next_); }
iterator end() { return iterator(dummy_cfd_); }
Cache* get_table_cache() { return table_cache_; }
WriteBufferManager* write_buffer_manager() { return write_buffer_manager_; }
WriteController* write_controller() { return write_controller_; }
private:
friend class ColumnFamilyData;
// helper function that gets called from cfd destructor
// REQUIRES: DB mutex held
void RemoveColumnFamily(ColumnFamilyData* cfd);
// column_families_ and column_family_data_ need to be protected:
// * when mutating both conditions have to be satisfied:
// 1. DB mutex locked
// 2. thread currently in single-threaded write thread
// * when reading, at least one condition needs to be satisfied:
// 1. DB mutex locked
// 2. accessed from a single-threaded write thread
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
2 years ago
UnorderedMap<std::string, uint32_t> column_families_;
UnorderedMap<uint32_t, ColumnFamilyData*> column_family_data_;
uint32_t max_column_family_;
const FileOptions file_options_;
ColumnFamilyData* dummy_cfd_;
// We don't hold the refcount here, since default column family always exists
// We are also not responsible for cleaning up default_cfd_cache_. This is
// just a cache that makes common case (accessing default column family)
// faster
ColumnFamilyData* default_cfd_cache_;
const std::string db_name_;
const ImmutableDBOptions* const db_options_;
Cache* table_cache_;
WriteBufferManager* write_buffer_manager_;
WriteController* write_controller_;
BlockCacheTracer* const block_cache_tracer_;
std::shared_ptr<IOTracer> io_tracer_;
const std::string& db_id_;
std::string db_session_id_;
};
// A wrapper for ColumnFamilySet that supports releasing DB mutex during each
// iteration over the iterator, because the cfd is Refed and Unrefed during
// each iteration to prevent concurrent CF drop from destroying it (until
// Unref).
class RefedColumnFamilySet {
public:
explicit RefedColumnFamilySet(ColumnFamilySet* cfs) : wrapped_(cfs) {}
class iterator {
public:
explicit iterator(ColumnFamilySet::iterator wrapped) : wrapped_(wrapped) {
MaybeRef(*wrapped_);
}
~iterator() { MaybeUnref(*wrapped_); }
inline void MaybeRef(ColumnFamilyData* cfd) {
if (cfd->GetID() != ColumnFamilyData::kDummyColumnFamilyDataId) {
cfd->Ref();
}
}
inline void MaybeUnref(ColumnFamilyData* cfd) {
if (cfd->GetID() != ColumnFamilyData::kDummyColumnFamilyDataId) {
cfd->UnrefAndTryDelete();
}
}
// NOTE: minimum operators for for-loop iteration
inline iterator& operator++() {
ColumnFamilyData* old = *wrapped_;
++wrapped_;
// Can only unref & potentially free cfd after accessing its next_
MaybeUnref(old);
MaybeRef(*wrapped_);
return *this;
}
inline bool operator!=(const iterator& other) const {
return this->wrapped_ != other.wrapped_;
}
inline ColumnFamilyData* operator*() { return *wrapped_; }
private:
ColumnFamilySet::iterator wrapped_;
};
iterator begin() { return iterator(wrapped_->begin()); }
iterator end() { return iterator(wrapped_->end()); }
private:
ColumnFamilySet* wrapped_;
};
// We use ColumnFamilyMemTablesImpl to provide WriteBatch a way to access
// memtables of different column families (specified by ID in the write batch)
class ColumnFamilyMemTablesImpl : public ColumnFamilyMemTables {
public:
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
explicit ColumnFamilyMemTablesImpl(ColumnFamilySet* column_family_set)
: column_family_set_(column_family_set), current_(nullptr) {}
// Constructs a ColumnFamilyMemTablesImpl equivalent to one constructed
// with the arguments used to construct *orig.
explicit ColumnFamilyMemTablesImpl(ColumnFamilyMemTablesImpl* orig)
: column_family_set_(orig->column_family_set_), current_(nullptr) {}
// sets current_ to ColumnFamilyData with column_family_id
// returns false if column family doesn't exist
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
// under a DB mutex OR from a write thread
bool Seek(uint32_t column_family_id) override;
// Returns log number of the selected column family
// REQUIRES: under a DB mutex OR from a write thread
uint64_t GetLogNumber() const override;
// REQUIRES: Seek() called first
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
// under a DB mutex OR from a write thread
virtual MemTable* GetMemTable() const override;
// Returns column family handle for the selected column family
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
// under a DB mutex OR from a write thread
virtual ColumnFamilyHandle* GetColumnFamilyHandle() override;
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// Cannot be called while another thread is calling Seek().
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
// under a DB mutex OR from a write thread
virtual ColumnFamilyData* current() override { return current_; }
private:
ColumnFamilySet* column_family_set_;
ColumnFamilyData* current_;
ColumnFamilyHandleInternal handle_;
};
extern uint32_t GetColumnFamilyID(ColumnFamilyHandle* column_family);
extern const Comparator* GetColumnFamilyUserComparator(
ColumnFamilyHandle* column_family);
} // namespace ROCKSDB_NAMESPACE