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/write_thread.h

441 lines
16 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).
#pragma once
#include <atomic>
#include <cassert>
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
#include <chrono>
#include <condition_variable>
#include <cstdint>
#include <mutex>
#include <type_traits>
#include <vector>
#include "db/dbformat.h"
Snapshots with user-specified timestamps (#9879) Summary: In RocksDB, keys are associated with (internal) sequence numbers which denote when the keys are written to the database. Sequence numbers in different RocksDB instances are unrelated, thus not comparable. It is nice if we can associate sequence numbers with their corresponding actual timestamps. One thing we can do is to support user-defined timestamp, which allows the applications to specify the format of custom timestamps and encode a timestamp with each key. More details can be found at https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp-%28Experimental%29. This PR provides a different but complementary approach. We can associate rocksdb snapshots (defined in https://github.com/facebook/rocksdb/blob/7.2.fb/include/rocksdb/snapshot.h#L20) with **user-specified** timestamps. Since a snapshot is essentially an object representing a sequence number, this PR establishes a bi-directional mapping between sequence numbers and timestamps. In the past, snapshots are usually taken by readers. The current super-version is grabbed, and a `rocksdb::Snapshot` object is created with the last published sequence number of the super-version. You can see that the reader actually has no good idea of what timestamp to assign to this snapshot, because by the time the `GetSnapshot()` is called, an arbitrarily long period of time may have already elapsed since the last write, which is when the last published sequence number is written. This observation motivates the creation of "timestamped" snapshots on the write path. Currently, this functionality is exposed only to the layer of `TransactionDB`. Application can tell RocksDB to create a snapshot when a transaction commits, effectively associating the last sequence number with a timestamp. It is also assumed that application will ensure any two snapshots with timestamps should satisfy the following: ``` snapshot1.seq < snapshot2.seq iff. snapshot1.ts < snapshot2.ts ``` If the application can guarantee that when a reader takes a timestamped snapshot, there is no active writes going on in the database, then we also allow the user to use a new API `TransactionDB::CreateTimestampedSnapshot()` to create a snapshot with associated timestamp. Code example ```cpp // Create a timestamped snapshot when committing transaction. txn->SetCommitTimestamp(100); txn->SetSnapshotOnNextOperation(); txn->Commit(); // A wrapper API for convenience Status Transaction::CommitAndTryCreateSnapshot( std::shared_ptr<TransactionNotifier> notifier, TxnTimestamp ts, std::shared_ptr<const Snapshot>* ret); // Create a timestamped snapshot if caller guarantees no concurrent writes std::pair<Status, std::shared_ptr<const Snapshot>> snapshot = txn_db->CreateTimestampedSnapshot(100); ``` The snapshots created in this way will be managed by RocksDB with ref-counting and potentially shared with other readers. We provide the following APIs for readers to retrieve a snapshot given a timestamp. ```cpp // Return the timestamped snapshot correponding to given timestamp. If ts is // kMaxTxnTimestamp, then we return the latest timestamped snapshot if present. // Othersise, we return the snapshot whose timestamp is equal to `ts`. If no // such snapshot exists, then we return null. std::shared_ptr<const Snapshot> TransactionDB::GetTimestampedSnapshot(TxnTimestamp ts) const; // Return the latest timestamped snapshot if present. std::shared_ptr<const Snapshot> TransactionDB::GetLatestTimestampedSnapshot() const; ``` We also provide two additional APIs for stats collection and reporting purposes. ```cpp Status TransactionDB::GetAllTimestampedSnapshots( std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; // Return timestamped snapshots whose timestamps fall in [ts_lb, ts_ub) and store them in `snapshots`. Status TransactionDB::GetTimestampedSnapshots( TxnTimestamp ts_lb, TxnTimestamp ts_ub, std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; ``` To prevent the number of timestamped snapshots from growing infinitely, we provide the following API to release timestamped snapshots whose timestamps are older than or equal to a given threshold. ```cpp void TransactionDB::ReleaseTimestampedSnapshotsOlderThan(TxnTimestamp ts); ``` Before shutdown, RocksDB will release all timestamped snapshots. Comparison with user-defined timestamp and how they can be combined: User-defined timestamp persists every key with a timestamp, while timestamped snapshots maintain a volatile mapping between snapshots (sequence numbers) and timestamps. Different internal keys with the same user key but different timestamps will be treated as different by compaction, thus a newer version will not hide older versions (with smaller timestamps) unless they are eligible for garbage collection. In contrast, taking a timestamped snapshot at a certain sequence number and timestamp prevents all the keys visible in this snapshot from been dropped by compaction. Here, visible means (seq < snapshot and most recent). The timestamped snapshot supports the semantics of reading at an exact point in time. Timestamped snapshots can also be used with user-defined timestamp. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9879 Test Plan: ``` make check TEST_TMPDIR=/dev/shm make crash_test_with_txn ``` Reviewed By: siying Differential Revision: D35783919 Pulled By: riversand963 fbshipit-source-id: 586ad905e169189e19d3bfc0cb0177a7239d1bd4
2 years ago
#include "db/post_memtable_callback.h"
#include "db/pre_release_callback.h"
#include "db/write_callback.h"
#include "monitoring/instrumented_mutex.h"
#include "rocksdb/options.h"
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
#include "rocksdb/status.h"
#include "rocksdb/types.h"
#include "rocksdb/write_batch.h"
#include "util/autovector.h"
namespace ROCKSDB_NAMESPACE {
class WriteThread {
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
enum State : uint8_t {
// The initial state of a writer. This is a Writer that is
// waiting in JoinBatchGroup. This state can be left when another
// thread informs the waiter that it has become a group leader
// (-> STATE_GROUP_LEADER), when a leader that has chosen to be
// non-parallel informs a follower that its writes have been committed
// (-> STATE_COMPLETED), or when a leader that has chosen to perform
// updates in parallel and needs this Writer to apply its batch (->
// STATE_PARALLEL_MEMTABLE_WRITER).
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
STATE_INIT = 1,
// The state used to inform a waiting Writer that it has become the
// leader, and it should now build a write batch group. Tricky:
// this state is not used if newest_writer_ is empty when a writer
// enqueues itself, because there is no need to wait (or even to
// create the mutex and condvar used to wait) in that case. This is
// a terminal state unless the leader chooses to make this a parallel
// batch, in which case the last parallel worker to finish will move
// the leader to STATE_COMPLETED.
STATE_GROUP_LEADER = 2,
// The state used to inform a waiting writer that it has become the
// leader of memtable writer group. The leader will either write
// memtable for the whole group, or launch a parallel group write
// to memtable by calling LaunchParallelMemTableWrite.
STATE_MEMTABLE_WRITER_LEADER = 4,
// The state used to inform a waiting writer that it has become a
// parallel memtable writer. It can be the group leader who launch the
// parallel writer group, or one of the followers. The writer should then
// apply its batch to the memtable concurrently and call
// CompleteParallelMemTableWriter.
STATE_PARALLEL_MEMTABLE_WRITER = 8,
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
// A follower whose writes have been applied, or a parallel leader
// whose followers have all finished their work. This is a terminal
// state.
STATE_COMPLETED = 16,
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
// A state indicating that the thread may be waiting using StateMutex()
// and StateCondVar()
STATE_LOCKED_WAITING = 32,
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
};
struct Writer;
struct WriteGroup {
Writer* leader = nullptr;
Writer* last_writer = nullptr;
SequenceNumber last_sequence;
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
// before running goes to zero, status needs leader->StateMutex()
Status status;
std::atomic<size_t> running;
size_t size = 0;
struct Iterator {
Writer* writer;
Writer* last_writer;
explicit Iterator(Writer* w, Writer* last)
: writer(w), last_writer(last) {}
Writer* operator*() const { return writer; }
Iterator& operator++() {
assert(writer != nullptr);
if (writer == last_writer) {
writer = nullptr;
} else {
writer = writer->link_newer;
}
return *this;
}
bool operator!=(const Iterator& other) const {
return writer != other.writer;
}
};
Iterator begin() const { return Iterator(leader, last_writer); }
Iterator end() const { return Iterator(nullptr, nullptr); }
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
};
// Information kept for every waiting writer.
struct Writer {
WriteBatch* batch;
bool sync;
bool no_slowdown;
bool disable_wal;
Rate-limit automatic WAL flush after each user write (#9607) Summary: **Context:** WAL flush is currently not rate-limited by `Options::rate_limiter`. This PR is to provide rate-limiting to auto WAL flush, the one that automatically happen after each user write operation (i.e, `Options::manual_wal_flush == false`), by adding `WriteOptions::rate_limiter_options`. Note that we are NOT rate-limiting WAL flush that do NOT automatically happen after each user write, such as `Options::manual_wal_flush == true + manual FlushWAL()` (rate-limiting multiple WAL flushes), for the benefits of: - being consistent with [ReadOptions::rate_limiter_priority](https://github.com/facebook/rocksdb/blob/7.0.fb/include/rocksdb/options.h#L515) - being able to turn off some WAL flush's rate-limiting but not all (e.g, turn off specific the WAL flush of a critical user write like a service's heartbeat) `WriteOptions::rate_limiter_options` only accept `Env::IO_USER` and `Env::IO_TOTAL` currently due to an implementation constraint. - The constraint is that we currently queue parallel writes (including WAL writes) based on FIFO policy which does not factor rate limiter priority into this layer's scheduling. If we allow lower priorities such as `Env::IO_HIGH/MID/LOW` and such writes specified with lower priorities occurs before ones specified with higher priorities (even just by a tiny bit in arrival time), the former would have blocked the latter, leading to a "priority inversion" issue and contradictory to what we promise for rate-limiting priority. Therefore we only allow `Env::IO_USER` and `Env::IO_TOTAL` right now before improving that scheduling. A pre-requisite to this feature is to support operation-level rate limiting in `WritableFileWriter`, which is also included in this PR. **Summary:** - Renamed test suite `DBRateLimiterTest to DBRateLimiterOnReadTest` for adding a new test suite - Accept `rate_limiter_priority` in `WritableFileWriter`'s private and public write functions - Passed `WriteOptions::rate_limiter_options` to `WritableFileWriter` in the path of automatic WAL flush. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9607 Test Plan: - Added new unit test to verify existing flush/compaction rate-limiting does not break, since `DBTest, RateLimitingTest` is disabled and current db-level rate-limiting tests focus on read only (e.g, `db_rate_limiter_test`, `DBTest2, RateLimitedCompactionReads`). - Added new unit test `DBRateLimiterOnWriteWALTest, AutoWalFlush` - `strace -ftt -e trace=write ./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -rate_limit_auto_wal_flush=1 -rate_limiter_bytes_per_sec=15 -rate_limiter_refill_period_us=1000000 -write_buffer_size=100000000 -disable_auto_compactions=1 -num=100` - verified that WAL flush(i.e, system-call _write_) were chunked into 15 bytes and each _write_ was roughly 1 second apart - verified the chunking disappeared when `-rate_limit_auto_wal_flush=0` - crash test: `python3 tools/db_crashtest.py blackbox --disable_wal=0 --rate_limit_auto_wal_flush=1 --rate_limiter_bytes_per_sec=10485760 --interval=10` killed as normal **Benchmarked on flush/compaction to ensure no performance regression:** - compaction with rate-limiting (see table 1, avg over 1280-run): pre-change: **915635 micros/op**; post-change: **907350 micros/op (improved by 0.106%)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f compact_bmk_output.txt compact_bmk_output_2.txt dont_care_output.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench --benchmarks=fillrandom -db=$TEST_TMPDIR -disable_auto_compactions=1 -write_buffer_size=6710886 > dont_care_output.txt && ./db_bench --benchmarks=compact -use_existing_db=1 -db=$TEST_TMPDIR -level0_file_num_compaction_trigger=1 -rate_limiter_bytes_per_sec=100000000 | egrep 'compact' done > compact_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' compact_bmk_output.txt >> compact_bmk_output_2.txt done ``` - compaction w/o rate-limiting (see table 2, avg over 640-run): pre-change: **822197 micros/op**; post-change: **823148 micros/op (regressed by 0.12%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` - flush with rate-limiting (see table 3, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench ): pre-change: **745752 micros/op**; post-change: **745331 micros/op (regressed by 0.06 %)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f flush_bmk_output.txt flush_bmk_output_2.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench -db=$TEST_TMPDIR -write_buffer_size=1048576000 -num=1000000 -rate_limiter_bytes_per_sec=100000000 -benchmarks=fillseq,flush | egrep 'flush' done > flush_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' flush_bmk_output.txt >> flush_bmk_output_2.txt done ``` - flush w/o rate-limiting (see table 4, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench): pre-change: **487512 micros/op**, post-change: **485856 micors/ops (improved by 0.34%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` | table 1 - compact with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 896978 | 16046.9 | 901242 | 15670.9 | 0.475373978 20 | 893718 | 15813 | 886505 | 17544.7 | -0.8070778478 40 | 900426 | 23882.2 | 894958 | 15104.5 | -0.6072681153 80 | 906635 | 21761.5 | 903332 | 23948.3 | -0.3643141948 160 | 898632 | 21098.9 | 907583 | 21145 | 0.9960695813 3.20E+02 | 905252 | 22785.5 | 908106 | 25325.5 | 0.3152713278 6.40E+02 | 905213 | 23598.6 | 906741 | 21370.5 | 0.1688000504 **1.28E+03** | **908316** | **23533.1** | **907350** | **24626.8** | **-0.1063506533** average over #-run | 901896.25 | 21064.9625 | 901977.125 | 20592.025 | 0.008967217682 | table 2 - compact w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 811211 | 26996.7 | 807586 | 28456.4 | -0.4468627768 20 | 815465 | 14803.7 | 814608 | 28719.7 | -0.105093413 40 | 809203 | 26187.1 | 797835 | 25492.1 | -1.404839082 80 | 822088 | 28765.3 | 822192 | 32840.4 | 0.01265071379 160 | 821719 | 36344.7 | 821664 | 29544.9 | -0.006693285661 3.20E+02 | 820921 | 27756.4 | 821403 | 28347.7 | 0.05871454135 **6.40E+02** | **822197** | **28960.6** | **823148** | **30055.1** | **0.1156657103** average over #-run | 8.18E+05 | 2.71E+04 | 8.15E+05 | 2.91E+04 | -0.25 | table 3 - flush with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 741721 | 11770.8 | 740345 | 5949.76 | -0.1855144994 20 | 735169 | 3561.83 | 743199 | 9755.77 | 1.09226586 40 | 743368 | 8891.03 | 742102 | 8683.22 | -0.1703059588 80 | 742129 | 8148.51 | 743417 | 9631.58| 0.1735547324 160 | 749045 | 9757.21 | 746256 | 9191.86 | -0.3723407806 **3.20E+02** | **745752** | **9819.65** | **745331** | **9840.62** | **-0.0564530836** 6.40E+02 | 749006 | 11080.5 | 748173 | 10578.7 | -0.1112140624 average over #-run | 743741.4286 | 9004.218571 | 744117.5714 | 9090.215714 | 0.05057441238 | table 4 - flush w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 477283 | 24719.6 | 473864 | 12379 | -0.7163464863 20 | 486743 | 20175.2 | 502296 | 23931.3 | 3.195320734 40 | 482846 | 15309.2 | 489820 | 22259.5 | 1.444352858 80 | 491490 | 21883.1 | 490071 | 23085.7 | -0.2887139108 160 | 493347 | 28074.3 | 483609 | 21211.7 | -1.973864238 **3.20E+02** | **487512** | **21401.5** | **485856** | **22195.2** | **-0.3396839462** 6.40E+02 | 490307 | 25418.6 | 485435 | 22405.2 | -0.9936631539 average over #-run | 4.87E+05 | 2.24E+04 | 4.87E+05 | 2.11E+04 | 0.00E+00 Reviewed By: ajkr Differential Revision: D34442441 Pulled By: hx235 fbshipit-source-id: 4790f13e1e5c0a95ae1d1cc93ffcf69dc6e78bdd
2 years ago
Env::IOPriority rate_limiter_priority;
bool disable_memtable;
size_t batch_cnt; // if non-zero, number of sub-batches in the write batch
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
size_t protection_bytes_per_key;
PreReleaseCallback* pre_release_callback;
Snapshots with user-specified timestamps (#9879) Summary: In RocksDB, keys are associated with (internal) sequence numbers which denote when the keys are written to the database. Sequence numbers in different RocksDB instances are unrelated, thus not comparable. It is nice if we can associate sequence numbers with their corresponding actual timestamps. One thing we can do is to support user-defined timestamp, which allows the applications to specify the format of custom timestamps and encode a timestamp with each key. More details can be found at https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp-%28Experimental%29. This PR provides a different but complementary approach. We can associate rocksdb snapshots (defined in https://github.com/facebook/rocksdb/blob/7.2.fb/include/rocksdb/snapshot.h#L20) with **user-specified** timestamps. Since a snapshot is essentially an object representing a sequence number, this PR establishes a bi-directional mapping between sequence numbers and timestamps. In the past, snapshots are usually taken by readers. The current super-version is grabbed, and a `rocksdb::Snapshot` object is created with the last published sequence number of the super-version. You can see that the reader actually has no good idea of what timestamp to assign to this snapshot, because by the time the `GetSnapshot()` is called, an arbitrarily long period of time may have already elapsed since the last write, which is when the last published sequence number is written. This observation motivates the creation of "timestamped" snapshots on the write path. Currently, this functionality is exposed only to the layer of `TransactionDB`. Application can tell RocksDB to create a snapshot when a transaction commits, effectively associating the last sequence number with a timestamp. It is also assumed that application will ensure any two snapshots with timestamps should satisfy the following: ``` snapshot1.seq < snapshot2.seq iff. snapshot1.ts < snapshot2.ts ``` If the application can guarantee that when a reader takes a timestamped snapshot, there is no active writes going on in the database, then we also allow the user to use a new API `TransactionDB::CreateTimestampedSnapshot()` to create a snapshot with associated timestamp. Code example ```cpp // Create a timestamped snapshot when committing transaction. txn->SetCommitTimestamp(100); txn->SetSnapshotOnNextOperation(); txn->Commit(); // A wrapper API for convenience Status Transaction::CommitAndTryCreateSnapshot( std::shared_ptr<TransactionNotifier> notifier, TxnTimestamp ts, std::shared_ptr<const Snapshot>* ret); // Create a timestamped snapshot if caller guarantees no concurrent writes std::pair<Status, std::shared_ptr<const Snapshot>> snapshot = txn_db->CreateTimestampedSnapshot(100); ``` The snapshots created in this way will be managed by RocksDB with ref-counting and potentially shared with other readers. We provide the following APIs for readers to retrieve a snapshot given a timestamp. ```cpp // Return the timestamped snapshot correponding to given timestamp. If ts is // kMaxTxnTimestamp, then we return the latest timestamped snapshot if present. // Othersise, we return the snapshot whose timestamp is equal to `ts`. If no // such snapshot exists, then we return null. std::shared_ptr<const Snapshot> TransactionDB::GetTimestampedSnapshot(TxnTimestamp ts) const; // Return the latest timestamped snapshot if present. std::shared_ptr<const Snapshot> TransactionDB::GetLatestTimestampedSnapshot() const; ``` We also provide two additional APIs for stats collection and reporting purposes. ```cpp Status TransactionDB::GetAllTimestampedSnapshots( std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; // Return timestamped snapshots whose timestamps fall in [ts_lb, ts_ub) and store them in `snapshots`. Status TransactionDB::GetTimestampedSnapshots( TxnTimestamp ts_lb, TxnTimestamp ts_ub, std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; ``` To prevent the number of timestamped snapshots from growing infinitely, we provide the following API to release timestamped snapshots whose timestamps are older than or equal to a given threshold. ```cpp void TransactionDB::ReleaseTimestampedSnapshotsOlderThan(TxnTimestamp ts); ``` Before shutdown, RocksDB will release all timestamped snapshots. Comparison with user-defined timestamp and how they can be combined: User-defined timestamp persists every key with a timestamp, while timestamped snapshots maintain a volatile mapping between snapshots (sequence numbers) and timestamps. Different internal keys with the same user key but different timestamps will be treated as different by compaction, thus a newer version will not hide older versions (with smaller timestamps) unless they are eligible for garbage collection. In contrast, taking a timestamped snapshot at a certain sequence number and timestamp prevents all the keys visible in this snapshot from been dropped by compaction. Here, visible means (seq < snapshot and most recent). The timestamped snapshot supports the semantics of reading at an exact point in time. Timestamped snapshots can also be used with user-defined timestamp. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9879 Test Plan: ``` make check TEST_TMPDIR=/dev/shm make crash_test_with_txn ``` Reviewed By: siying Differential Revision: D35783919 Pulled By: riversand963 fbshipit-source-id: 586ad905e169189e19d3bfc0cb0177a7239d1bd4
2 years ago
PostMemTableCallback* post_memtable_callback;
uint64_t log_used; // log number that this batch was inserted into
uint64_t log_ref; // log number that memtable insert should reference
WriteCallback* callback;
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
bool made_waitable; // records lazy construction of mutex and cv
std::atomic<uint8_t> state; // write under StateMutex() or pre-link
WriteGroup* write_group;
SequenceNumber sequence; // the sequence number to use for the first key
Status status;
Status callback_status; // status returned by callback->Callback()
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
std::aligned_storage<sizeof(std::mutex)>::type state_mutex_bytes;
std::aligned_storage<sizeof(std::condition_variable)>::type state_cv_bytes;
Writer* link_older; // read/write only before linking, or as leader
Writer* link_newer; // lazy, read/write only before linking, or as leader
Writer()
: batch(nullptr),
sync(false),
no_slowdown(false),
disable_wal(false),
Rate-limit automatic WAL flush after each user write (#9607) Summary: **Context:** WAL flush is currently not rate-limited by `Options::rate_limiter`. This PR is to provide rate-limiting to auto WAL flush, the one that automatically happen after each user write operation (i.e, `Options::manual_wal_flush == false`), by adding `WriteOptions::rate_limiter_options`. Note that we are NOT rate-limiting WAL flush that do NOT automatically happen after each user write, such as `Options::manual_wal_flush == true + manual FlushWAL()` (rate-limiting multiple WAL flushes), for the benefits of: - being consistent with [ReadOptions::rate_limiter_priority](https://github.com/facebook/rocksdb/blob/7.0.fb/include/rocksdb/options.h#L515) - being able to turn off some WAL flush's rate-limiting but not all (e.g, turn off specific the WAL flush of a critical user write like a service's heartbeat) `WriteOptions::rate_limiter_options` only accept `Env::IO_USER` and `Env::IO_TOTAL` currently due to an implementation constraint. - The constraint is that we currently queue parallel writes (including WAL writes) based on FIFO policy which does not factor rate limiter priority into this layer's scheduling. If we allow lower priorities such as `Env::IO_HIGH/MID/LOW` and such writes specified with lower priorities occurs before ones specified with higher priorities (even just by a tiny bit in arrival time), the former would have blocked the latter, leading to a "priority inversion" issue and contradictory to what we promise for rate-limiting priority. Therefore we only allow `Env::IO_USER` and `Env::IO_TOTAL` right now before improving that scheduling. A pre-requisite to this feature is to support operation-level rate limiting in `WritableFileWriter`, which is also included in this PR. **Summary:** - Renamed test suite `DBRateLimiterTest to DBRateLimiterOnReadTest` for adding a new test suite - Accept `rate_limiter_priority` in `WritableFileWriter`'s private and public write functions - Passed `WriteOptions::rate_limiter_options` to `WritableFileWriter` in the path of automatic WAL flush. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9607 Test Plan: - Added new unit test to verify existing flush/compaction rate-limiting does not break, since `DBTest, RateLimitingTest` is disabled and current db-level rate-limiting tests focus on read only (e.g, `db_rate_limiter_test`, `DBTest2, RateLimitedCompactionReads`). - Added new unit test `DBRateLimiterOnWriteWALTest, AutoWalFlush` - `strace -ftt -e trace=write ./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -rate_limit_auto_wal_flush=1 -rate_limiter_bytes_per_sec=15 -rate_limiter_refill_period_us=1000000 -write_buffer_size=100000000 -disable_auto_compactions=1 -num=100` - verified that WAL flush(i.e, system-call _write_) were chunked into 15 bytes and each _write_ was roughly 1 second apart - verified the chunking disappeared when `-rate_limit_auto_wal_flush=0` - crash test: `python3 tools/db_crashtest.py blackbox --disable_wal=0 --rate_limit_auto_wal_flush=1 --rate_limiter_bytes_per_sec=10485760 --interval=10` killed as normal **Benchmarked on flush/compaction to ensure no performance regression:** - compaction with rate-limiting (see table 1, avg over 1280-run): pre-change: **915635 micros/op**; post-change: **907350 micros/op (improved by 0.106%)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f compact_bmk_output.txt compact_bmk_output_2.txt dont_care_output.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench --benchmarks=fillrandom -db=$TEST_TMPDIR -disable_auto_compactions=1 -write_buffer_size=6710886 > dont_care_output.txt && ./db_bench --benchmarks=compact -use_existing_db=1 -db=$TEST_TMPDIR -level0_file_num_compaction_trigger=1 -rate_limiter_bytes_per_sec=100000000 | egrep 'compact' done > compact_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' compact_bmk_output.txt >> compact_bmk_output_2.txt done ``` - compaction w/o rate-limiting (see table 2, avg over 640-run): pre-change: **822197 micros/op**; post-change: **823148 micros/op (regressed by 0.12%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` - flush with rate-limiting (see table 3, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench ): pre-change: **745752 micros/op**; post-change: **745331 micros/op (regressed by 0.06 %)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f flush_bmk_output.txt flush_bmk_output_2.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench -db=$TEST_TMPDIR -write_buffer_size=1048576000 -num=1000000 -rate_limiter_bytes_per_sec=100000000 -benchmarks=fillseq,flush | egrep 'flush' done > flush_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' flush_bmk_output.txt >> flush_bmk_output_2.txt done ``` - flush w/o rate-limiting (see table 4, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench): pre-change: **487512 micros/op**, post-change: **485856 micors/ops (improved by 0.34%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` | table 1 - compact with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 896978 | 16046.9 | 901242 | 15670.9 | 0.475373978 20 | 893718 | 15813 | 886505 | 17544.7 | -0.8070778478 40 | 900426 | 23882.2 | 894958 | 15104.5 | -0.6072681153 80 | 906635 | 21761.5 | 903332 | 23948.3 | -0.3643141948 160 | 898632 | 21098.9 | 907583 | 21145 | 0.9960695813 3.20E+02 | 905252 | 22785.5 | 908106 | 25325.5 | 0.3152713278 6.40E+02 | 905213 | 23598.6 | 906741 | 21370.5 | 0.1688000504 **1.28E+03** | **908316** | **23533.1** | **907350** | **24626.8** | **-0.1063506533** average over #-run | 901896.25 | 21064.9625 | 901977.125 | 20592.025 | 0.008967217682 | table 2 - compact w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 811211 | 26996.7 | 807586 | 28456.4 | -0.4468627768 20 | 815465 | 14803.7 | 814608 | 28719.7 | -0.105093413 40 | 809203 | 26187.1 | 797835 | 25492.1 | -1.404839082 80 | 822088 | 28765.3 | 822192 | 32840.4 | 0.01265071379 160 | 821719 | 36344.7 | 821664 | 29544.9 | -0.006693285661 3.20E+02 | 820921 | 27756.4 | 821403 | 28347.7 | 0.05871454135 **6.40E+02** | **822197** | **28960.6** | **823148** | **30055.1** | **0.1156657103** average over #-run | 8.18E+05 | 2.71E+04 | 8.15E+05 | 2.91E+04 | -0.25 | table 3 - flush with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 741721 | 11770.8 | 740345 | 5949.76 | -0.1855144994 20 | 735169 | 3561.83 | 743199 | 9755.77 | 1.09226586 40 | 743368 | 8891.03 | 742102 | 8683.22 | -0.1703059588 80 | 742129 | 8148.51 | 743417 | 9631.58| 0.1735547324 160 | 749045 | 9757.21 | 746256 | 9191.86 | -0.3723407806 **3.20E+02** | **745752** | **9819.65** | **745331** | **9840.62** | **-0.0564530836** 6.40E+02 | 749006 | 11080.5 | 748173 | 10578.7 | -0.1112140624 average over #-run | 743741.4286 | 9004.218571 | 744117.5714 | 9090.215714 | 0.05057441238 | table 4 - flush w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 477283 | 24719.6 | 473864 | 12379 | -0.7163464863 20 | 486743 | 20175.2 | 502296 | 23931.3 | 3.195320734 40 | 482846 | 15309.2 | 489820 | 22259.5 | 1.444352858 80 | 491490 | 21883.1 | 490071 | 23085.7 | -0.2887139108 160 | 493347 | 28074.3 | 483609 | 21211.7 | -1.973864238 **3.20E+02** | **487512** | **21401.5** | **485856** | **22195.2** | **-0.3396839462** 6.40E+02 | 490307 | 25418.6 | 485435 | 22405.2 | -0.9936631539 average over #-run | 4.87E+05 | 2.24E+04 | 4.87E+05 | 2.11E+04 | 0.00E+00 Reviewed By: ajkr Differential Revision: D34442441 Pulled By: hx235 fbshipit-source-id: 4790f13e1e5c0a95ae1d1cc93ffcf69dc6e78bdd
2 years ago
rate_limiter_priority(Env::IOPriority::IO_TOTAL),
disable_memtable(false),
batch_cnt(0),
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
protection_bytes_per_key(0),
pre_release_callback(nullptr),
Snapshots with user-specified timestamps (#9879) Summary: In RocksDB, keys are associated with (internal) sequence numbers which denote when the keys are written to the database. Sequence numbers in different RocksDB instances are unrelated, thus not comparable. It is nice if we can associate sequence numbers with their corresponding actual timestamps. One thing we can do is to support user-defined timestamp, which allows the applications to specify the format of custom timestamps and encode a timestamp with each key. More details can be found at https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp-%28Experimental%29. This PR provides a different but complementary approach. We can associate rocksdb snapshots (defined in https://github.com/facebook/rocksdb/blob/7.2.fb/include/rocksdb/snapshot.h#L20) with **user-specified** timestamps. Since a snapshot is essentially an object representing a sequence number, this PR establishes a bi-directional mapping between sequence numbers and timestamps. In the past, snapshots are usually taken by readers. The current super-version is grabbed, and a `rocksdb::Snapshot` object is created with the last published sequence number of the super-version. You can see that the reader actually has no good idea of what timestamp to assign to this snapshot, because by the time the `GetSnapshot()` is called, an arbitrarily long period of time may have already elapsed since the last write, which is when the last published sequence number is written. This observation motivates the creation of "timestamped" snapshots on the write path. Currently, this functionality is exposed only to the layer of `TransactionDB`. Application can tell RocksDB to create a snapshot when a transaction commits, effectively associating the last sequence number with a timestamp. It is also assumed that application will ensure any two snapshots with timestamps should satisfy the following: ``` snapshot1.seq < snapshot2.seq iff. snapshot1.ts < snapshot2.ts ``` If the application can guarantee that when a reader takes a timestamped snapshot, there is no active writes going on in the database, then we also allow the user to use a new API `TransactionDB::CreateTimestampedSnapshot()` to create a snapshot with associated timestamp. Code example ```cpp // Create a timestamped snapshot when committing transaction. txn->SetCommitTimestamp(100); txn->SetSnapshotOnNextOperation(); txn->Commit(); // A wrapper API for convenience Status Transaction::CommitAndTryCreateSnapshot( std::shared_ptr<TransactionNotifier> notifier, TxnTimestamp ts, std::shared_ptr<const Snapshot>* ret); // Create a timestamped snapshot if caller guarantees no concurrent writes std::pair<Status, std::shared_ptr<const Snapshot>> snapshot = txn_db->CreateTimestampedSnapshot(100); ``` The snapshots created in this way will be managed by RocksDB with ref-counting and potentially shared with other readers. We provide the following APIs for readers to retrieve a snapshot given a timestamp. ```cpp // Return the timestamped snapshot correponding to given timestamp. If ts is // kMaxTxnTimestamp, then we return the latest timestamped snapshot if present. // Othersise, we return the snapshot whose timestamp is equal to `ts`. If no // such snapshot exists, then we return null. std::shared_ptr<const Snapshot> TransactionDB::GetTimestampedSnapshot(TxnTimestamp ts) const; // Return the latest timestamped snapshot if present. std::shared_ptr<const Snapshot> TransactionDB::GetLatestTimestampedSnapshot() const; ``` We also provide two additional APIs for stats collection and reporting purposes. ```cpp Status TransactionDB::GetAllTimestampedSnapshots( std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; // Return timestamped snapshots whose timestamps fall in [ts_lb, ts_ub) and store them in `snapshots`. Status TransactionDB::GetTimestampedSnapshots( TxnTimestamp ts_lb, TxnTimestamp ts_ub, std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; ``` To prevent the number of timestamped snapshots from growing infinitely, we provide the following API to release timestamped snapshots whose timestamps are older than or equal to a given threshold. ```cpp void TransactionDB::ReleaseTimestampedSnapshotsOlderThan(TxnTimestamp ts); ``` Before shutdown, RocksDB will release all timestamped snapshots. Comparison with user-defined timestamp and how they can be combined: User-defined timestamp persists every key with a timestamp, while timestamped snapshots maintain a volatile mapping between snapshots (sequence numbers) and timestamps. Different internal keys with the same user key but different timestamps will be treated as different by compaction, thus a newer version will not hide older versions (with smaller timestamps) unless they are eligible for garbage collection. In contrast, taking a timestamped snapshot at a certain sequence number and timestamp prevents all the keys visible in this snapshot from been dropped by compaction. Here, visible means (seq < snapshot and most recent). The timestamped snapshot supports the semantics of reading at an exact point in time. Timestamped snapshots can also be used with user-defined timestamp. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9879 Test Plan: ``` make check TEST_TMPDIR=/dev/shm make crash_test_with_txn ``` Reviewed By: siying Differential Revision: D35783919 Pulled By: riversand963 fbshipit-source-id: 586ad905e169189e19d3bfc0cb0177a7239d1bd4
2 years ago
post_memtable_callback(nullptr),
log_used(0),
log_ref(0),
callback(nullptr),
made_waitable(false),
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
state(STATE_INIT),
write_group(nullptr),
sequence(kMaxSequenceNumber),
link_older(nullptr),
link_newer(nullptr) {}
Writer(const WriteOptions& write_options, WriteBatch* _batch,
WriteCallback* _callback, uint64_t _log_ref, bool _disable_memtable,
size_t _batch_cnt = 0,
Snapshots with user-specified timestamps (#9879) Summary: In RocksDB, keys are associated with (internal) sequence numbers which denote when the keys are written to the database. Sequence numbers in different RocksDB instances are unrelated, thus not comparable. It is nice if we can associate sequence numbers with their corresponding actual timestamps. One thing we can do is to support user-defined timestamp, which allows the applications to specify the format of custom timestamps and encode a timestamp with each key. More details can be found at https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp-%28Experimental%29. This PR provides a different but complementary approach. We can associate rocksdb snapshots (defined in https://github.com/facebook/rocksdb/blob/7.2.fb/include/rocksdb/snapshot.h#L20) with **user-specified** timestamps. Since a snapshot is essentially an object representing a sequence number, this PR establishes a bi-directional mapping between sequence numbers and timestamps. In the past, snapshots are usually taken by readers. The current super-version is grabbed, and a `rocksdb::Snapshot` object is created with the last published sequence number of the super-version. You can see that the reader actually has no good idea of what timestamp to assign to this snapshot, because by the time the `GetSnapshot()` is called, an arbitrarily long period of time may have already elapsed since the last write, which is when the last published sequence number is written. This observation motivates the creation of "timestamped" snapshots on the write path. Currently, this functionality is exposed only to the layer of `TransactionDB`. Application can tell RocksDB to create a snapshot when a transaction commits, effectively associating the last sequence number with a timestamp. It is also assumed that application will ensure any two snapshots with timestamps should satisfy the following: ``` snapshot1.seq < snapshot2.seq iff. snapshot1.ts < snapshot2.ts ``` If the application can guarantee that when a reader takes a timestamped snapshot, there is no active writes going on in the database, then we also allow the user to use a new API `TransactionDB::CreateTimestampedSnapshot()` to create a snapshot with associated timestamp. Code example ```cpp // Create a timestamped snapshot when committing transaction. txn->SetCommitTimestamp(100); txn->SetSnapshotOnNextOperation(); txn->Commit(); // A wrapper API for convenience Status Transaction::CommitAndTryCreateSnapshot( std::shared_ptr<TransactionNotifier> notifier, TxnTimestamp ts, std::shared_ptr<const Snapshot>* ret); // Create a timestamped snapshot if caller guarantees no concurrent writes std::pair<Status, std::shared_ptr<const Snapshot>> snapshot = txn_db->CreateTimestampedSnapshot(100); ``` The snapshots created in this way will be managed by RocksDB with ref-counting and potentially shared with other readers. We provide the following APIs for readers to retrieve a snapshot given a timestamp. ```cpp // Return the timestamped snapshot correponding to given timestamp. If ts is // kMaxTxnTimestamp, then we return the latest timestamped snapshot if present. // Othersise, we return the snapshot whose timestamp is equal to `ts`. If no // such snapshot exists, then we return null. std::shared_ptr<const Snapshot> TransactionDB::GetTimestampedSnapshot(TxnTimestamp ts) const; // Return the latest timestamped snapshot if present. std::shared_ptr<const Snapshot> TransactionDB::GetLatestTimestampedSnapshot() const; ``` We also provide two additional APIs for stats collection and reporting purposes. ```cpp Status TransactionDB::GetAllTimestampedSnapshots( std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; // Return timestamped snapshots whose timestamps fall in [ts_lb, ts_ub) and store them in `snapshots`. Status TransactionDB::GetTimestampedSnapshots( TxnTimestamp ts_lb, TxnTimestamp ts_ub, std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; ``` To prevent the number of timestamped snapshots from growing infinitely, we provide the following API to release timestamped snapshots whose timestamps are older than or equal to a given threshold. ```cpp void TransactionDB::ReleaseTimestampedSnapshotsOlderThan(TxnTimestamp ts); ``` Before shutdown, RocksDB will release all timestamped snapshots. Comparison with user-defined timestamp and how they can be combined: User-defined timestamp persists every key with a timestamp, while timestamped snapshots maintain a volatile mapping between snapshots (sequence numbers) and timestamps. Different internal keys with the same user key but different timestamps will be treated as different by compaction, thus a newer version will not hide older versions (with smaller timestamps) unless they are eligible for garbage collection. In contrast, taking a timestamped snapshot at a certain sequence number and timestamp prevents all the keys visible in this snapshot from been dropped by compaction. Here, visible means (seq < snapshot and most recent). The timestamped snapshot supports the semantics of reading at an exact point in time. Timestamped snapshots can also be used with user-defined timestamp. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9879 Test Plan: ``` make check TEST_TMPDIR=/dev/shm make crash_test_with_txn ``` Reviewed By: siying Differential Revision: D35783919 Pulled By: riversand963 fbshipit-source-id: 586ad905e169189e19d3bfc0cb0177a7239d1bd4
2 years ago
PreReleaseCallback* _pre_release_callback = nullptr,
PostMemTableCallback* _post_memtable_callback = nullptr)
: batch(_batch),
sync(write_options.sync),
no_slowdown(write_options.no_slowdown),
disable_wal(write_options.disableWAL),
Rate-limit automatic WAL flush after each user write (#9607) Summary: **Context:** WAL flush is currently not rate-limited by `Options::rate_limiter`. This PR is to provide rate-limiting to auto WAL flush, the one that automatically happen after each user write operation (i.e, `Options::manual_wal_flush == false`), by adding `WriteOptions::rate_limiter_options`. Note that we are NOT rate-limiting WAL flush that do NOT automatically happen after each user write, such as `Options::manual_wal_flush == true + manual FlushWAL()` (rate-limiting multiple WAL flushes), for the benefits of: - being consistent with [ReadOptions::rate_limiter_priority](https://github.com/facebook/rocksdb/blob/7.0.fb/include/rocksdb/options.h#L515) - being able to turn off some WAL flush's rate-limiting but not all (e.g, turn off specific the WAL flush of a critical user write like a service's heartbeat) `WriteOptions::rate_limiter_options` only accept `Env::IO_USER` and `Env::IO_TOTAL` currently due to an implementation constraint. - The constraint is that we currently queue parallel writes (including WAL writes) based on FIFO policy which does not factor rate limiter priority into this layer's scheduling. If we allow lower priorities such as `Env::IO_HIGH/MID/LOW` and such writes specified with lower priorities occurs before ones specified with higher priorities (even just by a tiny bit in arrival time), the former would have blocked the latter, leading to a "priority inversion" issue and contradictory to what we promise for rate-limiting priority. Therefore we only allow `Env::IO_USER` and `Env::IO_TOTAL` right now before improving that scheduling. A pre-requisite to this feature is to support operation-level rate limiting in `WritableFileWriter`, which is also included in this PR. **Summary:** - Renamed test suite `DBRateLimiterTest to DBRateLimiterOnReadTest` for adding a new test suite - Accept `rate_limiter_priority` in `WritableFileWriter`'s private and public write functions - Passed `WriteOptions::rate_limiter_options` to `WritableFileWriter` in the path of automatic WAL flush. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9607 Test Plan: - Added new unit test to verify existing flush/compaction rate-limiting does not break, since `DBTest, RateLimitingTest` is disabled and current db-level rate-limiting tests focus on read only (e.g, `db_rate_limiter_test`, `DBTest2, RateLimitedCompactionReads`). - Added new unit test `DBRateLimiterOnWriteWALTest, AutoWalFlush` - `strace -ftt -e trace=write ./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -rate_limit_auto_wal_flush=1 -rate_limiter_bytes_per_sec=15 -rate_limiter_refill_period_us=1000000 -write_buffer_size=100000000 -disable_auto_compactions=1 -num=100` - verified that WAL flush(i.e, system-call _write_) were chunked into 15 bytes and each _write_ was roughly 1 second apart - verified the chunking disappeared when `-rate_limit_auto_wal_flush=0` - crash test: `python3 tools/db_crashtest.py blackbox --disable_wal=0 --rate_limit_auto_wal_flush=1 --rate_limiter_bytes_per_sec=10485760 --interval=10` killed as normal **Benchmarked on flush/compaction to ensure no performance regression:** - compaction with rate-limiting (see table 1, avg over 1280-run): pre-change: **915635 micros/op**; post-change: **907350 micros/op (improved by 0.106%)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f compact_bmk_output.txt compact_bmk_output_2.txt dont_care_output.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench --benchmarks=fillrandom -db=$TEST_TMPDIR -disable_auto_compactions=1 -write_buffer_size=6710886 > dont_care_output.txt && ./db_bench --benchmarks=compact -use_existing_db=1 -db=$TEST_TMPDIR -level0_file_num_compaction_trigger=1 -rate_limiter_bytes_per_sec=100000000 | egrep 'compact' done > compact_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' compact_bmk_output.txt >> compact_bmk_output_2.txt done ``` - compaction w/o rate-limiting (see table 2, avg over 640-run): pre-change: **822197 micros/op**; post-change: **823148 micros/op (regressed by 0.12%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` - flush with rate-limiting (see table 3, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench ): pre-change: **745752 micros/op**; post-change: **745331 micros/op (regressed by 0.06 %)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f flush_bmk_output.txt flush_bmk_output_2.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench -db=$TEST_TMPDIR -write_buffer_size=1048576000 -num=1000000 -rate_limiter_bytes_per_sec=100000000 -benchmarks=fillseq,flush | egrep 'flush' done > flush_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' flush_bmk_output.txt >> flush_bmk_output_2.txt done ``` - flush w/o rate-limiting (see table 4, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench): pre-change: **487512 micros/op**, post-change: **485856 micors/ops (improved by 0.34%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` | table 1 - compact with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 896978 | 16046.9 | 901242 | 15670.9 | 0.475373978 20 | 893718 | 15813 | 886505 | 17544.7 | -0.8070778478 40 | 900426 | 23882.2 | 894958 | 15104.5 | -0.6072681153 80 | 906635 | 21761.5 | 903332 | 23948.3 | -0.3643141948 160 | 898632 | 21098.9 | 907583 | 21145 | 0.9960695813 3.20E+02 | 905252 | 22785.5 | 908106 | 25325.5 | 0.3152713278 6.40E+02 | 905213 | 23598.6 | 906741 | 21370.5 | 0.1688000504 **1.28E+03** | **908316** | **23533.1** | **907350** | **24626.8** | **-0.1063506533** average over #-run | 901896.25 | 21064.9625 | 901977.125 | 20592.025 | 0.008967217682 | table 2 - compact w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 811211 | 26996.7 | 807586 | 28456.4 | -0.4468627768 20 | 815465 | 14803.7 | 814608 | 28719.7 | -0.105093413 40 | 809203 | 26187.1 | 797835 | 25492.1 | -1.404839082 80 | 822088 | 28765.3 | 822192 | 32840.4 | 0.01265071379 160 | 821719 | 36344.7 | 821664 | 29544.9 | -0.006693285661 3.20E+02 | 820921 | 27756.4 | 821403 | 28347.7 | 0.05871454135 **6.40E+02** | **822197** | **28960.6** | **823148** | **30055.1** | **0.1156657103** average over #-run | 8.18E+05 | 2.71E+04 | 8.15E+05 | 2.91E+04 | -0.25 | table 3 - flush with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 741721 | 11770.8 | 740345 | 5949.76 | -0.1855144994 20 | 735169 | 3561.83 | 743199 | 9755.77 | 1.09226586 40 | 743368 | 8891.03 | 742102 | 8683.22 | -0.1703059588 80 | 742129 | 8148.51 | 743417 | 9631.58| 0.1735547324 160 | 749045 | 9757.21 | 746256 | 9191.86 | -0.3723407806 **3.20E+02** | **745752** | **9819.65** | **745331** | **9840.62** | **-0.0564530836** 6.40E+02 | 749006 | 11080.5 | 748173 | 10578.7 | -0.1112140624 average over #-run | 743741.4286 | 9004.218571 | 744117.5714 | 9090.215714 | 0.05057441238 | table 4 - flush w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 477283 | 24719.6 | 473864 | 12379 | -0.7163464863 20 | 486743 | 20175.2 | 502296 | 23931.3 | 3.195320734 40 | 482846 | 15309.2 | 489820 | 22259.5 | 1.444352858 80 | 491490 | 21883.1 | 490071 | 23085.7 | -0.2887139108 160 | 493347 | 28074.3 | 483609 | 21211.7 | -1.973864238 **3.20E+02** | **487512** | **21401.5** | **485856** | **22195.2** | **-0.3396839462** 6.40E+02 | 490307 | 25418.6 | 485435 | 22405.2 | -0.9936631539 average over #-run | 4.87E+05 | 2.24E+04 | 4.87E+05 | 2.11E+04 | 0.00E+00 Reviewed By: ajkr Differential Revision: D34442441 Pulled By: hx235 fbshipit-source-id: 4790f13e1e5c0a95ae1d1cc93ffcf69dc6e78bdd
2 years ago
rate_limiter_priority(write_options.rate_limiter_priority),
disable_memtable(_disable_memtable),
batch_cnt(_batch_cnt),
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
protection_bytes_per_key(_batch->GetProtectionBytesPerKey()),
pre_release_callback(_pre_release_callback),
Snapshots with user-specified timestamps (#9879) Summary: In RocksDB, keys are associated with (internal) sequence numbers which denote when the keys are written to the database. Sequence numbers in different RocksDB instances are unrelated, thus not comparable. It is nice if we can associate sequence numbers with their corresponding actual timestamps. One thing we can do is to support user-defined timestamp, which allows the applications to specify the format of custom timestamps and encode a timestamp with each key. More details can be found at https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp-%28Experimental%29. This PR provides a different but complementary approach. We can associate rocksdb snapshots (defined in https://github.com/facebook/rocksdb/blob/7.2.fb/include/rocksdb/snapshot.h#L20) with **user-specified** timestamps. Since a snapshot is essentially an object representing a sequence number, this PR establishes a bi-directional mapping between sequence numbers and timestamps. In the past, snapshots are usually taken by readers. The current super-version is grabbed, and a `rocksdb::Snapshot` object is created with the last published sequence number of the super-version. You can see that the reader actually has no good idea of what timestamp to assign to this snapshot, because by the time the `GetSnapshot()` is called, an arbitrarily long period of time may have already elapsed since the last write, which is when the last published sequence number is written. This observation motivates the creation of "timestamped" snapshots on the write path. Currently, this functionality is exposed only to the layer of `TransactionDB`. Application can tell RocksDB to create a snapshot when a transaction commits, effectively associating the last sequence number with a timestamp. It is also assumed that application will ensure any two snapshots with timestamps should satisfy the following: ``` snapshot1.seq < snapshot2.seq iff. snapshot1.ts < snapshot2.ts ``` If the application can guarantee that when a reader takes a timestamped snapshot, there is no active writes going on in the database, then we also allow the user to use a new API `TransactionDB::CreateTimestampedSnapshot()` to create a snapshot with associated timestamp. Code example ```cpp // Create a timestamped snapshot when committing transaction. txn->SetCommitTimestamp(100); txn->SetSnapshotOnNextOperation(); txn->Commit(); // A wrapper API for convenience Status Transaction::CommitAndTryCreateSnapshot( std::shared_ptr<TransactionNotifier> notifier, TxnTimestamp ts, std::shared_ptr<const Snapshot>* ret); // Create a timestamped snapshot if caller guarantees no concurrent writes std::pair<Status, std::shared_ptr<const Snapshot>> snapshot = txn_db->CreateTimestampedSnapshot(100); ``` The snapshots created in this way will be managed by RocksDB with ref-counting and potentially shared with other readers. We provide the following APIs for readers to retrieve a snapshot given a timestamp. ```cpp // Return the timestamped snapshot correponding to given timestamp. If ts is // kMaxTxnTimestamp, then we return the latest timestamped snapshot if present. // Othersise, we return the snapshot whose timestamp is equal to `ts`. If no // such snapshot exists, then we return null. std::shared_ptr<const Snapshot> TransactionDB::GetTimestampedSnapshot(TxnTimestamp ts) const; // Return the latest timestamped snapshot if present. std::shared_ptr<const Snapshot> TransactionDB::GetLatestTimestampedSnapshot() const; ``` We also provide two additional APIs for stats collection and reporting purposes. ```cpp Status TransactionDB::GetAllTimestampedSnapshots( std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; // Return timestamped snapshots whose timestamps fall in [ts_lb, ts_ub) and store them in `snapshots`. Status TransactionDB::GetTimestampedSnapshots( TxnTimestamp ts_lb, TxnTimestamp ts_ub, std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; ``` To prevent the number of timestamped snapshots from growing infinitely, we provide the following API to release timestamped snapshots whose timestamps are older than or equal to a given threshold. ```cpp void TransactionDB::ReleaseTimestampedSnapshotsOlderThan(TxnTimestamp ts); ``` Before shutdown, RocksDB will release all timestamped snapshots. Comparison with user-defined timestamp and how they can be combined: User-defined timestamp persists every key with a timestamp, while timestamped snapshots maintain a volatile mapping between snapshots (sequence numbers) and timestamps. Different internal keys with the same user key but different timestamps will be treated as different by compaction, thus a newer version will not hide older versions (with smaller timestamps) unless they are eligible for garbage collection. In contrast, taking a timestamped snapshot at a certain sequence number and timestamp prevents all the keys visible in this snapshot from been dropped by compaction. Here, visible means (seq < snapshot and most recent). The timestamped snapshot supports the semantics of reading at an exact point in time. Timestamped snapshots can also be used with user-defined timestamp. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9879 Test Plan: ``` make check TEST_TMPDIR=/dev/shm make crash_test_with_txn ``` Reviewed By: siying Differential Revision: D35783919 Pulled By: riversand963 fbshipit-source-id: 586ad905e169189e19d3bfc0cb0177a7239d1bd4
2 years ago
post_memtable_callback(_post_memtable_callback),
log_used(0),
log_ref(_log_ref),
callback(_callback),
made_waitable(false),
state(STATE_INIT),
write_group(nullptr),
sequence(kMaxSequenceNumber),
link_older(nullptr),
link_newer(nullptr) {}
~Writer() {
if (made_waitable) {
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
StateMutex().~mutex();
StateCV().~condition_variable();
}
status.PermitUncheckedError();
callback_status.PermitUncheckedError();
}
bool CheckCallback(DB* db) {
if (callback != nullptr) {
callback_status = callback->Callback(db);
}
return callback_status.ok();
}
void CreateMutex() {
if (!made_waitable) {
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
// Note that made_waitable is tracked separately from state
// transitions, because we can't atomically create the mutex and
// link into the list.
made_waitable = true;
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
new (&state_mutex_bytes) std::mutex;
new (&state_cv_bytes) std::condition_variable;
}
}
// returns the aggregate status of this Writer
Status FinalStatus() {
if (!status.ok()) {
// a non-ok memtable write status takes presidence
assert(callback == nullptr || callback_status.ok());
return status;
} else if (!callback_status.ok()) {
// if the callback failed then that is the status we want
// because a memtable insert should not have been attempted
assert(callback != nullptr);
assert(status.ok());
return callback_status;
} else {
// if there is no callback then we only care about
// the memtable insert status
assert(callback == nullptr || callback_status.ok());
return status;
}
}
bool CallbackFailed() {
return (callback != nullptr) && !callback_status.ok();
}
bool ShouldWriteToMemtable() {
return status.ok() && !CallbackFailed() && !disable_memtable;
}
bool ShouldWriteToWAL() {
return status.ok() && !CallbackFailed() && !disable_wal;
}
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
// No other mutexes may be acquired while holding StateMutex(), it is
// always last in the order
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
std::mutex& StateMutex() {
assert(made_waitable);
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
return *static_cast<std::mutex*>(static_cast<void*>(&state_mutex_bytes));
}
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
std::condition_variable& StateCV() {
assert(made_waitable);
return *static_cast<std::condition_variable*>(
static_cast<void*>(&state_cv_bytes));
}
};
struct AdaptationContext {
const char* name;
std::atomic<int32_t> value;
explicit AdaptationContext(const char* name0) : name(name0), value(0) {}
};
explicit WriteThread(const ImmutableDBOptions& db_options);
virtual ~WriteThread() = default;
// IMPORTANT: None of the methods in this class rely on the db mutex
// for correctness. All of the methods except JoinBatchGroup and
// EnterUnbatched may be called either with or without the db mutex held.
// Correctness is maintained by ensuring that only a single thread is
// a leader at a time.
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
// Registers w as ready to become part of a batch group, waits until the
// caller should perform some work, and returns the current state of the
// writer. If w has become the leader of a write batch group, returns
// STATE_GROUP_LEADER. If w has been made part of a sequential batch
// group and the leader has performed the write, returns STATE_DONE.
// If w has been made part of a parallel batch group and is responsible
// for updating the memtable, returns STATE_PARALLEL_MEMTABLE_WRITER.
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
//
// The db mutex SHOULD NOT be held when calling this function, because
// it will block.
//
// Writer* w: Writer to be executed as part of a batch group
void JoinBatchGroup(Writer* w);
// Constructs a write batch group led by leader, which should be a
// Writer passed to JoinBatchGroup on the current thread.
//
// Writer* leader: Writer that is STATE_GROUP_LEADER
// WriteGroup* write_group: Out-param of group members
// returns: Total batch group byte size
size_t EnterAsBatchGroupLeader(Writer* leader, WriteGroup* write_group);
// Unlinks the Writer-s in a batch group, wakes up the non-leaders,
// and wakes up the next leader (if any).
//
// WriteGroup* write_group: the write group
// Status status: Status of write operation
void ExitAsBatchGroupLeader(WriteGroup& write_group, Status& status);
// Exit batch group on behalf of batch group leader.
void ExitAsBatchGroupFollower(Writer* w);
// Constructs a write batch group led by leader from newest_memtable_writers_
// list. The leader should either write memtable for the whole group and
// call ExitAsMemTableWriter, or launch parallel memtable write through
// LaunchParallelMemTableWriters.
void EnterAsMemTableWriter(Writer* leader, WriteGroup* write_grup);
// Memtable writer group leader, or the last finished writer in a parallel
// write group, exit from the newest_memtable_writers_ list, and wake up
// the next leader if needed.
void ExitAsMemTableWriter(Writer* self, WriteGroup& write_group);
// Causes JoinBatchGroup to return STATE_PARALLEL_MEMTABLE_WRITER for all of
// the non-leader members of this write batch group. Sets Writer::sequence
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
// before waking them up.
//
// WriteGroup* write_group: Extra state used to coordinate the parallel add
void LaunchParallelMemTableWriters(WriteGroup* write_group);
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
// Reports the completion of w's batch to the parallel group leader, and
// waits for the rest of the parallel batch to complete. Returns true
// if this thread is the last to complete, and hence should advance
// the sequence number and then call EarlyExitParallelGroup, false if
// someone else has already taken responsibility for that.
bool CompleteParallelMemTableWriter(Writer* w);
// Waits for all preceding writers (unlocking mu while waiting), then
// registers w as the currently proceeding writer.
//
// Writer* w: A Writer not eligible for batching
// InstrumentedMutex* mu: The db mutex, to unlock while waiting
// REQUIRES: db mutex held
void EnterUnbatched(Writer* w, InstrumentedMutex* mu);
// Completes a Writer begun with EnterUnbatched, unblocking subsequent
// writers.
void ExitUnbatched(Writer* w);
// Wait for all parallel memtable writers to finish, in case pipelined
// write is enabled.
void WaitForMemTableWriters();
SequenceNumber UpdateLastSequence(SequenceNumber sequence) {
if (sequence > last_sequence_) {
last_sequence_ = sequence;
}
return last_sequence_;
}
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
// Insert a dummy writer at the tail of the write queue to indicate a write
// stall, and fail any writers in the queue with no_slowdown set to true
void BeginWriteStall();
// Remove the dummy writer and wake up waiting writers
void EndWriteStall();
private:
// See AwaitState.
const uint64_t max_yield_usec_;
const uint64_t slow_yield_usec_;
// Allow multiple writers write to memtable concurrently.
const bool allow_concurrent_memtable_write_;
// Enable pipelined write to WAL and memtable.
const bool enable_pipelined_write_;
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// The maximum limit of number of bytes that are written in a single batch
// of WAL or memtable write. It is followed when the leader write size
// is larger than 1/8 of this limit.
const uint64_t max_write_batch_group_size_bytes;
// Points to the newest pending writer. Only leader can remove
// elements, adding can be done lock-free by anybody.
std::atomic<Writer*> newest_writer_;
// Points to the newest pending memtable writer. Used only when pipelined
// write is enabled.
std::atomic<Writer*> newest_memtable_writer_;
// The last sequence that have been consumed by a writer. The sequence
// is not necessary visible to reads because the writer can be ongoing.
SequenceNumber last_sequence_;
// A dummy writer to indicate a write stall condition. This will be inserted
// at the tail of the writer queue by the leader, so newer writers can just
// check for this and bail
Writer write_stall_dummy_;
// Mutex and condvar for writers to block on a write stall. During a write
// stall, writers with no_slowdown set to false will wait on this rather
// on the writer queue
port::Mutex stall_mu_;
port::CondVar stall_cv_;
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
// Waits for w->state & goal_mask using w->StateMutex(). Returns
// the state that satisfies goal_mask.
uint8_t BlockingAwaitState(Writer* w, uint8_t goal_mask);
// Blocks until w->state & goal_mask, returning the state value
// that satisfied the predicate. Uses ctx to adaptively use
// std::this_thread::yield() to avoid mutex overheads. ctx should be
// a context-dependent static.
uint8_t AwaitState(Writer* w, uint8_t goal_mask, AdaptationContext* ctx);
// Set writer state and wake the writer up if it is waiting.
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
void SetState(Writer* w, uint8_t new_state);
// Links w into the newest_writer list. Return true if w was linked directly
// into the leader position. Safe to call from multiple threads without
// external locking.
bool LinkOne(Writer* w, std::atomic<Writer*>* newest_writer);
// Link write group into the newest_writer list as a whole, while keeping the
// order of the writers unchanged. Return true if the group was linked
// directly into the leader position.
bool LinkGroup(WriteGroup& write_group, std::atomic<Writer*>* newest_writer);
// Computes any missing link_newer links. Should not be called
// concurrently with itself.
void CreateMissingNewerLinks(Writer* head);
// Set the leader in write_group to completed state and remove it from the
// write group.
void CompleteLeader(WriteGroup& write_group);
// Set a follower in write_group to completed state and remove it from the
// write group.
void CompleteFollower(Writer* w, WriteGroup& write_group);
};
} // namespace ROCKSDB_NAMESPACE