2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2013-10-16 23:59:46 +02:00
|
|
|
// This source code is licensed under the BSD-style license found in the
|
|
|
|
// LICENSE file in the root directory of this source tree. An additional grant
|
|
|
|
// of patent rights can be found in the PATENTS file in the same directory.
|
|
|
|
//
|
2011-03-18 23:37:00 +01:00
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
|
|
//
|
|
|
|
// WriteBatch::rep_ :=
|
|
|
|
// sequence: fixed64
|
|
|
|
// count: fixed32
|
|
|
|
// data: record[count]
|
|
|
|
// record :=
|
2013-03-21 23:59:47 +01:00
|
|
|
// kTypeValue varstring varstring
|
2011-03-18 23:37:00 +01:00
|
|
|
// kTypeDeletion varstring
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
// kTypeSingleDeletion varstring
|
|
|
|
// kTypeMerge varstring varstring
|
2014-01-07 23:41:42 +01:00
|
|
|
// kTypeColumnFamilyValue varint32 varstring varstring
|
|
|
|
// kTypeColumnFamilyDeletion varint32 varstring varstring
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
// kTypeColumnFamilySingleDeletion varint32 varstring varstring
|
|
|
|
// kTypeColumnFamilyMerge varint32 varstring varstring
|
2011-03-18 23:37:00 +01:00
|
|
|
// varstring :=
|
|
|
|
// len: varint32
|
|
|
|
// data: uint8[len]
|
|
|
|
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/write_batch.h"
|
2015-07-11 05:15:45 +02:00
|
|
|
|
|
|
|
#include <stack>
|
|
|
|
#include <stdexcept>
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
#include <vector>
|
2015-07-11 05:15:45 +02:00
|
|
|
|
2014-08-19 00:19:17 +02:00
|
|
|
#include "db/column_family.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/db_impl.h"
|
|
|
|
#include "db/dbformat.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
2015-08-15 01:59:07 +02:00
|
|
|
#include "db/flush_scheduler.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "db/memtable.h"
|
2015-08-07 02:59:05 +02:00
|
|
|
#include "db/snapshot_impl.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "db/write_batch_internal.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "rocksdb/merge_operator.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "util/coding.h"
|
2015-03-03 19:59:36 +01:00
|
|
|
#include "util/perf_context_imp.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "util/statistics.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
namespace rocksdb {
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
// anon namespace for file-local types
|
|
|
|
namespace {
|
|
|
|
|
|
|
|
enum ContentFlags : uint32_t {
|
|
|
|
DEFERRED = 1,
|
|
|
|
HAS_PUT = 2,
|
|
|
|
HAS_DELETE = 4,
|
|
|
|
HAS_SINGLE_DELETE = 8,
|
|
|
|
HAS_MERGE = 16,
|
|
|
|
};
|
|
|
|
|
|
|
|
struct BatchContentClassifier : public WriteBatch::Handler {
|
|
|
|
uint32_t content_flags = 0;
|
|
|
|
|
|
|
|
Status PutCF(uint32_t, const Slice&, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_PUT;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DeleteCF(uint32_t, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_DELETE;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status SingleDeleteCF(uint32_t, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_SINGLE_DELETE;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status MergeCF(uint32_t, const Slice&, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_MERGE;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
} // anon namespace
|
|
|
|
|
2012-03-09 01:23:21 +01:00
|
|
|
|
2015-07-11 05:15:45 +02:00
|
|
|
struct SavePoint {
|
|
|
|
size_t size; // size of rep_
|
|
|
|
int count; // count of elements in rep_
|
2015-11-06 16:03:30 +01:00
|
|
|
uint32_t content_flags;
|
2015-07-11 05:15:45 +02:00
|
|
|
};
|
|
|
|
|
|
|
|
struct SavePoints {
|
|
|
|
std::stack<SavePoint> stack;
|
|
|
|
};
|
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
WriteBatch::WriteBatch(size_t reserved_bytes)
|
|
|
|
: save_points_(nullptr), content_flags_(0), rep_() {
|
2016-04-02 00:23:46 +02:00
|
|
|
rep_.reserve((reserved_bytes > WriteBatchInternal::kHeader) ?
|
2016-03-31 04:56:55 +02:00
|
|
|
reserved_bytes : WriteBatchInternal::kHeader);
|
2016-03-30 19:35:22 +02:00
|
|
|
rep_.resize(WriteBatchInternal::kHeader);
|
2015-11-06 16:03:30 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
WriteBatch::WriteBatch(const std::string& rep)
|
|
|
|
: save_points_(nullptr),
|
|
|
|
content_flags_(ContentFlags::DEFERRED),
|
|
|
|
rep_(rep) {}
|
|
|
|
|
|
|
|
WriteBatch::WriteBatch(const WriteBatch& src)
|
|
|
|
: save_points_(src.save_points_),
|
|
|
|
content_flags_(src.content_flags_.load(std::memory_order_relaxed)),
|
|
|
|
rep_(src.rep_) {}
|
|
|
|
|
|
|
|
WriteBatch::WriteBatch(WriteBatch&& src)
|
|
|
|
: save_points_(std::move(src.save_points_)),
|
|
|
|
content_flags_(src.content_flags_.load(std::memory_order_relaxed)),
|
|
|
|
rep_(std::move(src.rep_)) {}
|
|
|
|
|
|
|
|
WriteBatch& WriteBatch::operator=(const WriteBatch& src) {
|
|
|
|
if (&src != this) {
|
|
|
|
this->~WriteBatch();
|
|
|
|
new (this) WriteBatch(src);
|
|
|
|
}
|
|
|
|
return *this;
|
|
|
|
}
|
|
|
|
|
|
|
|
WriteBatch& WriteBatch::operator=(WriteBatch&& src) {
|
|
|
|
if (&src != this) {
|
|
|
|
this->~WriteBatch();
|
|
|
|
new (this) WriteBatch(std::move(src));
|
|
|
|
}
|
|
|
|
return *this;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-12-16 00:26:20 +01:00
|
|
|
WriteBatch::~WriteBatch() { delete save_points_; }
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
WriteBatch::Handler::~Handler() { }
|
|
|
|
|
2013-08-15 01:32:46 +02:00
|
|
|
void WriteBatch::Handler::LogData(const Slice& blob) {
|
|
|
|
// If the user has not specified something to do with blobs, then we ignore
|
|
|
|
// them.
|
|
|
|
}
|
|
|
|
|
2013-08-22 03:27:48 +02:00
|
|
|
bool WriteBatch::Handler::Continue() {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
void WriteBatch::Clear() {
|
|
|
|
rep_.clear();
|
2016-03-30 19:43:00 +02:00
|
|
|
rep_.resize(WriteBatchInternal::kHeader);
|
2015-07-11 05:15:45 +02:00
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
content_flags_.store(0, std::memory_order_relaxed);
|
|
|
|
|
2015-07-11 05:15:45 +02:00
|
|
|
if (save_points_ != nullptr) {
|
|
|
|
while (!save_points_->stack.empty()) {
|
|
|
|
save_points_->stack.pop();
|
|
|
|
}
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2013-06-26 19:50:58 +02:00
|
|
|
int WriteBatch::Count() const {
|
|
|
|
return WriteBatchInternal::Count(this);
|
|
|
|
}
|
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
uint32_t WriteBatch::ComputeContentFlags() const {
|
|
|
|
auto rv = content_flags_.load(std::memory_order_relaxed);
|
|
|
|
if ((rv & ContentFlags::DEFERRED) != 0) {
|
|
|
|
BatchContentClassifier classifier;
|
|
|
|
Iterate(&classifier);
|
|
|
|
rv = classifier.content_flags;
|
|
|
|
|
|
|
|
// this method is conceptually const, because it is performing a lazy
|
|
|
|
// computation that doesn't affect the abstract state of the batch.
|
|
|
|
// content_flags_ is marked mutable so that we can perform the
|
|
|
|
// following assignment
|
|
|
|
content_flags_.store(rv, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
return rv;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool WriteBatch::HasPut() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_PUT) != 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool WriteBatch::HasDelete() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_DELETE) != 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool WriteBatch::HasSingleDelete() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_SINGLE_DELETE) != 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool WriteBatch::HasMerge() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_MERGE) != 0;
|
|
|
|
}
|
|
|
|
|
2016-04-02 00:23:46 +02:00
|
|
|
bool ReadKeyFromWriteBatchEntry(Slice* input, Slice* key, bool cf_record) {
|
|
|
|
assert(input != nullptr && key != nullptr);
|
|
|
|
// Skip tag byte
|
|
|
|
input->remove_prefix(1);
|
|
|
|
|
|
|
|
if (cf_record) {
|
|
|
|
// Skip column_family bytes
|
|
|
|
uint32_t cf;
|
|
|
|
if (!GetVarint32(input, &cf)) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Extract key
|
|
|
|
return GetLengthPrefixedSlice(input, key);
|
|
|
|
}
|
|
|
|
|
2014-08-19 00:19:17 +02:00
|
|
|
Status ReadRecordFromWriteBatch(Slice* input, char* tag,
|
|
|
|
uint32_t* column_family, Slice* key,
|
|
|
|
Slice* value, Slice* blob) {
|
|
|
|
assert(key != nullptr && value != nullptr);
|
|
|
|
*tag = (*input)[0];
|
|
|
|
input->remove_prefix(1);
|
|
|
|
*column_family = 0; // default
|
|
|
|
switch (*tag) {
|
|
|
|
case kTypeColumnFamilyValue:
|
|
|
|
if (!GetVarint32(input, column_family)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Put");
|
|
|
|
}
|
|
|
|
// intentional fallthrough
|
|
|
|
case kTypeValue:
|
|
|
|
if (!GetLengthPrefixedSlice(input, key) ||
|
|
|
|
!GetLengthPrefixedSlice(input, value)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Put");
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
case kTypeColumnFamilyDeletion:
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
case kTypeColumnFamilySingleDeletion:
|
2014-08-19 00:19:17 +02:00
|
|
|
if (!GetVarint32(input, column_family)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Delete");
|
|
|
|
}
|
|
|
|
// intentional fallthrough
|
|
|
|
case kTypeDeletion:
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
case kTypeSingleDeletion:
|
2014-08-19 00:19:17 +02:00
|
|
|
if (!GetLengthPrefixedSlice(input, key)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Delete");
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
case kTypeColumnFamilyMerge:
|
|
|
|
if (!GetVarint32(input, column_family)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Merge");
|
|
|
|
}
|
|
|
|
// intentional fallthrough
|
|
|
|
case kTypeMerge:
|
|
|
|
if (!GetLengthPrefixedSlice(input, key) ||
|
|
|
|
!GetLengthPrefixedSlice(input, value)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Merge");
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
case kTypeLogData:
|
|
|
|
assert(blob != nullptr);
|
|
|
|
if (!GetLengthPrefixedSlice(input, blob)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Blob");
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
default:
|
|
|
|
return Status::Corruption("unknown WriteBatch tag");
|
|
|
|
}
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
Status WriteBatch::Iterate(Handler* handler) const {
|
|
|
|
Slice input(rep_);
|
2016-03-30 19:43:00 +02:00
|
|
|
if (input.size() < WriteBatchInternal::kHeader) {
|
2011-05-21 04:17:43 +02:00
|
|
|
return Status::Corruption("malformed WriteBatch (too small)");
|
|
|
|
}
|
|
|
|
|
2016-03-30 19:43:00 +02:00
|
|
|
input.remove_prefix(WriteBatchInternal::kHeader);
|
2013-08-15 01:32:46 +02:00
|
|
|
Slice key, value, blob;
|
2011-05-21 04:17:43 +02:00
|
|
|
int found = 0;
|
2014-02-26 02:30:54 +01:00
|
|
|
Status s;
|
|
|
|
while (s.ok() && !input.empty() && handler->Continue()) {
|
2014-08-19 00:19:17 +02:00
|
|
|
char tag = 0;
|
2014-01-30 00:26:43 +01:00
|
|
|
uint32_t column_family = 0; // default
|
2014-08-19 00:19:17 +02:00
|
|
|
|
|
|
|
s = ReadRecordFromWriteBatch(&input, &tag, &column_family, &key, &value,
|
|
|
|
&blob);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
switch (tag) {
|
2014-01-07 23:41:42 +01:00
|
|
|
case kTypeColumnFamilyValue:
|
2011-05-21 04:17:43 +02:00
|
|
|
case kTypeValue:
|
2015-11-06 16:03:30 +01:00
|
|
|
assert(content_flags_.load(std::memory_order_relaxed) &
|
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_PUT));
|
2014-08-19 00:19:17 +02:00
|
|
|
s = handler->PutCF(column_family, key, value);
|
|
|
|
found++;
|
2011-05-21 04:17:43 +02:00
|
|
|
break;
|
2014-01-07 23:41:42 +01:00
|
|
|
case kTypeColumnFamilyDeletion:
|
2011-05-21 04:17:43 +02:00
|
|
|
case kTypeDeletion:
|
2015-11-06 16:03:30 +01:00
|
|
|
assert(content_flags_.load(std::memory_order_relaxed) &
|
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_DELETE));
|
2014-08-19 00:19:17 +02:00
|
|
|
s = handler->DeleteCF(column_family, key);
|
|
|
|
found++;
|
2011-05-21 04:17:43 +02:00
|
|
|
break;
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
case kTypeColumnFamilySingleDeletion:
|
|
|
|
case kTypeSingleDeletion:
|
2015-11-06 16:03:30 +01:00
|
|
|
assert(content_flags_.load(std::memory_order_relaxed) &
|
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_SINGLE_DELETE));
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
s = handler->SingleDeleteCF(column_family, key);
|
|
|
|
found++;
|
|
|
|
break;
|
2014-01-07 23:41:42 +01:00
|
|
|
case kTypeColumnFamilyMerge:
|
2013-03-21 23:59:47 +01:00
|
|
|
case kTypeMerge:
|
2015-11-06 16:03:30 +01:00
|
|
|
assert(content_flags_.load(std::memory_order_relaxed) &
|
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_MERGE));
|
2014-08-19 00:19:17 +02:00
|
|
|
s = handler->MergeCF(column_family, key, value);
|
|
|
|
found++;
|
2013-03-21 23:59:47 +01:00
|
|
|
break;
|
2013-08-15 01:32:46 +02:00
|
|
|
case kTypeLogData:
|
2014-08-19 00:19:17 +02:00
|
|
|
handler->LogData(blob);
|
2013-08-15 01:32:46 +02:00
|
|
|
break;
|
2011-05-21 04:17:43 +02:00
|
|
|
default:
|
|
|
|
return Status::Corruption("unknown WriteBatch tag");
|
|
|
|
}
|
|
|
|
}
|
2014-02-26 02:30:54 +01:00
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
if (found != WriteBatchInternal::Count(this)) {
|
2011-05-21 04:17:43 +02:00
|
|
|
return Status::Corruption("WriteBatch has wrong count");
|
|
|
|
} else {
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
int WriteBatchInternal::Count(const WriteBatch* b) {
|
|
|
|
return DecodeFixed32(b->rep_.data() + 8);
|
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatchInternal::SetCount(WriteBatch* b, int n) {
|
|
|
|
EncodeFixed32(&b->rep_[8], n);
|
|
|
|
}
|
|
|
|
|
|
|
|
SequenceNumber WriteBatchInternal::Sequence(const WriteBatch* b) {
|
|
|
|
return SequenceNumber(DecodeFixed64(b->rep_.data()));
|
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatchInternal::SetSequence(WriteBatch* b, SequenceNumber seq) {
|
|
|
|
EncodeFixed64(&b->rep_[0], seq);
|
|
|
|
}
|
|
|
|
|
2016-04-02 00:23:46 +02:00
|
|
|
size_t WriteBatchInternal::GetFirstOffset(WriteBatch* b) {
|
|
|
|
return WriteBatchInternal::kHeader;
|
2016-03-31 04:56:55 +02:00
|
|
|
}
|
2015-07-11 05:15:45 +02:00
|
|
|
|
2014-04-22 20:27:33 +02:00
|
|
|
void WriteBatchInternal::Put(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& key, const Slice& value) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeValue));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyValue));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, key);
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, value);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(
|
|
|
|
b->content_flags_.load(std::memory_order_relaxed) | ContentFlags::HAS_PUT,
|
|
|
|
std::memory_order_relaxed);
|
2014-04-22 20:27:33 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatch::Put(ColumnFamilyHandle* column_family, const Slice& key,
|
|
|
|
const Slice& value) {
|
|
|
|
WriteBatchInternal::Put(this, GetColumnFamilyID(column_family), key, value);
|
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatchInternal::Put(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& key, const SliceParts& value) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
2014-01-07 23:41:42 +01:00
|
|
|
if (column_family_id == 0) {
|
2014-04-22 20:27:33 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeValue));
|
2014-01-07 23:41:42 +01:00
|
|
|
} else {
|
2014-04-22 20:27:33 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyValue));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
2014-01-07 23:41:42 +01:00
|
|
|
}
|
2014-04-22 20:27:33 +02:00
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, key);
|
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, value);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(
|
|
|
|
b->content_flags_.load(std::memory_order_relaxed) | ContentFlags::HAS_PUT,
|
|
|
|
std::memory_order_relaxed);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2014-03-14 19:26:13 +01:00
|
|
|
void WriteBatch::Put(ColumnFamilyHandle* column_family, const SliceParts& key,
|
2014-01-07 23:41:42 +01:00
|
|
|
const SliceParts& value) {
|
2014-04-22 20:27:33 +02:00
|
|
|
WriteBatchInternal::Put(this, GetColumnFamilyID(column_family), key, value);
|
|
|
|
}
|
2014-03-14 19:26:13 +01:00
|
|
|
|
2014-04-22 20:27:33 +02:00
|
|
|
void WriteBatchInternal::Delete(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& key) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
2014-01-07 23:41:42 +01:00
|
|
|
if (column_family_id == 0) {
|
2014-04-22 20:27:33 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeDeletion));
|
2014-01-07 23:41:42 +01:00
|
|
|
} else {
|
2014-04-22 20:27:33 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyDeletion));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
2014-01-07 23:41:42 +01:00
|
|
|
}
|
2014-04-22 20:27:33 +02:00
|
|
|
PutLengthPrefixedSlice(&b->rep_, key);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_DELETE,
|
|
|
|
std::memory_order_relaxed);
|
2013-11-07 21:37:58 +01:00
|
|
|
}
|
|
|
|
|
2014-03-14 19:26:13 +01:00
|
|
|
void WriteBatch::Delete(ColumnFamilyHandle* column_family, const Slice& key) {
|
2014-04-22 20:27:33 +02:00
|
|
|
WriteBatchInternal::Delete(this, GetColumnFamilyID(column_family), key);
|
2014-07-10 18:31:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatchInternal::Delete(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& key) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeDeletion));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyDeletion));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, key);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_DELETE,
|
|
|
|
std::memory_order_relaxed);
|
2014-07-10 18:31:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatch::Delete(ColumnFamilyHandle* column_family,
|
|
|
|
const SliceParts& key) {
|
|
|
|
WriteBatchInternal::Delete(this, GetColumnFamilyID(column_family), key);
|
2014-04-22 20:27:33 +02:00
|
|
|
}
|
2014-03-14 19:26:13 +01:00
|
|
|
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
void WriteBatchInternal::SingleDelete(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& key) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeSingleDeletion));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilySingleDeletion));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, key);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_SINGLE_DELETE,
|
|
|
|
std::memory_order_relaxed);
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatch::SingleDelete(ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& key) {
|
|
|
|
WriteBatchInternal::SingleDelete(this, GetColumnFamilyID(column_family), key);
|
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatchInternal::SingleDelete(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& key) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeSingleDeletion));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilySingleDeletion));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, key);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_SINGLE_DELETE,
|
|
|
|
std::memory_order_relaxed);
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatch::SingleDelete(ColumnFamilyHandle* column_family,
|
|
|
|
const SliceParts& key) {
|
|
|
|
WriteBatchInternal::SingleDelete(this, GetColumnFamilyID(column_family), key);
|
|
|
|
}
|
|
|
|
|
2014-04-22 20:27:33 +02:00
|
|
|
void WriteBatchInternal::Merge(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& key, const Slice& value) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
2014-01-07 23:41:42 +01:00
|
|
|
if (column_family_id == 0) {
|
2014-04-22 20:27:33 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeMerge));
|
2014-01-07 23:41:42 +01:00
|
|
|
} else {
|
2014-04-22 20:27:33 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyMerge));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
2014-01-07 23:41:42 +01:00
|
|
|
}
|
2014-04-22 20:27:33 +02:00
|
|
|
PutLengthPrefixedSlice(&b->rep_, key);
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, value);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_MERGE,
|
|
|
|
std::memory_order_relaxed);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2014-03-14 19:26:13 +01:00
|
|
|
void WriteBatch::Merge(ColumnFamilyHandle* column_family, const Slice& key,
|
2014-01-07 23:41:42 +01:00
|
|
|
const Slice& value) {
|
2014-04-22 20:27:33 +02:00
|
|
|
WriteBatchInternal::Merge(this, GetColumnFamilyID(column_family), key, value);
|
2013-03-21 23:59:47 +01:00
|
|
|
}
|
|
|
|
|
2015-05-28 01:59:22 +02:00
|
|
|
void WriteBatchInternal::Merge(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& key,
|
|
|
|
const SliceParts& value) {
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeMerge));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyMerge));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, key);
|
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, value);
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_MERGE,
|
|
|
|
std::memory_order_relaxed);
|
2015-05-28 01:59:22 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void WriteBatch::Merge(ColumnFamilyHandle* column_family,
|
|
|
|
const SliceParts& key,
|
|
|
|
const SliceParts& value) {
|
|
|
|
WriteBatchInternal::Merge(this, GetColumnFamilyID(column_family),
|
|
|
|
key, value);
|
|
|
|
}
|
|
|
|
|
2013-08-15 01:32:46 +02:00
|
|
|
void WriteBatch::PutLogData(const Slice& blob) {
|
|
|
|
rep_.push_back(static_cast<char>(kTypeLogData));
|
|
|
|
PutLengthPrefixedSlice(&rep_, blob);
|
|
|
|
}
|
2013-03-21 23:59:47 +01:00
|
|
|
|
2015-07-11 05:15:45 +02:00
|
|
|
void WriteBatch::SetSavePoint() {
|
|
|
|
if (save_points_ == nullptr) {
|
|
|
|
save_points_ = new SavePoints();
|
|
|
|
}
|
|
|
|
// Record length and count of current batch of writes.
|
2015-11-06 16:03:30 +01:00
|
|
|
save_points_->stack.push(SavePoint{
|
|
|
|
GetDataSize(), Count(), content_flags_.load(std::memory_order_relaxed)});
|
2015-07-11 05:15:45 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
Status WriteBatch::RollbackToSavePoint() {
|
|
|
|
if (save_points_ == nullptr || save_points_->stack.size() == 0) {
|
|
|
|
return Status::NotFound();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Pop the most recent savepoint off the stack
|
|
|
|
SavePoint savepoint = save_points_->stack.top();
|
|
|
|
save_points_->stack.pop();
|
|
|
|
|
|
|
|
assert(savepoint.size <= rep_.size());
|
2015-11-06 16:03:30 +01:00
|
|
|
assert(savepoint.count <= Count());
|
2015-07-11 05:15:45 +02:00
|
|
|
|
|
|
|
if (savepoint.size == rep_.size()) {
|
|
|
|
// No changes to rollback
|
|
|
|
} else if (savepoint.size == 0) {
|
|
|
|
// Rollback everything
|
|
|
|
Clear();
|
|
|
|
} else {
|
|
|
|
rep_.resize(savepoint.size);
|
|
|
|
WriteBatchInternal::SetCount(this, savepoint.count);
|
2015-11-06 16:03:30 +01:00
|
|
|
content_flags_.store(savepoint.content_flags, std::memory_order_relaxed);
|
2015-07-11 05:15:45 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
namespace {
|
|
|
|
class MemTableInserter : public WriteBatch::Handler {
|
|
|
|
public:
|
|
|
|
SequenceNumber 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
2015-08-15 01:59:07 +02:00
|
|
|
ColumnFamilyMemTables* const cf_mems_;
|
|
|
|
FlushScheduler* const flush_scheduler_;
|
|
|
|
const bool ignore_missing_column_families_;
|
|
|
|
const uint64_t log_number_;
|
2013-07-13 01:56:52 +02:00
|
|
|
DBImpl* db_;
|
2014-02-06 01:02:48 +01:00
|
|
|
const bool dont_filter_deletes_;
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
const bool concurrent_memtable_writes_;
|
2011-05-21 04:17:43 +02:00
|
|
|
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
// cf_mems should not be shared with concurrent inserters
|
2014-01-28 20:05:04 +01:00
|
|
|
MemTableInserter(SequenceNumber sequence, ColumnFamilyMemTables* cf_mems,
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
FlushScheduler* flush_scheduler,
|
2014-09-02 22:29:05 +02:00
|
|
|
bool ignore_missing_column_families, uint64_t log_number,
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
DB* db, const bool dont_filter_deletes,
|
|
|
|
bool concurrent_memtable_writes)
|
2014-01-28 20:05:04 +01:00
|
|
|
: sequence_(sequence),
|
|
|
|
cf_mems_(cf_mems),
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
flush_scheduler_(flush_scheduler),
|
2014-09-02 22:29:05 +02:00
|
|
|
ignore_missing_column_families_(ignore_missing_column_families),
|
2014-02-06 01:02:48 +01:00
|
|
|
log_number_(log_number),
|
2014-01-28 20:05:04 +01:00
|
|
|
db_(reinterpret_cast<DBImpl*>(db)),
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
dont_filter_deletes_(dont_filter_deletes),
|
|
|
|
concurrent_memtable_writes_(concurrent_memtable_writes) {
|
|
|
|
assert(cf_mems_);
|
2014-02-06 01:02:48 +01:00
|
|
|
if (!dont_filter_deletes_) {
|
2014-01-28 20:05:04 +01:00
|
|
|
assert(db_);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-03-03 23:30:36 +01:00
|
|
|
bool SeekToColumnFamily(uint32_t column_family_id, Status* s) {
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
// If we are in a concurrent mode, it is the caller's responsibility
|
|
|
|
// to clone the original ColumnFamilyMemTables so that each thread
|
|
|
|
// has its own instance. Otherwise, it must be guaranteed that there
|
|
|
|
// is no concurrent access
|
2014-02-06 01:02:48 +01:00
|
|
|
bool found = cf_mems_->Seek(column_family_id);
|
2014-09-02 22:29:05 +02:00
|
|
|
if (!found) {
|
|
|
|
if (ignore_missing_column_families_) {
|
|
|
|
*s = Status::OK();
|
|
|
|
} else {
|
|
|
|
*s = Status::InvalidArgument(
|
|
|
|
"Invalid column family specified in write batch");
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
if (log_number_ != 0 && log_number_ < cf_mems_->GetLogNumber()) {
|
|
|
|
// This is true only in recovery environment (log_number_ is always 0 in
|
|
|
|
// non-recovery, regular write code-path)
|
2014-03-03 23:30:36 +01:00
|
|
|
// * If log_number_ < cf_mems_->GetLogNumber(), this means that column
|
|
|
|
// family already contains updates from this log. We can't apply updates
|
|
|
|
// twice because of update-in-place or merge workloads -- ignore the
|
|
|
|
// update
|
|
|
|
*s = Status::OK();
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
2015-11-06 16:29:10 +01:00
|
|
|
|
2014-03-03 23:30:36 +01:00
|
|
|
virtual Status PutCF(uint32_t column_family_id, const Slice& key,
|
2015-02-26 20:28:41 +01:00
|
|
|
const Slice& value) override {
|
2014-03-03 23:30:36 +01:00
|
|
|
Status seek_status;
|
|
|
|
if (!SeekToColumnFamily(column_family_id, &seek_status)) {
|
|
|
|
++sequence_;
|
|
|
|
return seek_status;
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
2014-02-06 01:02:48 +01:00
|
|
|
MemTable* mem = cf_mems_->GetMemTable();
|
2014-09-09 03:46:52 +02:00
|
|
|
auto* moptions = mem->GetMemTableOptions();
|
|
|
|
if (!moptions->inplace_update_support) {
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
mem->Add(sequence_, kTypeValue, key, value, concurrent_memtable_writes_);
|
2014-09-09 03:46:52 +02:00
|
|
|
} else if (moptions->inplace_callback == 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
2015-08-15 01:59:07 +02:00
|
|
|
assert(!concurrent_memtable_writes_);
|
2014-02-07 00:42:16 +01:00
|
|
|
mem->Update(sequence_, key, value);
|
2014-10-27 20:10:13 +01:00
|
|
|
RecordTick(moptions->statistics, NUMBER_KEYS_UPDATED);
|
In-place updates for equal keys and similar sized values
Summary:
Currently for each put, a fresh memory is allocated, and a new entry is added to the memtable with a new sequence number irrespective of whether the key already exists in the memtable. This diff is an attempt to update the value inplace for existing keys. It currently handles a very simple case:
1. Key already exists in the current memtable. Does not inplace update values in immutable memtable or snapshot
2. Latest value type is a 'put' ie kTypeValue
3. New value size is less than existing value, to avoid reallocating memory
TODO: For a put of an existing key, deallocate memory take by values, for other value types till a kTypeValue is found, ie. remove kTypeMerge.
TODO: Update the transaction log, to allow consistent reload of the memtable.
Test Plan: Added a unit test verifying the inplace update. But some other unit tests broken due to invalid sequence number checks. WIll fix them next.
Reviewers: xinyaohu, sumeet, haobo, dhruba
CC: leveldb
Differential Revision: https://reviews.facebook.net/D12423
Automatic commit by arc
2013-08-19 23:12:47 +02:00
|
|
|
} else {
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
assert(!concurrent_memtable_writes_);
|
2014-09-09 03:46:52 +02:00
|
|
|
if (mem->UpdateCallback(sequence_, key, value)) {
|
2014-01-14 16:55:16 +01:00
|
|
|
} else {
|
Allow callback to change size of existing value. Change return type of the callback function to an enum status to handle 3 cases.
Summary:
This diff fixes 2 hacks:
* The callback function can modify the existing value inplace, if the merged value fits within the existing buffer size. But currently the existing buffer size is not being modified. Now the callback recieves a int* allowing the size to be modified. Since size is encoded as a varint in the internal key for memtable. It might happen that the entire value might have be copied to the new location if the new size varint is smaller than the existing size varint.
* The callback function has 3 functionalities
1. Modify existing buffer inplace, and update size correspondingly. Now to indicate that, Returns 1.
2. Generate a new buffer indicating merged value. Returns 2.
3. Fails to do either of above, based on whatever application logic. Returns 0.
Test Plan: Just make all for now. I'm adding another unit test to test each scenario.
Reviewers: dhruba, haobo
Reviewed By: haobo
CC: leveldb, sdong, kailiu, xinyaohu, sumeet, danguo
Differential Revision: https://reviews.facebook.net/D15195
2014-01-17 00:11:19 +01:00
|
|
|
// key not found in memtable. Do sst get, update, add
|
2014-01-14 16:55:16 +01:00
|
|
|
SnapshotImpl read_from_snapshot;
|
|
|
|
read_from_snapshot.number_ = sequence_;
|
|
|
|
ReadOptions ropts;
|
|
|
|
ropts.snapshot = &read_from_snapshot;
|
|
|
|
|
|
|
|
std::string prev_value;
|
|
|
|
std::string merged_value;
|
2014-02-11 02:04:44 +01:00
|
|
|
|
|
|
|
auto cf_handle = cf_mems_->GetColumnFamilyHandle();
|
|
|
|
if (cf_handle == nullptr) {
|
|
|
|
cf_handle = db_->DefaultColumnFamily();
|
|
|
|
}
|
|
|
|
Status s = db_->Get(ropts, cf_handle, key, &prev_value);
|
|
|
|
|
2014-01-14 16:55:16 +01:00
|
|
|
char* prev_buffer = const_cast<char*>(prev_value.c_str());
|
2014-11-11 22:47:22 +01:00
|
|
|
uint32_t prev_size = static_cast<uint32_t>(prev_value.size());
|
2014-09-09 03:46:52 +02:00
|
|
|
auto status = moptions->inplace_callback(s.ok() ? prev_buffer : nullptr,
|
|
|
|
s.ok() ? &prev_size : nullptr,
|
|
|
|
value, &merged_value);
|
Allow callback to change size of existing value. Change return type of the callback function to an enum status to handle 3 cases.
Summary:
This diff fixes 2 hacks:
* The callback function can modify the existing value inplace, if the merged value fits within the existing buffer size. But currently the existing buffer size is not being modified. Now the callback recieves a int* allowing the size to be modified. Since size is encoded as a varint in the internal key for memtable. It might happen that the entire value might have be copied to the new location if the new size varint is smaller than the existing size varint.
* The callback function has 3 functionalities
1. Modify existing buffer inplace, and update size correspondingly. Now to indicate that, Returns 1.
2. Generate a new buffer indicating merged value. Returns 2.
3. Fails to do either of above, based on whatever application logic. Returns 0.
Test Plan: Just make all for now. I'm adding another unit test to test each scenario.
Reviewers: dhruba, haobo
Reviewed By: haobo
CC: leveldb, sdong, kailiu, xinyaohu, sumeet, danguo
Differential Revision: https://reviews.facebook.net/D15195
2014-01-17 00:11:19 +01:00
|
|
|
if (status == UpdateStatus::UPDATED_INPLACE) {
|
2014-01-14 16:55:16 +01:00
|
|
|
// prev_value is updated in-place with final value.
|
2014-02-07 00:42:16 +01:00
|
|
|
mem->Add(sequence_, kTypeValue, key, Slice(prev_buffer, prev_size));
|
2014-10-27 20:10:13 +01:00
|
|
|
RecordTick(moptions->statistics, NUMBER_KEYS_WRITTEN);
|
Allow callback to change size of existing value. Change return type of the callback function to an enum status to handle 3 cases.
Summary:
This diff fixes 2 hacks:
* The callback function can modify the existing value inplace, if the merged value fits within the existing buffer size. But currently the existing buffer size is not being modified. Now the callback recieves a int* allowing the size to be modified. Since size is encoded as a varint in the internal key for memtable. It might happen that the entire value might have be copied to the new location if the new size varint is smaller than the existing size varint.
* The callback function has 3 functionalities
1. Modify existing buffer inplace, and update size correspondingly. Now to indicate that, Returns 1.
2. Generate a new buffer indicating merged value. Returns 2.
3. Fails to do either of above, based on whatever application logic. Returns 0.
Test Plan: Just make all for now. I'm adding another unit test to test each scenario.
Reviewers: dhruba, haobo
Reviewed By: haobo
CC: leveldb, sdong, kailiu, xinyaohu, sumeet, danguo
Differential Revision: https://reviews.facebook.net/D15195
2014-01-17 00:11:19 +01:00
|
|
|
} else if (status == UpdateStatus::UPDATED) {
|
|
|
|
// merged_value contains the final value.
|
2014-02-07 00:42:16 +01:00
|
|
|
mem->Add(sequence_, kTypeValue, key, Slice(merged_value));
|
2014-10-27 20:10:13 +01:00
|
|
|
RecordTick(moptions->statistics, NUMBER_KEYS_WRITTEN);
|
2014-01-14 16:55:16 +01:00
|
|
|
}
|
|
|
|
}
|
In-place updates for equal keys and similar sized values
Summary:
Currently for each put, a fresh memory is allocated, and a new entry is added to the memtable with a new sequence number irrespective of whether the key already exists in the memtable. This diff is an attempt to update the value inplace for existing keys. It currently handles a very simple case:
1. Key already exists in the current memtable. Does not inplace update values in immutable memtable or snapshot
2. Latest value type is a 'put' ie kTypeValue
3. New value size is less than existing value, to avoid reallocating memory
TODO: For a put of an existing key, deallocate memory take by values, for other value types till a kTypeValue is found, ie. remove kTypeMerge.
TODO: Update the transaction log, to allow consistent reload of the memtable.
Test Plan: Added a unit test verifying the inplace update. But some other unit tests broken due to invalid sequence number checks. WIll fix them next.
Reviewers: xinyaohu, sumeet, haobo, dhruba
CC: leveldb
Differential Revision: https://reviews.facebook.net/D12423
Automatic commit by arc
2013-08-19 23:12:47 +02:00
|
|
|
}
|
Allow callback to change size of existing value. Change return type of the callback function to an enum status to handle 3 cases.
Summary:
This diff fixes 2 hacks:
* The callback function can modify the existing value inplace, if the merged value fits within the existing buffer size. But currently the existing buffer size is not being modified. Now the callback recieves a int* allowing the size to be modified. Since size is encoded as a varint in the internal key for memtable. It might happen that the entire value might have be copied to the new location if the new size varint is smaller than the existing size varint.
* The callback function has 3 functionalities
1. Modify existing buffer inplace, and update size correspondingly. Now to indicate that, Returns 1.
2. Generate a new buffer indicating merged value. Returns 2.
3. Fails to do either of above, based on whatever application logic. Returns 0.
Test Plan: Just make all for now. I'm adding another unit test to test each scenario.
Reviewers: dhruba, haobo
Reviewed By: haobo
CC: leveldb, sdong, kailiu, xinyaohu, sumeet, danguo
Differential Revision: https://reviews.facebook.net/D15195
2014-01-17 00:11:19 +01:00
|
|
|
// Since all Puts are logged in trasaction logs (if enabled), always bump
|
|
|
|
// sequence number. Even if the update eventually fails and does not result
|
|
|
|
// in memtable add/update.
|
2011-05-21 04:17:43 +02:00
|
|
|
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
2015-08-15 01:59:07 +02:00
|
|
|
CheckMemtableFull();
|
2014-02-26 02:30:54 +01:00
|
|
|
return Status::OK();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
2014-01-14 16:55:16 +01:00
|
|
|
|
2015-11-06 16:29:10 +01:00
|
|
|
Status DeleteImpl(uint32_t column_family_id, const Slice& key,
|
|
|
|
ValueType delete_type) {
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
Status seek_status;
|
|
|
|
if (!SeekToColumnFamily(column_family_id, &seek_status)) {
|
|
|
|
++sequence_;
|
|
|
|
return seek_status;
|
|
|
|
}
|
|
|
|
MemTable* mem = cf_mems_->GetMemTable();
|
|
|
|
auto* moptions = mem->GetMemTableOptions();
|
|
|
|
if (!dont_filter_deletes_ && moptions->filter_deletes) {
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
assert(!concurrent_memtable_writes_);
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
SnapshotImpl read_from_snapshot;
|
|
|
|
read_from_snapshot.number_ = sequence_;
|
|
|
|
ReadOptions ropts;
|
|
|
|
ropts.snapshot = &read_from_snapshot;
|
|
|
|
std::string value;
|
|
|
|
auto cf_handle = cf_mems_->GetColumnFamilyHandle();
|
|
|
|
if (cf_handle == nullptr) {
|
|
|
|
cf_handle = db_->DefaultColumnFamily();
|
|
|
|
}
|
|
|
|
if (!db_->KeyMayExist(ropts, cf_handle, key, &value)) {
|
|
|
|
RecordTick(moptions->statistics, NUMBER_FILTERED_DELETES);
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
}
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
mem->Add(sequence_, delete_type, key, Slice(), concurrent_memtable_writes_);
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
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
2015-08-15 01:59:07 +02:00
|
|
|
CheckMemtableFull();
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2015-11-06 16:29:10 +01:00
|
|
|
virtual Status DeleteCF(uint32_t column_family_id,
|
|
|
|
const Slice& key) override {
|
|
|
|
return DeleteImpl(column_family_id, key, kTypeDeletion);
|
|
|
|
}
|
|
|
|
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
virtual Status SingleDeleteCF(uint32_t column_family_id,
|
|
|
|
const Slice& key) override {
|
2015-11-06 16:29:10 +01:00
|
|
|
return DeleteImpl(column_family_id, key, kTypeSingleDeletion);
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
}
|
|
|
|
|
2014-02-26 02:30:54 +01:00
|
|
|
virtual Status MergeCF(uint32_t column_family_id, const Slice& key,
|
2015-02-26 20:28:41 +01:00
|
|
|
const Slice& value) override {
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
assert(!concurrent_memtable_writes_);
|
2014-03-03 23:30:36 +01:00
|
|
|
Status seek_status;
|
|
|
|
if (!SeekToColumnFamily(column_family_id, &seek_status)) {
|
|
|
|
++sequence_;
|
|
|
|
return seek_status;
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
2014-02-06 01:02:48 +01:00
|
|
|
MemTable* mem = cf_mems_->GetMemTable();
|
2014-09-09 03:46:52 +02:00
|
|
|
auto* moptions = mem->GetMemTableOptions();
|
2014-01-11 02:33:56 +01:00
|
|
|
bool perform_merge = false;
|
|
|
|
|
2014-09-09 03:46:52 +02:00
|
|
|
if (moptions->max_successive_merges > 0 && db_ != nullptr) {
|
2014-01-11 02:33:56 +01:00
|
|
|
LookupKey lkey(key, sequence_);
|
|
|
|
|
|
|
|
// Count the number of successive merges at the head
|
|
|
|
// of the key in the memtable
|
2014-01-28 20:05:04 +01:00
|
|
|
size_t num_merges = mem->CountSuccessiveMergeEntries(lkey);
|
2014-01-11 02:33:56 +01:00
|
|
|
|
2014-09-09 03:46:52 +02:00
|
|
|
if (num_merges >= moptions->max_successive_merges) {
|
2014-01-11 02:33:56 +01:00
|
|
|
perform_merge = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (perform_merge) {
|
|
|
|
// 1) Get the existing value
|
|
|
|
std::string get_value;
|
|
|
|
|
|
|
|
// Pass in the sequence number so that we also include previous merge
|
|
|
|
// operations in the same batch.
|
|
|
|
SnapshotImpl read_from_snapshot;
|
|
|
|
read_from_snapshot.number_ = sequence_;
|
|
|
|
ReadOptions read_options;
|
|
|
|
read_options.snapshot = &read_from_snapshot;
|
|
|
|
|
2014-02-11 02:04:44 +01:00
|
|
|
auto cf_handle = cf_mems_->GetColumnFamilyHandle();
|
|
|
|
if (cf_handle == nullptr) {
|
|
|
|
cf_handle = db_->DefaultColumnFamily();
|
|
|
|
}
|
|
|
|
db_->Get(read_options, cf_handle, key, &get_value);
|
2014-01-11 02:33:56 +01:00
|
|
|
Slice get_value_slice = Slice(get_value);
|
|
|
|
|
|
|
|
// 2) Apply this merge
|
2014-10-27 20:10:13 +01:00
|
|
|
auto merge_operator = moptions->merge_operator;
|
2014-01-11 02:33:56 +01:00
|
|
|
assert(merge_operator);
|
|
|
|
|
|
|
|
std::deque<std::string> operands;
|
|
|
|
operands.push_front(value.ToString());
|
|
|
|
std::string new_value;
|
2015-03-03 19:59:36 +01:00
|
|
|
bool merge_success = false;
|
|
|
|
{
|
|
|
|
StopWatchNano timer(Env::Default(), moptions->statistics != nullptr);
|
|
|
|
PERF_TIMER_GUARD(merge_operator_time_nanos);
|
|
|
|
merge_success = merge_operator->FullMerge(
|
|
|
|
key, &get_value_slice, operands, &new_value, moptions->info_log);
|
|
|
|
RecordTick(moptions->statistics, MERGE_OPERATION_TOTAL_TIME,
|
|
|
|
timer.ElapsedNanos());
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!merge_success) {
|
2014-01-11 02:33:56 +01:00
|
|
|
// Failed to merge!
|
2014-10-27 20:10:13 +01:00
|
|
|
RecordTick(moptions->statistics, NUMBER_MERGE_FAILURES);
|
2014-01-11 02:33:56 +01:00
|
|
|
|
2014-07-28 21:05:36 +02:00
|
|
|
// Store the delta in memtable
|
|
|
|
perform_merge = false;
|
2014-01-11 02:33:56 +01:00
|
|
|
} else {
|
|
|
|
// 3) Add value to memtable
|
2014-01-28 20:05:04 +01:00
|
|
|
mem->Add(sequence_, kTypeValue, key, new_value);
|
2014-01-11 02:33:56 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!perform_merge) {
|
|
|
|
// Add merge operator to memtable
|
2014-01-28 20:05:04 +01:00
|
|
|
mem->Add(sequence_, kTypeMerge, key, value);
|
2014-01-11 02:33:56 +01:00
|
|
|
}
|
|
|
|
|
2013-03-21 23:59:47 +01:00
|
|
|
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
2015-08-15 01:59:07 +02:00
|
|
|
CheckMemtableFull();
|
2014-02-26 02:30:54 +01:00
|
|
|
return Status::OK();
|
2013-03-21 23:59:47 +01:00
|
|
|
}
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
|
|
|
|
void CheckMemtableFull() {
|
|
|
|
if (flush_scheduler_ != nullptr) {
|
|
|
|
auto* cfd = cf_mems_->current();
|
|
|
|
assert(cfd != nullptr);
|
|
|
|
if (cfd->mem()->ShouldScheduleFlush() &&
|
|
|
|
cfd->mem()->MarkFlushScheduled()) {
|
|
|
|
// MarkFlushScheduled only returns true if we are the one that
|
|
|
|
// should take action, so no need to dedup further
|
|
|
|
flush_scheduler_->ScheduleFlush(cfd);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
};
|
2011-10-31 18:22:06 +01:00
|
|
|
} // namespace
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-01-06 21:44:21 +01:00
|
|
|
// This function can only be called in these conditions:
|
|
|
|
// 1) During Recovery()
|
2015-11-06 16:29:10 +01:00
|
|
|
// 2) During Write(), in a single-threaded write thread
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
// 3) During Write(), in a concurrent context where memtables has been cloned
|
2015-11-06 16:29:10 +01:00
|
|
|
// The reason is that it calls memtables->Seek(), which has a stateful cache
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
Status WriteBatchInternal::InsertInto(
|
2016-02-05 19:44:13 +01:00
|
|
|
const autovector<WriteThread::Writer*>& writers, SequenceNumber 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
2015-08-15 01:59:07 +02:00
|
|
|
ColumnFamilyMemTables* memtables, FlushScheduler* flush_scheduler,
|
|
|
|
bool ignore_missing_column_families, uint64_t log_number, DB* db,
|
|
|
|
const bool dont_filter_deletes, bool concurrent_memtable_writes) {
|
|
|
|
MemTableInserter inserter(sequence, memtables, flush_scheduler,
|
|
|
|
ignore_missing_column_families, log_number, db,
|
|
|
|
dont_filter_deletes, concurrent_memtable_writes);
|
2016-02-05 19:44:13 +01:00
|
|
|
|
|
|
|
for (size_t i = 0; i < writers.size(); i++) {
|
|
|
|
if (!writers[i]->CallbackFailed()) {
|
|
|
|
writers[i]->status = writers[i]->batch->Iterate(&inserter);
|
|
|
|
if (!writers[i]->status.ok()) {
|
|
|
|
return writers[i]->status;
|
|
|
|
}
|
|
|
|
}
|
2015-11-06 16:29:10 +01:00
|
|
|
}
|
2016-02-05 19:44:13 +01:00
|
|
|
return Status::OK();
|
2015-11-06 16:29:10 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
Status WriteBatchInternal::InsertInto(const WriteBatch* batch,
|
2014-01-28 20:05:04 +01:00
|
|
|
ColumnFamilyMemTables* memtables,
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
FlushScheduler* flush_scheduler,
|
2014-09-02 22:29:05 +02:00
|
|
|
bool ignore_missing_column_families,
|
|
|
|
uint64_t log_number, DB* db,
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
const bool dont_filter_deletes,
|
|
|
|
bool concurrent_memtable_writes) {
|
2015-11-06 16:29:10 +01:00
|
|
|
MemTableInserter inserter(WriteBatchInternal::Sequence(batch), memtables,
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
flush_scheduler, ignore_missing_column_families,
|
|
|
|
log_number, db, dont_filter_deletes,
|
|
|
|
concurrent_memtable_writes);
|
2015-11-06 16:29:10 +01:00
|
|
|
return batch->Iterate(&inserter);
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
void WriteBatchInternal::SetContents(WriteBatch* b, const Slice& contents) {
|
2016-03-30 19:43:00 +02:00
|
|
|
assert(contents.size() >= WriteBatchInternal::kHeader);
|
2011-03-18 23:37:00 +01:00
|
|
|
b->rep_.assign(contents.data(), contents.size());
|
2015-11-06 16:03:30 +01:00
|
|
|
b->content_flags_.store(ContentFlags::DEFERRED, std::memory_order_relaxed);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2012-03-09 01:23:21 +01:00
|
|
|
void WriteBatchInternal::Append(WriteBatch* dst, const WriteBatch* src) {
|
|
|
|
SetCount(dst, Count(dst) + Count(src));
|
2016-03-30 19:43:00 +02:00
|
|
|
assert(src->rep_.size() >= WriteBatchInternal::kHeader);
|
2016-04-02 00:23:46 +02:00
|
|
|
dst->rep_.append(src->rep_.data() + WriteBatchInternal::kHeader,
|
2016-03-31 04:56:55 +02:00
|
|
|
src->rep_.size() - WriteBatchInternal::kHeader);
|
2015-11-06 16:03:30 +01:00
|
|
|
dst->content_flags_.store(
|
|
|
|
dst->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
src->content_flags_.load(std::memory_order_relaxed),
|
|
|
|
std::memory_order_relaxed);
|
2012-03-09 01:23:21 +01:00
|
|
|
}
|
|
|
|
|
2015-11-06 16:29:10 +01:00
|
|
|
size_t WriteBatchInternal::AppendedByteSize(size_t leftByteSize,
|
|
|
|
size_t rightByteSize) {
|
|
|
|
if (leftByteSize == 0 || rightByteSize == 0) {
|
|
|
|
return leftByteSize + rightByteSize;
|
|
|
|
} else {
|
2016-03-30 19:43:00 +02:00
|
|
|
return leftByteSize + rightByteSize - WriteBatchInternal::kHeader;
|
2015-11-06 16:29:10 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
} // namespace rocksdb
|