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/memtable.cc

1676 lines
62 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.
#include "db/memtable.h"
#include <algorithm>
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
#include <array>
#include <limits>
#include <memory>
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/dbformat.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"
#include "db/merge_context.h"
#include "db/merge_helper.h"
#include "db/pinned_iterators_manager.h"
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
#include "db/range_tombstone_fragmenter.h"
#include "db/read_callback.h"
#include "db/wide/wide_column_serialization.h"
#include "logging/logging.h"
#include "memory/arena.h"
#include "memory/memory_usage.h"
#include "monitoring/perf_context_imp.h"
#include "monitoring/statistics.h"
#include "port/lang.h"
#include "port/port.h"
#include "rocksdb/comparator.h"
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/slice_transform.h"
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
#include "rocksdb/types.h"
#include "rocksdb/write_buffer_manager.h"
#include "table/internal_iterator.h"
#include "table/iterator_wrapper.h"
#include "table/merging_iterator.h"
#include "util/autovector.h"
#include "util/coding.h"
#include "util/mutexlock.h"
namespace ROCKSDB_NAMESPACE {
ImmutableMemTableOptions::ImmutableMemTableOptions(
const ImmutableOptions& ioptions,
const MutableCFOptions& mutable_cf_options)
: arena_block_size(mutable_cf_options.arena_block_size),
memtable_prefix_bloom_bits(
static_cast<uint32_t>(
static_cast<double>(mutable_cf_options.write_buffer_size) *
mutable_cf_options.memtable_prefix_bloom_size_ratio) *
8u),
memtable_huge_page_size(mutable_cf_options.memtable_huge_page_size),
memtable_whole_key_filtering(
mutable_cf_options.memtable_whole_key_filtering),
inplace_update_support(ioptions.inplace_update_support),
inplace_update_num_locks(mutable_cf_options.inplace_update_num_locks),
inplace_callback(ioptions.inplace_callback),
max_successive_merges(mutable_cf_options.max_successive_merges),
statistics(ioptions.stats),
merge_operator(ioptions.merge_operator.get()),
info_log(ioptions.logger),
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
allow_data_in_errors(ioptions.allow_data_in_errors),
protection_bytes_per_key(
mutable_cf_options.memtable_protection_bytes_per_key) {}
MemTable::MemTable(const InternalKeyComparator& cmp,
const ImmutableOptions& ioptions,
const MutableCFOptions& mutable_cf_options,
WriteBufferManager* write_buffer_manager,
SequenceNumber latest_seq, uint32_t column_family_id)
: comparator_(cmp),
moptions_(ioptions, mutable_cf_options),
refs_(0),
kArenaBlockSize(Arena::OptimizeBlockSize(moptions_.arena_block_size)),
mem_tracker_(write_buffer_manager),
arena_(moptions_.arena_block_size,
(write_buffer_manager != nullptr &&
(write_buffer_manager->enabled() ||
write_buffer_manager->cost_to_cache()))
? &mem_tracker_
: nullptr,
mutable_cf_options.memtable_huge_page_size),
table_(ioptions.memtable_factory->CreateMemTableRep(
comparator_, &arena_, mutable_cf_options.prefix_extractor.get(),
ioptions.logger, column_family_id)),
range_del_table_(SkipListFactory().CreateMemTableRep(
comparator_, &arena_, nullptr /* transform */, ioptions.logger,
column_family_id)),
is_range_del_table_empty_(true),
data_size_(0),
num_entries_(0),
num_deletes_(0),
write_buffer_size_(mutable_cf_options.write_buffer_size),
flush_in_progress_(false),
flush_completed_(false),
file_number_(0),
first_seqno_(0),
earliest_seqno_(latest_seq),
creation_seq_(latest_seq),
mem_next_logfile_number_(0),
min_prep_log_referenced_(0),
locks_(moptions_.inplace_update_support
? moptions_.inplace_update_num_locks
: 0),
prefix_extractor_(mutable_cf_options.prefix_extractor.get()),
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
flush_state_(FLUSH_NOT_REQUESTED),
clock_(ioptions.clock),
insert_with_hint_prefix_extractor_(
ioptions.memtable_insert_with_hint_prefix_extractor.get()),
oldest_key_time_(std::numeric_limits<uint64_t>::max()),
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
atomic_flush_seqno_(kMaxSequenceNumber),
approximate_memory_usage_(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
UpdateFlushState();
// something went wrong if we need to flush before inserting anything
assert(!ShouldScheduleFlush());
// use bloom_filter_ for both whole key and prefix bloom filter
if ((prefix_extractor_ || moptions_.memtable_whole_key_filtering) &&
moptions_.memtable_prefix_bloom_bits > 0) {
bloom_filter_.reset(
new DynamicBloom(&arena_, moptions_.memtable_prefix_bloom_bits,
Faster new DynamicBloom implementation (for memtable) (#5762) Summary: Since DynamicBloom is now only used in-memory, we're free to change it without schema compatibility issues. The new implementation is drawn from (with manifest permission) https://github.com/pdillinger/wormhashing/blob/303542a767437f56d8b66cea6ebecaac0e6a61e9/bloom_simulation_tests/foo.cc#L613 This has several speed advantages over the prior implementation: * Uses fastrange instead of % * Minimum logic to determine first (and all) probed memory addresses * (Major) Two probes per 64-bit memory fetch/write. * Very fast and effective (murmur-like) hash expansion/re-mixing. (At least on recent CPUs, integer multiplication is very cheap.) While a Bloom filter with 512-bit cache locality has about a 1.15x FP rate penalty (e.g. 0.84% to 0.97%), further restricting to two probes per 64 bits incurs an additional 1.12x FP rate penalty (e.g. 0.97% to 1.09%). Nevertheless, the unit tests show no "mediocre" FP rate samples, unlike the old implementation with more erratic FP rates. Especially for the memtable, we expect speed to outweigh somewhat higher FP rates. For example, a negative table query would have to be 1000x slower than a BF query to justify doubling BF query time to shave 10% off FP rate (working assumption around 1% FP rate). While that seems likely for SSTs, my data suggests a speed factor of roughly 50x for the memtable (vs. BF; ~1.5% lower write throughput when enabling memtable Bloom filter, after this change). Thus, it's probably not worth even 5% more time in the Bloom filter to shave off 1/10th of the Bloom FP rate, or 0.1% in absolute terms, and it's probably at least 20% slower to recoup that much FP rate from this new implementation. Because of this, we do not see a need for a 'locality' option that affects the MemTable Bloom filter and have decoupled the MemTable Bloom filter from Options::bloom_locality. Note that just 3% more memory to the Bloom filter (10.3 bits per key vs. just 10) is able to make up for the ~12% FP rate drop in the new implementation: [] # Nearly "ideal" FP-wise but reasonably fast cache-local implementation [~/wormhashing/bloom_simulation_tests] ./foo_gcc_IMPL_CACHE_WORM64_FROM32_any.out 10000000 6 10 $RANDOM 100000000 ./foo_gcc_IMPL_CACHE_WORM64_FROM32_any.out time: 3.29372 sampled_fp_rate: 0.00985956 ... [] # Close match to this new implementation [~/wormhashing/bloom_simulation_tests] ./foo_gcc_IMPL_CACHE_MUL64_BLOCK_FROM32_any.out 10000000 6 10.3 $RANDOM 100000000 ./foo_gcc_IMPL_CACHE_MUL64_BLOCK_FROM32_any.out time: 2.10072 sampled_fp_rate: 0.00985655 ... [] # Old locality=1 implementation [~/wormhashing/bloom_simulation_tests] ./foo_gcc_IMPL_CACHE_ROCKSDB_DYNAMIC_any.out 10000000 6 10 $RANDOM 100000000 ./foo_gcc_IMPL_CACHE_ROCKSDB_DYNAMIC_any.out time: 3.95472 sampled_fp_rate: 0.00988943 ... Also note the dramatic speed improvement vs. alternatives. -- Performance unit test: DynamicBloomTest.concurrent_with_perf is updated to report more precise timing data. (Measure running time of each thread, not just longest running thread, etc.) Results averaged over various sizes enabled with --enable_perf and 20 runs each; old dynamic bloom refers to locality=1, the faster of the old: old dynamic bloom, avg add latency = 65.6468 new dynamic bloom, avg add latency = 44.3809 old dynamic bloom, avg query latency = 50.6485 new dynamic bloom, avg query latency = 43.2186 old avg parallel add latency = 41.678 new avg parallel add latency = 24.5238 old avg parallel hit latency = 14.6322 new avg parallel hit latency = 12.3939 old avg parallel miss latency = 16.7289 new avg parallel miss latency = 12.2134 Tested on a dedicated 64-bit production machine at Facebook. Significant improvement all around. Despite now using std::atomic<uint64_t>, quick before-and-after test on a 32-bit machine (Intel Atom N270, released 2008) shows no regression in performance, in some cases modest improvement. -- Performance integration test (synthetic): with DEBUG_LEVEL=0, used TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=fillrandom,readmissing,readrandom,stats --num=2000000 and optionally with -memtable_whole_key_filtering -memtable_bloom_size_ratio=0.01 300 runs each configuration. Write throughput change by enabling memtable bloom: Old locality=0: -3.06% Old locality=1: -2.37% New: -1.50% conclusion -> seems to substantially close the gap Readmissing throughput change by enabling memtable bloom: Old locality=0: +34.47% Old locality=1: +34.80% New: +33.25% conclusion -> maybe a small new penalty from FP rate Readrandom throughput change by enabling memtable bloom: Old locality=0: +31.54% Old locality=1: +31.13% New: +30.60% conclusion -> maybe also from FP rate (after memtable flush) -- Another conclusion we can draw from this new implementation is that the existing 32-bit hash function is not inherently crippling the Bloom filter speed or accuracy, below about 5 million keys. For speed, the implementation is essentially the same whether starting with 32-bits or 64-bits of hash; it just determines whether the first multiplication after fastrange is a pseudorandom expansion or needed re-mix. Note that this multiplication can occur while memory is fetching. For accuracy, in a standard configuration, you need about 5 million keys before you have about a 1.1x FP penalty due to using a 32-bit hash vs. 64-bit: [~/wormhashing/bloom_simulation_tests] ./foo_gcc_IMPL_CACHE_MUL64_BLOCK_FROM32_any.out $((5 * 1000 * 1000 * 10)) 6 10 $RANDOM 100000000 ./foo_gcc_IMPL_CACHE_MUL64_BLOCK_FROM32_any.out time: 2.52069 sampled_fp_rate: 0.0118267 ... [~/wormhashing/bloom_simulation_tests] ./foo_gcc_IMPL_CACHE_MUL64_BLOCK_any.out $((5 * 1000 * 1000 * 10)) 6 10 $RANDOM 100000000 ./foo_gcc_IMPL_CACHE_MUL64_BLOCK_any.out time: 2.43871 sampled_fp_rate: 0.0109059 Pull Request resolved: https://github.com/facebook/rocksdb/pull/5762 Differential Revision: D17214194 Pulled By: pdillinger fbshipit-source-id: ad9da031772e985fd6b62a0e1db8e81892520595
5 years ago
6 /* hard coded 6 probes */,
moptions_.memtable_huge_page_size, ioptions.logger));
}
// Initialize cached_range_tombstone_ here since it could
// be read before it is constructed in MemTable::Add(), which could also lead
// to a data race on the global mutex table backing atomic shared_ptr.
auto new_cache = std::make_shared<FragmentedRangeTombstoneListCache>();
size_t size = cached_range_tombstone_.Size();
for (size_t i = 0; i < size; ++i) {
std::shared_ptr<FragmentedRangeTombstoneListCache>* local_cache_ref_ptr =
cached_range_tombstone_.AccessAtCore(i);
auto new_local_cache_ref = std::make_shared<
const std::shared_ptr<FragmentedRangeTombstoneListCache>>(new_cache);
std::atomic_store_explicit(
local_cache_ref_ptr,
std::shared_ptr<FragmentedRangeTombstoneListCache>(new_local_cache_ref,
new_cache.get()),
std::memory_order_relaxed);
}
}
MemTable::~MemTable() {
mem_tracker_.FreeMem();
assert(refs_ == 0);
}
size_t MemTable::ApproximateMemoryUsage() {
autovector<size_t> usages = {
arena_.ApproximateMemoryUsage(), table_->ApproximateMemoryUsage(),
range_del_table_->ApproximateMemoryUsage(),
ROCKSDB_NAMESPACE::ApproximateMemoryUsage(insert_hints_)};
size_t total_usage = 0;
for (size_t usage : usages) {
// If usage + total_usage >= kMaxSizet, return kMaxSizet.
// the following variation is to avoid numeric overflow.
if (usage >= std::numeric_limits<size_t>::max() - total_usage) {
return std::numeric_limits<size_t>::max();
}
total_usage += usage;
Add a new mem-table representation based on cuckoo hash. Summary: = Major Changes = * Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash. Cuckoo hash uses multiple hash functions. This allows each key to have multiple possible locations in the mem-table. - Put: When insert a key, it will try to find whether one of its possible locations is vacant and store the key. If none of its possible locations are available, then it will kick out a victim key and store at that location. The kicked-out victim key will then be stored at a vacant space of its possible locations or kick-out another victim. In this diff, the kick-out path (known as cuckoo-path) is found using BFS, which guarantees to be the shortest. - Get: Simply tries all possible locations of a key --- this guarantees worst-case constant time complexity. - Time complexity: O(1) for Get, and average O(1) for Put if the fullness of the mem-table is below 80%. - Default using two hash functions, the number of hash functions used by the cuckoo-hash may dynamically increase if it fails to find a short-enough kick-out path. - Currently, HashCuckooRep does not support iteration and snapshots, as our current main purpose of this is to optimize point access. = Minor Changes = * Add IsSnapshotSupported() to DB to indicate whether the current DB supports snapshots. If it returns false, then DB::GetSnapshot() will always return nullptr. Test Plan: Run existing tests. Will develop a test specifically for cuckoo hash in the next diff. Reviewers: sdong, haobo Reviewed By: sdong CC: leveldb, dhruba, igor Differential Revision: https://reviews.facebook.net/D16155
10 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
approximate_memory_usage_.store(total_usage, std::memory_order_relaxed);
Add a new mem-table representation based on cuckoo hash. Summary: = Major Changes = * Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash. Cuckoo hash uses multiple hash functions. This allows each key to have multiple possible locations in the mem-table. - Put: When insert a key, it will try to find whether one of its possible locations is vacant and store the key. If none of its possible locations are available, then it will kick out a victim key and store at that location. The kicked-out victim key will then be stored at a vacant space of its possible locations or kick-out another victim. In this diff, the kick-out path (known as cuckoo-path) is found using BFS, which guarantees to be the shortest. - Get: Simply tries all possible locations of a key --- this guarantees worst-case constant time complexity. - Time complexity: O(1) for Get, and average O(1) for Put if the fullness of the mem-table is below 80%. - Default using two hash functions, the number of hash functions used by the cuckoo-hash may dynamically increase if it fails to find a short-enough kick-out path. - Currently, HashCuckooRep does not support iteration and snapshots, as our current main purpose of this is to optimize point access. = Minor Changes = * Add IsSnapshotSupported() to DB to indicate whether the current DB supports snapshots. If it returns false, then DB::GetSnapshot() will always return nullptr. Test Plan: Run existing tests. Will develop a test specifically for cuckoo hash in the next diff. Reviewers: sdong, haobo Reviewed By: sdong CC: leveldb, dhruba, igor Differential Revision: https://reviews.facebook.net/D16155
10 years ago
// otherwise, return the actual usage
return total_usage;
}
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
bool MemTable::ShouldFlushNow() {
size_t write_buffer_size = write_buffer_size_.load(std::memory_order_relaxed);
// In a lot of times, we cannot allocate arena blocks that exactly matches the
// buffer size. Thus we have to decide if we should over-allocate or
// under-allocate.
// This constant variable can be interpreted as: if we still have more than
// "kAllowOverAllocationRatio * kArenaBlockSize" space left, we'd try to over
// allocate one more block.
const double kAllowOverAllocationRatio = 0.6;
// If arena still have room for new block allocation, we can safely say it
// shouldn't flush.
auto allocated_memory = table_->ApproximateMemoryUsage() +
range_del_table_->ApproximateMemoryUsage() +
arena_.MemoryAllocatedBytes();
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
approximate_memory_usage_.store(allocated_memory, std::memory_order_relaxed);
// if we can still allocate one more block without exceeding the
// over-allocation ratio, then we should not flush.
if (allocated_memory + kArenaBlockSize <
write_buffer_size + kArenaBlockSize * kAllowOverAllocationRatio) {
return false;
}
// if user keeps adding entries that exceeds write_buffer_size, we need to
// flush earlier even though we still have much available memory left.
if (allocated_memory >
write_buffer_size + kArenaBlockSize * kAllowOverAllocationRatio) {
return true;
}
// In this code path, Arena has already allocated its "last block", which
// means the total allocatedmemory size is either:
// (1) "moderately" over allocated the memory (no more than `0.6 * arena
// block size`. Or,
// (2) the allocated memory is less than write buffer size, but we'll stop
// here since if we allocate a new arena block, we'll over allocate too much
// more (half of the arena block size) memory.
//
// In either case, to avoid over-allocate, the last block will stop allocation
// when its usage reaches a certain ratio, which we carefully choose "0.75
// full" as the stop condition because it addresses the following issue with
// great simplicity: What if the next inserted entry's size is
// bigger than AllocatedAndUnused()?
//
// The answer is: if the entry size is also bigger than 0.25 *
// kArenaBlockSize, a dedicated block will be allocated for it; otherwise
// arena will anyway skip the AllocatedAndUnused() and allocate a new, empty
// and regular block. In either case, we *overly* over-allocated.
//
// Therefore, setting the last block to be at most "0.75 full" avoids both
// cases.
//
// NOTE: the average percentage of waste space of this approach can be counted
// as: "arena block size * 0.25 / write buffer size". User who specify a small
// write buffer size and/or big arena block size may suffer.
return arena_.AllocatedAndUnused() < kArenaBlockSize / 4;
}
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
void MemTable::UpdateFlushState() {
auto state = flush_state_.load(std::memory_order_relaxed);
if (state == FLUSH_NOT_REQUESTED && ShouldFlushNow()) {
// ignore CAS failure, because that means somebody else requested
// a flush
flush_state_.compare_exchange_strong(state, FLUSH_REQUESTED,
std::memory_order_relaxed,
std::memory_order_relaxed);
}
}
void MemTable::UpdateOldestKeyTime() {
uint64_t oldest_key_time = oldest_key_time_.load(std::memory_order_relaxed);
if (oldest_key_time == std::numeric_limits<uint64_t>::max()) {
int64_t current_time = 0;
auto s = clock_->GetCurrentTime(&current_time);
if (s.ok()) {
assert(current_time >= 0);
// If fail, the timestamp is already set.
oldest_key_time_.compare_exchange_strong(
oldest_key_time, static_cast<uint64_t>(current_time),
std::memory_order_relaxed, std::memory_order_relaxed);
}
}
}
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
Status MemTable::VerifyEntryChecksum(const char* entry,
size_t protection_bytes_per_key,
bool allow_data_in_errors) {
if (protection_bytes_per_key == 0) {
return Status::OK();
}
uint32_t key_length;
const char* key_ptr = GetVarint32Ptr(entry, entry + 5, &key_length);
if (key_ptr == nullptr) {
return Status::Corruption("Unable to parse internal key length");
}
if (key_length < 8) {
return Status::Corruption("Memtable entry internal key length too short.");
}
Slice user_key = Slice(key_ptr, key_length - 8);
const uint64_t tag = DecodeFixed64(key_ptr + key_length - 8);
ValueType type;
SequenceNumber seq;
UnPackSequenceAndType(tag, &seq, &type);
uint32_t value_length = 0;
const char* value_ptr = GetVarint32Ptr(
key_ptr + key_length, key_ptr + key_length + 5, &value_length);
if (value_ptr == nullptr) {
return Status::Corruption("Unable to parse internal key value");
}
Slice value = Slice(value_ptr, value_length);
const char* checksum_ptr = value_ptr + value_length;
uint64_t expected = ProtectionInfo64()
.ProtectKVO(user_key, value, type)
.ProtectS(seq)
.GetVal();
bool match = true;
switch (protection_bytes_per_key) {
case 1:
match = static_cast<uint8_t>(checksum_ptr[0]) ==
static_cast<uint8_t>(expected);
break;
case 2:
match = DecodeFixed16(checksum_ptr) == static_cast<uint16_t>(expected);
break;
case 4:
match = DecodeFixed32(checksum_ptr) == static_cast<uint32_t>(expected);
break;
case 8:
match = DecodeFixed64(checksum_ptr) == expected;
break;
default:
assert(false);
}
if (!match) {
std::string msg(
"Corrupted memtable entry, per key-value checksum verification "
"failed.");
if (allow_data_in_errors) {
msg.append("Unrecognized value type: " +
std::to_string(static_cast<int>(type)) + ". ");
msg.append("User key: " + user_key.ToString(/*hex=*/true) + ". ");
msg.append("seq: " + std::to_string(seq) + ".");
}
return Status::Corruption(msg.c_str());
}
return Status::OK();
}
int MemTable::KeyComparator::operator()(const char* prefix_len_key1,
const char* prefix_len_key2) const {
// Internal keys are encoded as length-prefixed strings.
Slice k1 = GetLengthPrefixedSlice(prefix_len_key1);
Slice k2 = GetLengthPrefixedSlice(prefix_len_key2);
return comparator.CompareKeySeq(k1, k2);
}
int MemTable::KeyComparator::operator()(
const char* prefix_len_key, const KeyComparator::DecodedType& key) const {
// Internal keys are encoded as length-prefixed strings.
Slice a = GetLengthPrefixedSlice(prefix_len_key);
return comparator.CompareKeySeq(a, key);
}
void MemTableRep::InsertConcurrently(KeyHandle /*handle*/) {
#ifndef ROCKSDB_LITE
throw std::runtime_error("concurrent insert not supported");
#else
abort();
#endif
}
Slice MemTableRep::UserKey(const char* key) const {
Slice slice = GetLengthPrefixedSlice(key);
return Slice(slice.data(), slice.size() - 8);
}
KeyHandle MemTableRep::Allocate(const size_t len, char** buf) {
*buf = allocator_->Allocate(len);
return static_cast<KeyHandle>(*buf);
}
// Encode a suitable internal key target for "target" and return it.
// Uses *scratch as scratch space, and the returned pointer will point
// into this scratch space.
const char* EncodeKey(std::string* scratch, const Slice& target) {
scratch->clear();
PutVarint32(scratch, static_cast<uint32_t>(target.size()));
scratch->append(target.data(), target.size());
return scratch->data();
}
class MemTableIterator : public InternalIterator {
public:
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
8 years ago
MemTableIterator(const MemTable& mem, const ReadOptions& read_options,
Arena* arena, bool use_range_del_table = false)
: bloom_(nullptr),
prefix_extractor_(mem.prefix_extractor_),
comparator_(mem.comparator_),
valid_(false),
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
8 years ago
arena_mode_(arena != nullptr),
value_pinned_(
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
!mem.GetImmutableMemTableOptions()->inplace_update_support),
protection_bytes_per_key_(mem.moptions_.protection_bytes_per_key),
status_(Status::OK()),
logger_(mem.moptions_.info_log) {
if (use_range_del_table) {
iter_ = mem.range_del_table_->GetIterator(arena);
} else if (prefix_extractor_ != nullptr && !read_options.total_order_seek &&
!read_options.auto_prefix_mode) {
// Auto prefix mode is not implemented in memtable yet.
bloom_ = mem.bloom_filter_.get();
iter_ = mem.table_->GetDynamicPrefixIterator(arena);
} else {
iter_ = mem.table_->GetIterator(arena);
}
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
status_.PermitUncheckedError();
}
// No copying allowed
MemTableIterator(const MemTableIterator&) = delete;
void operator=(const MemTableIterator&) = delete;
~MemTableIterator() override {
#ifndef NDEBUG
// Assert that the MemTableIterator is never deleted while
// Pinning is Enabled.
assert(!pinned_iters_mgr_ || !pinned_iters_mgr_->PinningEnabled());
#endif
if (arena_mode_) {
iter_->~Iterator();
} else {
delete iter_;
}
}
#ifndef NDEBUG
void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
pinned_iters_mgr_ = pinned_iters_mgr;
}
PinnedIteratorsManager* pinned_iters_mgr_ = nullptr;
#endif
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
bool Valid() const override { return valid_ && status_.ok(); }
void Seek(const Slice& k) override {
PERF_TIMER_GUARD(seek_on_memtable_time);
PERF_COUNTER_ADD(seek_on_memtable_count, 1);
if (bloom_) {
// iterator should only use prefix bloom filter
auto ts_sz = comparator_.comparator.user_comparator()->timestamp_size();
Slice user_k_without_ts(ExtractUserKeyAndStripTimestamp(k, ts_sz));
if (prefix_extractor_->InDomain(user_k_without_ts)) {
if (!bloom_->MayContain(
prefix_extractor_->Transform(user_k_without_ts))) {
PERF_COUNTER_ADD(bloom_memtable_miss_count, 1);
valid_ = false;
return;
} else {
PERF_COUNTER_ADD(bloom_memtable_hit_count, 1);
}
}
}
iter_->Seek(k, nullptr);
valid_ = iter_->Valid();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
VerifyEntryChecksum();
}
void SeekForPrev(const Slice& k) override {
PERF_TIMER_GUARD(seek_on_memtable_time);
PERF_COUNTER_ADD(seek_on_memtable_count, 1);
if (bloom_) {
auto ts_sz = comparator_.comparator.user_comparator()->timestamp_size();
Slice user_k_without_ts(ExtractUserKeyAndStripTimestamp(k, ts_sz));
if (prefix_extractor_->InDomain(user_k_without_ts)) {
if (!bloom_->MayContain(
prefix_extractor_->Transform(user_k_without_ts))) {
PERF_COUNTER_ADD(bloom_memtable_miss_count, 1);
valid_ = false;
return;
} else {
PERF_COUNTER_ADD(bloom_memtable_hit_count, 1);
}
}
}
iter_->Seek(k, nullptr);
valid_ = iter_->Valid();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
VerifyEntryChecksum();
if (!Valid() && status().ok()) {
SeekToLast();
}
while (Valid() && comparator_.comparator.Compare(k, key()) < 0) {
Prev();
}
}
void SeekToFirst() override {
iter_->SeekToFirst();
valid_ = iter_->Valid();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
VerifyEntryChecksum();
}
void SeekToLast() override {
iter_->SeekToLast();
valid_ = iter_->Valid();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
VerifyEntryChecksum();
}
void Next() override {
PERF_COUNTER_ADD(next_on_memtable_count, 1);
assert(Valid());
iter_->Next();
TEST_SYNC_POINT_CALLBACK("MemTableIterator::Next:0", iter_);
valid_ = iter_->Valid();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
VerifyEntryChecksum();
}
bool NextAndGetResult(IterateResult* result) override {
Next();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
bool is_valid = Valid();
if (is_valid) {
result->key = key();
result->bound_check_result = IterBoundCheck::kUnknown;
result->value_prepared = true;
}
return is_valid;
}
void Prev() override {
PERF_COUNTER_ADD(prev_on_memtable_count, 1);
assert(Valid());
iter_->Prev();
valid_ = iter_->Valid();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
VerifyEntryChecksum();
}
Slice key() const override {
assert(Valid());
return GetLengthPrefixedSlice(iter_->key());
}
Slice value() const override {
assert(Valid());
Slice key_slice = GetLengthPrefixedSlice(iter_->key());
return GetLengthPrefixedSlice(key_slice.data() + key_slice.size());
}
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
Status status() const override { return status_; }
bool IsKeyPinned() const override {
Introduce ReadOptions::pin_data (support zero copy for keys) Summary: This patch update the Iterator API to introduce new functions that allow users to keep the Slices returned by key() valid as long as the Iterator is not deleted ReadOptions::pin_data : If true keep loaded blocks in memory as long as the iterator is not deleted Iterator::IsKeyPinned() : If true, this mean that the Slice returned by key() is valid as long as the iterator is not deleted Also add a new option BlockBasedTableOptions::use_delta_encoding to allow users to disable delta_encoding if needed. Benchmark results (using https://phabricator.fb.com/P20083553) ``` // $ du -h /home/tec/local/normal.4K.Snappy/db10077 // 6.1G /home/tec/local/normal.4K.Snappy/db10077 // $ du -h /home/tec/local/zero.8K.LZ4/db10077 // 6.4G /home/tec/local/zero.8K.LZ4/db10077 // Benchmarks for shard db10077 // _build/opt/rocks/benchmark/rocks_copy_benchmark \ // --normal_db_path="/home/tec/local/normal.4K.Snappy/db10077" \ // --zero_db_path="/home/tec/local/zero.8K.LZ4/db10077" // First run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 1.73s 576.97m // BM_StringPiece 103.74% 1.67s 598.55m // ============================================================================ // Match rate : 1000000 / 1000000 // Second run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 611.99ms 1.63 // BM_StringPiece 203.76% 300.35ms 3.33 // ============================================================================ // Match rate : 1000000 / 1000000 ``` Test Plan: Unit tests Reviewers: sdong, igor, anthony, yhchiang, rven Reviewed By: rven Subscribers: dhruba, lovro, adsharma Differential Revision: https://reviews.facebook.net/D48999
9 years ago
// memtable data is always pinned
return true;
}
bool IsValuePinned() const override {
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
8 years ago
// memtable value is always pinned, except if we allow inplace update.
return value_pinned_;
}
private:
DynamicBloom* bloom_;
const SliceTransform* const prefix_extractor_;
const MemTable::KeyComparator comparator_;
MemTableRep::Iterator* iter_;
bool valid_;
bool arena_mode_;
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
8 years ago
bool value_pinned_;
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
size_t protection_bytes_per_key_;
Status status_;
Logger* logger_;
void VerifyEntryChecksum() {
if (protection_bytes_per_key_ > 0 && Valid()) {
status_ = MemTable::VerifyEntryChecksum(iter_->key(),
protection_bytes_per_key_);
if (!status_.ok()) {
ROCKS_LOG_ERROR(logger_, "In MemtableIterator: %s", status_.getState());
}
}
}
};
InternalIterator* MemTable::NewIterator(const ReadOptions& read_options,
Arena* arena) {
assert(arena != nullptr);
auto mem = arena->AllocateAligned(sizeof(MemTableIterator));
return new (mem) MemTableIterator(*this, read_options, arena);
}
FragmentedRangeTombstoneIterator* MemTable::NewRangeTombstoneIterator(
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
const ReadOptions& read_options, SequenceNumber read_seq,
bool immutable_memtable) {
if (read_options.ignore_range_deletions ||
is_range_del_table_empty_.load(std::memory_order_relaxed)) {
return nullptr;
}
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
return NewRangeTombstoneIteratorInternal(read_options, read_seq,
immutable_memtable);
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
}
FragmentedRangeTombstoneIterator* MemTable::NewRangeTombstoneIteratorInternal(
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
const ReadOptions& read_options, SequenceNumber read_seq,
bool immutable_memtable) {
if (immutable_memtable) {
// Note that caller should already have verified that
// !is_range_del_table_empty_
assert(IsFragmentedRangeTombstonesConstructed());
return new FragmentedRangeTombstoneIterator(
fragmented_range_tombstone_list_.get(), comparator_.comparator,
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
read_seq, read_options.timestamp);
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
}
Cache fragmented range tombstone list for mutable memtables (#10547) Summary: Each read from memtable used to read and fragment all the range tombstones into a `FragmentedRangeTombstoneList`. https://github.com/facebook/rocksdb/issues/10380 improved the inefficient here by caching a `FragmentedRangeTombstoneList` with each immutable memtable. This PR extends the caching to mutable memtables. The fragmented range tombstone can be constructed in either read (This PR) or write path (https://github.com/facebook/rocksdb/issues/10584). With both implementation, each `DeleteRange()` will invalidate the cache, and the difference is where the cache is re-constructed.`CoreLocalArray` is used to store the cache with each memtable so that multi-threaded reads can be efficient. More specifically, each core will have a shared_ptr to a shared_ptr pointing to the current cache. Each read thread will only update the reference count in its core-local shared_ptr, and this is only needed when reading from mutable memtables. The choice between write path and read path is not an easy one: they are both improvement compared to no caching in the current implementation, but they favor different operations and could cause regression in the other operation (read vs write). The write path caching in (https://github.com/facebook/rocksdb/issues/10584) leads to a cleaner implementation, but I chose the read path caching here to avoid significant regression in write performance when there is a considerable amount of range tombstones in a single memtable (the number from the benchmark below suggests >1000 with concurrent writers). Note that even though the fragmented range tombstone list is only constructed in `DeleteRange()` operations, it could block other writes from proceeding, and hence affects overall write performance. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10547 Test Plan: - TestGet() in stress test is updated in https://github.com/facebook/rocksdb/issues/10553 to compare Get() result against expected state: `./db_stress_branch --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Perf benchmark: tested read and write performance where a memtable has 0, 1, 10, 100 and 1000 range tombstones. ``` ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=200000 --reads=100000 --disable_auto_compactions --max_num_range_tombstones=1000 ``` Write perf regressed since the cost of constructing fragmented range tombstone list is shifted from every read to a single write. 6cbe5d8e172dc5f1ef65c9d0a6eedbd9987b2c72 is included in the last column as a reference to see performance impact on multi-thread reads if `CoreLocalArray` is not used. micros/op averaged over 5 runs: first 4 columns are for fillrandom, last 4 columns are for readrandom. | |fillrandom main | write path caching | read path caching |memtable V3 (https://github.com/facebook/rocksdb/issues/10308) | readrandom main | write path caching | read path caching |memtable V3 | |--- |--- |--- |--- |--- | --- | --- | --- | --- | | 0 |6.35 |6.15 |5.82 |6.12 |2.24 |2.26 |2.03 |2.07 | | 1 |5.99 |5.88 |5.77 |6.28 |2.65 |2.27 |2.24 |2.5 | | 10 |6.15 |6.02 |5.92 |5.95 |5.15 |2.61 |2.31 |2.53 | | 100 |5.95 |5.78 |5.88 |6.23 |28.31 |2.34 |2.45 |2.94 | | 100 25 threads |52.01 |45.85 |46.18 |47.52 |35.97 |3.34 |3.34 |3.56 | | 1000 |6.0 |7.07 |5.98 |6.08 |333.18 |2.86 |2.7 |3.6 | | 1000 25 threads |52.6 |148.86 |79.06 |45.52 |473.49 |3.66 |3.48 |4.38 | - Benchmark performance of`readwhilewriting` from https://github.com/facebook/rocksdb/issues/10552, 100 range tombstones are written: `./db_bench --benchmarks=readwhilewriting --writes_per_range_tombstone=500 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=100000 --reads=500000 --disable_auto_compactions --max_num_range_tombstones=10000 --finish_after_writes` readrandom micros/op: | |main |write path caching |read path caching |memtable V3 | |---|---|---|---|---| | single thread |48.28 |1.55 |1.52 |1.96 | | 25 threads |64.3 |2.55 |2.67 |2.64 | Reviewed By: ajkr Differential Revision: D38895410 Pulled By: cbi42 fbshipit-source-id: 930bfc309dd1b2f4e8e9042f5126785bba577559
2 years ago
// takes current cache
std::shared_ptr<FragmentedRangeTombstoneListCache> cache =
std::atomic_load_explicit(cached_range_tombstone_.Access(),
std::memory_order_relaxed);
// construct fragmented tombstone list if necessary
if (!cache->initialized.load(std::memory_order_acquire)) {
cache->reader_mutex.lock();
if (!cache->tombstones) {
auto* unfragmented_iter =
new MemTableIterator(*this, read_options, nullptr /* arena */,
true /* use_range_del_table */);
Improve FragmentTombstones() speed by lazily initializing `seq_set_` (#10848) Summary: FragmentedRangeTombstoneList has a member variable `seq_set_` that contains the sequence numbers of all range tombstones in a set. The set is constructed in `FragmentTombstones()` and is used only in `FragmentedRangeTombstoneList::ContainsRange()` which only happens during compaction. This PR moves the initialization of `seq_set_` to `FragmentedRangeTombstoneList::ContainsRange()`. This should speed up `FragmentTombstones()` when the range tombstone list is used for read/scan requests. Microbench shows the speed improvement to be ~45%. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10848 Test Plan: - Existing tests and stress test: `python3 tools/db_crashtest.py whitebox --simple --verify_iterator_with_expected_state_one_in=5`. - Microbench: update `range_del_aggregator_bench` to benchmark speed of `FragmentTombstones()`: ``` ./range_del_aggregator_bench --num_range_tombstones=1000 --tombstone_start_upper_bound=50000000 --num_runs=10000 --tombstone_width_mean=200 --should_deletes_per_run=100 --use_compaction_range_del_aggregator=true Before this PR: ========================= Fragment Tombstones: 270.286 us AddTombstones: 1.28933 us ShouldDelete (first): 0.525528 us ShouldDelete (rest): 0.0797519 us After this PR: time to fragment tombstones is pushed to AddTombstones() which only happen during compaction. ========================= Fragment Tombstones: 149.879 us AddTombstones: 102.131 us ShouldDelete (first): 0.565871 us ShouldDelete (rest): 0.0729444 us ``` - db_bench: this should improve speed for fragmenting range tombstones for mutable memtable: ``` ./db_bench --benchmarks=readwhilewriting --writes_per_range_tombstone=100 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=250000 --disable_auto_compactions --max_num_range_tombstones=100000 --finish_after_writes --write_buffer_size=1073741824 --threads=25 Before this PR: readwhilewriting : 18.301 micros/op 1310445 ops/sec 4.769 seconds 6250000 operations; 28.1 MB/s (41001 of 250000 found) After this PR: readwhilewriting : 16.943 micros/op 1439376 ops/sec 4.342 seconds 6250000 operations; 23.8 MB/s (28977 of 250000 found) ``` Reviewed By: ajkr Differential Revision: D40646227 Pulled By: cbi42 fbshipit-source-id: ea471667edb258f67d01cfd828588e80a89e4083
2 years ago
cache->tombstones.reset(new FragmentedRangeTombstoneList(
std::unique_ptr<InternalIterator>(unfragmented_iter),
comparator_.comparator));
Cache fragmented range tombstone list for mutable memtables (#10547) Summary: Each read from memtable used to read and fragment all the range tombstones into a `FragmentedRangeTombstoneList`. https://github.com/facebook/rocksdb/issues/10380 improved the inefficient here by caching a `FragmentedRangeTombstoneList` with each immutable memtable. This PR extends the caching to mutable memtables. The fragmented range tombstone can be constructed in either read (This PR) or write path (https://github.com/facebook/rocksdb/issues/10584). With both implementation, each `DeleteRange()` will invalidate the cache, and the difference is where the cache is re-constructed.`CoreLocalArray` is used to store the cache with each memtable so that multi-threaded reads can be efficient. More specifically, each core will have a shared_ptr to a shared_ptr pointing to the current cache. Each read thread will only update the reference count in its core-local shared_ptr, and this is only needed when reading from mutable memtables. The choice between write path and read path is not an easy one: they are both improvement compared to no caching in the current implementation, but they favor different operations and could cause regression in the other operation (read vs write). The write path caching in (https://github.com/facebook/rocksdb/issues/10584) leads to a cleaner implementation, but I chose the read path caching here to avoid significant regression in write performance when there is a considerable amount of range tombstones in a single memtable (the number from the benchmark below suggests >1000 with concurrent writers). Note that even though the fragmented range tombstone list is only constructed in `DeleteRange()` operations, it could block other writes from proceeding, and hence affects overall write performance. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10547 Test Plan: - TestGet() in stress test is updated in https://github.com/facebook/rocksdb/issues/10553 to compare Get() result against expected state: `./db_stress_branch --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Perf benchmark: tested read and write performance where a memtable has 0, 1, 10, 100 and 1000 range tombstones. ``` ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=200000 --reads=100000 --disable_auto_compactions --max_num_range_tombstones=1000 ``` Write perf regressed since the cost of constructing fragmented range tombstone list is shifted from every read to a single write. 6cbe5d8e172dc5f1ef65c9d0a6eedbd9987b2c72 is included in the last column as a reference to see performance impact on multi-thread reads if `CoreLocalArray` is not used. micros/op averaged over 5 runs: first 4 columns are for fillrandom, last 4 columns are for readrandom. | |fillrandom main | write path caching | read path caching |memtable V3 (https://github.com/facebook/rocksdb/issues/10308) | readrandom main | write path caching | read path caching |memtable V3 | |--- |--- |--- |--- |--- | --- | --- | --- | --- | | 0 |6.35 |6.15 |5.82 |6.12 |2.24 |2.26 |2.03 |2.07 | | 1 |5.99 |5.88 |5.77 |6.28 |2.65 |2.27 |2.24 |2.5 | | 10 |6.15 |6.02 |5.92 |5.95 |5.15 |2.61 |2.31 |2.53 | | 100 |5.95 |5.78 |5.88 |6.23 |28.31 |2.34 |2.45 |2.94 | | 100 25 threads |52.01 |45.85 |46.18 |47.52 |35.97 |3.34 |3.34 |3.56 | | 1000 |6.0 |7.07 |5.98 |6.08 |333.18 |2.86 |2.7 |3.6 | | 1000 25 threads |52.6 |148.86 |79.06 |45.52 |473.49 |3.66 |3.48 |4.38 | - Benchmark performance of`readwhilewriting` from https://github.com/facebook/rocksdb/issues/10552, 100 range tombstones are written: `./db_bench --benchmarks=readwhilewriting --writes_per_range_tombstone=500 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=100000 --reads=500000 --disable_auto_compactions --max_num_range_tombstones=10000 --finish_after_writes` readrandom micros/op: | |main |write path caching |read path caching |memtable V3 | |---|---|---|---|---| | single thread |48.28 |1.55 |1.52 |1.96 | | 25 threads |64.3 |2.55 |2.67 |2.64 | Reviewed By: ajkr Differential Revision: D38895410 Pulled By: cbi42 fbshipit-source-id: 930bfc309dd1b2f4e8e9042f5126785bba577559
2 years ago
cache->initialized.store(true, std::memory_order_release);
}
cache->reader_mutex.unlock();
}
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
auto* fragmented_iter = new FragmentedRangeTombstoneIterator(
cache, comparator_.comparator, read_seq, read_options.timestamp);
return fragmented_iter;
}
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
void MemTable::ConstructFragmentedRangeTombstones() {
assert(!IsFragmentedRangeTombstonesConstructed(false));
// There should be no concurrent Construction
if (!is_range_del_table_empty_.load(std::memory_order_relaxed)) {
auto* unfragmented_iter =
new MemTableIterator(*this, ReadOptions(), nullptr /* arena */,
true /* use_range_del_table */);
fragmented_range_tombstone_list_ =
std::make_unique<FragmentedRangeTombstoneList>(
std::unique_ptr<InternalIterator>(unfragmented_iter),
comparator_.comparator);
}
}
port::RWMutex* MemTable::GetLock(const Slice& key) {
return &locks_[GetSliceRangedNPHash(key, locks_.size())];
}
MemTable::MemTableStats MemTable::ApproximateStats(const Slice& start_ikey,
const Slice& end_ikey) {
uint64_t entry_count = table_->ApproximateNumEntries(start_ikey, end_ikey);
entry_count += range_del_table_->ApproximateNumEntries(start_ikey, end_ikey);
if (entry_count == 0) {
return {0, 0};
}
uint64_t n = num_entries_.load(std::memory_order_relaxed);
if (n == 0) {
return {0, 0};
}
if (entry_count > n) {
// (range_del_)table_->ApproximateNumEntries() is just an estimate so it can
// be larger than actual entries we have. Cap it to entries we have to limit
// the inaccuracy.
entry_count = n;
}
uint64_t data_size = data_size_.load(std::memory_order_relaxed);
return {entry_count * (data_size / n), entry_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
Status MemTable::VerifyEncodedEntry(Slice encoded,
const ProtectionInfoKVOS64& kv_prot_info) {
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
uint32_t ikey_len = 0;
if (!GetVarint32(&encoded, &ikey_len)) {
return Status::Corruption("Unable to parse internal key length");
}
size_t ts_sz = GetInternalKeyComparator().user_comparator()->timestamp_size();
if (ikey_len < 8 + ts_sz) {
return Status::Corruption("Internal key length too short");
}
if (ikey_len > encoded.size()) {
return Status::Corruption("Internal key length too long");
}
uint32_t value_len = 0;
const size_t user_key_len = ikey_len - 8;
Slice key(encoded.data(), user_key_len);
encoded.remove_prefix(user_key_len);
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
uint64_t packed = DecodeFixed64(encoded.data());
ValueType value_type = kMaxValue;
SequenceNumber sequence_number = kMaxSequenceNumber;
UnPackSequenceAndType(packed, &sequence_number, &value_type);
encoded.remove_prefix(8);
if (!GetVarint32(&encoded, &value_len)) {
return Status::Corruption("Unable to parse value length");
}
if (value_len < encoded.size()) {
return Status::Corruption("Value length too short");
}
if (value_len > encoded.size()) {
return Status::Corruption("Value length too long");
}
Slice value(encoded.data(), value_len);
return kv_prot_info.StripS(sequence_number)
.StripKVO(key, value, value_type)
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
.GetStatus();
}
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
void MemTable::UpdateEntryChecksum(const ProtectionInfoKVOS64* kv_prot_info,
const Slice& key, const Slice& value,
ValueType type, SequenceNumber s,
char* checksum_ptr) {
if (moptions_.protection_bytes_per_key == 0) {
return;
}
uint64_t checksum = 0;
if (kv_prot_info == nullptr) {
checksum =
ProtectionInfo64().ProtectKVO(key, value, type).ProtectS(s).GetVal();
} else {
checksum = kv_prot_info->GetVal();
}
switch (moptions_.protection_bytes_per_key) {
case 1:
checksum_ptr[0] = static_cast<uint8_t>(checksum);
break;
case 2:
EncodeFixed16(checksum_ptr, static_cast<uint16_t>(checksum));
break;
case 4:
EncodeFixed32(checksum_ptr, static_cast<uint32_t>(checksum));
break;
case 8:
EncodeFixed64(checksum_ptr, checksum);
break;
default:
assert(false);
}
}
Status MemTable::Add(SequenceNumber s, ValueType type,
const Slice& key, /* user key */
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
const Slice& value,
const ProtectionInfoKVOS64* kv_prot_info,
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
bool allow_concurrent,
MemTablePostProcessInfo* post_process_info, void** hint) {
// Format of an entry is concatenation of:
// key_size : varint32 of internal_key.size()
// key bytes : char[internal_key.size()]
// value_size : varint32 of value.size()
// value bytes : char[value.size()]
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
// checksum : char[moptions_.protection_bytes_per_key]
uint32_t key_size = static_cast<uint32_t>(key.size());
uint32_t val_size = static_cast<uint32_t>(value.size());
uint32_t internal_key_size = key_size + 8;
const uint32_t encoded_len = VarintLength(internal_key_size) +
internal_key_size + VarintLength(val_size) +
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
val_size + moptions_.protection_bytes_per_key;
char* buf = nullptr;
std::unique_ptr<MemTableRep>& table =
type == kTypeRangeDeletion ? range_del_table_ : table_;
KeyHandle handle = table->Allocate(encoded_len, &buf);
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
char* p = EncodeVarint32(buf, internal_key_size);
memcpy(p, key.data(), key_size);
Slice key_slice(p, key_size);
p += key_size;
uint64_t packed = PackSequenceAndType(s, type);
EncodeFixed64(p, packed);
p += 8;
p = EncodeVarint32(p, val_size);
memcpy(p, value.data(), val_size);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
assert((unsigned)(p + val_size - buf + moptions_.protection_bytes_per_key) ==
(unsigned)encoded_len);
UpdateEntryChecksum(kv_prot_info, key, value, type, s,
buf + encoded_len - moptions_.protection_bytes_per_key);
Slice encoded(buf, encoded_len - moptions_.protection_bytes_per_key);
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
if (kv_prot_info != nullptr) {
TEST_SYNC_POINT_CALLBACK("MemTable::Add:Encoded", &encoded);
Status status = VerifyEncodedEntry(encoded, *kv_prot_info);
if (!status.ok()) {
return status;
}
}
Add support for timestamp in Get/Put (#5079) Summary: It's useful to be able to (optionally) associate key-value pairs with user-provided timestamps. This PR is an early effort towards this goal and continues the work of facebook#4942. A suite of new unit tests exist in DBBasicTestWithTimestampWithParam. Support for timestamp requires the user to provide timestamp as a slice in `ReadOptions` and `WriteOptions`. All timestamps of the same database must share the same length, format, etc. The format of the timestamp is the same throughout the same database, and the user is responsible for providing a comparator function (Comparator) to order the <key, timestamp> tuples. Once created, the format and length of the timestamp cannot change (at least for now). Test plan (on devserver): ``` $COMPILE_WITH_ASAN=1 make -j32 all $./db_basic_test --gtest_filter=Timestamp/DBBasicTestWithTimestampWithParam.PutAndGet/* $make check ``` All tests must pass. We also run the following db_bench tests to verify whether there is regression on Get/Put while timestamp is not enabled. ``` $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillseq,readrandom -num=1000000 $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=1000000 ``` Repeat for 6 times for both versions. Results are as follows: ``` | | readrandom | fillrandom | | master | 16.77 MB/s | 47.05 MB/s | | PR5079 | 16.44 MB/s | 47.03 MB/s | ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/5079 Differential Revision: D15132946 Pulled By: riversand963 fbshipit-source-id: 833a0d657eac21182f0f206c910a6438154c742c
5 years ago
size_t ts_sz = GetInternalKeyComparator().user_comparator()->timestamp_size();
Slice key_without_ts = StripTimestampFromUserKey(key, ts_sz);
Add support for timestamp in Get/Put (#5079) Summary: It's useful to be able to (optionally) associate key-value pairs with user-provided timestamps. This PR is an early effort towards this goal and continues the work of facebook#4942. A suite of new unit tests exist in DBBasicTestWithTimestampWithParam. Support for timestamp requires the user to provide timestamp as a slice in `ReadOptions` and `WriteOptions`. All timestamps of the same database must share the same length, format, etc. The format of the timestamp is the same throughout the same database, and the user is responsible for providing a comparator function (Comparator) to order the <key, timestamp> tuples. Once created, the format and length of the timestamp cannot change (at least for now). Test plan (on devserver): ``` $COMPILE_WITH_ASAN=1 make -j32 all $./db_basic_test --gtest_filter=Timestamp/DBBasicTestWithTimestampWithParam.PutAndGet/* $make check ``` All tests must pass. We also run the following db_bench tests to verify whether there is regression on Get/Put while timestamp is not enabled. ``` $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillseq,readrandom -num=1000000 $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=1000000 ``` Repeat for 6 times for both versions. Results are as follows: ``` | | readrandom | fillrandom | | master | 16.77 MB/s | 47.05 MB/s | | PR5079 | 16.44 MB/s | 47.03 MB/s | ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/5079 Differential Revision: D15132946 Pulled By: riversand963 fbshipit-source-id: 833a0d657eac21182f0f206c910a6438154c742c
5 years ago
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 (!allow_concurrent) {
// Extract prefix for insert with hint.
if (insert_with_hint_prefix_extractor_ != nullptr &&
insert_with_hint_prefix_extractor_->InDomain(key_slice)) {
Slice prefix = insert_with_hint_prefix_extractor_->Transform(key_slice);
bool res = table->InsertKeyWithHint(handle, &insert_hints_[prefix]);
if (UNLIKELY(!res)) {
return Status::TryAgain("key+seq exists");
}
} else {
bool res = table->InsertKey(handle);
if (UNLIKELY(!res)) {
return Status::TryAgain("key+seq exists");
}
}
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
// this is a bit ugly, but is the way to avoid locked instructions
// when incrementing an atomic
num_entries_.store(num_entries_.load(std::memory_order_relaxed) + 1,
std::memory_order_relaxed);
data_size_.store(data_size_.load(std::memory_order_relaxed) + encoded_len,
std::memory_order_relaxed);
if (type == kTypeDeletion || type == kTypeSingleDeletion ||
type == kTypeDeletionWithTimestamp) {
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
num_deletes_.store(num_deletes_.load(std::memory_order_relaxed) + 1,
std::memory_order_relaxed);
}
if (bloom_filter_ && prefix_extractor_ &&
prefix_extractor_->InDomain(key_without_ts)) {
bloom_filter_->Add(prefix_extractor_->Transform(key_without_ts));
}
if (bloom_filter_ && moptions_.memtable_whole_key_filtering) {
bloom_filter_->Add(key_without_ts);
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
}
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
// The first sequence number inserted into the memtable
assert(first_seqno_ == 0 || s >= first_seqno_);
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 (first_seqno_ == 0) {
first_seqno_.store(s, std::memory_order_relaxed);
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 (earliest_seqno_ == kMaxSequenceNumber) {
earliest_seqno_.store(GetFirstSequenceNumber(),
std::memory_order_relaxed);
}
assert(first_seqno_.load() >= earliest_seqno_.load());
}
assert(post_process_info == nullptr);
UpdateFlushState();
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
} else {
bool res = (hint == nullptr)
? table->InsertKeyConcurrently(handle)
: table->InsertKeyWithHintConcurrently(handle, hint);
if (UNLIKELY(!res)) {
return Status::TryAgain("key+seq exists");
}
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
assert(post_process_info != nullptr);
post_process_info->num_entries++;
post_process_info->data_size += encoded_len;
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 (type == kTypeDeletion) {
post_process_info->num_deletes++;
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
9 years ago
}
if (bloom_filter_ && prefix_extractor_ &&
prefix_extractor_->InDomain(key_without_ts)) {
bloom_filter_->AddConcurrently(
prefix_extractor_->Transform(key_without_ts));
}
if (bloom_filter_ && moptions_.memtable_whole_key_filtering) {
bloom_filter_->AddConcurrently(key_without_ts);
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
}
// atomically update first_seqno_ and earliest_seqno_.
uint64_t cur_seq_num = first_seqno_.load(std::memory_order_relaxed);
while ((cur_seq_num == 0 || s < cur_seq_num) &&
!first_seqno_.compare_exchange_weak(cur_seq_num, s)) {
}
uint64_t cur_earliest_seqno =
earliest_seqno_.load(std::memory_order_relaxed);
while (
(cur_earliest_seqno == kMaxSequenceNumber || s < cur_earliest_seqno) &&
!first_seqno_.compare_exchange_weak(cur_earliest_seqno, s)) {
}
}
if (type == kTypeRangeDeletion) {
Cache fragmented range tombstone list for mutable memtables (#10547) Summary: Each read from memtable used to read and fragment all the range tombstones into a `FragmentedRangeTombstoneList`. https://github.com/facebook/rocksdb/issues/10380 improved the inefficient here by caching a `FragmentedRangeTombstoneList` with each immutable memtable. This PR extends the caching to mutable memtables. The fragmented range tombstone can be constructed in either read (This PR) or write path (https://github.com/facebook/rocksdb/issues/10584). With both implementation, each `DeleteRange()` will invalidate the cache, and the difference is where the cache is re-constructed.`CoreLocalArray` is used to store the cache with each memtable so that multi-threaded reads can be efficient. More specifically, each core will have a shared_ptr to a shared_ptr pointing to the current cache. Each read thread will only update the reference count in its core-local shared_ptr, and this is only needed when reading from mutable memtables. The choice between write path and read path is not an easy one: they are both improvement compared to no caching in the current implementation, but they favor different operations and could cause regression in the other operation (read vs write). The write path caching in (https://github.com/facebook/rocksdb/issues/10584) leads to a cleaner implementation, but I chose the read path caching here to avoid significant regression in write performance when there is a considerable amount of range tombstones in a single memtable (the number from the benchmark below suggests >1000 with concurrent writers). Note that even though the fragmented range tombstone list is only constructed in `DeleteRange()` operations, it could block other writes from proceeding, and hence affects overall write performance. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10547 Test Plan: - TestGet() in stress test is updated in https://github.com/facebook/rocksdb/issues/10553 to compare Get() result against expected state: `./db_stress_branch --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Perf benchmark: tested read and write performance where a memtable has 0, 1, 10, 100 and 1000 range tombstones. ``` ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=200000 --reads=100000 --disable_auto_compactions --max_num_range_tombstones=1000 ``` Write perf regressed since the cost of constructing fragmented range tombstone list is shifted from every read to a single write. 6cbe5d8e172dc5f1ef65c9d0a6eedbd9987b2c72 is included in the last column as a reference to see performance impact on multi-thread reads if `CoreLocalArray` is not used. micros/op averaged over 5 runs: first 4 columns are for fillrandom, last 4 columns are for readrandom. | |fillrandom main | write path caching | read path caching |memtable V3 (https://github.com/facebook/rocksdb/issues/10308) | readrandom main | write path caching | read path caching |memtable V3 | |--- |--- |--- |--- |--- | --- | --- | --- | --- | | 0 |6.35 |6.15 |5.82 |6.12 |2.24 |2.26 |2.03 |2.07 | | 1 |5.99 |5.88 |5.77 |6.28 |2.65 |2.27 |2.24 |2.5 | | 10 |6.15 |6.02 |5.92 |5.95 |5.15 |2.61 |2.31 |2.53 | | 100 |5.95 |5.78 |5.88 |6.23 |28.31 |2.34 |2.45 |2.94 | | 100 25 threads |52.01 |45.85 |46.18 |47.52 |35.97 |3.34 |3.34 |3.56 | | 1000 |6.0 |7.07 |5.98 |6.08 |333.18 |2.86 |2.7 |3.6 | | 1000 25 threads |52.6 |148.86 |79.06 |45.52 |473.49 |3.66 |3.48 |4.38 | - Benchmark performance of`readwhilewriting` from https://github.com/facebook/rocksdb/issues/10552, 100 range tombstones are written: `./db_bench --benchmarks=readwhilewriting --writes_per_range_tombstone=500 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=100000 --reads=500000 --disable_auto_compactions --max_num_range_tombstones=10000 --finish_after_writes` readrandom micros/op: | |main |write path caching |read path caching |memtable V3 | |---|---|---|---|---| | single thread |48.28 |1.55 |1.52 |1.96 | | 25 threads |64.3 |2.55 |2.67 |2.64 | Reviewed By: ajkr Differential Revision: D38895410 Pulled By: cbi42 fbshipit-source-id: 930bfc309dd1b2f4e8e9042f5126785bba577559
2 years ago
auto new_cache = std::make_shared<FragmentedRangeTombstoneListCache>();
size_t size = cached_range_tombstone_.Size();
if (allow_concurrent) {
range_del_mutex_.lock();
}
for (size_t i = 0; i < size; ++i) {
std::shared_ptr<FragmentedRangeTombstoneListCache>* local_cache_ref_ptr =
cached_range_tombstone_.AccessAtCore(i);
auto new_local_cache_ref = std::make_shared<
const std::shared_ptr<FragmentedRangeTombstoneListCache>>(new_cache);
// It is okay for some reader to load old cache during invalidation as
// the new sequence number is not published yet.
// Each core will have a shared_ptr to a shared_ptr to the cached
// fragmented range tombstones, so that ref count is maintianed locally
// per-core using the per-core shared_ptr.
std::atomic_store_explicit(
local_cache_ref_ptr,
std::shared_ptr<FragmentedRangeTombstoneListCache>(
new_local_cache_ref, new_cache.get()),
std::memory_order_relaxed);
}
if (allow_concurrent) {
range_del_mutex_.unlock();
}
is_range_del_table_empty_.store(false, std::memory_order_relaxed);
}
UpdateOldestKeyTime();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
TEST_SYNC_POINT_CALLBACK("MemTable::Add:BeforeReturn:Encoded", &encoded);
return Status::OK();
}
// Callback from MemTable::Get()
namespace {
struct Saver {
Status* status;
const LookupKey* key;
bool* found_final_value; // Is value set correctly? Used by KeyMayExist
bool* merge_in_progress;
std::string* value;
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
PinnableWideColumns* columns;
SequenceNumber seq;
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
4 years ago
std::string* timestamp;
const MergeOperator* merge_operator;
// the merge operations encountered;
MergeContext* merge_context;
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
SequenceNumber max_covering_tombstone_seq;
MemTable* mem;
Logger* logger;
Statistics* statistics;
bool inplace_update_support;
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
bool do_merge;
SystemClock* clock;
ReadCallback* callback_;
bool* is_blob_index;
bool allow_data_in_errors;
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
size_t protection_bytes_per_key;
bool CheckCallback(SequenceNumber _seq) {
if (callback_) {
return callback_->IsVisible(_seq);
}
return true;
}
};
} // anonymous namespace
static bool SaveValue(void* arg, const char* entry) {
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
TEST_SYNC_POINT_CALLBACK("Memtable::SaveValue:Begin:entry", &entry);
Saver* s = reinterpret_cast<Saver*>(arg);
assert(s != nullptr);
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
assert(!s->value || !s->columns);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
if (s->protection_bytes_per_key > 0) {
*(s->status) = MemTable::VerifyEntryChecksum(
entry, s->protection_bytes_per_key, s->allow_data_in_errors);
if (!s->status->ok()) {
ROCKS_LOG_ERROR(s->logger, "In SaveValue: %s", s->status->getState());
// Memtable entry corrupted
return false;
}
}
MergeContext* merge_context = s->merge_context;
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
SequenceNumber max_covering_tombstone_seq = s->max_covering_tombstone_seq;
const MergeOperator* merge_operator = s->merge_operator;
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
assert(merge_context != nullptr);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
// Refer to comments under MemTable::Add() for entry format.
// Check that it belongs to same user key.
uint32_t key_length = 0;
const char* key_ptr = GetVarint32Ptr(entry, entry + 5, &key_length);
assert(key_length >= 8);
Add support for timestamp in Get/Put (#5079) Summary: It's useful to be able to (optionally) associate key-value pairs with user-provided timestamps. This PR is an early effort towards this goal and continues the work of facebook#4942. A suite of new unit tests exist in DBBasicTestWithTimestampWithParam. Support for timestamp requires the user to provide timestamp as a slice in `ReadOptions` and `WriteOptions`. All timestamps of the same database must share the same length, format, etc. The format of the timestamp is the same throughout the same database, and the user is responsible for providing a comparator function (Comparator) to order the <key, timestamp> tuples. Once created, the format and length of the timestamp cannot change (at least for now). Test plan (on devserver): ``` $COMPILE_WITH_ASAN=1 make -j32 all $./db_basic_test --gtest_filter=Timestamp/DBBasicTestWithTimestampWithParam.PutAndGet/* $make check ``` All tests must pass. We also run the following db_bench tests to verify whether there is regression on Get/Put while timestamp is not enabled. ``` $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillseq,readrandom -num=1000000 $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=1000000 ``` Repeat for 6 times for both versions. Results are as follows: ``` | | readrandom | fillrandom | | master | 16.77 MB/s | 47.05 MB/s | | PR5079 | 16.44 MB/s | 47.03 MB/s | ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/5079 Differential Revision: D15132946 Pulled By: riversand963 fbshipit-source-id: 833a0d657eac21182f0f206c910a6438154c742c
5 years ago
Slice user_key_slice = Slice(key_ptr, key_length - 8);
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
4 years ago
const Comparator* user_comparator =
s->mem->GetInternalKeyComparator().user_comparator();
size_t ts_sz = user_comparator->timestamp_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
if (ts_sz && s->timestamp && max_covering_tombstone_seq > 0) {
// timestamp should already be set to range tombstone timestamp
assert(s->timestamp->size() == ts_sz);
}
Enable backward iterator for keys with user-defined timestamp (#8035) Summary: This PR does the following: - Enable backward iteration for keys with user-defined timestamp. Note that merge, single delete, range delete are not supported yet. - Introduces a new helper API `Comparator::EqualWithoutTimestamp()`. - Fix a typo in `SetTimestamp()`. - Add/update unit tests Run db_bench (built with DEBUG_LEVEL=0) to demonstrate that no overhead is introduced for CPU-intensive workloads with a lot of `Prev()`. Also provided results of iterating keys with timestamps. 1. Disable timestamp, run: ``` ./db_bench -db=/dev/shm/rocksdb -disable_wal=1 -benchmarks=fillseq,seekrandom[-W1-X6] -reverse_iterator=1 -seek_nexts=5 ``` Results: > Baseline > - seekrandom [AVG 6 runs] : 96115 ops/sec; 53.2 MB/sec > - seekrandom [MEDIAN 6 runs] : 98075 ops/sec; 54.2 MB/sec > > This PR > - seekrandom [AVG 6 runs] : 95521 ops/sec; 52.8 MB/sec > - seekrandom [MEDIAN 6 runs] : 96338 ops/sec; 53.3 MB/sec 2. Enable timestamp, run: ``` ./db_bench -user_timestamp_size=8 -db=/dev/shm/rocksdb -disable_wal=1 -benchmarks=fillseq,seekrandom[-W1-X6] -reverse_iterator=1 -seek_nexts=5 ``` Result: > Baseline: not supported > > This PR > - seekrandom [AVG 6 runs] : 90514 ops/sec; 50.1 MB/sec > - seekrandom [MEDIAN 6 runs] : 90834 ops/sec; 50.2 MB/sec Pull Request resolved: https://github.com/facebook/rocksdb/pull/8035 Reviewed By: ltamasi Differential Revision: D26926668 Pulled By: riversand963 fbshipit-source-id: 95330cc2242397c03e09d29e5417dfb0adc98ef5
3 years ago
if (user_comparator->EqualWithoutTimestamp(user_key_slice,
s->key->user_key())) {
// Correct user key
const uint64_t tag = DecodeFixed64(key_ptr + key_length - 8);
ValueType type;
SequenceNumber seq;
UnPackSequenceAndType(tag, &seq, &type);
// If the value is not in the snapshot, skip it
if (!s->CheckCallback(seq)) {
return true; // to continue to the next seq
}
if (s->seq == kMaxSequenceNumber) {
s->seq = seq;
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
if (s->seq > max_covering_tombstone_seq) {
if (ts_sz && s->timestamp != nullptr) {
// `timestamp` was set to range tombstone's timestamp before
// `SaveValue` is ever called. This key has a higher sequence number
// than range tombstone, and is the key with the highest seqno across
// all keys with this user_key, so we update timestamp here.
Slice ts = ExtractTimestampFromUserKey(user_key_slice, ts_sz);
s->timestamp->assign(ts.data(), ts_sz);
}
} else {
s->seq = max_covering_tombstone_seq;
}
}
if (ts_sz > 0 && s->timestamp != nullptr) {
if (!s->timestamp->empty()) {
assert(ts_sz == s->timestamp->size());
}
// TODO optimize for smaller size ts
const std::string kMaxTs(ts_sz, '\xff');
if (s->timestamp->empty() ||
user_comparator->CompareTimestamp(*(s->timestamp), kMaxTs) == 0) {
Slice ts = ExtractTimestampFromUserKey(user_key_slice, ts_sz);
s->timestamp->assign(ts.data(), ts_sz);
}
}
if ((type == kTypeValue || type == kTypeMerge || type == kTypeBlobIndex ||
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
type == kTypeWideColumnEntity || type == kTypeDeletion ||
type == kTypeSingleDeletion || type == kTypeDeletionWithTimestamp) &&
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
max_covering_tombstone_seq > seq) {
type = kTypeRangeDeletion;
}
switch (type) {
case kTypeBlobIndex: {
if (!s->do_merge) {
*(s->status) = Status::NotSupported(
"GetMergeOperands not supported by stacked BlobDB");
*(s->found_final_value) = true;
return false;
}
if (*(s->merge_in_progress)) {
*(s->status) = Status::NotSupported(
"Merge operator not supported by stacked BlobDB");
*(s->found_final_value) = true;
return false;
}
if (s->is_blob_index == nullptr) {
ROCKS_LOG_ERROR(s->logger, "Encountered unexpected blob index.");
*(s->status) = Status::NotSupported(
"Encountered unexpected blob index. Please open DB with "
"ROCKSDB_NAMESPACE::blob_db::BlobDB.");
*(s->found_final_value) = true;
return false;
}
if (s->inplace_update_support) {
s->mem->GetLock(s->key->user_key())->ReadLock();
}
Slice v = GetLengthPrefixedSlice(key_ptr + key_length);
*(s->status) = Status::OK();
if (s->value) {
s->value->assign(v.data(), v.size());
} else if (s->columns) {
s->columns->SetPlainValue(v);
}
if (s->inplace_update_support) {
s->mem->GetLock(s->key->user_key())->ReadUnlock();
}
*(s->found_final_value) = true;
*(s->is_blob_index) = true;
return false;
}
case kTypeValue: {
if (s->inplace_update_support) {
s->mem->GetLock(s->key->user_key())->ReadLock();
}
Slice v = GetLengthPrefixedSlice(key_ptr + key_length);
*(s->status) = Status::OK();
if (!s->do_merge) {
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
// Preserve the value with the goal of returning it as part of
// raw merge operands to the user
// TODO(yanqin) update MergeContext so that timestamps information
// can also be retained.
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
merge_context->PushOperand(
v, s->inplace_update_support == false /* operand_pinned */);
} else if (*(s->merge_in_progress)) {
assert(s->do_merge);
if (s->value || s->columns) {
std::string result;
*(s->status) = MergeHelper::TimedFullMerge(
merge_operator, s->key->user_key(), &v,
merge_context->GetOperands(), &result, s->logger, s->statistics,
s->clock, /* result_operand */ nullptr,
/* update_num_ops_stats */ true);
if (s->status->ok()) {
if (s->value) {
*(s->value) = std::move(result);
} else {
assert(s->columns);
s->columns->SetPlainValue(result);
}
}
}
} else if (s->value) {
s->value->assign(v.data(), v.size());
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
} else if (s->columns) {
s->columns->SetPlainValue(v);
}
if (s->inplace_update_support) {
s->mem->GetLock(s->key->user_key())->ReadUnlock();
}
*(s->found_final_value) = true;
if (s->is_blob_index != nullptr) {
*(s->is_blob_index) = false;
}
return false;
}
case kTypeWideColumnEntity: {
if (s->inplace_update_support) {
s->mem->GetLock(s->key->user_key())->ReadLock();
}
Slice v = GetLengthPrefixedSlice(key_ptr + key_length);
*(s->status) = Status::OK();
if (!s->do_merge) {
// Preserve the value with the goal of returning it as part of
// raw merge operands to the user
Slice value_of_default;
*(s->status) = WideColumnSerialization::GetValueOfDefaultColumn(
v, value_of_default);
if (s->status->ok()) {
merge_context->PushOperand(
value_of_default,
s->inplace_update_support == false /* operand_pinned */);
}
} else if (*(s->merge_in_progress)) {
assert(s->do_merge);
if (s->value) {
Slice value_of_default;
*(s->status) = WideColumnSerialization::GetValueOfDefaultColumn(
v, value_of_default);
if (s->status->ok()) {
*(s->status) = MergeHelper::TimedFullMerge(
merge_operator, s->key->user_key(), &value_of_default,
merge_context->GetOperands(), s->value, s->logger,
s->statistics, s->clock, /* result_operand */ nullptr,
/* update_num_ops_stats */ true);
}
} else if (s->columns) {
std::string result;
*(s->status) = MergeHelper::TimedFullMergeWithEntity(
merge_operator, s->key->user_key(), v,
merge_context->GetOperands(), &result, s->logger, s->statistics,
s->clock, /* update_num_ops_stats */ true);
if (s->status->ok()) {
*(s->status) = s->columns->SetWideColumnValue(result);
}
}
} else if (s->value) {
Slice value_of_default;
*(s->status) = WideColumnSerialization::GetValueOfDefaultColumn(
v, value_of_default);
if (s->status->ok()) {
s->value->assign(value_of_default.data(), value_of_default.size());
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
}
} else if (s->columns) {
*(s->status) = s->columns->SetWideColumnValue(v);
}
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
if (s->inplace_update_support) {
s->mem->GetLock(s->key->user_key())->ReadUnlock();
}
*(s->found_final_value) = true;
if (s->is_blob_index != nullptr) {
*(s->is_blob_index) = false;
}
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
4 years ago
return false;
}
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
case kTypeDeletion:
case kTypeDeletionWithTimestamp:
case kTypeSingleDeletion:
case kTypeRangeDeletion: {
if (*(s->merge_in_progress)) {
if (s->value || s->columns) {
std::string result;
*(s->status) = MergeHelper::TimedFullMerge(
merge_operator, s->key->user_key(), nullptr,
merge_context->GetOperands(), &result, s->logger, s->statistics,
s->clock, /* result_operand */ nullptr,
/* update_num_ops_stats */ true);
if (s->status->ok()) {
if (s->value) {
*(s->value) = std::move(result);
} else {
assert(s->columns);
s->columns->SetPlainValue(result);
}
}
}
} else {
*(s->status) = Status::NotFound();
}
*(s->found_final_value) = true;
return false;
}
case kTypeMerge: {
if (!merge_operator) {
*(s->status) = Status::InvalidArgument(
"merge_operator is not properly initialized.");
// Normally we continue the loop (return true) when we see a merge
// operand. But in case of an error, we should stop the loop
// immediately and pretend we have found the value to stop further
// seek. Otherwise, the later call will override this error status.
*(s->found_final_value) = true;
return false;
}
Slice v = GetLengthPrefixedSlice(key_ptr + key_length);
*(s->merge_in_progress) = true;
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
8 years ago
merge_context->PushOperand(
v, s->inplace_update_support == false /* operand_pinned */);
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
if (s->do_merge && merge_operator->ShouldMerge(
merge_context->GetOperandsDirectionBackward())) {
if (s->value || s->columns) {
std::string result;
*(s->status) = MergeHelper::TimedFullMerge(
merge_operator, s->key->user_key(), nullptr,
merge_context->GetOperands(), &result, s->logger, s->statistics,
s->clock, /* result_operand */ nullptr,
/* update_num_ops_stats */ true);
if (s->status->ok()) {
if (s->value) {
*(s->value) = std::move(result);
} else {
assert(s->columns);
s->columns->SetPlainValue(result);
}
}
}
Introduce conditional merge-operator invocation in point lookups Summary: For every merge operand encountered for a key in the read path we now have the ability to decide whether to look further (to retrieve more merge operands for the key) or stop and invoke the merge operator to return the value. The user needs to override `ShouldMerge()` method with a condition to terminate search when true to avail this facility. This has a couple of advantages: 1. It helps in limiting the number of merge operands that are looked at to compute a value as part of a user Get operation. 2. It allows to peek at a merge key-value to see if further merge operands need to look at. Example: Limiting the number of merge operands that are looked at: Lets say you have 10 merge operands for a key spread over various levels. If you only want RocksDB to look at the latest two merge operands instead of all 10 to compute the value, it is now possible with this PR. You can set the condition in `ShouldMerge()` to return true when the size of the operand list is 2. Look at the example implementation in the unit test. Without this PR, a Get might look at all the 10 merge operands in different levels before invoking the merge-operator. Added a new unit test. Made sure that there is no perf regression by running benchmarks. Command line to Load data: ``` TEST_TMPDIR=/dev/shm ./db_bench --benchmarks="mergerandom" --merge_operator="uint64add" --num=10000000 ... mergerandom : 12.861 micros/op 77757 ops/sec; 8.6 MB/s ( updates:10000000) ``` **ReadRandomMergeRandom bechmark results:** Command line: ``` TEST_TMPDIR=/dev/shm ./db_bench --benchmarks="readrandommergerandom" --merge_operator="uint64add" --num=10000000 ``` Base -- Without this code change (on commit fc7476b): ``` readrandommergerandom : 38.586 micros/op 25916 ops/sec; (reads:3001599 merges:6998401 total:10000000 hits:842235 maxlength:8) ``` With this code change: ``` readrandommergerandom : 38.653 micros/op 25870 ops/sec; (reads:3001599 merges:6998401 total:10000000 hits:842235 maxlength:8) ``` Closes https://github.com/facebook/rocksdb/pull/2923 Differential Revision: D5898239 Pulled By: sagar0 fbshipit-source-id: daefa325019f77968639a75c851d46352c2303ef
7 years ago
*(s->found_final_value) = true;
return false;
}
return true;
}
default: {
std::string msg("Corrupted value not expected.");
if (s->allow_data_in_errors) {
msg.append("Unrecognized value type: " +
std::to_string(static_cast<int>(type)) + ". ");
msg.append("User key: " + user_key_slice.ToString(/*hex=*/true) +
". ");
msg.append("seq: " + std::to_string(seq) + ".");
}
*(s->status) = Status::Corruption(msg.c_str());
return false;
}
}
}
// s->state could be Corrupt, merge or notfound
return false;
}
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
4 years ago
bool MemTable::Get(const LookupKey& key, std::string* value,
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
PinnableWideColumns* columns, std::string* timestamp,
Status* s, MergeContext* merge_context,
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
SequenceNumber* max_covering_tombstone_seq,
SequenceNumber* seq, const ReadOptions& read_opts,
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
bool immutable_memtable, ReadCallback* callback,
bool* is_blob_index, bool do_merge) {
// The sequence number is updated synchronously in version_set.h
if (IsEmpty()) {
// Avoiding recording stats for speed.
return false;
}
PERF_TIMER_GUARD(get_from_memtable_time);
std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
NewRangeTombstoneIterator(read_opts,
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
GetInternalKeySeqno(key.internal_key()),
immutable_memtable));
if (range_del_iter != nullptr) {
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
SequenceNumber covering_seq =
range_del_iter->MaxCoveringTombstoneSeqnum(key.user_key());
if (covering_seq > *max_covering_tombstone_seq) {
*max_covering_tombstone_seq = covering_seq;
if (timestamp) {
// Will be overwritten in SaveValue() if there is a point key with
// a higher seqno.
timestamp->assign(range_del_iter->timestamp().data(),
range_del_iter->timestamp().size());
}
}
}
bool found_final_value = false;
bool merge_in_progress = s->IsMergeInProgress();
bool may_contain = true;
Add support for timestamp in Get/Put (#5079) Summary: It's useful to be able to (optionally) associate key-value pairs with user-provided timestamps. This PR is an early effort towards this goal and continues the work of facebook#4942. A suite of new unit tests exist in DBBasicTestWithTimestampWithParam. Support for timestamp requires the user to provide timestamp as a slice in `ReadOptions` and `WriteOptions`. All timestamps of the same database must share the same length, format, etc. The format of the timestamp is the same throughout the same database, and the user is responsible for providing a comparator function (Comparator) to order the <key, timestamp> tuples. Once created, the format and length of the timestamp cannot change (at least for now). Test plan (on devserver): ``` $COMPILE_WITH_ASAN=1 make -j32 all $./db_basic_test --gtest_filter=Timestamp/DBBasicTestWithTimestampWithParam.PutAndGet/* $make check ``` All tests must pass. We also run the following db_bench tests to verify whether there is regression on Get/Put while timestamp is not enabled. ``` $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillseq,readrandom -num=1000000 $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=1000000 ``` Repeat for 6 times for both versions. Results are as follows: ``` | | readrandom | fillrandom | | master | 16.77 MB/s | 47.05 MB/s | | PR5079 | 16.44 MB/s | 47.03 MB/s | ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/5079 Differential Revision: D15132946 Pulled By: riversand963 fbshipit-source-id: 833a0d657eac21182f0f206c910a6438154c742c
5 years ago
size_t ts_sz = GetInternalKeyComparator().user_comparator()->timestamp_size();
Slice user_key_without_ts = StripTimestampFromUserKey(key.user_key(), ts_sz);
bool bloom_checked = false;
if (bloom_filter_) {
// when both memtable_whole_key_filtering and prefix_extractor_ are set,
// only do whole key filtering for Get() to save CPU
if (moptions_.memtable_whole_key_filtering) {
may_contain = bloom_filter_->MayContain(user_key_without_ts);
bloom_checked = true;
} else {
assert(prefix_extractor_);
if (prefix_extractor_->InDomain(user_key_without_ts)) {
may_contain = bloom_filter_->MayContain(
prefix_extractor_->Transform(user_key_without_ts));
bloom_checked = true;
}
}
}
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
if (bloom_filter_ && !may_contain) {
// iter is null if prefix bloom says the key does not exist
PERF_COUNTER_ADD(bloom_memtable_miss_count, 1);
*seq = kMaxSequenceNumber;
} else {
if (bloom_checked) {
PERF_COUNTER_ADD(bloom_memtable_hit_count, 1);
}
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
GetFromTable(key, *max_covering_tombstone_seq, do_merge, callback,
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
is_blob_index, value, columns, timestamp, s, merge_context,
seq, &found_final_value, &merge_in_progress);
}
[RocksDB] [MergeOperator] The new Merge Interface! Uses merge sequences. Summary: Here are the major changes to the Merge Interface. It has been expanded to handle cases where the MergeOperator is not associative. It does so by stacking up merge operations while scanning through the key history (i.e.: during Get() or Compaction), until a valid Put/Delete/end-of-history is encountered; it then applies all of the merge operations in the correct sequence starting with the base/sentinel value. I have also introduced an "AssociativeMerge" function which allows the user to take advantage of associative merge operations (such as in the case of counters). The implementation will always attempt to merge the operations/operands themselves together when they are encountered, and will resort to the "stacking" method if and only if the "associative-merge" fails. This implementation is conjectured to allow MergeOperator to handle the general case, while still providing the user with the ability to take advantage of certain efficiencies in their own merge-operator / data-structure. NOTE: This is a preliminary diff. This must still go through a lot of review, revision, and testing. Feedback welcome! Test Plan: -This is a preliminary diff. I have only just begun testing/debugging it. -I will be testing this with the existing MergeOperator use-cases and unit-tests (counters, string-append, and redis-lists) -I will be "desk-checking" and walking through the code with the help gdb. -I will find a way of stress-testing the new interface / implementation using db_bench, db_test, merge_test, and/or db_stress. -I will ensure that my tests cover all cases: Get-Memtable, Get-Immutable-Memtable, Get-from-Disk, Iterator-Range-Scan, Flush-Memtable-to-L0, Compaction-L0-L1, Compaction-Ln-L(n+1), Put/Delete found, Put/Delete not-found, end-of-history, end-of-file, etc. -A lot of feedback from the reviewers. Reviewers: haobo, dhruba, zshao, emayanke Reviewed By: haobo CC: leveldb Differential Revision: https://reviews.facebook.net/D11499
11 years ago
// No change to value, since we have not yet found a Put/Delete
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
// Propagate corruption error
if (!found_final_value && merge_in_progress && !s->IsCorruption()) {
*s = Status::MergeInProgress();
}
PERF_COUNTER_ADD(get_from_memtable_count, 1);
return found_final_value;
}
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
void MemTable::GetFromTable(const LookupKey& key,
SequenceNumber max_covering_tombstone_seq,
bool do_merge, ReadCallback* callback,
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
4 years ago
bool* is_blob_index, std::string* value,
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
PinnableWideColumns* columns,
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
4 years ago
std::string* timestamp, Status* s,
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
MergeContext* merge_context, SequenceNumber* seq,
bool* found_final_value, bool* merge_in_progress) {
Saver saver;
saver.status = s;
saver.found_final_value = found_final_value;
saver.merge_in_progress = merge_in_progress;
saver.key = &key;
saver.value = value;
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
saver.columns = columns;
return timestamp from get (#6409) Summary: Added new Get() methods that return timestamp. Dummy implementation is given so that classes derived from DB don't need to be touched to provide their implementation. MultiGet is not included. ReadRandom perf test (10 minutes) on the same development machine ram drive with the same DB data shows no regression (within marge of error). The test is adapted from https://github.com/facebook/rocksdb/wiki/RocksDB-In-Memory-Workload-Performance-Benchmarks. base line (commit 72ee067b9): 101.712 micros/op 314602 ops/sec; 36.0 MB/s (5658999 of 5658999 found) This PR: 100.288 micros/op 319071 ops/sec; 36.5 MB/s (5674999 of 5674999 found) ./db_bench --db=r:\rocksdb.github --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --cache_size=2147483648 --cache_numshardbits=6 --compression_type=none --compression_ratio=1 --min_level_to_compress=-1 --disable_seek_compaction=1 --hard_rate_limit=2 --write_buffer_size=134217728 --max_write_buffer_number=2 --level0_file_num_compaction_trigger=8 --target_file_size_base=134217728 --max_bytes_for_level_base=1073741824 --disable_wal=0 --wal_dir=r:\rocksdb.github\WAL_LOG --sync=0 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_background_compactions=4 --max_background_flushes=0 --level0_slowdown_writes_trigger=16 --level0_stop_writes_trigger=24 --statistics=0 --stats_per_interval=0 --stats_interval=1048576 --histogram=0 --use_plain_table=1 --open_files=-1 --mmap_read=1 --mmap_write=0 --memtablerep=prefix_hash --bloom_bits=10 --bloom_locality=1 --duration=600 --benchmarks=readrandom --use_existing_db=1 --num=25000000 --threads=32 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6409 Differential Revision: D20200086 Pulled By: riversand963 fbshipit-source-id: 490edd74d924f62bd8ae9c29c2a6bbbb8410ca50
4 years ago
saver.timestamp = timestamp;
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
saver.seq = kMaxSequenceNumber;
saver.mem = this;
saver.merge_context = merge_context;
saver.max_covering_tombstone_seq = max_covering_tombstone_seq;
saver.merge_operator = moptions_.merge_operator;
saver.logger = moptions_.info_log;
saver.inplace_update_support = moptions_.inplace_update_support;
saver.statistics = moptions_.statistics;
saver.clock = clock_;
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
saver.callback_ = callback;
saver.is_blob_index = is_blob_index;
saver.do_merge = do_merge;
saver.allow_data_in_errors = moptions_.allow_data_in_errors;
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
saver.protection_bytes_per_key = moptions_.protection_bytes_per_key;
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
table_->Get(key, &saver, SaveValue);
*seq = saver.seq;
}
void MemTable::MultiGet(const ReadOptions& read_options, MultiGetRange* range,
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
ReadCallback* callback, bool immutable_memtable) {
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
// The sequence number is updated synchronously in version_set.h
if (IsEmpty()) {
// Avoiding recording stats for speed.
return;
}
PERF_TIMER_GUARD(get_from_memtable_time);
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
// For now, memtable Bloom filter is effectively disabled if there are any
// range tombstones. This is the simplest way to ensure range tombstones are
// handled. TODO: allow Bloom checks where max_covering_tombstone_seq==0
bool no_range_del = read_options.ignore_range_deletions ||
is_range_del_table_empty_.load(std::memory_order_relaxed);
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
MultiGetRange temp_range(*range, range->begin(), range->end());
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
if (bloom_filter_ && no_range_del) {
bool whole_key =
!prefix_extractor_ || moptions_.memtable_whole_key_filtering;
std::array<Slice, MultiGetContext::MAX_BATCH_SIZE> bloom_keys;
std::array<bool, MultiGetContext::MAX_BATCH_SIZE> may_match;
std::array<size_t, MultiGetContext::MAX_BATCH_SIZE> range_indexes;
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
int num_keys = 0;
for (auto iter = temp_range.begin(); iter != temp_range.end(); ++iter) {
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
if (whole_key) {
bloom_keys[num_keys] = iter->ukey_without_ts;
range_indexes[num_keys++] = iter.index();
} else if (prefix_extractor_->InDomain(iter->ukey_without_ts)) {
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
bloom_keys[num_keys] =
prefix_extractor_->Transform(iter->ukey_without_ts);
range_indexes[num_keys++] = iter.index();
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
}
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
}
bloom_filter_->MayContain(num_keys, &bloom_keys[0], &may_match[0]);
for (int i = 0; i < num_keys; ++i) {
if (!may_match[i]) {
temp_range.SkipIndex(range_indexes[i]);
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
PERF_COUNTER_ADD(bloom_memtable_miss_count, 1);
} else {
PERF_COUNTER_ADD(bloom_memtable_hit_count, 1);
}
}
}
for (auto iter = temp_range.begin(); iter != temp_range.end(); ++iter) {
bool found_final_value{false};
bool merge_in_progress = iter->s->IsMergeInProgress();
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
if (!no_range_del) {
std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
NewRangeTombstoneIteratorInternal(
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
read_options, GetInternalKeySeqno(iter->lkey->internal_key()),
immutable_memtable));
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
SequenceNumber covering_seq =
range_del_iter->MaxCoveringTombstoneSeqnum(iter->lkey->user_key());
if (covering_seq > iter->max_covering_tombstone_seq) {
iter->max_covering_tombstone_seq = covering_seq;
if (iter->timestamp) {
// Will be overwritten in SaveValue() if there is a point key with
// a higher seqno.
iter->timestamp->assign(range_del_iter->timestamp().data(),
range_del_iter->timestamp().size());
}
}
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
}
Fix major bug with MultiGet, DeleteRange, and memtable Bloom (#9453) Summary: MemTable::MultiGet was not considering range tombstones before querying Bloom filter. This means range tombstones would be skipped for keys (or prefixes) with no other entries in the memtable. This could cause old values for a key (in SST files) to still show up until the range tombstone covering it has been flushed. This is fixed by essentially disabling the memtable Bloom filter when there are any range tombstones. (This could be better optimized in the future, but good enough for now.) Did some other cleanup/optimization in the same code to (more than) offset the cost of checking on range tombstones in more cases. There is now notable improvement when memtable_whole_key_filtering and prefix_extractor are used together (unusual), and this makes MultiGet closer to the Get implementation. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9453 Test Plan: new unit test added. Added memtable Bloom to crash test. Performance testing -------------------- Build WAL-only DB (recovers to memtable): ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=1000000 -write_buffer_size=250000000 ``` Query test command, to maximize sensitivity to the changed code: ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=multireadrandom -num=10000000 -write_buffer_size=250000000 -memtable_bloom_size_ratio=0.015 -multiread_batched -batch_size=24 -threads=8 -memtable_whole_key_filtering=$MWKF -prefix_size=$PXS ``` (Note -num here is 10x larger for mostly memtable misses) Before & after run simultaneously, average over 10 iterations per data point, ops/sec. MWKF=0 PXS=0 (Bloom disabled) Before: 5724844 After: 6722066 MWKF=0 PXS=7 (prefixes hardly unique; Bloom not useful) Before: 9981319 After: 10237990 MWKF=0 PXS=8 (prefixes unique; Bloom useful) Before: 12081715 After: 12117603 MWKF=1 PXS=0 (whole key Bloom useful) Before: 11944354 After: 12096085 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes not useful in old version) Before: 9444299 After: 11826029 MWKF=1 PXS=7 (whole key Bloom useful in new version; prefixes useful in old version) Before: 11784465 After: 11778591 Only in this last case is the 'before' *slightly* faster, perhaps because hashing prefixes is slightly faster than hashing whole keys. Otherwise, 'after' is faster. Reviewed By: ajkr Differential Revision: D33805025 Pulled By: pdillinger fbshipit-source-id: 597523cae4f4eafdf6ae6bb2bc6cb46f83b017bf
2 years ago
SequenceNumber dummy_seq;
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
GetFromTable(*(iter->lkey), iter->max_covering_tombstone_seq, true,
callback, &iter->is_blob_index, iter->value->GetSelf(),
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
/*columns=*/nullptr, iter->timestamp, iter->s,
&(iter->merge_context), &dummy_seq, &found_final_value,
&merge_in_progress);
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
if (!found_final_value && merge_in_progress) {
*(iter->s) = Status::MergeInProgress();
}
if (found_final_value) {
iter->value->PinSelf();
range->AddValueSize(iter->value->size());
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
range->MarkKeyDone(iter);
RecordTick(moptions_.statistics, MEMTABLE_HIT);
if (range->GetValueSize() > read_options.value_size_soft_limit) {
// Set all remaining keys in range to Abort
for (auto range_iter = range->begin(); range_iter != range->end();
++range_iter) {
range->MarkKeyDone(range_iter);
*(range_iter->s) = Status::Aborted();
}
break;
}
MultiGet batching in memtable (#5818) Summary: RocksDB has a MultiGet() API that implements batched key lookup for higher performance (https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L468). Currently, batching is implemented in BlockBasedTableReader::MultiGet() for SST file lookups. One of the ways it improves performance is by pipelining bloom filter lookups (by prefetching required cachelines for all the keys in the batch, and then doing the probe) and thus hiding the cache miss latency. The same concept can be extended to the memtable as well. This PR involves implementing a pipelined bloom filter lookup in DynamicBloom, and implementing MemTable::MultiGet() that can leverage it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5818 Test Plan: Existing tests Performance Test: Ran the below command which fills up the memtable and makes sure there are no flushes and then call multiget. Ran it on master and on the new change and see atleast 1% performance improvement across all the test runs I did. Sometimes the improvement was upto 5%. TEST_TMPDIR=/data/users/$USER/benchmarks/feature/ numactl -C 10 ./db_bench -benchmarks="fillseq,multireadrandom" -num=600000 -compression_type="none" -level_compaction_dynamic_level_bytes -write_buffer_size=200000000 -target_file_size_base=200000000 -max_bytes_for_level_base=16777216 -reads=90000 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 -statistics -memtable_whole_key_filtering=true -memtable_bloom_size_ratio=10 Differential Revision: D17578869 Pulled By: vjnadimpalli fbshipit-source-id: 23dc651d9bf49db11d22375bf435708875a1f192
5 years ago
}
}
PERF_COUNTER_ADD(get_from_memtable_count, 1);
}
Status MemTable::Update(SequenceNumber seq, ValueType value_type,
const Slice& key, const Slice& value,
const ProtectionInfoKVOS64* kv_prot_info) {
LookupKey lkey(key, seq);
Slice mem_key = lkey.memtable_key();
std::unique_ptr<MemTableRep::Iterator> iter(
table_->GetDynamicPrefixIterator());
iter->Seek(lkey.internal_key(), mem_key.data());
if (iter->Valid()) {
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
// Refer to comments under MemTable::Add() for entry format.
// Check that it belongs to same user key. We do not check the
// sequence number since the Seek() call above should have skipped
// all entries with overly large sequence numbers.
const char* entry = iter->key();
uint32_t key_length = 0;
const char* key_ptr = GetVarint32Ptr(entry, entry + 5, &key_length);
if (comparator_.comparator.user_comparator()->Equal(
Slice(key_ptr, key_length - 8), lkey.user_key())) {
// Correct user key
const uint64_t tag = DecodeFixed64(key_ptr + key_length - 8);
ValueType type;
SequenceNumber existing_seq;
UnPackSequenceAndType(tag, &existing_seq, &type);
assert(existing_seq != seq);
if (type == value_type) {
Slice prev_value = GetLengthPrefixedSlice(key_ptr + key_length);
uint32_t prev_size = static_cast<uint32_t>(prev_value.size());
uint32_t new_size = static_cast<uint32_t>(value.size());
// Update value, if new value size <= previous value size
if (new_size <= prev_size) {
char* p =
EncodeVarint32(const_cast<char*>(key_ptr) + key_length, new_size);
WriteLock wl(GetLock(lkey.user_key()));
memcpy(p, value.data(), value.size());
assert((unsigned)((p + value.size()) - entry) ==
(unsigned)(VarintLength(key_length) + key_length +
VarintLength(value.size()) + value.size()));
RecordTick(moptions_.statistics, NUMBER_KEYS_UPDATED);
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
if (kv_prot_info != nullptr) {
ProtectionInfoKVOS64 updated_kv_prot_info(*kv_prot_info);
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
// `seq` is swallowed and `existing_seq` prevails.
updated_kv_prot_info.UpdateS(seq, existing_seq);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
UpdateEntryChecksum(&updated_kv_prot_info, key, value, type,
existing_seq, p + value.size());
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
Slice encoded(entry, p + value.size() - entry);
return VerifyEncodedEntry(encoded, updated_kv_prot_info);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
} else {
UpdateEntryChecksum(nullptr, key, value, type, existing_seq,
p + value.size());
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
}
return Status::OK();
}
}
}
}
// The latest value is not value_type or key doesn't exist
return Add(seq, value_type, key, value, kv_prot_info);
}
Status MemTable::UpdateCallback(SequenceNumber seq, const Slice& key,
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
3 years ago
const Slice& delta,
const ProtectionInfoKVOS64* kv_prot_info) {
LookupKey lkey(key, seq);
Slice memkey = lkey.memtable_key();
std::unique_ptr<MemTableRep::Iterator> iter(
table_->GetDynamicPrefixIterator());
iter->Seek(lkey.internal_key(), memkey.data());
if (iter->Valid()) {
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
// Refer to comments under MemTable::Add() for entry format.
// Check that it belongs to same user key. We do not check the
// sequence number since the Seek() call above should have skipped
// all entries with overly large sequence numbers.
const char* entry = iter->key();
uint32_t key_length = 0;
const char* key_ptr = GetVarint32Ptr(entry, entry + 5, &key_length);
if (comparator_.comparator.user_comparator()->Equal(
Slice(key_ptr, key_length - 8), lkey.user_key())) {
// Correct user key
const uint64_t tag = DecodeFixed64(key_ptr + key_length - 8);
ValueType type;
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
uint64_t existing_seq;
UnPackSequenceAndType(tag, &existing_seq, &type);
if (type == kTypeValue) {
Slice prev_value = GetLengthPrefixedSlice(key_ptr + key_length);
uint32_t prev_size = static_cast<uint32_t>(prev_value.size());
char* prev_buffer = const_cast<char*>(prev_value.data());
uint32_t new_prev_size = prev_size;
std::string str_value;
WriteLock wl(GetLock(lkey.user_key()));
auto status = moptions_.inplace_callback(prev_buffer, &new_prev_size,
delta, &str_value);
if (status == UpdateStatus::UPDATED_INPLACE) {
// Value already updated by callback.
assert(new_prev_size <= prev_size);
if (new_prev_size < prev_size) {
// overwrite the new prev_size
char* p = EncodeVarint32(const_cast<char*>(key_ptr) + key_length,
new_prev_size);
if (VarintLength(new_prev_size) < VarintLength(prev_size)) {
// shift the value buffer as well.
memcpy(p, prev_buffer, new_prev_size);
prev_buffer = p;
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
}
}
RecordTick(moptions_.statistics, NUMBER_KEYS_UPDATED);
UpdateFlushState();
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
Slice new_value(prev_buffer, new_prev_size);
if (kv_prot_info != nullptr) {
ProtectionInfoKVOS64 updated_kv_prot_info(*kv_prot_info);
// `seq` is swallowed and `existing_seq` prevails.
updated_kv_prot_info.UpdateS(seq, existing_seq);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
updated_kv_prot_info.UpdateV(delta, new_value);
Slice encoded(entry, prev_buffer + new_prev_size - entry);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
UpdateEntryChecksum(&updated_kv_prot_info, key, new_value, type,
existing_seq, prev_buffer + new_prev_size);
return VerifyEncodedEntry(encoded, updated_kv_prot_info);
Add memtable per key-value checksum (#10281) Summary: Append per key-value checksum to internal key. These checksums are verified on read paths including Get, Iterator and during Flush. Get and Iterator will return `Corruption` status if there is a checksum verification failure. Flush will make DB become read-only upon memtable entry checksum verification failure. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10281 Test Plan: - Added new unit test cases: `make check` - Benchmark on memtable insert ``` TEST_TMPDIR=/dev/shm/memtable_write ./db_bench -benchmarks=fillseq -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 # avg over 10 runs Baseline: 1166936 ops/sec memtable 2 bytes kv checksum : 1.11674e+06 ops/sec (-4%) memtable 2 bytes kv checksum + write batch 8 bytes kv checksum: 1.08579e+06 ops/sec (-6.95%) write batch 8 bytes kv checksum: 1.17979e+06 ops/sec (+1.1%) ``` - Benchmark on only memtable read: ops/sec dropped 31% for `readseq` due to time spend on verifying checksum. ops/sec for `readrandom` dropped ~6.8%. ``` # Readseq sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillseq,readseq"[-X20]" -disable_wal=true -max_write_buffer_number=100 -num=10000000 -min_write_buffer_number_to_merge=100 readseq [AVG 20 runs] : 7432840 (± 212005) ops/sec; 822.3 (± 23.5) MB/sec readseq [MEDIAN 20 runs] : 7573878 ops/sec; 837.9 MB/sec With -memtable_protection_bytes_per_key=2: readseq [AVG 20 runs] : 5134607 (± 119596) ops/sec; 568.0 (± 13.2) MB/sec readseq [MEDIAN 20 runs] : 5232946 ops/sec; 578.9 MB/sec # Readrandom sudo TEST_TMPDIR=/dev/shm/memtable_read ./db_bench -benchmarks=fillrandom,readrandom"[-X10]" -disable_wal=true -max_write_buffer_number=100 -num=1000000 -min_write_buffer_number_to_merge=100 readrandom [AVG 10 runs] : 140236 (± 3938) ops/sec; 9.8 (± 0.3) MB/sec readrandom [MEDIAN 10 runs] : 140545 ops/sec; 9.8 MB/sec With -memtable_protection_bytes_per_key=2: readrandom [AVG 10 runs] : 130632 (± 2738) ops/sec; 9.1 (± 0.2) MB/sec readrandom [MEDIAN 10 runs] : 130341 ops/sec; 9.1 MB/sec ``` - Stress test: `python3 -u tools/db_crashtest.py whitebox --duration=1800` Reviewed By: ajkr Differential Revision: D37607896 Pulled By: cbi42 fbshipit-source-id: fdaefb475629d2471780d4a5f5bf81b44ee56113
2 years ago
} else {
UpdateEntryChecksum(nullptr, key, new_value, type, existing_seq,
prev_buffer + new_prev_size);
}
return Status::OK();
} else if (status == UpdateStatus::UPDATED) {
Status s;
if (kv_prot_info != nullptr) {
ProtectionInfoKVOS64 updated_kv_prot_info(*kv_prot_info);
updated_kv_prot_info.UpdateV(delta, str_value);
s = Add(seq, kTypeValue, key, Slice(str_value),
&updated_kv_prot_info);
} else {
s = Add(seq, kTypeValue, key, Slice(str_value),
nullptr /* kv_prot_info */);
}
RecordTick(moptions_.statistics, NUMBER_KEYS_WRITTEN);
UpdateFlushState();
return s;
} else if (status == UpdateStatus::UPDATE_FAILED) {
// `UPDATE_FAILED` is named incorrectly. It indicates no update
// happened. It does not indicate a failure happened.
UpdateFlushState();
return Status::OK();
}
}
}
}
// The latest value is not `kTypeValue` or key doesn't exist
return Status::NotFound();
}
size_t MemTable::CountSuccessiveMergeEntries(const LookupKey& key) {
Slice memkey = key.memtable_key();
// A total ordered iterator is costly for some memtablerep (prefix aware
// reps). By passing in the user key, we allow efficient iterator creation.
// The iterator only needs to be ordered within the same user key.
std::unique_ptr<MemTableRep::Iterator> iter(
table_->GetDynamicPrefixIterator());
iter->Seek(key.internal_key(), memkey.data());
size_t num_successive_merges = 0;
for (; iter->Valid(); iter->Next()) {
const char* entry = iter->key();
uint32_t key_length = 0;
const char* iter_key_ptr = GetVarint32Ptr(entry, entry + 5, &key_length);
if (!comparator_.comparator.user_comparator()->Equal(
Slice(iter_key_ptr, key_length - 8), key.user_key())) {
break;
}
const uint64_t tag = DecodeFixed64(iter_key_ptr + key_length - 8);
ValueType type;
uint64_t unused;
UnPackSequenceAndType(tag, &unused, &type);
if (type != kTypeMerge) {
break;
}
++num_successive_merges;
}
return num_successive_merges;
}
void MemTableRep::Get(const LookupKey& k, void* callback_args,
bool (*callback_func)(void* arg, const char* entry)) {
auto iter = GetDynamicPrefixIterator();
for (iter->Seek(k.internal_key(), k.memtable_key().data());
iter->Valid() && callback_func(callback_args, iter->key());
iter->Next()) {
}
}
void MemTable::RefLogContainingPrepSection(uint64_t log) {
assert(log > 0);
auto cur = min_prep_log_referenced_.load();
while ((log < cur || cur == 0) &&
!min_prep_log_referenced_.compare_exchange_strong(cur, log)) {
cur = min_prep_log_referenced_.load();
}
}
uint64_t MemTable::GetMinLogContainingPrepSection() {
return min_prep_log_referenced_.load();
}
} // namespace ROCKSDB_NAMESPACE