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

407 lines
14 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#pragma once
#include <array>
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
#include <vector>
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
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
5 years ago
#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
3 years ago
#include "db/kv_checksum.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
5 years ago
#include "db/trim_history_scheduler.h"
#include "db/write_thread.h"
#include "rocksdb/db.h"
#include "rocksdb/options.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
5 years ago
#include "rocksdb/types.h"
#include "rocksdb/write_batch.h"
#include "util/autovector.h"
#include "util/cast_util.h"
namespace ROCKSDB_NAMESPACE {
class MemTable;
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
class FlushScheduler;
class ColumnFamilyData;
class ColumnFamilyMemTables {
public:
virtual ~ColumnFamilyMemTables() {}
virtual bool Seek(uint32_t column_family_id) = 0;
// returns true if the update to memtable should be ignored
// (useful when recovering from log whose updates have already
// been processed)
virtual uint64_t GetLogNumber() const = 0;
virtual MemTable* GetMemTable() const = 0;
virtual ColumnFamilyHandle* GetColumnFamilyHandle() = 0;
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
virtual ColumnFamilyData* current() { return nullptr; }
};
class ColumnFamilyMemTablesDefault : public ColumnFamilyMemTables {
public:
explicit ColumnFamilyMemTablesDefault(MemTable* mem)
: ok_(false), mem_(mem) {}
bool Seek(uint32_t column_family_id) override {
ok_ = (column_family_id == 0);
return ok_;
}
uint64_t GetLogNumber() const override { return 0; }
MemTable* GetMemTable() const override {
assert(ok_);
return mem_;
}
ColumnFamilyHandle* GetColumnFamilyHandle() override { return nullptr; }
private:
bool ok_;
MemTable* mem_;
};
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
struct WriteBatch::ProtectionInfo {
// `WriteBatch` usually doesn't contain a huge number of keys so protecting
// with a fixed, non-configurable eight bytes per key may work well enough.
autovector<ProtectionInfoKVOC64> entries_;
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
size_t GetBytesPerKey() const { return 8; }
};
// WriteBatchInternal provides static methods for manipulating a
// WriteBatch that we don't want in the public WriteBatch interface.
class WriteBatchInternal {
public:
// WriteBatch header has an 8-byte sequence number followed by a 4-byte count.
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
static constexpr size_t kHeader = 12;
// WriteBatch methods with column_family_id instead of ColumnFamilyHandle*
static Status Put(WriteBatch* batch, uint32_t column_family_id,
const Slice& key, const Slice& value);
static Status Put(WriteBatch* batch, uint32_t column_family_id,
const SliceParts& key, const SliceParts& value);
static Status PutEntity(WriteBatch* batch, uint32_t column_family_id,
const Slice& key, const WideColumns& columns);
static Status Delete(WriteBatch* batch, uint32_t column_family_id,
const SliceParts& key);
static Status Delete(WriteBatch* batch, uint32_t column_family_id,
const Slice& key);
static Status SingleDelete(WriteBatch* batch, uint32_t column_family_id,
const SliceParts& 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
9 years ago
static Status SingleDelete(WriteBatch* batch, uint32_t column_family_id,
const Slice& 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
9 years ago
static Status DeleteRange(WriteBatch* b, uint32_t column_family_id,
const Slice& begin_key, const Slice& end_key);
static Status DeleteRange(WriteBatch* b, uint32_t column_family_id,
const SliceParts& begin_key,
const SliceParts& end_key);
static Status Merge(WriteBatch* batch, uint32_t column_family_id,
const Slice& key, const Slice& value);
static Status Merge(WriteBatch* batch, uint32_t column_family_id,
const SliceParts& key, const SliceParts& value);
static Status PutBlobIndex(WriteBatch* batch, uint32_t column_family_id,
const Slice& key, const Slice& value);
static Status MarkEndPrepare(WriteBatch* batch, const Slice& xid,
const bool write_after_commit = true,
const bool unprepared_batch = false);
static Status MarkRollback(WriteBatch* batch, const Slice& xid);
static Status MarkCommit(WriteBatch* batch, const Slice& xid);
static Status MarkCommitWithTimestamp(WriteBatch* batch, const Slice& xid,
const Slice& commit_ts);
static Status InsertNoop(WriteBatch* batch);
// Return the number of entries in the batch.
static uint32_t Count(const WriteBatch* batch);
// Set the count for the number of entries in the batch.
static void SetCount(WriteBatch* batch, uint32_t n);
// Return the sequence number for the start of this batch.
static SequenceNumber Sequence(const WriteBatch* batch);
// Store the specified number as the sequence number for the start of
// this batch.
static void SetSequence(WriteBatch* batch, SequenceNumber seq);
// Returns the offset of the first entry in the batch.
// This offset is only valid if the batch is not empty.
static size_t GetFirstOffset(WriteBatch* batch);
static Slice Contents(const WriteBatch* batch) {
return Slice(batch->rep_);
}
static size_t ByteSize(const WriteBatch* batch) {
return batch->rep_.size();
}
static Status SetContents(WriteBatch* batch, const Slice& contents);
static Status CheckSlicePartsLength(const SliceParts& key,
const SliceParts& value);
// Inserts batches[i] into memtable, for i in 0..num_batches-1 inclusive.
//
// If ignore_missing_column_families == true. WriteBatch
// referencing non-existing column family will be ignored.
// If ignore_missing_column_families == false, processing of the
// batches will be stopped if a reference is found to a non-existing
// column family and InvalidArgument() will be returned. The writes
// in batches may be only partially applied at that point.
//
// If log_number is non-zero, the memtable will be updated only if
// memtables->GetLogNumber() >= log_number.
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
//
// If flush_scheduler is non-null, it will be invoked if the memtable
// should be flushed.
//
// Under concurrent use, the caller is responsible for making sure that
// the memtables object itself is thread-local.
static Status 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
5 years ago
TrimHistoryScheduler* trim_history_scheduler,
bool ignore_missing_column_families = false, uint64_t log_number = 0,
DB* db = nullptr, bool concurrent_memtable_writes = false,
bool seq_per_batch = false, bool batch_per_txn = true);
// Convenience form of InsertInto when you have only one batch
// next_seq returns the seq after last sequence number used in MemTable insert
static Status InsertInto(
const WriteBatch* batch, 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
5 years ago
TrimHistoryScheduler* trim_history_scheduler,
bool ignore_missing_column_families = false, uint64_t log_number = 0,
DB* db = nullptr, bool concurrent_memtable_writes = false,
SequenceNumber* next_seq = nullptr, bool* has_valid_writes = nullptr,
bool seq_per_batch = false, bool batch_per_txn = true);
static Status 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
5 years ago
TrimHistoryScheduler* trim_history_scheduler,
bool ignore_missing_column_families = false,
uint64_t log_number = 0, DB* db = nullptr,
bool concurrent_memtable_writes = false,
bool seq_per_batch = false, size_t batch_cnt = 0,
bool batch_per_txn = true,
bool hint_per_batch = false);
// Appends src write batch to dst write batch and updates count in dst
// write batch. Returns OK if the append is successful. Checks number of
// checksum against count in dst and src write batches, and returns Corruption
// if the count is inconsistent.
static Status Append(WriteBatch* dst, const WriteBatch* src,
const bool WAL_only = false);
// Returns the byte size of appending a WriteBatch with ByteSize
// leftByteSize and a WriteBatch with ByteSize rightByteSize
static size_t AppendedByteSize(size_t leftByteSize, size_t rightByteSize);
// Iterate over [begin, end) range of a write batch
static Status Iterate(const WriteBatch* wb, WriteBatch::Handler* handler,
size_t begin, size_t end);
// This write batch includes the latest state that should be persisted. Such
// state meant to be used only during recovery.
static void SetAsLatestPersistentState(WriteBatch* b);
static bool IsLatestPersistentState(const WriteBatch* b);
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
static std::tuple<Status, uint32_t, size_t> GetColumnFamilyIdAndTimestampSize(
WriteBatch* b, ColumnFamilyHandle* column_family);
static bool TimestampsUpdateNeeded(const WriteBatch& wb) {
return wb.needs_in_place_update_ts_;
}
static bool HasKeyWithTimestamp(const WriteBatch& wb) {
return wb.has_key_with_ts_;
}
Handoff checksum during WAL replay (#10212) Summary: Added checksum protection for write batch content read from WAL to when per key-value checksum is computed on the write batch. This gives full coverage on write batch integrity of WAL replay to memtable. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10212 Test Plan: - Added unit test and the existing tests (replay code path covers the change in this PR): `make -j32 check` - Stress test: ran `db_stress` for 30min. - Perf regression: ``` # setup TEST_TMPDIR=/dev/shm/100MB_WAL_DB/ ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576000 # benchmark db open time TEST_TMPDIR=/dev/shm/100MB_WAL_DB/ /usr/bin/time ./db_bench -use_existing_db=true -benchmarks=overwrite -write_buffer_size=1048576000 -writes=1 -report_open_timing=true For 20 runs, pre-PR avg: 3734.31ms, post-PR avg: 3790.06 ms (~1.5% regression). Pre-PR OpenDb: 3714.36 milliseconds OpenDb: 3622.71 milliseconds OpenDb: 3591.17 milliseconds OpenDb: 3674.7 milliseconds OpenDb: 3615.79 milliseconds OpenDb: 3982.83 milliseconds OpenDb: 3650.6 milliseconds OpenDb: 3809.26 milliseconds OpenDb: 3576.44 milliseconds OpenDb: 3638.12 milliseconds OpenDb: 3845.68 milliseconds OpenDb: 3677.32 milliseconds OpenDb: 3659.64 milliseconds OpenDb: 3837.55 milliseconds OpenDb: 3899.64 milliseconds OpenDb: 3840.72 milliseconds OpenDb: 3802.71 milliseconds OpenDb: 3573.27 milliseconds OpenDb: 3895.76 milliseconds OpenDb: 3778.02 milliseconds Post-PR: OpenDb: 3880.46 milliseconds OpenDb: 3709.02 milliseconds OpenDb: 3954.67 milliseconds OpenDb: 3955.64 milliseconds OpenDb: 3958.64 milliseconds OpenDb: 3631.28 milliseconds OpenDb: 3721 milliseconds OpenDb: 3729.89 milliseconds OpenDb: 3730.55 milliseconds OpenDb: 3966.32 milliseconds OpenDb: 3685.54 milliseconds OpenDb: 3573.17 milliseconds OpenDb: 3703.75 milliseconds OpenDb: 3873.62 milliseconds OpenDb: 3704.4 milliseconds OpenDb: 3820.98 milliseconds OpenDb: 3721.62 milliseconds OpenDb: 3770.86 milliseconds OpenDb: 3949.78 milliseconds OpenDb: 3760.07 milliseconds ``` Reviewed By: ajkr Differential Revision: D37302092 Pulled By: cbi42 fbshipit-source-id: 7346e625f453ce4c0e5d708776cd1fb2af6b068b
2 years ago
// Update per-key value protection information on this write batch.
// If checksum is provided, the batch content is verfied against the checksum.
static Status UpdateProtectionInfo(WriteBatch* wb, size_t bytes_per_key,
uint64_t* checksum = nullptr);
};
// LocalSavePoint is similar to a scope guard
class LocalSavePoint {
public:
explicit LocalSavePoint(WriteBatch* batch)
: batch_(batch),
savepoint_(batch->GetDataSize(), batch->Count(),
batch->content_flags_.load(std::memory_order_relaxed))
#ifndef NDEBUG
,
committed_(false)
#endif
{
}
#ifndef NDEBUG
~LocalSavePoint() { assert(committed_); }
#endif
Status commit() {
#ifndef NDEBUG
committed_ = true;
#endif
if (batch_->max_bytes_ && batch_->rep_.size() > batch_->max_bytes_) {
batch_->rep_.resize(savepoint_.size);
WriteBatchInternal::SetCount(batch_, savepoint_.count);
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
if (batch_->prot_info_ != nullptr) {
batch_->prot_info_->entries_.resize(savepoint_.count);
}
batch_->content_flags_.store(savepoint_.content_flags,
std::memory_order_relaxed);
return Status::MemoryLimit();
}
return Status::OK();
}
private:
WriteBatch* batch_;
SavePoint savepoint_;
#ifndef NDEBUG
bool committed_;
#endif
};
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
template <typename TimestampSizeFuncType>
class TimestampUpdater : public WriteBatch::Handler {
public:
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
explicit TimestampUpdater(WriteBatch::ProtectionInfo* prot_info,
TimestampSizeFuncType&& ts_sz_func, const Slice& ts)
: prot_info_(prot_info),
ts_sz_func_(std::move(ts_sz_func)),
timestamp_(ts) {
assert(!timestamp_.empty());
}
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
~TimestampUpdater() override {}
Status PutCF(uint32_t cf, const Slice& key, const Slice&) override {
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
return UpdateTimestamp(cf, key);
}
Status DeleteCF(uint32_t cf, const Slice& key) override {
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
return UpdateTimestamp(cf, key);
}
Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
return UpdateTimestamp(cf, key);
}
Status DeleteRangeCF(uint32_t cf, const Slice& begin_key,
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
const Slice& end_key) override {
Status s = UpdateTimestamp(cf, begin_key, true /* is_key */);
if (s.ok()) {
s = UpdateTimestamp(cf, end_key, false /* is_key */);
}
return s;
}
Status MergeCF(uint32_t cf, const Slice& key, const Slice&) override {
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
return UpdateTimestamp(cf, key);
}
Status PutBlobIndexCF(uint32_t cf, const Slice& key, const Slice&) override {
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
return UpdateTimestamp(cf, key);
}
Status MarkBeginPrepare(bool) override { return Status::OK(); }
Status MarkEndPrepare(const Slice&) override { return Status::OK(); }
Status MarkCommit(const Slice&) override { return Status::OK(); }
Status MarkCommitWithTimestamp(const Slice&, const Slice&) override {
return Status::OK();
}
Status MarkRollback(const Slice&) override { return Status::OK(); }
Status MarkNoop(bool /*empty_batch*/) override { return Status::OK(); }
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
private:
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
// @param is_key specifies whether the update is for key or value.
Status UpdateTimestamp(uint32_t cf, const Slice& buf, bool is_key = true) {
Status s = UpdateTimestampImpl(cf, buf, idx_, is_key);
++idx_;
return s;
}
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
Status UpdateTimestampImpl(uint32_t cf, const Slice& buf, size_t /*idx*/,
bool is_key) {
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
if (timestamp_.empty()) {
return Status::InvalidArgument("Timestamp is empty");
}
size_t cf_ts_sz = ts_sz_func_(cf);
if (0 == cf_ts_sz) {
// Skip this column family.
return Status::OK();
} else if (std::numeric_limits<size_t>::max() == cf_ts_sz) {
// Column family timestamp info not found.
return Status::NotFound();
} else if (cf_ts_sz != timestamp_.size()) {
return Status::InvalidArgument("timestamp size mismatch");
}
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
UpdateProtectionInformationIfNeeded(buf, timestamp_, is_key);
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
char* ptr = const_cast<char*>(buf.data() + buf.size() - cf_ts_sz);
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
assert(ptr);
memcpy(ptr, timestamp_.data(), timestamp_.size());
return Status::OK();
}
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
void UpdateProtectionInformationIfNeeded(const Slice& buf, const Slice& ts,
bool is_key) {
if (prot_info_ != nullptr) {
const size_t ts_sz = ts.size();
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
SliceParts old(&buf, 1);
Slice old_no_ts(buf.data(), buf.size() - ts_sz);
std::array<Slice, 2> new_key_cmpts{{old_no_ts, ts}};
SliceParts new_parts(new_key_cmpts.data(), 2);
if (is_key) {
prot_info_->entries_[idx_].UpdateK(old, new_parts);
} else {
prot_info_->entries_[idx_].UpdateV(old, new_parts);
}
}
}
// No copy or move.
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
TimestampUpdater(const TimestampUpdater&) = delete;
TimestampUpdater(TimestampUpdater&&) = delete;
TimestampUpdater& operator=(const TimestampUpdater&) = delete;
TimestampUpdater& operator=(TimestampUpdater&&) = delete;
WriteBatch::ProtectionInfo* const prot_info_ = nullptr;
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
const TimestampSizeFuncType ts_sz_func_{};
const Slice timestamp_;
Revise APIs related to user-defined timestamp (#8946) Summary: ajkr reminded me that we have a rule of not including per-kv related data in `WriteOptions`. Namely, `WriteOptions` should not include information about "what-to-write", but should just include information about "how-to-write". According to this rule, `WriteOptions::timestamp` (experimental) is clearly a violation. Therefore, this PR removes `WriteOptions::timestamp` for compliance. After the removal, we need to pass timestamp info via another set of APIs. This PR proposes a set of overloaded functions `Put(write_opts, key, value, ts)`, `Delete(write_opts, key, ts)`, and `SingleDelete(write_opts, key, ts)`. Planned to add `Write(write_opts, batch, ts)`, but its complexity made me reconsider doing it in another PR (maybe). For better checking and returning error early, we also add a new set of APIs to `WriteBatch` that take extra `timestamp` information when writing to `WriteBatch`es. These set of APIs in `WriteBatchWithIndex` are currently not supported, and are on our TODO list. Removed `WriteBatch::AssignTimestamps()` and renamed `WriteBatch::AssignTimestamp()` to `WriteBatch::UpdateTimestamps()` since this method require that all keys have space for timestamps allocated already and multiple timestamps can be updated. The constructor of `WriteBatch` now takes a fourth argument `default_cf_ts_sz` which is the timestamp size of the default column family. This will be used to allocate space when calling APIs that do not specify a column family handle. Also, updated `DB::Get()`, `DB::MultiGet()`, `DB::NewIterator()`, `DB::NewIterators()` methods, replacing some assertions about timestamp to returning Status code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8946 Test Plan: make check ./db_bench -benchmarks=fillseq,fillrandom,readrandom,readseq,deleterandom -user_timestamp_size=8 ./db_stress --user_timestamp_size=8 -nooverwritepercent=0 -test_secondary=0 -secondary_catch_up_one_in=0 -continuous_verification_interval=0 Make sure there is no perf regression by running the following ``` ./db_bench_opt -db=/dev/shm/rocksdb -use_existing_db=0 -level0_stop_writes_trigger=256 -level0_slowdown_writes_trigger=256 -level0_file_num_compaction_trigger=256 -disable_wal=1 -duration=10 -benchmarks=fillrandom ``` Before this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.831 micros/op 546235 ops/sec; 60.4 MB/s ``` After this PR ``` DB path: [/dev/shm/rocksdb] fillrandom : 1.820 micros/op 549404 ops/sec; 60.8 MB/s ``` Reviewed By: ltamasi Differential Revision: D33721359 Pulled By: riversand963 fbshipit-source-id: c131561534272c120ffb80711d42748d21badf09
2 years ago
size_t idx_ = 0;
};
} // namespace ROCKSDB_NAMESPACE