2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2017-07-16 01:03:42 +02:00
|
|
|
// This source code is licensed under both the GPLv2 (found in the
|
|
|
|
// COPYING file in the root directory) and Apache 2.0 License
|
|
|
|
// (found in the LICENSE.Apache file in the root directory).
|
2013-10-16 23:59:46 +02:00
|
|
|
//
|
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
|
2018-01-23 21:46:12 +01:00
|
|
|
// kTypeRangeDeletion 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
|
|
|
// kTypeMerge varstring varstring
|
2014-01-07 23:41:42 +01:00
|
|
|
// kTypeColumnFamilyValue varint32 varstring varstring
|
2018-01-23 21:46:12 +01:00
|
|
|
// kTypeColumnFamilyDeletion varint32 varstring
|
|
|
|
// kTypeColumnFamilySingleDeletion varint32 varstring
|
|
|
|
// kTypeColumnFamilyRangeDeletion 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
|
|
|
// kTypeColumnFamilyMerge varint32 varstring varstring
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
// kTypeBeginPrepareXID varstring
|
|
|
|
// kTypeEndPrepareXID
|
|
|
|
// kTypeCommitXID varstring
|
|
|
|
// kTypeRollbackXID varstring
|
2017-11-11 20:23:43 +01:00
|
|
|
// kTypeBeginPersistedPrepareXID varstring
|
2018-06-29 03:46:39 +02:00
|
|
|
// kTypeBeginUnprepareXID varstring
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
// kTypeNoop
|
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
|
|
|
|
2016-07-07 23:45:29 +02:00
|
|
|
#include <map>
|
2015-07-11 05:15:45 +02:00
|
|
|
#include <stack>
|
|
|
|
#include <stdexcept>
|
2017-03-22 19:07:52 +01:00
|
|
|
#include <type_traits>
|
2019-09-13 01:53:31 +02:00
|
|
|
#include <unordered_map>
|
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"
|
2019-06-01 00:21:36 +02:00
|
|
|
#include "db/db_impl/db_impl.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#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"
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
#include "db/kv_checksum.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "db/memtable.h"
|
2016-06-14 01:17:26 +02:00
|
|
|
#include "db/merge_context.h"
|
2015-08-07 02:59:05 +02:00
|
|
|
#include "db/snapshot_impl.h"
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
#include "db/trim_history_scheduler.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "db/write_batch_internal.h"
|
2017-04-06 04:02:00 +02:00
|
|
|
#include "monitoring/perf_context_imp.h"
|
|
|
|
#include "monitoring/statistics.h"
|
2020-04-20 22:21:34 +02:00
|
|
|
#include "port/lang.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "rocksdb/merge_operator.h"
|
2021-01-26 07:07:26 +01:00
|
|
|
#include "rocksdb/system_clock.h"
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
#include "util/autovector.h"
|
2019-08-27 19:57:28 +02:00
|
|
|
#include "util/cast_util.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "util/coding.h"
|
2018-03-06 08:48:23 +01:00
|
|
|
#include "util/duplicate_detector.h"
|
2016-11-16 00:18:56 +01:00
|
|
|
#include "util/string_util.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
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 {
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
DEFERRED = 1 << 0,
|
|
|
|
HAS_PUT = 1 << 1,
|
|
|
|
HAS_DELETE = 1 << 2,
|
|
|
|
HAS_SINGLE_DELETE = 1 << 3,
|
|
|
|
HAS_MERGE = 1 << 4,
|
|
|
|
HAS_BEGIN_PREPARE = 1 << 5,
|
|
|
|
HAS_END_PREPARE = 1 << 6,
|
|
|
|
HAS_COMMIT = 1 << 7,
|
|
|
|
HAS_ROLLBACK = 1 << 8,
|
2016-08-16 17:16:04 +02:00
|
|
|
HAS_DELETE_RANGE = 1 << 9,
|
2017-10-03 18:08:07 +02:00
|
|
|
HAS_BLOB_INDEX = 1 << 10,
|
2018-07-07 02:17:36 +02:00
|
|
|
HAS_BEGIN_UNPREPARE = 1 << 11,
|
2015-11-06 16:03:30 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
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();
|
|
|
|
}
|
|
|
|
|
2016-08-16 17:16:04 +02:00
|
|
|
Status DeleteRangeCF(uint32_t, const Slice&, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_DELETE_RANGE;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
Status MergeCF(uint32_t, const Slice&, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_MERGE;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
|
2017-10-03 18:08:07 +02:00
|
|
|
Status PutBlobIndexCF(uint32_t, const Slice&, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_BLOB_INDEX;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2018-07-07 02:17:36 +02:00
|
|
|
Status MarkBeginPrepare(bool unprepare) override {
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
content_flags |= ContentFlags::HAS_BEGIN_PREPARE;
|
2018-07-07 02:17:36 +02:00
|
|
|
if (unprepare) {
|
|
|
|
content_flags |= ContentFlags::HAS_BEGIN_UNPREPARE;
|
|
|
|
}
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status MarkEndPrepare(const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_END_PREPARE;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status MarkCommit(const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_COMMIT;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2021-12-10 20:03:39 +01:00
|
|
|
Status MarkCommitWithTimestamp(const Slice&, const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_COMMIT;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
Status MarkRollback(const Slice&) override {
|
|
|
|
content_flags |= ContentFlags::HAS_ROLLBACK;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
2015-11-06 16:03:30 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
} // anon namespace
|
|
|
|
|
2015-07-11 05:15:45 +02:00
|
|
|
struct SavePoints {
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
std::stack<SavePoint, autovector<SavePoint>> stack;
|
2015-07-11 05:15:45 +02:00
|
|
|
};
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
WriteBatch::WriteBatch(size_t reserved_bytes, size_t max_bytes)
|
2021-09-13 00:33:15 +02:00
|
|
|
: content_flags_(0), max_bytes_(max_bytes), rep_() {
|
2019-07-26 00:23:46 +02:00
|
|
|
rep_.reserve((reserved_bytes > WriteBatchInternal::kHeader)
|
|
|
|
? reserved_bytes
|
|
|
|
: WriteBatchInternal::kHeader);
|
|
|
|
rep_.resize(WriteBatchInternal::kHeader);
|
|
|
|
}
|
|
|
|
|
2021-09-13 00:33:15 +02:00
|
|
|
WriteBatch::WriteBatch(size_t reserved_bytes, size_t max_bytes,
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
size_t protection_bytes_per_key)
|
2021-09-13 00:33:15 +02:00
|
|
|
: content_flags_(0), max_bytes_(max_bytes), rep_() {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
// Currently `protection_bytes_per_key` can only be enabled at 8 bytes per
|
|
|
|
// entry.
|
|
|
|
assert(protection_bytes_per_key == 0 || protection_bytes_per_key == 8);
|
|
|
|
if (protection_bytes_per_key != 0) {
|
|
|
|
prot_info_.reset(new WriteBatch::ProtectionInfo());
|
|
|
|
}
|
|
|
|
rep_.reserve((reserved_bytes > WriteBatchInternal::kHeader)
|
|
|
|
? reserved_bytes
|
|
|
|
: WriteBatchInternal::kHeader);
|
|
|
|
rep_.resize(WriteBatchInternal::kHeader);
|
|
|
|
}
|
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
WriteBatch::WriteBatch(const std::string& rep)
|
2021-09-13 00:33:15 +02:00
|
|
|
: content_flags_(ContentFlags::DEFERRED), max_bytes_(0), rep_(rep) {}
|
2015-11-06 16:03:30 +01:00
|
|
|
|
2018-01-12 00:35:21 +01:00
|
|
|
WriteBatch::WriteBatch(std::string&& rep)
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
: content_flags_(ContentFlags::DEFERRED),
|
2018-01-12 00:35:21 +01:00
|
|
|
max_bytes_(0),
|
2021-09-13 00:33:15 +02:00
|
|
|
rep_(std::move(rep)) {}
|
2018-01-12 00:35:21 +01:00
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
WriteBatch::WriteBatch(const WriteBatch& src)
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
: wal_term_point_(src.wal_term_point_),
|
2015-11-06 16:03:30 +01:00
|
|
|
content_flags_(src.content_flags_.load(std::memory_order_relaxed)),
|
2017-04-11 00:38:34 +02:00
|
|
|
max_bytes_(src.max_bytes_),
|
2021-09-13 00:33:15 +02:00
|
|
|
rep_(src.rep_) {
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
if (src.save_points_ != nullptr) {
|
|
|
|
save_points_.reset(new SavePoints());
|
|
|
|
save_points_->stack = src.save_points_->stack;
|
|
|
|
}
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (src.prot_info_ != nullptr) {
|
|
|
|
prot_info_.reset(new WriteBatch::ProtectionInfo());
|
|
|
|
prot_info_->entries_ = src.prot_info_->entries_;
|
|
|
|
}
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
}
|
2015-11-06 16:03:30 +01:00
|
|
|
|
2017-09-18 23:36:53 +02:00
|
|
|
WriteBatch::WriteBatch(WriteBatch&& src) noexcept
|
2015-11-06 16:03:30 +01:00
|
|
|
: save_points_(std::move(src.save_points_)),
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
wal_term_point_(std::move(src.wal_term_point_)),
|
2015-11-06 16:03:30 +01:00
|
|
|
content_flags_(src.content_flags_.load(std::memory_order_relaxed)),
|
2017-04-11 00:38:34 +02:00
|
|
|
max_bytes_(src.max_bytes_),
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
prot_info_(std::move(src.prot_info_)),
|
2021-09-13 00:33:15 +02:00
|
|
|
rep_(std::move(src.rep_)) {}
|
2015-11-06 16:03:30 +01:00
|
|
|
|
|
|
|
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
|
|
|
}
|
|
|
|
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
WriteBatch::~WriteBatch() { }
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
WriteBatch::Handler::~Handler() { }
|
|
|
|
|
2018-03-05 22:08:17 +01:00
|
|
|
void WriteBatch::Handler::LogData(const Slice& /*blob*/) {
|
2013-08-15 01:32:46 +02:00
|
|
|
// 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();
|
|
|
|
}
|
|
|
|
}
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (prot_info_ != nullptr) {
|
|
|
|
prot_info_->entries_.clear();
|
|
|
|
}
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
wal_term_point_.clear();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2019-09-20 21:00:55 +02:00
|
|
|
uint32_t WriteBatch::Count() const { return WriteBatchInternal::Count(this); }
|
2013-06-26 19:50:58 +02:00
|
|
|
|
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;
|
2020-10-02 00:57:28 +02:00
|
|
|
// Should we handle status here?
|
|
|
|
Iterate(&classifier).PermitUncheckedError();
|
2015-11-06 16:03:30 +01:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
void WriteBatch::MarkWalTerminationPoint() {
|
|
|
|
wal_term_point_.size = GetDataSize();
|
|
|
|
wal_term_point_.count = Count();
|
|
|
|
wal_term_point_.content_flags = content_flags_;
|
|
|
|
}
|
|
|
|
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
size_t WriteBatch::GetProtectionBytesPerKey() const {
|
|
|
|
if (prot_info_ != nullptr) {
|
|
|
|
return prot_info_->GetBytesPerKey();
|
|
|
|
}
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2016-08-16 17:16:04 +02:00
|
|
|
bool WriteBatch::HasDeleteRange() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_DELETE_RANGE) != 0;
|
|
|
|
}
|
|
|
|
|
2015-11-06 16:03:30 +01:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
bool WriteBatch::HasBeginPrepare() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_BEGIN_PREPARE) != 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool WriteBatch::HasEndPrepare() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_END_PREPARE) != 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool WriteBatch::HasCommit() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_COMMIT) != 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool WriteBatch::HasRollback() const {
|
|
|
|
return (ComputeContentFlags() & ContentFlags::HAS_ROLLBACK) != 0;
|
|
|
|
}
|
|
|
|
|
2014-08-19 00:19:17 +02:00
|
|
|
Status ReadRecordFromWriteBatch(Slice* input, char* tag,
|
|
|
|
uint32_t* column_family, Slice* key,
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
Slice* value, Slice* blob, Slice* xid) {
|
2014-08-19 00:19:17 +02:00
|
|
|
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");
|
|
|
|
}
|
2018-07-13 19:47:49 +02:00
|
|
|
FALLTHROUGH_INTENDED;
|
2014-08-19 00:19:17 +02:00
|
|
|
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");
|
|
|
|
}
|
2018-07-13 19:47:49 +02:00
|
|
|
FALLTHROUGH_INTENDED;
|
2014-08-19 00:19:17 +02:00
|
|
|
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;
|
2016-08-16 17:16:04 +02:00
|
|
|
case kTypeColumnFamilyRangeDeletion:
|
|
|
|
if (!GetVarint32(input, column_family)) {
|
|
|
|
return Status::Corruption("bad WriteBatch DeleteRange");
|
|
|
|
}
|
2018-07-13 19:47:49 +02:00
|
|
|
FALLTHROUGH_INTENDED;
|
2016-08-16 17:16:04 +02:00
|
|
|
case kTypeRangeDeletion:
|
|
|
|
// for range delete, "key" is begin_key, "value" is end_key
|
|
|
|
if (!GetLengthPrefixedSlice(input, key) ||
|
|
|
|
!GetLengthPrefixedSlice(input, value)) {
|
|
|
|
return Status::Corruption("bad WriteBatch DeleteRange");
|
|
|
|
}
|
|
|
|
break;
|
2014-08-19 00:19:17 +02:00
|
|
|
case kTypeColumnFamilyMerge:
|
|
|
|
if (!GetVarint32(input, column_family)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Merge");
|
|
|
|
}
|
2018-07-13 19:47:49 +02:00
|
|
|
FALLTHROUGH_INTENDED;
|
2014-08-19 00:19:17 +02:00
|
|
|
case kTypeMerge:
|
|
|
|
if (!GetLengthPrefixedSlice(input, key) ||
|
|
|
|
!GetLengthPrefixedSlice(input, value)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Merge");
|
|
|
|
}
|
|
|
|
break;
|
2017-10-03 18:08:07 +02:00
|
|
|
case kTypeColumnFamilyBlobIndex:
|
|
|
|
if (!GetVarint32(input, column_family)) {
|
|
|
|
return Status::Corruption("bad WriteBatch BlobIndex");
|
|
|
|
}
|
2018-07-13 19:47:49 +02:00
|
|
|
FALLTHROUGH_INTENDED;
|
2017-10-03 18:08:07 +02:00
|
|
|
case kTypeBlobIndex:
|
|
|
|
if (!GetLengthPrefixedSlice(input, key) ||
|
|
|
|
!GetLengthPrefixedSlice(input, value)) {
|
|
|
|
return Status::Corruption("bad WriteBatch BlobIndex");
|
|
|
|
}
|
|
|
|
break;
|
2014-08-19 00:19:17 +02:00
|
|
|
case kTypeLogData:
|
|
|
|
assert(blob != nullptr);
|
|
|
|
if (!GetLengthPrefixedSlice(input, blob)) {
|
|
|
|
return Status::Corruption("bad WriteBatch Blob");
|
|
|
|
}
|
|
|
|
break;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
case kTypeNoop:
|
|
|
|
case kTypeBeginPrepareXID:
|
2017-11-11 20:23:43 +01:00
|
|
|
// This indicates that the prepared batch is also persisted in the db.
|
|
|
|
// This is used in WritePreparedTxn
|
|
|
|
case kTypeBeginPersistedPrepareXID:
|
2018-06-29 03:46:39 +02:00
|
|
|
// This is used in WriteUnpreparedTxn
|
|
|
|
case kTypeBeginUnprepareXID:
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
break;
|
|
|
|
case kTypeEndPrepareXID:
|
|
|
|
if (!GetLengthPrefixedSlice(input, xid)) {
|
|
|
|
return Status::Corruption("bad EndPrepare XID");
|
|
|
|
}
|
|
|
|
break;
|
2021-12-10 20:03:39 +01:00
|
|
|
case kTypeCommitXIDAndTimestamp:
|
|
|
|
if (!GetLengthPrefixedSlice(input, key)) {
|
|
|
|
return Status::Corruption("bad commit timestamp");
|
|
|
|
}
|
|
|
|
FALLTHROUGH_INTENDED;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
case kTypeCommitXID:
|
|
|
|
if (!GetLengthPrefixedSlice(input, xid)) {
|
|
|
|
return Status::Corruption("bad Commit XID");
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
case kTypeRollbackXID:
|
|
|
|
if (!GetLengthPrefixedSlice(input, xid)) {
|
|
|
|
return Status::Corruption("bad Rollback XID");
|
|
|
|
}
|
|
|
|
break;
|
2014-08-19 00:19:17 +02:00
|
|
|
default:
|
|
|
|
return Status::Corruption("unknown WriteBatch tag");
|
|
|
|
}
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
Status WriteBatch::Iterate(Handler* handler) const {
|
2019-07-31 22:36:22 +02:00
|
|
|
if (rep_.size() < WriteBatchInternal::kHeader) {
|
2011-05-21 04:17:43 +02:00
|
|
|
return Status::Corruption("malformed WriteBatch (too small)");
|
|
|
|
}
|
|
|
|
|
2019-07-31 22:36:22 +02:00
|
|
|
return WriteBatchInternal::Iterate(this, handler, WriteBatchInternal::kHeader,
|
|
|
|
rep_.size());
|
|
|
|
}
|
|
|
|
|
|
|
|
Status WriteBatchInternal::Iterate(const WriteBatch* wb,
|
|
|
|
WriteBatch::Handler* handler, size_t begin,
|
|
|
|
size_t end) {
|
|
|
|
if (begin > wb->rep_.size() || end > wb->rep_.size() || end < begin) {
|
|
|
|
return Status::Corruption("Invalid start/end bounds for Iterate");
|
|
|
|
}
|
|
|
|
assert(begin <= end);
|
|
|
|
Slice input(wb->rep_.data() + begin, static_cast<size_t>(end - begin));
|
|
|
|
bool whole_batch =
|
|
|
|
(begin == WriteBatchInternal::kHeader) && (end == wb->rep_.size());
|
|
|
|
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
Slice key, value, blob, xid;
|
2017-09-29 01:43:04 +02:00
|
|
|
// Sometimes a sub-batch starts with a Noop. We want to exclude such Noops as
|
2018-03-08 19:18:34 +01:00
|
|
|
// the batch boundary symbols otherwise we would mis-count the number of
|
2017-09-29 01:43:04 +02:00
|
|
|
// batches. We do that by checking whether the accumulated batch is empty
|
|
|
|
// before seeing the next Noop.
|
|
|
|
bool empty_batch = true;
|
2019-09-09 20:22:28 +02:00
|
|
|
uint32_t found = 0;
|
2014-02-26 02:30:54 +01:00
|
|
|
Status s;
|
2018-02-06 03:32:54 +01:00
|
|
|
char tag = 0;
|
|
|
|
uint32_t column_family = 0; // default
|
2018-04-21 00:13:47 +02:00
|
|
|
bool last_was_try_again = false;
|
2018-10-11 05:55:42 +02:00
|
|
|
bool handler_continue = true;
|
|
|
|
while (((s.ok() && !input.empty()) || UNLIKELY(s.IsTryAgain()))) {
|
|
|
|
handler_continue = handler->Continue();
|
|
|
|
if (!handler_continue) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
2018-02-16 17:36:47 +01:00
|
|
|
if (LIKELY(!s.IsTryAgain())) {
|
2018-04-21 00:13:47 +02:00
|
|
|
last_was_try_again = false;
|
2018-02-06 03:32:54 +01:00
|
|
|
tag = 0;
|
|
|
|
column_family = 0; // default
|
|
|
|
|
|
|
|
s = ReadRecordFromWriteBatch(&input, &tag, &column_family, &key, &value,
|
|
|
|
&blob, &xid);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
assert(s.IsTryAgain());
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(!last_was_try_again); // to detect infinite loop bugs
|
2018-04-21 00:13:47 +02:00
|
|
|
if (UNLIKELY(last_was_try_again)) {
|
|
|
|
return Status::Corruption(
|
|
|
|
"two consecutive TryAgain in WriteBatch handler; this is either a "
|
|
|
|
"software bug or data corruption.");
|
|
|
|
}
|
|
|
|
last_was_try_again = true;
|
2018-02-06 03:32:54 +01:00
|
|
|
s = Status::OK();
|
2014-08-19 00:19:17 +02:00
|
|
|
}
|
|
|
|
|
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:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2015-11-06 16:03:30 +01:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_PUT));
|
2014-08-19 00:19:17 +02:00
|
|
|
s = handler->PutCF(column_family, key, value);
|
2018-02-16 17:36:47 +01:00
|
|
|
if (LIKELY(s.ok())) {
|
2018-02-06 03:32:54 +01:00
|
|
|
empty_batch = false;
|
|
|
|
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:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2015-11-06 16:03:30 +01:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_DELETE));
|
2014-08-19 00:19:17 +02:00
|
|
|
s = handler->DeleteCF(column_family, key);
|
2018-02-16 17:36:47 +01:00
|
|
|
if (LIKELY(s.ok())) {
|
2018-02-06 03:32:54 +01:00
|
|
|
empty_batch = false;
|
|
|
|
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:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2015-11-06 16:03:30 +01:00
|
|
|
(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);
|
2018-02-16 17:36:47 +01:00
|
|
|
if (LIKELY(s.ok())) {
|
2018-02-06 03:32:54 +01:00
|
|
|
empty_batch = false;
|
|
|
|
found++;
|
|
|
|
}
|
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
|
|
|
break;
|
2016-08-16 17:16:04 +02:00
|
|
|
case kTypeColumnFamilyRangeDeletion:
|
|
|
|
case kTypeRangeDeletion:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2016-08-16 17:16:04 +02:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_DELETE_RANGE));
|
|
|
|
s = handler->DeleteRangeCF(column_family, key, value);
|
2018-02-16 17:36:47 +01:00
|
|
|
if (LIKELY(s.ok())) {
|
2018-02-06 03:32:54 +01:00
|
|
|
empty_batch = false;
|
|
|
|
found++;
|
|
|
|
}
|
2016-08-16 17:16:04 +02:00
|
|
|
break;
|
2014-01-07 23:41:42 +01:00
|
|
|
case kTypeColumnFamilyMerge:
|
2013-03-21 23:59:47 +01:00
|
|
|
case kTypeMerge:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2015-11-06 16:03:30 +01:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_MERGE));
|
2014-08-19 00:19:17 +02:00
|
|
|
s = handler->MergeCF(column_family, key, value);
|
2018-02-16 17:36:47 +01:00
|
|
|
if (LIKELY(s.ok())) {
|
2018-02-06 03:32:54 +01:00
|
|
|
empty_batch = false;
|
|
|
|
found++;
|
|
|
|
}
|
2013-03-21 23:59:47 +01:00
|
|
|
break;
|
2017-10-03 18:08:07 +02:00
|
|
|
case kTypeColumnFamilyBlobIndex:
|
|
|
|
case kTypeBlobIndex:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2017-10-03 18:08:07 +02:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_BLOB_INDEX));
|
|
|
|
s = handler->PutBlobIndexCF(column_family, key, value);
|
2018-02-16 17:36:47 +01:00
|
|
|
if (LIKELY(s.ok())) {
|
2018-02-06 03:32:54 +01:00
|
|
|
found++;
|
|
|
|
}
|
2017-10-03 18:08:07 +02:00
|
|
|
break;
|
2013-08-15 01:32:46 +02:00
|
|
|
case kTypeLogData:
|
2014-08-19 00:19:17 +02:00
|
|
|
handler->LogData(blob);
|
2017-12-01 08:39:56 +01:00
|
|
|
// A batch might have nothing but LogData. It is still a batch.
|
|
|
|
empty_batch = false;
|
2013-08-15 01:32:46 +02:00
|
|
|
break;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
case kTypeBeginPrepareXID:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_BEGIN_PREPARE));
|
2020-10-20 22:17:17 +02:00
|
|
|
s = handler->MarkBeginPrepare();
|
|
|
|
assert(s.ok());
|
2017-09-29 01:43:04 +02:00
|
|
|
empty_batch = false;
|
2017-11-11 20:23:43 +01:00
|
|
|
if (!handler->WriteAfterCommit()) {
|
|
|
|
s = Status::NotSupported(
|
|
|
|
"WriteCommitted txn tag when write_after_commit_ is disabled (in "
|
2018-06-29 03:46:39 +02:00
|
|
|
"WritePrepared/WriteUnprepared mode). If it is not due to "
|
|
|
|
"corruption, the WAL must be emptied before changing the "
|
|
|
|
"WritePolicy.");
|
|
|
|
}
|
|
|
|
if (handler->WriteBeforePrepare()) {
|
|
|
|
s = Status::NotSupported(
|
|
|
|
"WriteCommitted txn tag when write_before_prepare_ is enabled "
|
|
|
|
"(in WriteUnprepared mode). If it is not due to corruption, the "
|
|
|
|
"WAL must be emptied before changing the WritePolicy.");
|
2017-11-11 20:23:43 +01:00
|
|
|
}
|
|
|
|
break;
|
|
|
|
case kTypeBeginPersistedPrepareXID:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2017-11-11 20:23:43 +01:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_BEGIN_PREPARE));
|
2020-10-20 22:17:17 +02:00
|
|
|
s = handler->MarkBeginPrepare();
|
|
|
|
assert(s.ok());
|
2017-11-11 20:23:43 +01:00
|
|
|
empty_batch = false;
|
|
|
|
if (handler->WriteAfterCommit()) {
|
|
|
|
s = Status::NotSupported(
|
2018-06-29 03:46:39 +02:00
|
|
|
"WritePrepared/WriteUnprepared txn tag when write_after_commit_ "
|
|
|
|
"is enabled (in default WriteCommitted mode). If it is not due "
|
|
|
|
"to corruption, the WAL must be emptied before changing the "
|
|
|
|
"WritePolicy.");
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
case kTypeBeginUnprepareXID:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
2018-07-07 02:17:36 +02:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_BEGIN_UNPREPARE));
|
2020-10-20 22:17:17 +02:00
|
|
|
s = handler->MarkBeginPrepare(true /* unprepared */);
|
|
|
|
assert(s.ok());
|
2018-06-29 03:46:39 +02:00
|
|
|
empty_batch = false;
|
|
|
|
if (handler->WriteAfterCommit()) {
|
|
|
|
s = Status::NotSupported(
|
|
|
|
"WriteUnprepared txn tag when write_after_commit_ is enabled (in "
|
2017-11-11 20:23:43 +01:00
|
|
|
"default WriteCommitted mode). If it is not due to corruption, "
|
|
|
|
"the WAL must be emptied before changing the WritePolicy.");
|
|
|
|
}
|
2018-06-29 03:46:39 +02:00
|
|
|
if (!handler->WriteBeforePrepare()) {
|
|
|
|
s = Status::NotSupported(
|
|
|
|
"WriteUnprepared txn tag when write_before_prepare_ is disabled "
|
|
|
|
"(in WriteCommitted/WritePrepared mode). If it is not due to "
|
|
|
|
"corruption, the WAL must be emptied before changing the "
|
|
|
|
"WritePolicy.");
|
|
|
|
}
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
break;
|
|
|
|
case kTypeEndPrepareXID:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_END_PREPARE));
|
2020-10-20 22:17:17 +02:00
|
|
|
s = handler->MarkEndPrepare(xid);
|
|
|
|
assert(s.ok());
|
2017-09-29 01:43:04 +02:00
|
|
|
empty_batch = true;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
break;
|
|
|
|
case kTypeCommitXID:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_COMMIT));
|
2020-10-20 22:17:17 +02:00
|
|
|
s = handler->MarkCommit(xid);
|
|
|
|
assert(s.ok());
|
2017-09-29 01:43:04 +02:00
|
|
|
empty_batch = true;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
break;
|
2021-12-10 20:03:39 +01:00
|
|
|
case kTypeCommitXIDAndTimestamp:
|
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_COMMIT));
|
|
|
|
// key stores the commit timestamp.
|
|
|
|
assert(!key.empty());
|
|
|
|
s = handler->MarkCommitWithTimestamp(xid, key);
|
|
|
|
if (LIKELY(s.ok())) {
|
|
|
|
empty_batch = true;
|
|
|
|
}
|
|
|
|
break;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
case kTypeRollbackXID:
|
2019-07-31 22:36:22 +02:00
|
|
|
assert(wb->content_flags_.load(std::memory_order_relaxed) &
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
(ContentFlags::DEFERRED | ContentFlags::HAS_ROLLBACK));
|
2020-10-20 22:17:17 +02:00
|
|
|
s = handler->MarkRollback(xid);
|
|
|
|
assert(s.ok());
|
2017-09-29 01:43:04 +02:00
|
|
|
empty_batch = true;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
break;
|
|
|
|
case kTypeNoop:
|
2020-10-20 22:17:17 +02:00
|
|
|
s = handler->MarkNoop(empty_batch);
|
|
|
|
assert(s.ok());
|
2017-09-29 01:43:04 +02:00
|
|
|
empty_batch = true;
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +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;
|
|
|
|
}
|
2019-07-31 22:36:22 +02:00
|
|
|
if (handler_continue && whole_batch &&
|
|
|
|
found != WriteBatchInternal::Count(wb)) {
|
2011-05-21 04:17:43 +02:00
|
|
|
return Status::Corruption("WriteBatch has wrong count");
|
|
|
|
} else {
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-11-02 01:23:52 +01:00
|
|
|
bool WriteBatchInternal::IsLatestPersistentState(const WriteBatch* b) {
|
|
|
|
return b->is_latest_persistent_state_;
|
|
|
|
}
|
|
|
|
|
2021-07-30 21:06:47 +02:00
|
|
|
void WriteBatchInternal::SetAsLatestPersistentState(WriteBatch* b) {
|
2017-11-02 01:23:52 +01:00
|
|
|
b->is_latest_persistent_state_ = true;
|
|
|
|
}
|
|
|
|
|
2019-09-09 20:22:28 +02:00
|
|
|
uint32_t WriteBatchInternal::Count(const WriteBatch* b) {
|
2011-03-18 23:37:00 +01:00
|
|
|
return DecodeFixed32(b->rep_.data() + 8);
|
|
|
|
}
|
|
|
|
|
2019-09-09 20:22:28 +02:00
|
|
|
void WriteBatchInternal::SetCount(WriteBatch* b, uint32_t n) {
|
2011-03-18 23:37:00 +01:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2018-03-05 22:08:17 +01:00
|
|
|
size_t WriteBatchInternal::GetFirstOffset(WriteBatch* /*b*/) {
|
2016-04-02 00:23:46 +02:00
|
|
|
return WriteBatchInternal::kHeader;
|
2016-03-31 04:56:55 +02:00
|
|
|
}
|
2015-07-11 05:15:45 +02:00
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::Put(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& key, const Slice& value) {
|
2018-02-09 23:50:09 +01:00
|
|
|
if (key.size() > size_t{port::kMaxUint32}) {
|
|
|
|
return Status::InvalidArgument("key is too large");
|
|
|
|
}
|
|
|
|
if (value.size() > size_t{port::kMaxUint32}) {
|
|
|
|
return Status::InvalidArgument("value is too large");
|
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
LocalSavePoint save(b);
|
2014-04-22 20:27:33 +02:00
|
|
|
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);
|
|
|
|
}
|
2021-09-13 00:33:15 +02:00
|
|
|
PutLengthPrefixedSlice(&b->rep_, key);
|
2014-04-22 20:27:33 +02:00
|
|
|
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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// Technically the optype could've been `kTypeColumnFamilyValue` with the
|
|
|
|
// CF ID encoded in the `WriteBatch`. That distinction is unimportant
|
|
|
|
// however since we verify CF ID is correct, as well as all other fields
|
|
|
|
// (a missing/extra encoded CF ID would corrupt another field). It is
|
|
|
|
// convenient to consolidate on `kTypeValue` here as that is what will be
|
|
|
|
// inserted into memtable.
|
2021-09-14 22:13:36 +02:00
|
|
|
b->prot_info_->entries_.emplace_back(ProtectionInfo64()
|
|
|
|
.ProtectKVO(key, value, kTypeValue)
|
|
|
|
.ProtectC(column_family_id));
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2014-04-22 20:27:33 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::Put(ColumnFamilyHandle* column_family, const Slice& key,
|
|
|
|
const Slice& value) {
|
|
|
|
return WriteBatchInternal::Put(this, GetColumnFamilyID(column_family), key,
|
|
|
|
value);
|
2014-04-22 20:27:33 +02:00
|
|
|
}
|
|
|
|
|
2018-02-09 23:50:09 +01:00
|
|
|
Status WriteBatchInternal::CheckSlicePartsLength(const SliceParts& key,
|
|
|
|
const SliceParts& value) {
|
|
|
|
size_t total_key_bytes = 0;
|
|
|
|
for (int i = 0; i < key.num_parts; ++i) {
|
|
|
|
total_key_bytes += key.parts[i].size();
|
|
|
|
}
|
|
|
|
if (total_key_bytes >= size_t{port::kMaxUint32}) {
|
|
|
|
return Status::InvalidArgument("key is too large");
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t total_value_bytes = 0;
|
|
|
|
for (int i = 0; i < value.num_parts; ++i) {
|
|
|
|
total_value_bytes += value.parts[i].size();
|
|
|
|
}
|
|
|
|
if (total_value_bytes >= size_t{port::kMaxUint32}) {
|
|
|
|
return Status::InvalidArgument("value is too large");
|
|
|
|
}
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::Put(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& key, const SliceParts& value) {
|
2018-02-09 23:50:09 +01:00
|
|
|
Status s = CheckSlicePartsLength(key, value);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
LocalSavePoint save(b);
|
2014-04-22 20:27:33 +02:00
|
|
|
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
|
|
|
}
|
2021-09-13 00:33:15 +02:00
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, key);
|
2014-04-22 20:27:33 +02:00
|
|
|
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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
|
|
|
b->prot_info_->entries_.emplace_back(ProtectionInfo64()
|
|
|
|
.ProtectKVO(key, value, kTypeValue)
|
|
|
|
.ProtectC(column_family_id));
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::Put(ColumnFamilyHandle* column_family, const SliceParts& key,
|
|
|
|
const SliceParts& value) {
|
|
|
|
return WriteBatchInternal::Put(this, GetColumnFamilyID(column_family), key,
|
|
|
|
value);
|
2014-04-22 20:27:33 +02:00
|
|
|
}
|
2014-03-14 19:26:13 +01:00
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::InsertNoop(WriteBatch* b) {
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeNoop));
|
2017-04-11 00:38:34 +02:00
|
|
|
return Status::OK();
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
}
|
|
|
|
|
2017-11-11 20:23:43 +01:00
|
|
|
Status WriteBatchInternal::MarkEndPrepare(WriteBatch* b, const Slice& xid,
|
2018-06-29 03:46:39 +02:00
|
|
|
bool write_after_commit,
|
|
|
|
bool unprepared_batch) {
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
// a manually constructed batch can only contain one prepare section
|
|
|
|
assert(b->rep_[12] == static_cast<char>(kTypeNoop));
|
|
|
|
|
|
|
|
// all savepoints up to this point are cleared
|
|
|
|
if (b->save_points_ != nullptr) {
|
|
|
|
while (!b->save_points_->stack.empty()) {
|
|
|
|
b->save_points_->stack.pop();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// rewrite noop as begin marker
|
2018-06-29 03:46:39 +02:00
|
|
|
b->rep_[12] = static_cast<char>(
|
|
|
|
write_after_commit ? kTypeBeginPrepareXID
|
|
|
|
: (unprepared_batch ? kTypeBeginUnprepareXID
|
|
|
|
: kTypeBeginPersistedPrepareXID));
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeEndPrepareXID));
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, xid);
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_END_PREPARE |
|
|
|
|
ContentFlags::HAS_BEGIN_PREPARE,
|
|
|
|
std::memory_order_relaxed);
|
2018-07-24 09:09:18 +02:00
|
|
|
if (unprepared_batch) {
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_BEGIN_UNPREPARE,
|
|
|
|
std::memory_order_relaxed);
|
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return Status::OK();
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::MarkCommit(WriteBatch* b, const Slice& xid) {
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeCommitXID));
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, xid);
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_COMMIT,
|
|
|
|
std::memory_order_relaxed);
|
2017-04-11 00:38:34 +02:00
|
|
|
return Status::OK();
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
}
|
|
|
|
|
2021-12-10 20:03:39 +01:00
|
|
|
Status WriteBatchInternal::MarkCommitWithTimestamp(WriteBatch* b,
|
|
|
|
const Slice& xid,
|
|
|
|
const Slice& commit_ts) {
|
|
|
|
assert(!commit_ts.empty());
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeCommitXIDAndTimestamp));
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, commit_ts);
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, xid);
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_COMMIT,
|
|
|
|
std::memory_order_relaxed);
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::MarkRollback(WriteBatch* b, const Slice& xid) {
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
b->rep_.push_back(static_cast<char>(kTypeRollbackXID));
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, xid);
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_ROLLBACK,
|
|
|
|
std::memory_order_relaxed);
|
2017-04-11 00:38:34 +02:00
|
|
|
return Status::OK();
|
Modification of WriteBatch to support two phase commit
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.
Test Plan: single unit test in write_batch_test.
Reviewers: hermanlee4, sdong, anthony
Subscribers: leveldb, dhruba, vasilep, andrewkr
Differential Revision: https://reviews.facebook.net/D57867
2016-04-08 08:35:51 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::Delete(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& key) {
|
|
|
|
LocalSavePoint save(b);
|
2014-04-22 20:27:33 +02:00
|
|
|
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
|
|
|
}
|
2021-09-13 00:33:15 +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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
b->prot_info_->entries_.emplace_back(
|
|
|
|
ProtectionInfo64()
|
2021-09-14 22:13:36 +02:00
|
|
|
.ProtectKVO(key, "" /* value */, kTypeDeletion)
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
.ProtectC(column_family_id));
|
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2013-11-07 21:37:58 +01:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::Delete(ColumnFamilyHandle* column_family, const Slice& key) {
|
|
|
|
return WriteBatchInternal::Delete(this, GetColumnFamilyID(column_family),
|
|
|
|
key);
|
2014-07-10 18:31:42 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::Delete(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& key) {
|
|
|
|
LocalSavePoint save(b);
|
2014-07-10 18:31:42 +02:00
|
|
|
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);
|
|
|
|
}
|
2021-09-13 00:33:15 +02:00
|
|
|
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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
b->prot_info_->entries_.emplace_back(
|
|
|
|
ProtectionInfo64()
|
2021-09-14 22:13:36 +02:00
|
|
|
.ProtectKVO(key,
|
|
|
|
SliceParts(nullptr /* _parts */, 0 /* _num_parts */),
|
|
|
|
kTypeDeletion)
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
.ProtectC(column_family_id));
|
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2014-07-10 18:31:42 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::Delete(ColumnFamilyHandle* column_family,
|
|
|
|
const SliceParts& key) {
|
|
|
|
return WriteBatchInternal::Delete(this, GetColumnFamilyID(column_family),
|
|
|
|
key);
|
2014-04-22 20:27:33 +02:00
|
|
|
}
|
2014-03-14 19:26:13 +01:00
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::SingleDelete(WriteBatch* b,
|
|
|
|
uint32_t column_family_id,
|
|
|
|
const Slice& key) {
|
|
|
|
LocalSavePoint save(b);
|
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
|
|
|
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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
2021-09-13 00:33:15 +02:00
|
|
|
b->prot_info_->entries_.emplace_back(
|
|
|
|
ProtectionInfo64()
|
2021-09-14 22:13:36 +02:00
|
|
|
.ProtectKVO(key, "" /* value */, kTypeSingleDeletion)
|
2021-09-13 00:33:15 +02:00
|
|
|
.ProtectC(column_family_id));
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
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
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::SingleDelete(ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& key) {
|
|
|
|
return WriteBatchInternal::SingleDelete(
|
|
|
|
this, GetColumnFamilyID(column_family), key);
|
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
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::SingleDelete(WriteBatch* b,
|
|
|
|
uint32_t column_family_id,
|
|
|
|
const SliceParts& key) {
|
|
|
|
LocalSavePoint save(b);
|
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
|
|
|
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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
b->prot_info_->entries_.emplace_back(
|
|
|
|
ProtectionInfo64()
|
2021-09-14 22:13:36 +02:00
|
|
|
.ProtectKVO(key,
|
|
|
|
SliceParts(nullptr /* _parts */,
|
|
|
|
0 /* _num_parts */) /* value */,
|
|
|
|
kTypeSingleDeletion)
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
.ProtectC(column_family_id));
|
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
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
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::SingleDelete(ColumnFamilyHandle* column_family,
|
|
|
|
const SliceParts& key) {
|
|
|
|
return WriteBatchInternal::SingleDelete(
|
|
|
|
this, GetColumnFamilyID(column_family), key);
|
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
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::DeleteRange(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& begin_key,
|
|
|
|
const Slice& end_key) {
|
|
|
|
LocalSavePoint save(b);
|
2016-08-16 17:16:04 +02:00
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeRangeDeletion));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyRangeDeletion));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, begin_key);
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, end_key);
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_DELETE_RANGE,
|
|
|
|
std::memory_order_relaxed);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
// In `DeleteRange()`, the end key is treated as the value.
|
2021-09-13 00:33:15 +02:00
|
|
|
b->prot_info_->entries_.emplace_back(
|
|
|
|
ProtectionInfo64()
|
2021-09-14 22:13:36 +02:00
|
|
|
.ProtectKVO(begin_key, end_key, kTypeRangeDeletion)
|
2021-09-13 00:33:15 +02:00
|
|
|
.ProtectC(column_family_id));
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2016-08-16 17:16:04 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::DeleteRange(ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& begin_key, const Slice& end_key) {
|
|
|
|
return WriteBatchInternal::DeleteRange(this, GetColumnFamilyID(column_family),
|
|
|
|
begin_key, end_key);
|
2016-08-16 17:16:04 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::DeleteRange(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& begin_key,
|
|
|
|
const SliceParts& end_key) {
|
|
|
|
LocalSavePoint save(b);
|
2016-08-16 17:16:04 +02:00
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeRangeDeletion));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyRangeDeletion));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, begin_key);
|
|
|
|
PutLengthPrefixedSliceParts(&b->rep_, end_key);
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_DELETE_RANGE,
|
|
|
|
std::memory_order_relaxed);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
// In `DeleteRange()`, the end key is treated as the value.
|
2021-09-13 00:33:15 +02:00
|
|
|
b->prot_info_->entries_.emplace_back(
|
|
|
|
ProtectionInfo64()
|
2021-09-14 22:13:36 +02:00
|
|
|
.ProtectKVO(begin_key, end_key, kTypeRangeDeletion)
|
2021-09-13 00:33:15 +02:00
|
|
|
.ProtectC(column_family_id));
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2016-08-16 17:16:04 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::DeleteRange(ColumnFamilyHandle* column_family,
|
|
|
|
const SliceParts& begin_key,
|
|
|
|
const SliceParts& end_key) {
|
|
|
|
return WriteBatchInternal::DeleteRange(this, GetColumnFamilyID(column_family),
|
|
|
|
begin_key, end_key);
|
2016-08-16 17:16:04 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::Merge(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const Slice& key, const Slice& value) {
|
2018-02-09 23:50:09 +01:00
|
|
|
if (key.size() > size_t{port::kMaxUint32}) {
|
|
|
|
return Status::InvalidArgument("key is too large");
|
|
|
|
}
|
|
|
|
if (value.size() > size_t{port::kMaxUint32}) {
|
|
|
|
return Status::InvalidArgument("value is too large");
|
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
LocalSavePoint save(b);
|
2014-04-22 20:27:33 +02:00
|
|
|
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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
|
|
|
b->prot_info_->entries_.emplace_back(ProtectionInfo64()
|
|
|
|
.ProtectKVO(key, value, kTypeMerge)
|
|
|
|
.ProtectC(column_family_id));
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::Merge(ColumnFamilyHandle* column_family, const Slice& key,
|
|
|
|
const Slice& value) {
|
|
|
|
return WriteBatchInternal::Merge(this, GetColumnFamilyID(column_family), key,
|
|
|
|
value);
|
2013-03-21 23:59:47 +01:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::Merge(WriteBatch* b, uint32_t column_family_id,
|
|
|
|
const SliceParts& key,
|
|
|
|
const SliceParts& value) {
|
2018-02-09 23:50:09 +01:00
|
|
|
Status s = CheckSlicePartsLength(key, value);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
LocalSavePoint save(b);
|
2015-05-28 01:59:22 +02:00
|
|
|
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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
|
|
|
b->prot_info_->entries_.emplace_back(ProtectionInfo64()
|
|
|
|
.ProtectKVO(key, value, kTypeMerge)
|
|
|
|
.ProtectC(column_family_id));
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
}
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2015-05-28 01:59:22 +02:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::Merge(ColumnFamilyHandle* column_family,
|
|
|
|
const SliceParts& key, const SliceParts& value) {
|
|
|
|
return WriteBatchInternal::Merge(this, GetColumnFamilyID(column_family), key,
|
|
|
|
value);
|
2015-05-28 01:59:22 +02:00
|
|
|
}
|
|
|
|
|
2017-10-03 18:08:07 +02:00
|
|
|
Status WriteBatchInternal::PutBlobIndex(WriteBatch* b,
|
|
|
|
uint32_t column_family_id,
|
|
|
|
const Slice& key, const Slice& value) {
|
|
|
|
LocalSavePoint save(b);
|
|
|
|
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
|
|
|
|
if (column_family_id == 0) {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeBlobIndex));
|
|
|
|
} else {
|
|
|
|
b->rep_.push_back(static_cast<char>(kTypeColumnFamilyBlobIndex));
|
|
|
|
PutVarint32(&b->rep_, column_family_id);
|
|
|
|
}
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, key);
|
|
|
|
PutLengthPrefixedSlice(&b->rep_, value);
|
|
|
|
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
|
|
|
|
ContentFlags::HAS_BLOB_INDEX,
|
|
|
|
std::memory_order_relaxed);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (b->prot_info_ != nullptr) {
|
|
|
|
// See comment in first `WriteBatchInternal::Put()` overload concerning the
|
2021-09-14 22:13:36 +02:00
|
|
|
// `ValueType` argument passed to `ProtectKVO()`.
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
b->prot_info_->entries_.emplace_back(
|
|
|
|
ProtectionInfo64()
|
2021-09-14 22:13:36 +02:00
|
|
|
.ProtectKVO(key, value, kTypeBlobIndex)
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
.ProtectC(column_family_id));
|
|
|
|
}
|
2017-10-03 18:08:07 +02:00
|
|
|
return save.commit();
|
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatch::PutLogData(const Slice& blob) {
|
|
|
|
LocalSavePoint save(this);
|
2013-08-15 01:32:46 +02:00
|
|
|
rep_.push_back(static_cast<char>(kTypeLogData));
|
|
|
|
PutLengthPrefixedSlice(&rep_, blob);
|
2017-04-11 00:38:34 +02:00
|
|
|
return save.commit();
|
2013-08-15 01:32:46 +02:00
|
|
|
}
|
2013-03-21 23:59:47 +01:00
|
|
|
|
2015-07-11 05:15:45 +02:00
|
|
|
void WriteBatch::SetSavePoint() {
|
|
|
|
if (save_points_ == nullptr) {
|
refactor SavePoints (#5192)
Summary:
Savepoints are assumed to be used in a stack-wise fashion (only
the top element should be used), so they were stored by `WriteBatch`
in a member variable `save_points` using an std::stack.
Conceptually this is fine, but the implementation had a few issues:
- the `save_points_` instance variable was a plain pointer to a heap-
allocated `SavePoints` struct. The destructor of `WriteBatch` simply
deletes this pointer. However, the copy constructor of WriteBatch
just copied that pointer, meaning that copying a WriteBatch with
active savepoints will very likely have crashed before. Now a proper
copy of the savepoints is made in the copy constructor, and not just
a copy of the pointer
- `save_points_` was an std::stack, which defaults to `std::deque` for
the underlying container. A deque is a bit over the top here, as we
only need access to the most recent savepoint (i.e. stack.top()) but
never any elements at the front. std::deque is rather expensive to
initialize in common environments. For example, the STL implementation
shipped with GNU g++ will perform a heap allocation of more than 500
bytes to create an empty deque object. Although the `save_points_`
container is created lazily by RocksDB, moving from a deque to a plain
`std::vector` is much more memory-efficient. So `save_points_` is now
a vector.
- `save_points_` was changed from a plain pointer to an `std::unique_ptr`,
making ownership more explicit.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5192
Differential Revision: D15024074
Pulled By: maysamyabandeh
fbshipit-source-id: 5b128786d3789cde94e46465c9e91badd07a25d7
2019-04-20 05:30:03 +02:00
|
|
|
save_points_.reset(new SavePoints());
|
2015-07-11 05:15:45 +02:00
|
|
|
}
|
|
|
|
// Record length and count of current batch of writes.
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02: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());
|
2019-09-09 20:22:28 +02:00
|
|
|
assert(static_cast<uint32_t>(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);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (prot_info_ != nullptr) {
|
|
|
|
prot_info_->entries_.resize(savepoint.count);
|
|
|
|
}
|
2015-07-11 05:15:45 +02:00
|
|
|
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();
|
|
|
|
}
|
|
|
|
|
2017-05-03 19:54:07 +02:00
|
|
|
Status WriteBatch::PopSavePoint() {
|
|
|
|
if (save_points_ == nullptr || save_points_->stack.size() == 0) {
|
|
|
|
return Status::NotFound();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Pop the most recent savepoint off the stack
|
|
|
|
save_points_->stack.pop();
|
|
|
|
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2021-12-11 05:32:31 +01:00
|
|
|
Status WriteBatch::AssignTimestamp(
|
|
|
|
const Slice& ts, std::function<Status(uint32_t, size_t&)> checker) {
|
|
|
|
TimestampAssigner ts_assigner(prot_info_.get(), std::move(checker), ts);
|
|
|
|
return Iterate(&ts_assigner);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status WriteBatch::AssignTimestamps(
|
|
|
|
const std::vector<Slice>& ts_list,
|
|
|
|
std::function<Status(uint32_t, size_t&)> checker) {
|
|
|
|
SimpleListTimestampAssigner ts_assigner(prot_info_.get(), std::move(checker),
|
|
|
|
ts_list);
|
|
|
|
return Iterate(&ts_assigner);
|
|
|
|
}
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
class MemTableInserter : public WriteBatch::Handler {
|
2017-03-22 19:07:52 +01:00
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
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_;
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
TrimHistoryScheduler* const trim_history_scheduler_;
|
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 ignore_missing_column_families_;
|
2016-04-18 20:11:51 +02:00
|
|
|
const uint64_t recovering_log_number_;
|
|
|
|
// log number that all Memtables inserted into should reference
|
|
|
|
uint64_t log_number_ref_;
|
2013-07-13 01:56:52 +02:00
|
|
|
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
|
|
|
const bool concurrent_memtable_writes_;
|
2017-03-22 19:07:52 +01:00
|
|
|
bool post_info_created_;
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const WriteBatch::ProtectionInfo* prot_info_;
|
|
|
|
size_t prot_info_idx_;
|
2017-03-22 19:07:52 +01:00
|
|
|
|
Ignore stale logs while restarting DBs
Summary:
Stale log files can be deleted out of order. This can happen for various reasons. One of the reason is that no data is ever inserted to a column family and we have an optimization to update its log number, but not all the old log files are cleaned up (the case shown in the unit tests added). It can also happen when we simply delete multiple log files out of order.
This causes data corruption because we simply increase seqID after processing the next row and we may end up with writing data with smaller seqID than what is already flushed to memtables.
In DB recovery, for the oldest files we are replaying, if there it contains no data for any column family, we ignore the sequence IDs in the file.
Test Plan: Add two unit tests that fail without the fix.
Reviewers: IslamAbdelRahman, igor, yiwu
Reviewed By: yiwu
Subscribers: hermanlee4, yoshinorim, leveldb, andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D60891
2016-07-19 20:48:00 +02:00
|
|
|
bool* has_valid_writes_;
|
2017-03-22 19:07:52 +01:00
|
|
|
// On some (!) platforms just default creating
|
|
|
|
// a map is too expensive in the Write() path as they
|
|
|
|
// cause memory allocations though unused.
|
|
|
|
// Make creation optional but do not incur
|
2018-11-09 20:17:34 +01:00
|
|
|
// std::unique_ptr additional allocation
|
2017-10-03 05:40:23 +02:00
|
|
|
using MemPostInfoMap = std::map<MemTable*, MemTablePostProcessInfo>;
|
|
|
|
using PostMapType = std::aligned_storage<sizeof(MemPostInfoMap)>::type;
|
2017-03-22 19:07:52 +01:00
|
|
|
PostMapType mem_post_info_map_;
|
2016-04-18 20:11:51 +02:00
|
|
|
// current recovered transaction we are rebuilding (recovery)
|
|
|
|
WriteBatch* rebuilding_trx_;
|
2017-09-29 01:43:04 +02:00
|
|
|
SequenceNumber rebuilding_trx_seq_;
|
2017-09-18 23:36:53 +02:00
|
|
|
// Increase seq number once per each write batch. Otherwise increase it once
|
|
|
|
// per key.
|
|
|
|
bool seq_per_batch_;
|
2017-09-29 01:43:04 +02:00
|
|
|
// Whether the memtable write will be done only after the commit
|
|
|
|
bool write_after_commit_;
|
2018-06-29 03:46:39 +02:00
|
|
|
// Whether memtable write can be done before prepare
|
|
|
|
bool write_before_prepare_;
|
2018-07-07 02:17:36 +02:00
|
|
|
// Whether this batch was unprepared or not
|
|
|
|
bool unprepared_batch_;
|
2018-03-14 08:55:04 +01:00
|
|
|
using DupDetector = std::aligned_storage<sizeof(DuplicateDetector)>::type;
|
|
|
|
DupDetector duplicate_detector_;
|
|
|
|
bool dup_dectector_on_;
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2019-09-13 01:53:31 +02:00
|
|
|
bool hint_per_batch_;
|
|
|
|
bool hint_created_;
|
|
|
|
// Hints for this batch
|
|
|
|
using HintMap = std::unordered_map<MemTable*, void*>;
|
|
|
|
using HintMapType = std::aligned_storage<sizeof(HintMap)>::type;
|
|
|
|
HintMapType hint_;
|
|
|
|
|
|
|
|
HintMap& GetHintMap() {
|
|
|
|
assert(hint_per_batch_);
|
|
|
|
if (!hint_created_) {
|
|
|
|
new (&hint_) HintMap();
|
|
|
|
hint_created_ = true;
|
|
|
|
}
|
|
|
|
return *reinterpret_cast<HintMap*>(&hint_);
|
|
|
|
}
|
|
|
|
|
2017-03-22 19:07:52 +01:00
|
|
|
MemPostInfoMap& GetPostMap() {
|
|
|
|
assert(concurrent_memtable_writes_);
|
|
|
|
if(!post_info_created_) {
|
|
|
|
new (&mem_post_info_map_) MemPostInfoMap();
|
|
|
|
post_info_created_ = true;
|
|
|
|
}
|
|
|
|
return *reinterpret_cast<MemPostInfoMap*>(&mem_post_info_map_);
|
|
|
|
}
|
|
|
|
|
2018-03-14 08:55:04 +01:00
|
|
|
bool IsDuplicateKeySeq(uint32_t column_family_id, const Slice& key) {
|
|
|
|
assert(!write_after_commit_);
|
|
|
|
assert(rebuilding_trx_ != nullptr);
|
|
|
|
if (!dup_dectector_on_) {
|
|
|
|
new (&duplicate_detector_) DuplicateDetector(db_);
|
|
|
|
dup_dectector_on_ = true;
|
|
|
|
}
|
|
|
|
return reinterpret_cast<DuplicateDetector*>
|
|
|
|
(&duplicate_detector_)->IsDuplicateKeySeq(column_family_id, key, sequence_);
|
|
|
|
}
|
|
|
|
|
2021-09-14 22:13:36 +02:00
|
|
|
const ProtectionInfoKVOC64* NextProtectionInfo() {
|
|
|
|
const ProtectionInfoKVOC64* res = nullptr;
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (prot_info_ != nullptr) {
|
|
|
|
assert(prot_info_idx_ < prot_info_->entries_.size());
|
|
|
|
res = &prot_info_->entries_[prot_info_idx_];
|
|
|
|
++prot_info_idx_;
|
|
|
|
}
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2017-11-11 20:23:43 +01:00
|
|
|
protected:
|
2019-02-19 22:36:04 +01:00
|
|
|
bool WriteBeforePrepare() const override { return write_before_prepare_; }
|
|
|
|
bool WriteAfterCommit() const override { return write_after_commit_; }
|
2017-11-11 20:23:43 +01:00
|
|
|
|
2017-09-18 23:36:53 +02:00
|
|
|
public:
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
// cf_mems should not be shared with concurrent inserters
|
2017-09-18 23:36:53 +02:00
|
|
|
MemTableInserter(SequenceNumber _sequence, ColumnFamilyMemTables* cf_mems,
|
|
|
|
FlushScheduler* flush_scheduler,
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
TrimHistoryScheduler* trim_history_scheduler,
|
2017-09-18 23:36:53 +02:00
|
|
|
bool ignore_missing_column_families,
|
|
|
|
uint64_t recovering_log_number, DB* db,
|
|
|
|
bool concurrent_memtable_writes,
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const WriteBatch::ProtectionInfo* prot_info,
|
2018-06-29 03:46:39 +02:00
|
|
|
bool* has_valid_writes = nullptr, bool seq_per_batch = false,
|
2019-09-13 01:53:31 +02:00
|
|
|
bool batch_per_txn = true, bool hint_per_batch = false)
|
2017-09-18 23:36:53 +02:00
|
|
|
: sequence_(_sequence),
|
|
|
|
cf_mems_(cf_mems),
|
|
|
|
flush_scheduler_(flush_scheduler),
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
trim_history_scheduler_(trim_history_scheduler),
|
2017-09-18 23:36:53 +02:00
|
|
|
ignore_missing_column_families_(ignore_missing_column_families),
|
|
|
|
recovering_log_number_(recovering_log_number),
|
|
|
|
log_number_ref_(0),
|
2020-04-29 22:06:27 +02:00
|
|
|
db_(static_cast_with_check<DBImpl>(db)),
|
2017-09-18 23:36:53 +02:00
|
|
|
concurrent_memtable_writes_(concurrent_memtable_writes),
|
|
|
|
post_info_created_(false),
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
prot_info_(prot_info),
|
|
|
|
prot_info_idx_(0),
|
2017-09-18 23:36:53 +02:00
|
|
|
has_valid_writes_(has_valid_writes),
|
|
|
|
rebuilding_trx_(nullptr),
|
2017-12-07 20:52:12 +01:00
|
|
|
rebuilding_trx_seq_(0),
|
2017-09-29 01:43:04 +02:00
|
|
|
seq_per_batch_(seq_per_batch),
|
|
|
|
// Write after commit currently uses one seq per key (instead of per
|
|
|
|
// batch). So seq_per_batch being false indicates write_after_commit
|
|
|
|
// approach.
|
2018-03-05 19:48:29 +01:00
|
|
|
write_after_commit_(!seq_per_batch),
|
2018-06-29 03:46:39 +02:00
|
|
|
// WriteUnprepared can write WriteBatches per transaction, so
|
|
|
|
// batch_per_txn being false indicates write_before_prepare.
|
|
|
|
write_before_prepare_(!batch_per_txn),
|
2018-07-07 02:17:36 +02:00
|
|
|
unprepared_batch_(false),
|
2018-03-14 08:55:04 +01:00
|
|
|
duplicate_detector_(),
|
2019-09-13 01:53:31 +02:00
|
|
|
dup_dectector_on_(false),
|
|
|
|
hint_per_batch_(hint_per_batch),
|
|
|
|
hint_created_(false) {
|
2017-09-18 23:36:53 +02:00
|
|
|
assert(cf_mems_);
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
|
|
|
|
2019-02-19 22:36:04 +01:00
|
|
|
~MemTableInserter() override {
|
2018-03-14 08:55:04 +01:00
|
|
|
if (dup_dectector_on_) {
|
|
|
|
reinterpret_cast<DuplicateDetector*>
|
|
|
|
(&duplicate_detector_)->~DuplicateDetector();
|
|
|
|
}
|
2017-03-22 19:07:52 +01:00
|
|
|
if (post_info_created_) {
|
|
|
|
reinterpret_cast<MemPostInfoMap*>
|
|
|
|
(&mem_post_info_map_)->~MemPostInfoMap();
|
|
|
|
}
|
2019-09-13 01:53:31 +02:00
|
|
|
if (hint_created_) {
|
|
|
|
for (auto iter : GetHintMap()) {
|
|
|
|
delete[] reinterpret_cast<char*>(iter.second);
|
|
|
|
}
|
|
|
|
reinterpret_cast<HintMap*>(&hint_)->~HintMap();
|
|
|
|
}
|
2017-11-14 17:42:14 +01:00
|
|
|
delete rebuilding_trx_;
|
2017-03-22 19:07:52 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
MemTableInserter(const MemTableInserter&) = delete;
|
|
|
|
MemTableInserter& operator=(const MemTableInserter&) = delete;
|
|
|
|
|
2017-11-15 17:19:57 +01:00
|
|
|
// The batch seq is regularly restarted; In normal mode it is set when
|
|
|
|
// MemTableInserter is constructed in the write thread and in recovery mode it
|
|
|
|
// is set when a batch, which is tagged with seq, is read from the WAL.
|
|
|
|
// Within a sequenced batch, which could be a merge of multiple batches, we
|
|
|
|
// have two policies to advance the seq: i) seq_per_key (default) and ii)
|
2018-03-08 19:18:34 +01:00
|
|
|
// seq_per_batch. To implement the latter we need to mark the boundary between
|
2017-11-15 17:19:57 +01:00
|
|
|
// the individual batches. The approach is this: 1) Use the terminating
|
2018-03-08 19:18:34 +01:00
|
|
|
// markers to indicate the boundary (kTypeEndPrepareXID, kTypeCommitXID,
|
|
|
|
// kTypeRollbackXID) 2) Terminate a batch with kTypeNoop in the absence of a
|
|
|
|
// natural boundary marker.
|
2017-09-18 23:36:53 +02:00
|
|
|
void MaybeAdvanceSeq(bool batch_boundry = false) {
|
|
|
|
if (batch_boundry == seq_per_batch_) {
|
|
|
|
sequence_++;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-04-18 20:11:51 +02:00
|
|
|
void set_log_number_ref(uint64_t log) { log_number_ref_ = log; }
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
void set_prot_info(const WriteBatch::ProtectionInfo* prot_info) {
|
|
|
|
prot_info_ = prot_info;
|
|
|
|
prot_info_idx_ = 0;
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2017-05-31 19:45:47 +02:00
|
|
|
SequenceNumber sequence() const { return sequence_; }
|
[rocksdb] Recovery path sequence miscount fix
Summary:
Consider the following WAL with 4 batch entries prefixed with their sequence at time of memtable insert.
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(a)]
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(b)]
[4: COMMIT(a)]
[7: COMMIT(b)]
The first two batches do not consume any sequence numbers so are both prefixed with seq=1.
For 2pc commit, memtable insertion takes place before COMMIT batch is written to WAL.
We can see that sequence number consumption takes place between WAL entries giving us the seemingly sparse sequence prefix for WAL entries.
This is a valid WAL.
Because with 2PC markers one WriteBatch points to another batch containing its inserts a writebatch can consume more or less sequence numbers than the number of sequence consuming entries that it contains.
We can see that, given the entries in the WAL, 6 sequence ids were consumed. Yet on recovery the maximum sequence consumed would be 7 + 3 (the number of sequence numbers consumed by COMMIT(b))
So, now upon recovery we must track the actual consumption of sequence numbers.
In the provided scenario there will be no sequence gaps, but it is possible to produce a sequence gap. This should not be a problem though. correct?
Test Plan: provided test.
Reviewers: sdong
Subscribers: andrewkr, leveldb, dhruba, hermanlee4
Differential Revision: https://reviews.facebook.net/D57645
2016-05-04 23:02:27 +02:00
|
|
|
|
2016-07-07 23:45:29 +02:00
|
|
|
void PostProcess() {
|
2017-03-22 19:07:52 +01:00
|
|
|
assert(concurrent_memtable_writes_);
|
|
|
|
// If post info was not created there is nothing
|
|
|
|
// to process and no need to create on demand
|
|
|
|
if(post_info_created_) {
|
|
|
|
for (auto& pair : GetPostMap()) {
|
|
|
|
pair.first->BatchPostProcess(pair.second);
|
|
|
|
}
|
2016-07-07 23:45:29 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
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;
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
if (recovering_log_number_ != 0 &&
|
|
|
|
recovering_log_number_ < cf_mems_->GetLogNumber()) {
|
|
|
|
// This is true only in recovery environment (recovering_log_number_ is
|
|
|
|
// always 0 in
|
2014-09-02 22:29:05 +02:00
|
|
|
// non-recovery, regular write code-path)
|
2016-04-18 20:11:51 +02:00
|
|
|
// * If recovering_log_number_ < cf_mems_->GetLogNumber(), this means that
|
|
|
|
// column
|
2014-03-03 23:30:36 +01:00
|
|
|
// 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;
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
|
Ignore stale logs while restarting DBs
Summary:
Stale log files can be deleted out of order. This can happen for various reasons. One of the reason is that no data is ever inserted to a column family and we have an optimization to update its log number, but not all the old log files are cleaned up (the case shown in the unit tests added). It can also happen when we simply delete multiple log files out of order.
This causes data corruption because we simply increase seqID after processing the next row and we may end up with writing data with smaller seqID than what is already flushed to memtables.
In DB recovery, for the oldest files we are replaying, if there it contains no data for any column family, we ignore the sequence IDs in the file.
Test Plan: Add two unit tests that fail without the fix.
Reviewers: IslamAbdelRahman, igor, yiwu
Reviewed By: yiwu
Subscribers: hermanlee4, yoshinorim, leveldb, andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D60891
2016-07-19 20:48:00 +02:00
|
|
|
if (has_valid_writes_ != nullptr) {
|
|
|
|
*has_valid_writes_ = true;
|
|
|
|
}
|
|
|
|
|
2016-04-18 20:11:51 +02:00
|
|
|
if (log_number_ref_ > 0) {
|
|
|
|
cf_mems_->GetMemTable()->RefLogContainingPrepSection(log_number_ref_);
|
|
|
|
}
|
|
|
|
|
2014-03-03 23:30:36 +01:00
|
|
|
return true;
|
|
|
|
}
|
2015-11-06 16:29:10 +01:00
|
|
|
|
2017-10-03 18:08:07 +02:00
|
|
|
Status PutCFImpl(uint32_t column_family_id, const Slice& key,
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const Slice& value, ValueType value_type,
|
2021-09-14 22:13:36 +02:00
|
|
|
const ProtectionInfoKVOS64* kv_prot_info) {
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
|
|
|
if (UNLIKELY(write_after_commit_ && rebuilding_trx_ != nullptr)) {
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
return WriteBatchInternal::Put(rebuilding_trx_, column_family_id, key,
|
|
|
|
value);
|
2017-09-29 01:43:04 +02:00
|
|
|
// else insert the values to the memtable right away
|
2016-05-05 21:48:52 +02:00
|
|
|
}
|
|
|
|
|
2020-08-25 01:41:42 +02:00
|
|
|
Status ret_status;
|
|
|
|
if (UNLIKELY(!SeekToColumnFamily(column_family_id, &ret_status))) {
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok() && rebuilding_trx_ != nullptr) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2018-03-08 19:18:34 +01:00
|
|
|
// The CF is probably flushed and hence no need for insert but we still
|
2018-03-05 19:48:29 +01:00
|
|
|
// need to keep track of the keys for upcoming rollback/commit.
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::Put(rebuilding_trx_, column_family_id,
|
|
|
|
key, value);
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(IsDuplicateKeySeq(column_family_id, key));
|
|
|
|
}
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(false /* batch_boundary */);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
2020-08-25 01:41:42 +02:00
|
|
|
return ret_status;
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
assert(ret_status.ok());
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2014-02-06 01:02:48 +01:00
|
|
|
MemTable* mem = cf_mems_->GetMemTable();
|
2017-11-03 06:16:23 +01:00
|
|
|
auto* moptions = mem->GetImmutableMemTableOptions();
|
2018-02-06 03:32:54 +01:00
|
|
|
// inplace_update_support is inconsistent with snapshots, and therefore with
|
|
|
|
// any kind of transactions including the ones that use seq_per_batch
|
|
|
|
assert(!seq_per_batch_ || !moptions->inplace_update_support);
|
2014-09-09 03:46:52 +02:00
|
|
|
if (!moptions->inplace_update_support) {
|
2020-11-24 01:27:46 +01:00
|
|
|
ret_status =
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
mem->Add(sequence_, value_type, key, value, kv_prot_info,
|
2019-09-13 01:53:31 +02:00
|
|
|
concurrent_memtable_writes_, get_post_process_info(mem),
|
|
|
|
hint_per_batch_ ? &GetHintMap()[mem] : nullptr);
|
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_);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
ret_status = mem->Update(sequence_, key, value, kv_prot_info);
|
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_);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
ret_status = mem->UpdateCallback(sequence_, key, value, kv_prot_info);
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.IsNotFound()) {
|
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;
|
2017-10-03 05:40:23 +02:00
|
|
|
// it's going to be overwritten for sure, so no point caching data block
|
|
|
|
// containing the old version
|
|
|
|
ropts.fill_cache = false;
|
2014-01-14 16:55:16 +01:00
|
|
|
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();
|
2020-11-24 01:27:46 +01:00
|
|
|
Status get_status = Status::NotSupported();
|
2016-11-10 20:10:23 +01:00
|
|
|
if (db_ != nullptr && recovering_log_number_ == 0) {
|
2016-09-21 20:05:07 +02:00
|
|
|
if (cf_handle == nullptr) {
|
|
|
|
cf_handle = db_->DefaultColumnFamily();
|
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
get_status = db_->Get(ropts, cf_handle, key, &prev_value);
|
|
|
|
}
|
|
|
|
// Intentionally overwrites the `NotFound` in `ret_status`.
|
|
|
|
if (!get_status.ok() && !get_status.IsNotFound()) {
|
|
|
|
ret_status = get_status;
|
|
|
|
} else {
|
|
|
|
ret_status = Status::OK();
|
2014-02-11 02:04:44 +01:00
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
UpdateStatus update_status;
|
|
|
|
char* prev_buffer = const_cast<char*>(prev_value.c_str());
|
|
|
|
uint32_t prev_size = static_cast<uint32_t>(prev_value.size());
|
|
|
|
if (get_status.ok()) {
|
|
|
|
update_status = moptions->inplace_callback(prev_buffer, &prev_size,
|
|
|
|
value, &merged_value);
|
|
|
|
} else {
|
|
|
|
update_status = moptions->inplace_callback(
|
|
|
|
nullptr /* existing_value */, nullptr /* existing_value_size */,
|
|
|
|
value, &merged_value);
|
|
|
|
}
|
|
|
|
if (update_status == UpdateStatus::UPDATED_INPLACE) {
|
|
|
|
assert(get_status.ok());
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (kv_prot_info != nullptr) {
|
2021-09-14 22:13:36 +02:00
|
|
|
ProtectionInfoKVOS64 updated_kv_prot_info(*kv_prot_info);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
updated_kv_prot_info.UpdateV(value,
|
|
|
|
Slice(prev_buffer, prev_size));
|
|
|
|
// prev_value is updated in-place with final value.
|
|
|
|
ret_status = mem->Add(sequence_, value_type, key,
|
|
|
|
Slice(prev_buffer, prev_size),
|
|
|
|
&updated_kv_prot_info);
|
|
|
|
} else {
|
|
|
|
ret_status = mem->Add(sequence_, value_type, key,
|
|
|
|
Slice(prev_buffer, prev_size),
|
|
|
|
nullptr /* kv_prot_info */);
|
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
RecordTick(moptions->statistics, NUMBER_KEYS_WRITTEN);
|
|
|
|
}
|
|
|
|
} else if (update_status == UpdateStatus::UPDATED) {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (kv_prot_info != nullptr) {
|
2021-09-14 22:13:36 +02:00
|
|
|
ProtectionInfoKVOS64 updated_kv_prot_info(*kv_prot_info);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
updated_kv_prot_info.UpdateV(value, merged_value);
|
|
|
|
// merged_value contains the final value.
|
|
|
|
ret_status = mem->Add(sequence_, value_type, key,
|
|
|
|
Slice(merged_value), &updated_kv_prot_info);
|
|
|
|
} else {
|
|
|
|
// merged_value contains the final value.
|
|
|
|
ret_status =
|
|
|
|
mem->Add(sequence_, value_type, key, Slice(merged_value),
|
|
|
|
nullptr /* kv_prot_info */);
|
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
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
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
if (UNLIKELY(ret_status.IsTryAgain())) {
|
|
|
|
assert(seq_per_batch_);
|
|
|
|
const bool kBatchBoundary = true;
|
|
|
|
MaybeAdvanceSeq(kBatchBoundary);
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq();
|
|
|
|
CheckMemtableFull();
|
|
|
|
}
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
2020-11-24 01:27:46 +01:00
|
|
|
// If `ret_status` is `TryAgain` then the next (successful) try will add
|
|
|
|
// the key to the rebuilding transaction object. If `ret_status` is
|
|
|
|
// another non-OK `Status`, then the `rebuilding_trx_` will be thrown
|
|
|
|
// away. So we only need to add to it when `ret_status.ok()`.
|
|
|
|
if (UNLIKELY(ret_status.ok() && rebuilding_trx_ != nullptr)) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::Put(rebuilding_trx_, column_family_id,
|
|
|
|
key, value);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
2018-02-06 03:32:54 +01:00
|
|
|
return ret_status;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
2014-01-14 16:55:16 +01:00
|
|
|
|
2019-02-19 22:36:04 +01:00
|
|
|
Status PutCF(uint32_t column_family_id, const Slice& key,
|
|
|
|
const Slice& value) override {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const auto* kv_prot_info = NextProtectionInfo();
|
|
|
|
if (kv_prot_info != nullptr) {
|
|
|
|
// Memtable needs seqno, doesn't need CF ID
|
|
|
|
auto mem_kv_prot_info =
|
|
|
|
kv_prot_info->StripC(column_family_id).ProtectS(sequence_);
|
|
|
|
return PutCFImpl(column_family_id, key, value, kTypeValue,
|
|
|
|
&mem_kv_prot_info);
|
|
|
|
}
|
|
|
|
return PutCFImpl(column_family_id, key, value, kTypeValue,
|
|
|
|
nullptr /* kv_prot_info */);
|
2017-10-03 18:08:07 +02:00
|
|
|
}
|
|
|
|
|
2018-03-05 22:08:17 +01:00
|
|
|
Status DeleteImpl(uint32_t /*column_family_id*/, const Slice& key,
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const Slice& value, ValueType delete_type,
|
2021-09-14 22:13:36 +02:00
|
|
|
const ProtectionInfoKVOS64* kv_prot_info) {
|
2018-02-06 03:32:54 +01:00
|
|
|
Status ret_status;
|
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
|
|
|
MemTable* mem = cf_mems_->GetMemTable();
|
2020-11-24 01:27:46 +01:00
|
|
|
ret_status =
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
mem->Add(sequence_, delete_type, key, value, kv_prot_info,
|
2019-09-13 01:53:31 +02:00
|
|
|
concurrent_memtable_writes_, get_post_process_info(mem),
|
|
|
|
hint_per_batch_ ? &GetHintMap()[mem] : nullptr);
|
2020-11-24 01:27:46 +01:00
|
|
|
if (UNLIKELY(ret_status.IsTryAgain())) {
|
2018-02-06 03:32:54 +01:00
|
|
|
assert(seq_per_batch_);
|
2020-11-24 01:27:46 +01:00
|
|
|
const bool kBatchBoundary = true;
|
|
|
|
MaybeAdvanceSeq(kBatchBoundary);
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq();
|
|
|
|
CheckMemtableFull();
|
2018-02-06 03:32:54 +01:00
|
|
|
}
|
|
|
|
return ret_status;
|
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
|
|
|
}
|
|
|
|
|
2019-02-19 22:36:04 +01:00
|
|
|
Status DeleteCF(uint32_t column_family_id, const Slice& key) override {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const auto* kv_prot_info = NextProtectionInfo();
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
|
|
|
if (UNLIKELY(write_after_commit_ && rebuilding_trx_ != nullptr)) {
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
return WriteBatchInternal::Delete(rebuilding_trx_, column_family_id, key);
|
2017-09-29 01:43:04 +02:00
|
|
|
// else insert the values to the memtable right away
|
2016-05-05 21:48:52 +02:00
|
|
|
}
|
|
|
|
|
2020-08-25 01:41:42 +02:00
|
|
|
Status ret_status;
|
|
|
|
if (UNLIKELY(!SeekToColumnFamily(column_family_id, &ret_status))) {
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok() && rebuilding_trx_ != nullptr) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2018-03-08 19:18:34 +01:00
|
|
|
// The CF is probably flushed and hence no need for insert but we still
|
2018-03-05 19:48:29 +01:00
|
|
|
// need to keep track of the keys for upcoming rollback/commit.
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status =
|
|
|
|
WriteBatchInternal::Delete(rebuilding_trx_, column_family_id, key);
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(IsDuplicateKeySeq(column_family_id, key));
|
|
|
|
}
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(false /* batch_boundary */);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
2020-08-25 01:41:42 +02:00
|
|
|
return ret_status;
|
2016-04-18 20:11:51 +02:00
|
|
|
}
|
|
|
|
|
2020-05-28 19:37:57 +02:00
|
|
|
ColumnFamilyData* cfd = cf_mems_->current();
|
|
|
|
assert(!cfd || cfd->user_comparator());
|
|
|
|
const size_t ts_sz = (cfd && cfd->user_comparator())
|
|
|
|
? cfd->user_comparator()->timestamp_size()
|
|
|
|
: 0;
|
|
|
|
const ValueType delete_type =
|
|
|
|
(0 == ts_sz) ? kTypeDeletion : kTypeDeletionWithTimestamp;
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (kv_prot_info != nullptr) {
|
|
|
|
auto mem_kv_prot_info =
|
|
|
|
kv_prot_info->StripC(column_family_id).ProtectS(sequence_);
|
|
|
|
mem_kv_prot_info.UpdateO(kTypeDeletion, delete_type);
|
|
|
|
ret_status = DeleteImpl(column_family_id, key, Slice(), delete_type,
|
|
|
|
&mem_kv_prot_info);
|
|
|
|
} else {
|
|
|
|
ret_status = DeleteImpl(column_family_id, key, Slice(), delete_type,
|
|
|
|
nullptr /* kv_prot_info */);
|
|
|
|
}
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
2020-11-24 01:27:46 +01:00
|
|
|
// If `ret_status` is `TryAgain` then the next (successful) try will add
|
|
|
|
// the key to the rebuilding transaction object. If `ret_status` is
|
|
|
|
// another non-OK `Status`, then the `rebuilding_trx_` will be thrown
|
|
|
|
// away. So we only need to add to it when `ret_status.ok()`.
|
|
|
|
if (UNLIKELY(ret_status.ok() && rebuilding_trx_ != nullptr)) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status =
|
|
|
|
WriteBatchInternal::Delete(rebuilding_trx_, column_family_id, key);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
|
|
|
return ret_status;
|
2015-11-06 16:29:10 +01:00
|
|
|
}
|
|
|
|
|
2019-02-19 22:36:04 +01:00
|
|
|
Status SingleDeleteCF(uint32_t column_family_id, const Slice& key) override {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const auto* kv_prot_info = NextProtectionInfo();
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
|
|
|
if (UNLIKELY(write_after_commit_ && rebuilding_trx_ != nullptr)) {
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
return WriteBatchInternal::SingleDelete(rebuilding_trx_, column_family_id,
|
|
|
|
key);
|
2017-09-29 01:43:04 +02:00
|
|
|
// else insert the values to the memtable right away
|
2016-05-05 21:48:52 +02:00
|
|
|
}
|
|
|
|
|
2020-08-25 01:41:42 +02:00
|
|
|
Status ret_status;
|
|
|
|
if (UNLIKELY(!SeekToColumnFamily(column_family_id, &ret_status))) {
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok() && rebuilding_trx_ != nullptr) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2018-03-08 19:18:34 +01:00
|
|
|
// The CF is probably flushed and hence no need for insert but we still
|
2018-03-05 19:48:29 +01:00
|
|
|
// need to keep track of the keys for upcoming rollback/commit.
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::SingleDelete(rebuilding_trx_,
|
|
|
|
column_family_id, key);
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(IsDuplicateKeySeq(column_family_id, key));
|
|
|
|
}
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(false /* batch_boundary */);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
2020-08-25 01:41:42 +02:00
|
|
|
return ret_status;
|
2016-04-18 20:11:51 +02:00
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
assert(ret_status.ok());
|
2016-04-18 20:11:51 +02:00
|
|
|
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (kv_prot_info != nullptr) {
|
|
|
|
auto mem_kv_prot_info =
|
|
|
|
kv_prot_info->StripC(column_family_id).ProtectS(sequence_);
|
|
|
|
ret_status = DeleteImpl(column_family_id, key, Slice(),
|
|
|
|
kTypeSingleDeletion, &mem_kv_prot_info);
|
|
|
|
} else {
|
|
|
|
ret_status = DeleteImpl(column_family_id, key, Slice(),
|
|
|
|
kTypeSingleDeletion, nullptr /* kv_prot_info */);
|
|
|
|
}
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
2020-11-24 01:27:46 +01:00
|
|
|
// If `ret_status` is `TryAgain` then the next (successful) try will add
|
|
|
|
// the key to the rebuilding transaction object. If `ret_status` is
|
|
|
|
// another non-OK `Status`, then the `rebuilding_trx_` will be thrown
|
|
|
|
// away. So we only need to add to it when `ret_status.ok()`.
|
|
|
|
if (UNLIKELY(ret_status.ok() && rebuilding_trx_ != nullptr)) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::SingleDelete(rebuilding_trx_,
|
|
|
|
column_family_id, key);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
|
|
|
return ret_status;
|
2016-08-16 17:16:04 +02:00
|
|
|
}
|
|
|
|
|
2019-02-19 22:36:04 +01:00
|
|
|
Status DeleteRangeCF(uint32_t column_family_id, const Slice& begin_key,
|
|
|
|
const Slice& end_key) override {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const auto* kv_prot_info = NextProtectionInfo();
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
|
|
|
if (UNLIKELY(write_after_commit_ && rebuilding_trx_ != nullptr)) {
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
return WriteBatchInternal::DeleteRange(rebuilding_trx_, column_family_id,
|
|
|
|
begin_key, end_key);
|
2017-09-29 01:43:04 +02:00
|
|
|
// else insert the values to the memtable right away
|
2016-08-16 17:16:04 +02:00
|
|
|
}
|
|
|
|
|
2020-08-25 01:41:42 +02:00
|
|
|
Status ret_status;
|
|
|
|
if (UNLIKELY(!SeekToColumnFamily(column_family_id, &ret_status))) {
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok() && rebuilding_trx_ != nullptr) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2018-03-08 19:18:34 +01:00
|
|
|
// The CF is probably flushed and hence no need for insert but we still
|
2018-03-05 19:48:29 +01:00
|
|
|
// need to keep track of the keys for upcoming rollback/commit.
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::DeleteRange(
|
|
|
|
rebuilding_trx_, column_family_id, begin_key, end_key);
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(IsDuplicateKeySeq(column_family_id, begin_key));
|
|
|
|
}
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(false /* batch_boundary */);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
2020-08-25 01:41:42 +02:00
|
|
|
return ret_status;
|
2016-08-16 17:16:04 +02:00
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
assert(ret_status.ok());
|
|
|
|
|
2016-11-16 00:18:56 +01:00
|
|
|
if (db_ != nullptr) {
|
|
|
|
auto cf_handle = cf_mems_->GetColumnFamilyHandle();
|
|
|
|
if (cf_handle == nullptr) {
|
|
|
|
cf_handle = db_->DefaultColumnFamily();
|
|
|
|
}
|
2020-07-03 04:24:25 +02:00
|
|
|
auto* cfd =
|
|
|
|
static_cast_with_check<ColumnFamilyHandleImpl>(cf_handle)->cfd();
|
2016-11-16 00:18:56 +01:00
|
|
|
if (!cfd->is_delete_range_supported()) {
|
2020-11-24 01:27:46 +01:00
|
|
|
// TODO(ajkr): refactor `SeekToColumnFamily()` so it returns a `Status`.
|
|
|
|
ret_status.PermitUncheckedError();
|
2016-11-16 00:18:56 +01:00
|
|
|
return Status::NotSupported(
|
|
|
|
std::string("DeleteRange not supported for table type ") +
|
|
|
|
cfd->ioptions()->table_factory->Name() + " in CF " +
|
|
|
|
cfd->GetName());
|
|
|
|
}
|
2020-05-07 20:53:32 +02:00
|
|
|
int cmp = cfd->user_comparator()->Compare(begin_key, end_key);
|
|
|
|
if (cmp > 0) {
|
2020-11-24 01:27:46 +01:00
|
|
|
// TODO(ajkr): refactor `SeekToColumnFamily()` so it returns a `Status`.
|
|
|
|
ret_status.PermitUncheckedError();
|
2020-05-07 20:53:32 +02:00
|
|
|
// It's an empty range where endpoints appear mistaken. Don't bother
|
|
|
|
// applying it to the DB, and return an error to the user.
|
|
|
|
return Status::InvalidArgument("end key comes before start key");
|
|
|
|
} else if (cmp == 0) {
|
2020-11-24 01:27:46 +01:00
|
|
|
// TODO(ajkr): refactor `SeekToColumnFamily()` so it returns a `Status`.
|
|
|
|
ret_status.PermitUncheckedError();
|
2020-05-07 20:53:32 +02:00
|
|
|
// It's an empty range. Don't bother applying it to the DB.
|
|
|
|
return Status::OK();
|
|
|
|
}
|
2016-11-16 00:18:56 +01:00
|
|
|
}
|
2016-08-16 17:16:04 +02:00
|
|
|
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (kv_prot_info != nullptr) {
|
|
|
|
auto mem_kv_prot_info =
|
|
|
|
kv_prot_info->StripC(column_family_id).ProtectS(sequence_);
|
|
|
|
ret_status = DeleteImpl(column_family_id, begin_key, end_key,
|
|
|
|
kTypeRangeDeletion, &mem_kv_prot_info);
|
|
|
|
} else {
|
|
|
|
ret_status = DeleteImpl(column_family_id, begin_key, end_key,
|
|
|
|
kTypeRangeDeletion, nullptr /* kv_prot_info */);
|
|
|
|
}
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
2020-11-24 01:27:46 +01:00
|
|
|
// If `ret_status` is `TryAgain` then the next (successful) try will add
|
|
|
|
// the key to the rebuilding transaction object. If `ret_status` is
|
|
|
|
// another non-OK `Status`, then the `rebuilding_trx_` will be thrown
|
|
|
|
// away. So we only need to add to it when `ret_status.ok()`.
|
2018-03-05 19:48:29 +01:00
|
|
|
if (UNLIKELY(!ret_status.IsTryAgain() && rebuilding_trx_ != nullptr)) {
|
|
|
|
assert(!write_after_commit_);
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::DeleteRange(
|
|
|
|
rebuilding_trx_, column_family_id, begin_key, end_key);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
|
|
|
return ret_status;
|
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
|
|
|
}
|
|
|
|
|
2019-02-19 22:36:04 +01:00
|
|
|
Status MergeCF(uint32_t column_family_id, const Slice& key,
|
|
|
|
const Slice& value) override {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const auto* kv_prot_info = NextProtectionInfo();
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
|
|
|
if (UNLIKELY(write_after_commit_ && rebuilding_trx_ != nullptr)) {
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
return WriteBatchInternal::Merge(rebuilding_trx_, column_family_id, key,
|
|
|
|
value);
|
2017-09-29 01:43:04 +02:00
|
|
|
// else insert the values to the memtable right away
|
2016-05-05 21:48:52 +02:00
|
|
|
}
|
|
|
|
|
2020-08-25 01:41:42 +02:00
|
|
|
Status ret_status;
|
|
|
|
if (UNLIKELY(!SeekToColumnFamily(column_family_id, &ret_status))) {
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok() && rebuilding_trx_ != nullptr) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2018-03-08 19:18:34 +01:00
|
|
|
// The CF is probably flushed and hence no need for insert but we still
|
2018-03-05 19:48:29 +01:00
|
|
|
// need to keep track of the keys for upcoming rollback/commit.
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::Merge(rebuilding_trx_,
|
|
|
|
column_family_id, key, value);
|
2020-11-24 01:27:46 +01:00
|
|
|
if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(IsDuplicateKeySeq(column_family_id, key));
|
|
|
|
}
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq(false /* batch_boundary */);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
2020-08-25 01:41:42 +02:00
|
|
|
return ret_status;
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
assert(ret_status.ok());
|
2016-05-05 21:48:52 +02:00
|
|
|
|
2014-02-06 01:02:48 +01:00
|
|
|
MemTable* mem = cf_mems_->GetMemTable();
|
2017-11-03 06:16:23 +01:00
|
|
|
auto* moptions = mem->GetImmutableMemTableOptions();
|
2020-11-17 02:58:59 +01:00
|
|
|
if (moptions->merge_operator == nullptr) {
|
|
|
|
return Status::InvalidArgument(
|
|
|
|
"Merge requires `ColumnFamilyOptions::merge_operator != nullptr`");
|
|
|
|
}
|
2014-01-11 02:33:56 +01:00
|
|
|
bool perform_merge = false;
|
2019-05-14 02:43:47 +02:00
|
|
|
assert(!concurrent_memtable_writes_ ||
|
|
|
|
moptions->max_successive_merges == 0);
|
2014-01-11 02:33:56 +01:00
|
|
|
|
2016-09-21 20:05:07 +02:00
|
|
|
// If we pass DB through and options.max_successive_merges is hit
|
|
|
|
// during recovery, Get() will be issued which will try to acquire
|
|
|
|
// DB mutex and cause deadlock, as DB mutex is already held.
|
|
|
|
// So we disable merge in recovery
|
|
|
|
if (moptions->max_successive_merges > 0 && db_ != nullptr &&
|
|
|
|
recovering_log_number_ == 0) {
|
2019-05-14 02:43:47 +02:00
|
|
|
assert(!concurrent_memtable_writes_);
|
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();
|
|
|
|
}
|
2020-11-24 01:27:46 +01:00
|
|
|
Status get_status = db_->Get(read_options, cf_handle, key, &get_value);
|
|
|
|
if (!get_status.ok()) {
|
|
|
|
// Failed to read a key we know exists. Store the delta in memtable.
|
2014-07-28 21:05:36 +02:00
|
|
|
perform_merge = false;
|
2014-01-11 02:33:56 +01:00
|
|
|
} else {
|
2020-11-24 01:27:46 +01:00
|
|
|
Slice get_value_slice = Slice(get_value);
|
|
|
|
|
|
|
|
// 2) Apply this merge
|
|
|
|
auto merge_operator = moptions->merge_operator;
|
|
|
|
assert(merge_operator);
|
|
|
|
|
|
|
|
std::string new_value;
|
|
|
|
Status merge_status = MergeHelper::TimedFullMerge(
|
|
|
|
merge_operator, key, &get_value_slice, {value}, &new_value,
|
2021-03-15 12:32:24 +01:00
|
|
|
moptions->info_log, moptions->statistics,
|
|
|
|
SystemClock::Default().get());
|
2020-11-24 01:27:46 +01:00
|
|
|
|
|
|
|
if (!merge_status.ok()) {
|
|
|
|
// Failed to merge!
|
|
|
|
// Store the delta in memtable
|
|
|
|
perform_merge = false;
|
|
|
|
} else {
|
|
|
|
// 3) Add value to memtable
|
|
|
|
assert(!concurrent_memtable_writes_);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (kv_prot_info != nullptr) {
|
|
|
|
auto merged_kv_prot_info =
|
|
|
|
kv_prot_info->StripC(column_family_id).ProtectS(sequence_);
|
|
|
|
merged_kv_prot_info.UpdateV(value, new_value);
|
|
|
|
merged_kv_prot_info.UpdateO(kTypeMerge, kTypeValue);
|
|
|
|
ret_status = mem->Add(sequence_, kTypeValue, key, new_value,
|
|
|
|
&merged_kv_prot_info);
|
|
|
|
} else {
|
|
|
|
ret_status = mem->Add(sequence_, kTypeValue, key, new_value,
|
|
|
|
nullptr /* kv_prot_info */);
|
|
|
|
}
|
2018-02-06 03:32:54 +01:00
|
|
|
}
|
2014-01-11 02:33:56 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!perform_merge) {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
assert(ret_status.ok());
|
2020-11-24 01:27:46 +01:00
|
|
|
// Add merge operand to memtable
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (kv_prot_info != nullptr) {
|
|
|
|
auto mem_kv_prot_info =
|
|
|
|
kv_prot_info->StripC(column_family_id).ProtectS(sequence_);
|
|
|
|
ret_status =
|
|
|
|
mem->Add(sequence_, kTypeMerge, key, value, &mem_kv_prot_info,
|
|
|
|
concurrent_memtable_writes_, get_post_process_info(mem));
|
|
|
|
} else {
|
|
|
|
ret_status = mem->Add(
|
|
|
|
sequence_, kTypeMerge, key, value, nullptr /* kv_prot_info */,
|
|
|
|
concurrent_memtable_writes_, get_post_process_info(mem));
|
|
|
|
}
|
2014-01-11 02:33:56 +01:00
|
|
|
}
|
|
|
|
|
2020-11-24 01:27:46 +01:00
|
|
|
if (UNLIKELY(ret_status.IsTryAgain())) {
|
|
|
|
assert(seq_per_batch_);
|
|
|
|
const bool kBatchBoundary = true;
|
|
|
|
MaybeAdvanceSeq(kBatchBoundary);
|
|
|
|
} else if (ret_status.ok()) {
|
|
|
|
MaybeAdvanceSeq();
|
|
|
|
CheckMemtableFull();
|
|
|
|
}
|
2018-03-05 19:48:29 +01:00
|
|
|
// optimize for non-recovery mode
|
2020-11-24 01:27:46 +01:00
|
|
|
// If `ret_status` is `TryAgain` then the next (successful) try will add
|
|
|
|
// the key to the rebuilding transaction object. If `ret_status` is
|
|
|
|
// another non-OK `Status`, then the `rebuilding_trx_` will be thrown
|
|
|
|
// away. So we only need to add to it when `ret_status.ok()`.
|
|
|
|
if (UNLIKELY(ret_status.ok() && rebuilding_trx_ != nullptr)) {
|
2018-03-05 19:48:29 +01:00
|
|
|
assert(!write_after_commit_);
|
2021-09-14 22:13:36 +02:00
|
|
|
// TODO(ajkr): propagate `ProtectionInfoKVOS64`.
|
2020-10-20 22:17:17 +02:00
|
|
|
ret_status = WriteBatchInternal::Merge(rebuilding_trx_, column_family_id,
|
|
|
|
key, value);
|
2018-03-05 19:48:29 +01:00
|
|
|
}
|
2018-02-06 03:32:54 +01:00
|
|
|
return ret_status;
|
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
|
|
|
|
2019-02-19 22:36:04 +01:00
|
|
|
Status PutBlobIndexCF(uint32_t column_family_id, const Slice& key,
|
|
|
|
const Slice& value) override {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
const auto* kv_prot_info = NextProtectionInfo();
|
|
|
|
if (kv_prot_info != nullptr) {
|
|
|
|
// Memtable needs seqno, doesn't need CF ID
|
|
|
|
auto mem_kv_prot_info =
|
|
|
|
kv_prot_info->StripC(column_family_id).ProtectS(sequence_);
|
|
|
|
// Same as PutCF except for value type.
|
|
|
|
return PutCFImpl(column_family_id, key, value, kTypeBlobIndex,
|
|
|
|
&mem_kv_prot_info);
|
|
|
|
} else {
|
|
|
|
return PutCFImpl(column_family_id, key, value, kTypeBlobIndex,
|
|
|
|
nullptr /* kv_prot_info */);
|
|
|
|
}
|
2017-10-03 18:08:07 +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
|
|
|
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
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
flush_scheduler_->ScheduleWork(cfd);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
// check if memtable_list size exceeds max_write_buffer_size_to_maintain
|
|
|
|
if (trim_history_scheduler_ != nullptr) {
|
|
|
|
auto* cfd = cf_mems_->current();
|
2019-12-14 04:09:53 +01:00
|
|
|
|
|
|
|
assert(cfd);
|
|
|
|
assert(cfd->ioptions());
|
|
|
|
|
|
|
|
const size_t size_to_maintain = static_cast<size_t>(
|
|
|
|
cfd->ioptions()->max_write_buffer_size_to_maintain);
|
|
|
|
|
|
|
|
if (size_to_maintain > 0) {
|
|
|
|
MemTableList* const imm = cfd->imm();
|
|
|
|
assert(imm);
|
|
|
|
|
2019-12-16 22:13:42 +01:00
|
|
|
if (imm->HasHistory()) {
|
2019-12-14 04:09:53 +01:00
|
|
|
const MemTable* const mem = cfd->mem();
|
|
|
|
assert(mem);
|
|
|
|
|
2021-12-02 20:44:29 +01:00
|
|
|
if (mem->MemoryAllocatedBytes() +
|
|
|
|
imm->MemoryAllocatedBytesExcludingLast() >=
|
2019-12-14 04:09:53 +01:00
|
|
|
size_to_maintain &&
|
|
|
|
imm->MarkTrimHistoryNeeded()) {
|
|
|
|
trim_history_scheduler_->ScheduleWork(cfd);
|
|
|
|
}
|
|
|
|
}
|
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
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2018-07-07 02:17:36 +02:00
|
|
|
// The write batch handler calls MarkBeginPrepare with unprepare set to true
|
|
|
|
// if it encounters the kTypeBeginUnprepareXID marker.
|
|
|
|
Status MarkBeginPrepare(bool unprepare) override {
|
2016-04-18 20:11:51 +02:00
|
|
|
assert(rebuilding_trx_ == nullptr);
|
|
|
|
assert(db_);
|
|
|
|
|
|
|
|
if (recovering_log_number_ != 0) {
|
|
|
|
// during recovery we rebuild a hollow transaction
|
|
|
|
// from all encountered prepare sections of the wal
|
|
|
|
if (db_->allow_2pc() == false) {
|
|
|
|
return Status::NotSupported(
|
|
|
|
"WAL contains prepared transactions. Open with "
|
|
|
|
"TransactionDB::Open().");
|
|
|
|
}
|
|
|
|
|
|
|
|
// we are now iterating through a prepared section
|
|
|
|
rebuilding_trx_ = new WriteBatch();
|
2017-09-29 01:43:04 +02:00
|
|
|
rebuilding_trx_seq_ = sequence_;
|
2020-02-14 03:50:01 +01:00
|
|
|
// Verify that we have matching MarkBeginPrepare/MarkEndPrepare markers.
|
|
|
|
// unprepared_batch_ should be false because it is false by default, and
|
|
|
|
// gets reset to false in MarkEndPrepare.
|
2018-07-07 02:17:36 +02:00
|
|
|
assert(!unprepared_batch_);
|
|
|
|
unprepared_batch_ = unprepare;
|
|
|
|
|
Ignore stale logs while restarting DBs
Summary:
Stale log files can be deleted out of order. This can happen for various reasons. One of the reason is that no data is ever inserted to a column family and we have an optimization to update its log number, but not all the old log files are cleaned up (the case shown in the unit tests added). It can also happen when we simply delete multiple log files out of order.
This causes data corruption because we simply increase seqID after processing the next row and we may end up with writing data with smaller seqID than what is already flushed to memtables.
In DB recovery, for the oldest files we are replaying, if there it contains no data for any column family, we ignore the sequence IDs in the file.
Test Plan: Add two unit tests that fail without the fix.
Reviewers: IslamAbdelRahman, igor, yiwu
Reviewed By: yiwu
Subscribers: hermanlee4, yoshinorim, leveldb, andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D60891
2016-07-19 20:48:00 +02:00
|
|
|
if (has_valid_writes_ != nullptr) {
|
|
|
|
*has_valid_writes_ = true;
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status MarkEndPrepare(const Slice& name) override {
|
|
|
|
assert(db_);
|
|
|
|
assert((rebuilding_trx_ != nullptr) == (recovering_log_number_ != 0));
|
|
|
|
|
|
|
|
if (recovering_log_number_ != 0) {
|
|
|
|
assert(db_->allow_2pc());
|
2018-03-05 19:48:29 +01:00
|
|
|
size_t batch_cnt =
|
|
|
|
write_after_commit_
|
|
|
|
? 0 // 0 will disable further checks
|
|
|
|
: static_cast<size_t>(sequence_ - rebuilding_trx_seq_ + 1);
|
2016-04-18 20:11:51 +02:00
|
|
|
db_->InsertRecoveredTransaction(recovering_log_number_, name.ToString(),
|
2018-03-05 19:48:29 +01:00
|
|
|
rebuilding_trx_, rebuilding_trx_seq_,
|
2018-07-07 02:17:36 +02:00
|
|
|
batch_cnt, unprepared_batch_);
|
2020-02-14 03:50:01 +01:00
|
|
|
unprepared_batch_ = false;
|
2016-04-18 20:11:51 +02:00
|
|
|
rebuilding_trx_ = nullptr;
|
|
|
|
} else {
|
|
|
|
assert(rebuilding_trx_ == nullptr);
|
|
|
|
}
|
2017-09-18 23:36:53 +02:00
|
|
|
const bool batch_boundry = true;
|
|
|
|
MaybeAdvanceSeq(batch_boundry);
|
|
|
|
|
|
|
|
return Status::OK();
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2017-09-29 01:43:04 +02:00
|
|
|
Status MarkNoop(bool empty_batch) override {
|
2017-09-18 23:36:53 +02:00
|
|
|
// A hack in pessimistic transaction could result into a noop at the start
|
|
|
|
// of the write batch, that should be ignored.
|
2017-09-29 01:43:04 +02:00
|
|
|
if (!empty_batch) {
|
2017-09-18 23:36:53 +02:00
|
|
|
// In the absence of Prepare markers, a kTypeNoop tag indicates the end of
|
|
|
|
// a batch. This happens when write batch commits skipping the prepare
|
|
|
|
// phase.
|
|
|
|
const bool batch_boundry = true;
|
|
|
|
MaybeAdvanceSeq(batch_boundry);
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status MarkCommit(const Slice& name) override {
|
|
|
|
assert(db_);
|
|
|
|
|
|
|
|
Status s;
|
|
|
|
|
|
|
|
if (recovering_log_number_ != 0) {
|
2021-12-10 20:03:39 +01:00
|
|
|
// We must hold db mutex in recovery.
|
|
|
|
db_->mutex()->AssertHeld();
|
2016-04-18 20:11:51 +02:00
|
|
|
// in recovery when we encounter a commit marker
|
|
|
|
// we lookup this transaction in our set of rebuilt transactions
|
|
|
|
// and commit.
|
|
|
|
auto trx = db_->GetRecoveredTransaction(name.ToString());
|
|
|
|
|
2018-03-08 19:18:34 +01:00
|
|
|
// the log containing the prepared section may have
|
2016-04-18 20:11:51 +02:00
|
|
|
// been released in the last incarnation because the
|
|
|
|
// data was flushed to L0
|
|
|
|
if (trx != nullptr) {
|
|
|
|
// at this point individual CF lognumbers will prevent
|
|
|
|
// duplicate re-insertion of values.
|
|
|
|
assert(log_number_ref_ == 0);
|
2017-09-29 01:43:04 +02:00
|
|
|
if (write_after_commit_) {
|
2018-07-07 02:17:36 +02:00
|
|
|
// write_after_commit_ can only have one batch in trx.
|
|
|
|
assert(trx->batches_.size() == 1);
|
|
|
|
const auto& batch_info = trx->batches_.begin()->second;
|
2018-03-08 19:18:34 +01:00
|
|
|
// all inserts must reference this trx log number
|
2018-07-07 02:17:36 +02:00
|
|
|
log_number_ref_ = batch_info.log_number_;
|
|
|
|
s = batch_info.batch_->Iterate(this);
|
2017-09-29 01:43:04 +02:00
|
|
|
log_number_ref_ = 0;
|
|
|
|
}
|
|
|
|
// else the values are already inserted before the commit
|
2016-04-18 20:11:51 +02:00
|
|
|
|
|
|
|
if (s.ok()) {
|
|
|
|
db_->DeleteRecoveredTransaction(name.ToString());
|
|
|
|
}
|
Ignore stale logs while restarting DBs
Summary:
Stale log files can be deleted out of order. This can happen for various reasons. One of the reason is that no data is ever inserted to a column family and we have an optimization to update its log number, but not all the old log files are cleaned up (the case shown in the unit tests added). It can also happen when we simply delete multiple log files out of order.
This causes data corruption because we simply increase seqID after processing the next row and we may end up with writing data with smaller seqID than what is already flushed to memtables.
In DB recovery, for the oldest files we are replaying, if there it contains no data for any column family, we ignore the sequence IDs in the file.
Test Plan: Add two unit tests that fail without the fix.
Reviewers: IslamAbdelRahman, igor, yiwu
Reviewed By: yiwu
Subscribers: hermanlee4, yoshinorim, leveldb, andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D60891
2016-07-19 20:48:00 +02:00
|
|
|
if (has_valid_writes_ != nullptr) {
|
|
|
|
*has_valid_writes_ = true;
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
}
|
|
|
|
} else {
|
2017-09-29 01:43:04 +02:00
|
|
|
// When writes are not delayed until commit, there is no disconnect
|
|
|
|
// between a memtable write and the WAL that supports it. So the commit
|
|
|
|
// need not reference any log as the only log to which it depends.
|
|
|
|
assert(!write_after_commit_ || log_number_ref_ > 0);
|
2016-04-18 20:11:51 +02:00
|
|
|
}
|
2017-09-18 23:36:53 +02:00
|
|
|
const bool batch_boundry = true;
|
|
|
|
MaybeAdvanceSeq(batch_boundry);
|
2016-04-18 20:11:51 +02:00
|
|
|
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2021-12-10 20:03:39 +01:00
|
|
|
Status MarkCommitWithTimestamp(const Slice& name,
|
|
|
|
const Slice& commit_ts) override {
|
|
|
|
assert(db_);
|
|
|
|
|
|
|
|
Status s;
|
|
|
|
|
|
|
|
if (recovering_log_number_ != 0) {
|
|
|
|
// In recovery, db mutex must be held.
|
|
|
|
db_->mutex()->AssertHeld();
|
|
|
|
// in recovery when we encounter a commit marker
|
|
|
|
// we lookup this transaction in our set of rebuilt transactions
|
|
|
|
// and commit.
|
|
|
|
auto trx = db_->GetRecoveredTransaction(name.ToString());
|
|
|
|
// the log containing the prepared section may have
|
|
|
|
// been released in the last incarnation because the
|
|
|
|
// data was flushed to L0
|
|
|
|
if (trx) {
|
|
|
|
// at this point individual CF lognumbers will prevent
|
|
|
|
// duplicate re-insertion of values.
|
|
|
|
assert(0 == log_number_ref_);
|
|
|
|
if (write_after_commit_) {
|
|
|
|
// write_after_commit_ can only have one batch in trx.
|
|
|
|
assert(trx->batches_.size() == 1);
|
|
|
|
const auto& batch_info = trx->batches_.begin()->second;
|
|
|
|
// all inserts must reference this trx log number
|
|
|
|
log_number_ref_ = batch_info.log_number_;
|
|
|
|
const auto checker = [this](uint32_t cf, size_t& ts_sz) {
|
|
|
|
assert(db_);
|
|
|
|
VersionSet* const vset = db_->GetVersionSet();
|
|
|
|
assert(vset);
|
|
|
|
ColumnFamilySet* const cf_set = vset->GetColumnFamilySet();
|
|
|
|
assert(cf_set);
|
|
|
|
ColumnFamilyData* cfd = cf_set->GetColumnFamily(cf);
|
|
|
|
assert(cfd);
|
|
|
|
const auto* const ucmp = cfd->user_comparator();
|
|
|
|
assert(ucmp);
|
|
|
|
if (ucmp->timestamp_size() == 0) {
|
|
|
|
ts_sz = 0;
|
|
|
|
} else if (ucmp->timestamp_size() != ts_sz) {
|
|
|
|
return Status::InvalidArgument("Timestamp size mismatch");
|
|
|
|
}
|
|
|
|
return Status::OK();
|
|
|
|
};
|
|
|
|
s = batch_info.batch_->AssignTimestamp(commit_ts, checker);
|
|
|
|
if (s.ok()) {
|
|
|
|
s = batch_info.batch_->Iterate(this);
|
|
|
|
log_number_ref_ = 0;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
// else the values are already inserted before the commit
|
|
|
|
|
|
|
|
if (s.ok()) {
|
|
|
|
db_->DeleteRecoveredTransaction(name.ToString());
|
|
|
|
}
|
|
|
|
if (has_valid_writes_) {
|
|
|
|
*has_valid_writes_ = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// When writes are not delayed until commit, there is no connection
|
|
|
|
// between a memtable write and the WAL that supports it. So the commit
|
|
|
|
// need not reference any log as the only log to which it depends.
|
|
|
|
assert(!write_after_commit_ || log_number_ref_ > 0);
|
|
|
|
}
|
|
|
|
constexpr bool batch_boundary = true;
|
|
|
|
MaybeAdvanceSeq(batch_boundary);
|
|
|
|
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2016-04-18 20:11:51 +02:00
|
|
|
Status MarkRollback(const Slice& name) override {
|
|
|
|
assert(db_);
|
|
|
|
|
|
|
|
if (recovering_log_number_ != 0) {
|
|
|
|
auto trx = db_->GetRecoveredTransaction(name.ToString());
|
|
|
|
|
|
|
|
// the log containing the transactions prep section
|
|
|
|
// may have been released in the previous incarnation
|
|
|
|
// because we knew it had been rolled back
|
|
|
|
if (trx != nullptr) {
|
|
|
|
db_->DeleteRecoveredTransaction(name.ToString());
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// in non recovery we simply ignore this tag
|
|
|
|
}
|
|
|
|
|
2017-11-15 17:19:57 +01:00
|
|
|
const bool batch_boundry = true;
|
|
|
|
MaybeAdvanceSeq(batch_boundry);
|
|
|
|
|
2016-04-18 20:11:51 +02:00
|
|
|
return Status::OK();
|
|
|
|
}
|
2016-07-07 23:45:29 +02:00
|
|
|
|
|
|
|
private:
|
|
|
|
MemTablePostProcessInfo* get_post_process_info(MemTable* mem) {
|
|
|
|
if (!concurrent_memtable_writes_) {
|
|
|
|
// No need to batch counters locally if we don't use concurrent mode.
|
|
|
|
return nullptr;
|
|
|
|
}
|
2017-03-22 19:07:52 +01:00
|
|
|
return &GetPostMap()[mem];
|
2016-07-07 23:45:29 +02:00
|
|
|
}
|
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
|
2017-09-18 23:36:53 +02:00
|
|
|
Status WriteBatchInternal::InsertInto(
|
|
|
|
WriteThread::WriteGroup& write_group, SequenceNumber sequence,
|
|
|
|
ColumnFamilyMemTables* memtables, FlushScheduler* flush_scheduler,
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
TrimHistoryScheduler* trim_history_scheduler,
|
2017-09-18 23:36:53 +02:00
|
|
|
bool ignore_missing_column_families, uint64_t recovery_log_number, DB* db,
|
2018-06-29 03:46:39 +02:00
|
|
|
bool concurrent_memtable_writes, bool seq_per_batch, bool batch_per_txn) {
|
|
|
|
MemTableInserter inserter(
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
sequence, memtables, flush_scheduler, trim_history_scheduler,
|
|
|
|
ignore_missing_column_families, recovery_log_number, db,
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
concurrent_memtable_writes, nullptr /* prot_info */,
|
|
|
|
nullptr /*has_valid_writes*/, seq_per_batch, batch_per_txn);
|
2017-05-19 23:24:23 +02:00
|
|
|
for (auto w : write_group) {
|
2017-11-15 17:22:54 +01:00
|
|
|
if (w->CallbackFailed()) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
w->sequence = inserter.sequence();
|
2016-04-18 20:11:51 +02:00
|
|
|
if (!w->ShouldWriteToMemtable()) {
|
2017-11-15 17:22:54 +01:00
|
|
|
// In seq_per_batch_ mode this advances the seq by one.
|
2017-10-06 23:18:30 +02:00
|
|
|
inserter.MaybeAdvanceSeq(true);
|
2016-04-18 20:11:51 +02:00
|
|
|
continue;
|
|
|
|
}
|
2017-05-31 19:45:47 +02:00
|
|
|
SetSequence(w->batch, inserter.sequence());
|
2016-04-18 20:11:51 +02:00
|
|
|
inserter.set_log_number_ref(w->log_ref);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
inserter.set_prot_info(w->batch->prot_info_.get());
|
2016-04-18 20:11:51 +02:00
|
|
|
w->status = w->batch->Iterate(&inserter);
|
|
|
|
if (!w->status.ok()) {
|
|
|
|
return w->status;
|
2016-02-05 19:44:13 +01:00
|
|
|
}
|
2018-02-06 03:32:54 +01:00
|
|
|
assert(!seq_per_batch || w->batch_cnt != 0);
|
|
|
|
assert(!seq_per_batch || inserter.sequence() - w->sequence == w->batch_cnt);
|
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
|
|
|
}
|
|
|
|
|
2017-10-06 23:18:30 +02:00
|
|
|
Status WriteBatchInternal::InsertInto(
|
|
|
|
WriteThread::Writer* writer, SequenceNumber sequence,
|
|
|
|
ColumnFamilyMemTables* memtables, FlushScheduler* flush_scheduler,
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
TrimHistoryScheduler* trim_history_scheduler,
|
2017-10-06 23:18:30 +02:00
|
|
|
bool ignore_missing_column_families, uint64_t log_number, DB* db,
|
2018-06-29 03:46:39 +02:00
|
|
|
bool concurrent_memtable_writes, bool seq_per_batch, size_t batch_cnt,
|
2019-09-13 01:53:31 +02:00
|
|
|
bool batch_per_txn, bool hint_per_batch) {
|
2018-04-13 02:55:14 +02:00
|
|
|
#ifdef NDEBUG
|
|
|
|
(void)batch_cnt;
|
|
|
|
#endif
|
2017-05-31 19:45:47 +02:00
|
|
|
assert(writer->ShouldWriteToMemtable());
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
MemTableInserter inserter(sequence, memtables, flush_scheduler,
|
|
|
|
trim_history_scheduler,
|
|
|
|
ignore_missing_column_families, log_number, db,
|
|
|
|
concurrent_memtable_writes, nullptr /* prot_info */,
|
|
|
|
nullptr /*has_valid_writes*/, seq_per_batch,
|
|
|
|
batch_per_txn, hint_per_batch);
|
2017-05-31 19:45:47 +02:00
|
|
|
SetSequence(writer->batch, sequence);
|
2016-04-18 20:11:51 +02:00
|
|
|
inserter.set_log_number_ref(writer->log_ref);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
inserter.set_prot_info(writer->batch->prot_info_.get());
|
2016-07-07 23:45:29 +02:00
|
|
|
Status s = writer->batch->Iterate(&inserter);
|
2018-02-06 03:32:54 +01:00
|
|
|
assert(!seq_per_batch || batch_cnt != 0);
|
|
|
|
assert(!seq_per_batch || inserter.sequence() - sequence == batch_cnt);
|
2016-07-07 23:45:29 +02:00
|
|
|
if (concurrent_memtable_writes) {
|
|
|
|
inserter.PostProcess();
|
|
|
|
}
|
|
|
|
return s;
|
2016-04-18 20:11:51 +02:00
|
|
|
}
|
|
|
|
|
Ignore stale logs while restarting DBs
Summary:
Stale log files can be deleted out of order. This can happen for various reasons. One of the reason is that no data is ever inserted to a column family and we have an optimization to update its log number, but not all the old log files are cleaned up (the case shown in the unit tests added). It can also happen when we simply delete multiple log files out of order.
This causes data corruption because we simply increase seqID after processing the next row and we may end up with writing data with smaller seqID than what is already flushed to memtables.
In DB recovery, for the oldest files we are replaying, if there it contains no data for any column family, we ignore the sequence IDs in the file.
Test Plan: Add two unit tests that fail without the fix.
Reviewers: IslamAbdelRahman, igor, yiwu
Reviewed By: yiwu
Subscribers: hermanlee4, yoshinorim, leveldb, andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D60891
2016-07-19 20:48:00 +02:00
|
|
|
Status WriteBatchInternal::InsertInto(
|
|
|
|
const WriteBatch* batch, ColumnFamilyMemTables* memtables,
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
FlushScheduler* flush_scheduler,
|
|
|
|
TrimHistoryScheduler* trim_history_scheduler,
|
|
|
|
bool ignore_missing_column_families, uint64_t log_number, DB* db,
|
|
|
|
bool concurrent_memtable_writes, SequenceNumber* next_seq,
|
|
|
|
bool* has_valid_writes, bool seq_per_batch, bool batch_per_txn) {
|
2017-05-31 19:45:47 +02:00
|
|
|
MemTableInserter inserter(Sequence(batch), memtables, flush_scheduler,
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
trim_history_scheduler,
|
2017-05-31 19:45:47 +02:00
|
|
|
ignore_missing_column_families, log_number, db,
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
concurrent_memtable_writes, batch->prot_info_.get(),
|
|
|
|
has_valid_writes, seq_per_batch, batch_per_txn);
|
[rocksdb] Recovery path sequence miscount fix
Summary:
Consider the following WAL with 4 batch entries prefixed with their sequence at time of memtable insert.
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(a)]
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(b)]
[4: COMMIT(a)]
[7: COMMIT(b)]
The first two batches do not consume any sequence numbers so are both prefixed with seq=1.
For 2pc commit, memtable insertion takes place before COMMIT batch is written to WAL.
We can see that sequence number consumption takes place between WAL entries giving us the seemingly sparse sequence prefix for WAL entries.
This is a valid WAL.
Because with 2PC markers one WriteBatch points to another batch containing its inserts a writebatch can consume more or less sequence numbers than the number of sequence consuming entries that it contains.
We can see that, given the entries in the WAL, 6 sequence ids were consumed. Yet on recovery the maximum sequence consumed would be 7 + 3 (the number of sequence numbers consumed by COMMIT(b))
So, now upon recovery we must track the actual consumption of sequence numbers.
In the provided scenario there will be no sequence gaps, but it is possible to produce a sequence gap. This should not be a problem though. correct?
Test Plan: provided test.
Reviewers: sdong
Subscribers: andrewkr, leveldb, dhruba, hermanlee4
Differential Revision: https://reviews.facebook.net/D57645
2016-05-04 23:02:27 +02:00
|
|
|
Status s = batch->Iterate(&inserter);
|
2017-09-18 23:36:53 +02:00
|
|
|
if (next_seq != nullptr) {
|
|
|
|
*next_seq = inserter.sequence();
|
[rocksdb] Recovery path sequence miscount fix
Summary:
Consider the following WAL with 4 batch entries prefixed with their sequence at time of memtable insert.
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(a)]
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(b)]
[4: COMMIT(a)]
[7: COMMIT(b)]
The first two batches do not consume any sequence numbers so are both prefixed with seq=1.
For 2pc commit, memtable insertion takes place before COMMIT batch is written to WAL.
We can see that sequence number consumption takes place between WAL entries giving us the seemingly sparse sequence prefix for WAL entries.
This is a valid WAL.
Because with 2PC markers one WriteBatch points to another batch containing its inserts a writebatch can consume more or less sequence numbers than the number of sequence consuming entries that it contains.
We can see that, given the entries in the WAL, 6 sequence ids were consumed. Yet on recovery the maximum sequence consumed would be 7 + 3 (the number of sequence numbers consumed by COMMIT(b))
So, now upon recovery we must track the actual consumption of sequence numbers.
In the provided scenario there will be no sequence gaps, but it is possible to produce a sequence gap. This should not be a problem though. correct?
Test Plan: provided test.
Reviewers: sdong
Subscribers: andrewkr, leveldb, dhruba, hermanlee4
Differential Revision: https://reviews.facebook.net/D57645
2016-05-04 23:02:27 +02:00
|
|
|
}
|
2016-07-07 23:45:29 +02:00
|
|
|
if (concurrent_memtable_writes) {
|
|
|
|
inserter.PostProcess();
|
|
|
|
}
|
[rocksdb] Recovery path sequence miscount fix
Summary:
Consider the following WAL with 4 batch entries prefixed with their sequence at time of memtable insert.
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(a)]
[1: BEGIN_PREPARE, PUT, PUT, PUT, PUT, END_PREPARE(b)]
[4: COMMIT(a)]
[7: COMMIT(b)]
The first two batches do not consume any sequence numbers so are both prefixed with seq=1.
For 2pc commit, memtable insertion takes place before COMMIT batch is written to WAL.
We can see that sequence number consumption takes place between WAL entries giving us the seemingly sparse sequence prefix for WAL entries.
This is a valid WAL.
Because with 2PC markers one WriteBatch points to another batch containing its inserts a writebatch can consume more or less sequence numbers than the number of sequence consuming entries that it contains.
We can see that, given the entries in the WAL, 6 sequence ids were consumed. Yet on recovery the maximum sequence consumed would be 7 + 3 (the number of sequence numbers consumed by COMMIT(b))
So, now upon recovery we must track the actual consumption of sequence numbers.
In the provided scenario there will be no sequence gaps, but it is possible to produce a sequence gap. This should not be a problem though. correct?
Test Plan: provided test.
Reviewers: sdong
Subscribers: andrewkr, leveldb, dhruba, hermanlee4
Differential Revision: https://reviews.facebook.net/D57645
2016-05-04 23:02:27 +02:00
|
|
|
return s;
|
2014-01-28 20:05:04 +01:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::SetContents(WriteBatch* b, const Slice& contents) {
|
2016-03-30 19:43:00 +02:00
|
|
|
assert(contents.size() >= WriteBatchInternal::kHeader);
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
assert(b->prot_info_ == nullptr);
|
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);
|
2017-04-11 00:38:34 +02:00
|
|
|
return Status::OK();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2017-04-11 00:38:34 +02:00
|
|
|
Status WriteBatchInternal::Append(WriteBatch* dst, const WriteBatch* src,
|
|
|
|
const bool wal_only) {
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
assert(dst->Count() == 0 ||
|
|
|
|
(dst->prot_info_ == nullptr) == (src->prot_info_ == nullptr));
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
size_t src_len;
|
|
|
|
int src_count;
|
|
|
|
uint32_t src_flags;
|
|
|
|
|
|
|
|
const SavePoint& batch_end = src->GetWalTerminationPoint();
|
|
|
|
|
|
|
|
if (wal_only && !batch_end.is_cleared()) {
|
|
|
|
src_len = batch_end.size - WriteBatchInternal::kHeader;
|
|
|
|
src_count = batch_end.count;
|
|
|
|
src_flags = batch_end.content_flags;
|
|
|
|
} else {
|
|
|
|
src_len = src->rep_.size() - WriteBatchInternal::kHeader;
|
|
|
|
src_count = Count(src);
|
|
|
|
src_flags = src->content_flags_.load(std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
Integrity protection for live updates to WriteBatch (#7748)
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).
The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.
When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748
Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.
Reviewed By: pdillinger
Differential Revision: D25754492
Pulled By: ajkr
fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
2021-01-29 21:17:17 +01:00
|
|
|
if (dst->prot_info_ != nullptr) {
|
|
|
|
std::copy(src->prot_info_->entries_.begin(),
|
|
|
|
src->prot_info_->entries_.begin() + src_count,
|
|
|
|
std::back_inserter(dst->prot_info_->entries_));
|
|
|
|
} else if (src->prot_info_ != nullptr) {
|
|
|
|
dst->prot_info_.reset(new WriteBatch::ProtectionInfo(*src->prot_info_));
|
|
|
|
}
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
SetCount(dst, Count(dst) + src_count);
|
2016-03-30 19:43:00 +02:00
|
|
|
assert(src->rep_.size() >= WriteBatchInternal::kHeader);
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
dst->rep_.append(src->rep_.data() + WriteBatchInternal::kHeader, src_len);
|
2015-11-06 16:03:30 +01:00
|
|
|
dst->content_flags_.store(
|
Add facility to write only a portion of WriteBatch to WAL
Summary:
When constructing a write batch a client may now call MarkWalTerminationPoint() on that batch. No batch operations after this call will be added written to the WAL but will still be inserted into the Memtable. This facility is used to remove one of the three WriteImpl calls in 2PC transactions. This produces a ~1% perf improvement.
```
RocksDB - unoptimized 2pc, sync_binlog=1, disable_2pc=off
INFO 2016-08-31 14:30:38,814 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2619 seconds. Requests/second = 28628
RocksDB - optimized 2pc , sync_binlog=1, disable_2pc=off
INFO 2016-08-31 16:26:59,442 [main]: REQUEST PHASE COMPLETED. 75000000 requests done in 2581 seconds. Requests/second = 29054
```
Test Plan: Two unit tests added.
Reviewers: sdong, yiwu, IslamAbdelRahman
Reviewed By: yiwu
Subscribers: hermanlee4, dhruba, andrewkr
Differential Revision: https://reviews.facebook.net/D64599
2016-10-07 20:31:26 +02:00
|
|
|
dst->content_flags_.load(std::memory_order_relaxed) | src_flags,
|
2015-11-06 16:03:30 +01:00
|
|
|
std::memory_order_relaxed);
|
2017-04-11 00:38:34 +02:00
|
|
|
return Status::OK();
|
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
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|