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

868 lines
31 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#pragma once
#include <stdio.h>
Introduce a new MultiGet batching implementation (#5011) Summary: This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching. Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to - 1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch() 2. Bloom filter cachelines can be prefetched, hiding the cache miss latency The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress. Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32). Batch Sizes 1 | 2 | 4 | 8 | 16 | 32 Random pattern (Stride length 0) 4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get 4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching) 4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching) Good locality (Stride length 16) 4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753 4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781 4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135 Good locality (Stride length 256) 4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232 4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268 4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62 Medium locality (Stride length 4096) 4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555 4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465 4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891 dbbench command used (on a DB with 4 levels, 12 million keys)- TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -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 Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011 Differential Revision: D14348703 Pulled By: anand1976 fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
5 years ago
#include <memory>
#include <string>
#include <utility>
#include "rocksdb/comparator.h"
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "rocksdb/types.h"
#include "util/coding.h"
#include "util/user_comparator_wrapper.h"
namespace ROCKSDB_NAMESPACE {
// The file declares data structures and functions that deal with internal
// keys.
// Each internal key contains a user key, a sequence number (SequenceNumber)
// and a type (ValueType), and they are usually encoded together.
// There are some related helper classes here.
class InternalKey;
// Value types encoded as the last component of internal keys.
// DO NOT CHANGE THESE ENUM VALUES: they are embedded in the on-disk
// data structures.
// The highest bit of the value type needs to be reserved to SST tables
// for them to do more flexible encoding.
enum ValueType : unsigned char {
kTypeDeletion = 0x0,
kTypeValue = 0x1,
kTypeMerge = 0x2,
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
kTypeLogData = 0x3, // WAL only.
kTypeColumnFamilyDeletion = 0x4, // WAL only.
kTypeColumnFamilyValue = 0x5, // WAL only.
kTypeColumnFamilyMerge = 0x6, // WAL only.
kTypeSingleDeletion = 0x7,
kTypeColumnFamilySingleDeletion = 0x8, // WAL only.
kTypeBeginPrepareXID = 0x9, // WAL only.
kTypeEndPrepareXID = 0xA, // WAL only.
kTypeCommitXID = 0xB, // WAL only.
kTypeRollbackXID = 0xC, // WAL only.
kTypeNoop = 0xD, // WAL only.
kTypeColumnFamilyRangeDeletion = 0xE, // WAL only.
kTypeRangeDeletion = 0xF, // meta block
kTypeColumnFamilyBlobIndex = 0x10, // Blob DB only
kTypeBlobIndex = 0x11, // Blob DB only
// When the prepared record is also persisted in db, we use a different
// record. This is to ensure that the WAL that is generated by a WritePolicy
// is not mistakenly read by another, which would result into data
// inconsistency.
kTypeBeginPersistedPrepareXID = 0x12, // WAL only.
// Similar to kTypeBeginPersistedPrepareXID, this is to ensure that WAL
// generated by WriteUnprepared write policy is not mistakenly read by
// another.
kTypeBeginUnprepareXID = 0x13, // WAL only.
kTypeDeletionWithTimestamp = 0x14,
kTypeCommitXIDAndTimestamp = 0x15, // WAL only
kTypeWideColumnEntity = 0x16,
kTypeColumnFamilyWideColumnEntity = 0x17, // WAL only
kTypeMaxValid, // Should be after the last valid type, only used for
// validation
kMaxValue = 0x7F // Not used for storing records.
};
// Defined in dbformat.cc
extern const ValueType kValueTypeForSeek;
extern const ValueType kValueTypeForSeekForPrev;
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
// Checks whether a type is an inline value type
// (i.e. a type used in memtable skiplist and sst file datablock).
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
inline bool IsValueType(ValueType t) {
return t <= kTypeMerge || kTypeSingleDeletion == t || kTypeBlobIndex == t ||
kTypeDeletionWithTimestamp == t || kTypeWideColumnEntity == t;
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
}
// Checks whether a type is from user operation
// kTypeRangeDeletion is in meta block so this API is separated from above
inline bool IsExtendedValueType(ValueType t) {
return IsValueType(t) || t == kTypeRangeDeletion;
}
// We leave eight bits empty at the bottom so a type and sequence#
// can be packed together into 64-bits.
static const SequenceNumber kMaxSequenceNumber = ((0x1ull << 56) - 1);
static const SequenceNumber kDisableGlobalSequenceNumber =
std::numeric_limits<uint64_t>::max();
constexpr uint64_t kNumInternalBytes = 8;
// Defined in dbformat.cc
extern const std::string kDisableUserTimestamp;
// The data structure that represents an internal key in the way that user_key,
// sequence number and type are stored in separated forms.
struct ParsedInternalKey {
Slice user_key;
SequenceNumber sequence;
ValueType type;
ParsedInternalKey()
: sequence(kMaxSequenceNumber),
type(kTypeDeletion) // Make code analyzer happy
{} // Intentionally left uninitialized (for speed)
Iterator with timestamp (#6255) Summary: Preliminary support for iterator with user timestamp. Current implementation does not consider merge operator and reverse iterator. Auto compaction is also disabled in unit tests. Create an iterator with timestamp. ``` ... read_opts.timestamp = &ts; auto* iter = db->NewIterator(read_opts); // target is key without timestamp. for (iter->Seek(target); iter->Valid(); iter->Next()) {} for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {} delete iter; read_opts.timestamp = &ts1; // lower_bound and upper_bound are without timestamp. read_opts.iterate_lower_bound = &lower_bound; read_opts.iterate_upper_bound = &upper_bound; auto* iter1 = db->NewIterator(read_opts); // Do Seek or SeekToFirst() delete iter1; ``` Test plan (dev server) ``` $make check ``` Simple benchmarking (dev server) 1. The overhead introduced by this PR even when timestamp is disabled. key size: 16 bytes value size: 100 bytes Entries: 1000000 Data reside in main memory, and try to stress iterator. Repeated three times on master and this PR. - Seek without next ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 ``` master: 159047.0 ops/sec this PR: 158922.3 ops/sec (2% drop in throughput) - Seek and next 10 times ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 -seek_nexts=10 ``` master: 109539.3 ops/sec this PR: 107519.7 ops/sec (2% drop in throughput) Pull Request resolved: https://github.com/facebook/rocksdb/pull/6255 Differential Revision: D19438227 Pulled By: riversand963 fbshipit-source-id: b66b4979486f8474619f4aa6bdd88598870b0746
4 years ago
// u contains timestamp if user timestamp feature is enabled.
ParsedInternalKey(const Slice& u, const SequenceNumber& seq, ValueType t)
: user_key(u), sequence(seq), type(t) {}
std::string DebugString(bool log_err_key, bool hex) const;
void clear() {
user_key.clear();
sequence = 0;
type = kTypeDeletion;
}
Allow compaction iterator to perform garbage collection (#7556) Summary: Add a threshold timestamp, full_history_ts_low_ of type `std::string*` to `CompactionIterator`, so that RocksDB can also perform garbage collection during compaction. * If full_history_ts_low_ is nullptr, then compaction iterator does not perform GC, preserving all timestamp history for all keys. Compaction iterator will treat user key with different timestamps as different user keys. * If full_history_ts_low_ is not nullptr, then compaction iterator performs GC. GC will look at keys older than `*full_history_ts_low_` and determine their eligibility based on factors including snapshots. Current rules of GC: * If an internal key is in the same snapshot as a previous counterpart with the same user key, and this key is eligible for GC, and the key is not single-delete or merge operand, then this key can be dropped. Note that the previous internal key cannot be a merge operand either. * If a tombstone is the most recent one in the earliest snapshot and it is eligible for GC, and keyNotExistsBeyondLevel() is true, then this tombstone can be dropped. * If a tombstone is the most recent one in a snapshot and it is eligible for GC, and the compaction is at bottommost level, then all other older internal keys of the same user key must also be eligible for GC, thus can be dropped * Single-delete, delete-range and merge are not currently supported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7556 Test Plan: make check Reviewed By: ltamasi Differential Revision: D24507728 Pulled By: riversand963 fbshipit-source-id: 3c09c7301f41eed76dfcf4d1527e68cf6e0a8bb3
4 years ago
void SetTimestamp(const Slice& ts) {
assert(ts.size() <= user_key.size());
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
const char* addr = user_key.data() + user_key.size() - ts.size();
Allow compaction iterator to perform garbage collection (#7556) Summary: Add a threshold timestamp, full_history_ts_low_ of type `std::string*` to `CompactionIterator`, so that RocksDB can also perform garbage collection during compaction. * If full_history_ts_low_ is nullptr, then compaction iterator does not perform GC, preserving all timestamp history for all keys. Compaction iterator will treat user key with different timestamps as different user keys. * If full_history_ts_low_ is not nullptr, then compaction iterator performs GC. GC will look at keys older than `*full_history_ts_low_` and determine their eligibility based on factors including snapshots. Current rules of GC: * If an internal key is in the same snapshot as a previous counterpart with the same user key, and this key is eligible for GC, and the key is not single-delete or merge operand, then this key can be dropped. Note that the previous internal key cannot be a merge operand either. * If a tombstone is the most recent one in the earliest snapshot and it is eligible for GC, and keyNotExistsBeyondLevel() is true, then this tombstone can be dropped. * If a tombstone is the most recent one in a snapshot and it is eligible for GC, and the compaction is at bottommost level, then all other older internal keys of the same user key must also be eligible for GC, thus can be dropped * Single-delete, delete-range and merge are not currently supported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7556 Test Plan: make check Reviewed By: ltamasi Differential Revision: D24507728 Pulled By: riversand963 fbshipit-source-id: 3c09c7301f41eed76dfcf4d1527e68cf6e0a8bb3
4 years ago
memcpy(const_cast<char*>(addr), ts.data(), ts.size());
}
Slice GetTimestamp(size_t ts_sz) {
assert(ts_sz <= user_key.size());
const char* addr = user_key.data() + user_key.size() - ts_sz;
return Slice(const_cast<char*>(addr), ts_sz);
}
};
// Return the length of the encoding of "key".
inline size_t InternalKeyEncodingLength(const ParsedInternalKey& key) {
return key.user_key.size() + kNumInternalBytes;
}
// Pack a sequence number and a ValueType into a uint64_t
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
inline uint64_t PackSequenceAndType(uint64_t seq, ValueType t) {
assert(seq <= kMaxSequenceNumber);
Skip swaths of range tombstone covered keys in merging iterator (2022 edition) (#10449) Summary: Delete range logic is moved from `DBIter` to `MergingIterator`, and `MergingIterator` will seek to the end of a range deletion if possible instead of scanning through each key and check with `RangeDelAggregator`. With the invariant that a key in level L (consider memtable as the first level, each immutable and L0 as a separate level) has a larger sequence number than all keys in any level >L, a range tombstone `[start, end)` from level L covers all keys in its range in any level >L. This property motivates optimizations in iterator: - in `Seek(target)`, if level L has a range tombstone `[start, end)` that covers `target.UserKey`, then for all levels > L, we can do Seek() on `end` instead of `target` to skip some range tombstone covered keys. - in `Next()/Prev()`, if the current key is covered by a range tombstone `[start, end)` from level L, we can do `Seek` to `end` for all levels > L. This PR implements the above optimizations in `MergingIterator`. As all range tombstone covered keys are now skipped in `MergingIterator`, the range tombstone logic is removed from `DBIter`. The idea in this PR is similar to https://github.com/facebook/rocksdb/issues/7317, but this PR leaves `InternalIterator` interface mostly unchanged. **Credit**: the cascading seek optimization and the sentinel key (discussed below) are inspired by [Pebble](https://github.com/cockroachdb/pebble/blob/master/merging_iter.go) and suggested by ajkr in https://github.com/facebook/rocksdb/issues/7317. The two optimizations are mostly implemented in `SeekImpl()/SeekForPrevImpl()` and `IsNextDeleted()/IsPrevDeleted()` in `merging_iterator.cc`. See comments for each method for more detail. One notable change is that the minHeap/maxHeap used by `MergingIterator` now contains range tombstone end keys besides point key iterators. This helps to reduce the number of key comparisons. For example, for a range tombstone `[start, end)`, a `start` and an `end` `HeapItem` are inserted into the heap. When a `HeapItem` for range tombstone start key is popped from the minHeap, we know this range tombstone becomes "active" in the sense that, before the range tombstone's end key is popped from the minHeap, all the keys popped from this heap is covered by the range tombstone's internal key range `[start, end)`. Another major change, *delete range sentinel key*, is made to `LevelIterator`. Before this PR, when all point keys in an SST file are iterated through in `MergingIterator`, a level iterator would advance to the next SST file in its level. In the case when an SST file has a range tombstone that covers keys beyond the SST file's last point key, advancing to the next SST file would lose this range tombstone. Consequently, `MergingIterator` could return keys that should have been deleted by some range tombstone. We prevent this by pretending that file boundaries in each SST file are sentinel keys. A `LevelIterator` now only advance the file iterator once the sentinel key is processed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10449 Test Plan: - Added many unit tests in db_range_del_test - Stress test: `./db_stress --readpercent=5 --prefixpercent=19 --writepercent=20 -delpercent=10 --iterpercent=44 --delrangepercent=2` - Additional iterator stress test is added to verify against iterators against expected state: https://github.com/facebook/rocksdb/issues/10538. This is based on ajkr's previous attempt https://github.com/facebook/rocksdb/pull/5506#issuecomment-506021913. ``` python3 ./tools/db_crashtest.py blackbox --simple --write_buffer_size=524288 --target_file_size_base=524288 --max_bytes_for_level_base=2097152 --compression_type=none --max_background_compactions=8 --value_size_mult=33 --max_key=5000000 --interval=10 --duration=7200 --delrangepercent=3 --delpercent=9 --iterpercent=25 --writepercent=60 --readpercent=3 --prefixpercent=0 --num_iterations=1000 --range_deletion_width=100 --verify_iterator_with_expected_state_one_in=1 ``` - Performance benchmark: I used a similar setup as in the blog [post](http://rocksdb.org/blog/2018/11/21/delete-range.html) that introduced DeleteRange, "a database with 5 million data keys, and 10000 range tombstones (ignoring those dropped during compaction) that were written in regular intervals after 4.5 million data keys were written". As expected, the performance with this PR depends on the range tombstone width. ``` # Setup: TEST_TMPDIR=/dev/shm ./db_bench_main --benchmarks=fillrandom --writes=4500000 --num=5000000 TEST_TMPDIR=/dev/shm ./db_bench_main --benchmarks=overwrite --writes=500000 --num=5000000 --use_existing_db=true --writes_per_range_tombstone=50 # Scan entire DB TEST_TMPDIR=/dev/shm ./db_bench_main --benchmarks=readseq[-X5] --use_existing_db=true --num=5000000 --disable_auto_compactions=true # Short range scan (10 Next()) TEST_TMPDIR=/dev/shm/width-100/ ./db_bench_main --benchmarks=seekrandom[-X5] --use_existing_db=true --num=500000 --reads=100000 --seek_nexts=10 --disable_auto_compactions=true # Long range scan(1000 Next()) TEST_TMPDIR=/dev/shm/width-100/ ./db_bench_main --benchmarks=seekrandom[-X5] --use_existing_db=true --num=500000 --reads=2500 --seek_nexts=1000 --disable_auto_compactions=true ``` Avg over of 10 runs (some slower tests had fews runs): For the first column (tombstone), 0 means no range tombstone, 100-10000 means width of the 10k range tombstones, and 1 means there is a single range tombstone in the entire DB (width is 1000). The 1 tombstone case is to test regression when there's very few range tombstones in the DB, as no range tombstone is likely to take a different code path than with range tombstones. - Scan entire DB | tombstone width | Pre-PR ops/sec | Post-PR ops/sec | ±% | | ------------- | ------------- | ------------- | ------------- | | 0 range tombstone |2525600 (± 43564) |2486917 (± 33698) |-1.53% | | 100 |1853835 (± 24736) |2073884 (± 32176) |+11.87% | | 1000 |422415 (± 7466) |1115801 (± 22781) |+164.15% | | 10000 |22384 (± 227) |227919 (± 6647) |+918.22% | | 1 range tombstone |2176540 (± 39050) |2434954 (± 24563) |+11.87% | - Short range scan | tombstone width | Pre-PR ops/sec | Post-PR ops/sec | ±% | | ------------- | ------------- | ------------- | ------------- | | 0 range tombstone |35398 (± 533) |35338 (± 569) |-0.17% | | 100 |28276 (± 664) |31684 (± 331) |+12.05% | | 1000 |7637 (± 77) |25422 (± 277) |+232.88% | | 10000 |1367 |28667 |+1997.07% | | 1 range tombstone |32618 (± 581) |32748 (± 506) |+0.4% | - Long range scan | tombstone width | Pre-PR ops/sec | Post-PR ops/sec | ±% | | ------------- | ------------- | ------------- | ------------- | | 0 range tombstone |2262 (± 33) |2353 (± 20) |+4.02% | | 100 |1696 (± 26) |1926 (± 18) |+13.56% | | 1000 |410 (± 6) |1255 (± 29) |+206.1% | | 10000 |25 |414 |+1556.0% | | 1 range tombstone |1957 (± 30) |2185 (± 44) |+11.65% | - Microbench does not show significant regression: https://gist.github.com/cbi42/59f280f85a59b678e7e5d8561e693b61 Reviewed By: ajkr Differential Revision: D38450331 Pulled By: cbi42 fbshipit-source-id: b5ef12e8d8c289ed2e163ccdf277f5039b511fca
2 years ago
// kTypeMaxValid is used in TruncatedRangeDelIterator, see its constructor.
assert(IsExtendedValueType(t) || t == kTypeMaxValid);
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
return (seq << 8) | t;
}
// Given the result of PackSequenceAndType, store the sequence number in *seq
// and the ValueType in *t.
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
inline void UnPackSequenceAndType(uint64_t packed, uint64_t* seq,
ValueType* t) {
*seq = packed >> 8;
*t = static_cast<ValueType>(packed & 0xff);
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
// Commented the following two assertions in order to test key-value checksum
// on corrupted keys without crashing ("DbKvChecksumTest").
// assert(*seq <= kMaxSequenceNumber);
// assert(IsExtendedValueType(*t));
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
}
EntryType GetEntryType(ValueType value_type);
// Append the serialization of "key" to *result.
extern void AppendInternalKey(std::string* result,
const ParsedInternalKey& key);
Iterator with timestamp (#6255) Summary: Preliminary support for iterator with user timestamp. Current implementation does not consider merge operator and reverse iterator. Auto compaction is also disabled in unit tests. Create an iterator with timestamp. ``` ... read_opts.timestamp = &ts; auto* iter = db->NewIterator(read_opts); // target is key without timestamp. for (iter->Seek(target); iter->Valid(); iter->Next()) {} for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {} delete iter; read_opts.timestamp = &ts1; // lower_bound and upper_bound are without timestamp. read_opts.iterate_lower_bound = &lower_bound; read_opts.iterate_upper_bound = &upper_bound; auto* iter1 = db->NewIterator(read_opts); // Do Seek or SeekToFirst() delete iter1; ``` Test plan (dev server) ``` $make check ``` Simple benchmarking (dev server) 1. The overhead introduced by this PR even when timestamp is disabled. key size: 16 bytes value size: 100 bytes Entries: 1000000 Data reside in main memory, and try to stress iterator. Repeated three times on master and this PR. - Seek without next ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 ``` master: 159047.0 ops/sec this PR: 158922.3 ops/sec (2% drop in throughput) - Seek and next 10 times ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 -seek_nexts=10 ``` master: 109539.3 ops/sec this PR: 107519.7 ops/sec (2% drop in throughput) Pull Request resolved: https://github.com/facebook/rocksdb/pull/6255 Differential Revision: D19438227 Pulled By: riversand963 fbshipit-source-id: b66b4979486f8474619f4aa6bdd88598870b0746
4 years ago
// Append the serialization of "key" to *result, replacing the original
// timestamp with argument ts.
extern void AppendInternalKeyWithDifferentTimestamp(
std::string* result, const ParsedInternalKey& key, const Slice& ts);
// Serialized internal key consists of user key followed by footer.
// This function appends the footer to *result, assuming that *result already
// contains the user key at the end.
extern void AppendInternalKeyFooter(std::string* result, SequenceNumber s,
ValueType t);
// Append the key and a minimal timestamp to *result
extern void AppendKeyWithMinTimestamp(std::string* result, const Slice& key,
size_t ts_sz);
// Append the key and a maximal timestamp to *result
extern void AppendKeyWithMaxTimestamp(std::string* result, const Slice& key,
size_t ts_sz);
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
// `key` is a user key with timestamp. Append the user key without timestamp
// and the maximal timestamp to *result.
extern void AppendUserKeyWithMaxTimestamp(std::string* result, const Slice& key,
size_t ts_sz);
// Attempt to parse an internal key from "internal_key". On success,
// stores the parsed data in "*result", and returns true.
//
// On error, returns false, leaves "*result" in an undefined state.
extern Status ParseInternalKey(const Slice& internal_key,
ParsedInternalKey* result, bool log_err_key);
// Returns the user key portion of an internal key.
inline Slice ExtractUserKey(const Slice& internal_key) {
assert(internal_key.size() >= kNumInternalBytes);
return Slice(internal_key.data(), internal_key.size() - kNumInternalBytes);
}
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
inline Slice ExtractUserKeyAndStripTimestamp(const Slice& internal_key,
size_t ts_sz) {
Slice ret = internal_key;
ret.remove_suffix(kNumInternalBytes + ts_sz);
return ret;
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
}
inline Slice StripTimestampFromUserKey(const Slice& user_key, size_t ts_sz) {
Slice ret = user_key;
ret.remove_suffix(ts_sz);
return ret;
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
}
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
inline Slice ExtractTimestampFromUserKey(const Slice& user_key, size_t ts_sz) {
assert(user_key.size() >= ts_sz);
return Slice(user_key.data() + user_key.size() - ts_sz, ts_sz);
}
inline Slice ExtractTimestampFromKey(const Slice& internal_key, size_t ts_sz) {
const size_t key_size = internal_key.size();
assert(key_size >= kNumInternalBytes + ts_sz);
return Slice(internal_key.data() + key_size - ts_sz - kNumInternalBytes,
ts_sz);
}
inline uint64_t ExtractInternalKeyFooter(const Slice& internal_key) {
assert(internal_key.size() >= kNumInternalBytes);
const size_t n = internal_key.size();
return DecodeFixed64(internal_key.data() + n - kNumInternalBytes);
}
inline ValueType ExtractValueType(const Slice& internal_key) {
uint64_t num = ExtractInternalKeyFooter(internal_key);
unsigned char c = num & 0xff;
return static_cast<ValueType>(c);
}
// A comparator for internal keys that uses a specified comparator for
// the user key portion and breaks ties by decreasing sequence number.
class InternalKeyComparator
#ifdef NDEBUG
final
#endif
: public CompareInterface {
private:
UserComparatorWrapper user_comparator_;
public:
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
// `InternalKeyComparator`s constructed with the default constructor are not
// usable and will segfault on any attempt to use them for comparisons.
InternalKeyComparator() = default;
// @param named If true, assign a name to this comparator based on the
// underlying comparator's name. This involves an allocation and copy in
// this constructor to precompute the result of `Name()`. To avoid this
// overhead, set `named` to false. In that case, `Name()` will return a
// generic name that is non-specific to the underlying comparator.
explicit InternalKeyComparator(const Comparator* c) : user_comparator_(c) {}
virtual ~InternalKeyComparator() {}
int Compare(const Slice& a, const Slice& b) const override;
bool Equal(const Slice& a, const Slice& b) const {
// TODO Use user_comparator_.Equal(). Perhaps compare seqno before
// comparing the user key too.
return Compare(a, b) == 0;
}
// Same as Compare except that it excludes the value type from comparison
int CompareKeySeq(const Slice& a, const Slice& b) const;
const Comparator* user_comparator() const {
return user_comparator_.user_comparator();
}
int Compare(const InternalKey& a, const InternalKey& b) const;
int Compare(const ParsedInternalKey& a, const ParsedInternalKey& b) const;
Remove copying of range tombstones keys in iterator (#10878) Summary: In MergingIterator, if a range tombstone's start or end key is added to minHeap/maxHeap, the key is copied. This PR removes the copying of range tombstone keys by adding InternalKey comparator that compares `Slice` for internal key and `ParsedInternalKey` directly. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10878 Test Plan: - existing UT - ran all flavors of stress test through sandcastle - benchmarks: I did not get improvement when compiling with DEBUG_LEVEL=0, and saw many noise. With `OPTIMIZE_LEVEL="-O3" USE_LTO=1` I do see improvement. ``` # Favorable set up: half of the writes are DeleteRange. TEST_TMPDIR=/tmp/rocksdb-rangedel-test-all-tombstone ./db_bench --benchmarks=fillseq,levelstats --writes_per_range_tombstone=1 --max_num_range_tombstones=1000000 --range_tombstone_width=2 --num=1000000 --max_bytes_for_level_base=4194304 --disable_auto_compactions --write_buffer_size=33554432 --key_size=50 # benchmark command TEST_TMPDIR=/tmp/rocksdb-rangedel-test-all-tombstone ./db_bench --benchmarks=readseq[-W1][-X5],levelstats --use_existing_db=true --cache_size=3221225472 --disable_auto_compactions=true --avoid_flush_during_recovery=true --seek_nexts=100 --reads=1000000 --num=1000000 --threads=25 # main readseq [AVG 5 runs] : 26017977 (± 371077) ops/sec; 3721.9 (± 53.1) MB/sec readseq [MEDIAN 5 runs] : 26096905 ops/sec; 3733.2 MB/sec # this PR readseq [AVG 5 runs] : 27481724 (± 568758) ops/sec; 3931.3 (± 81.4) MB/sec readseq [MEDIAN 5 runs] : 27323957 ops/sec; 3908.7 MB/sec ``` Reviewed By: ajkr Differential Revision: D40711170 Pulled By: cbi42 fbshipit-source-id: 708cb584e2bd085a9ce0d2ef6a420489f721717f
2 years ago
int Compare(const Slice& a, const ParsedInternalKey& b) const;
int Compare(const ParsedInternalKey& a, const Slice& b) const;
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
// In this `Compare()` overload, the sequence numbers provided in
// `a_global_seqno` and `b_global_seqno` override the sequence numbers in `a`
// and `b`, respectively. To disable sequence number override(s), provide the
// value `kDisableGlobalSequenceNumber`.
int Compare(const Slice& a, SequenceNumber a_global_seqno, const Slice& b,
SequenceNumber b_global_seqno) const;
};
// The class represent the internal key in encoded form.
class InternalKey {
private:
std::string rep_;
public:
InternalKey() {} // Leave rep_ as empty to indicate it is invalid
InternalKey(const Slice& _user_key, SequenceNumber s, ValueType t) {
AppendInternalKey(&rep_, ParsedInternalKey(_user_key, s, t));
}
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
InternalKey(const Slice& _user_key, SequenceNumber s, ValueType t, Slice ts) {
AppendInternalKeyWithDifferentTimestamp(
&rep_, ParsedInternalKey(_user_key, s, t), ts);
}
// sets the internal key to be bigger or equal to all internal keys with this
// user key
void SetMaxPossibleForUserKey(const Slice& _user_key) {
AppendInternalKey(
&rep_, ParsedInternalKey(_user_key, 0, static_cast<ValueType>(0)));
}
// sets the internal key to be smaller or equal to all internal keys with this
// user key
void SetMinPossibleForUserKey(const Slice& _user_key) {
AppendInternalKey(&rep_, ParsedInternalKey(_user_key, kMaxSequenceNumber,
kValueTypeForSeek));
}
[fix] SIGSEGV when VersionEdit in MANIFEST is corrupted Summary: This was reported by our customers in task #4295529. Cause: * MANIFEST file contains a VersionEdit, which contains file entries whose 'smallest' and 'largest' internal keys are empty. String with zero characters. Root cause of corruption was not investigated. We should report corruption when this happens. However, we currently SIGSEGV. Here's what happens: * VersionEdit encodes zero-strings happily and stores them in smallest and largest InternalKeys. InternalKey::Encode() does assert when `rep_.empty()`, but we don't assert in production environemnts. Also, we should never assert as a result of DB corruption. * As part of our ConsistencyCheck, we call GetLiveFilesMetaData() * GetLiveFilesMetadata() calls `file->largest.user_key().ToString()` * user_key() function does: 1. assert(size > 8) (ooops, no assert), 2. returns `Slice(internal_key.data(), internal_key.size() - 8)` * since `internal_key.size()` is unsigned int, this call translates to `Slice(whatever, 1298471928561892576182756)`. Bazinga. Fix: * VersionEdit checks if InternalKey is valid in `VersionEdit::GetInternalKey()`. If it's invalid, returns corruption. Lessons learned: * Always keep in mind that even if you `assert()`, production code will continue execution even if assert fails. * Never `assert` based on DB corruption. Assert only if the code should guarantee that assert can't fail. Test Plan: dumped offending manifest. Before: assert. Now: corruption Reviewers: dhruba, haobo, sdong Reviewed By: dhruba CC: leveldb Differential Revision: https://reviews.facebook.net/D18507
10 years ago
bool Valid() const {
ParsedInternalKey parsed;
return (ParseInternalKey(Slice(rep_), &parsed, false /* log_err_key */)
.ok()); // TODO
[fix] SIGSEGV when VersionEdit in MANIFEST is corrupted Summary: This was reported by our customers in task #4295529. Cause: * MANIFEST file contains a VersionEdit, which contains file entries whose 'smallest' and 'largest' internal keys are empty. String with zero characters. Root cause of corruption was not investigated. We should report corruption when this happens. However, we currently SIGSEGV. Here's what happens: * VersionEdit encodes zero-strings happily and stores them in smallest and largest InternalKeys. InternalKey::Encode() does assert when `rep_.empty()`, but we don't assert in production environemnts. Also, we should never assert as a result of DB corruption. * As part of our ConsistencyCheck, we call GetLiveFilesMetaData() * GetLiveFilesMetadata() calls `file->largest.user_key().ToString()` * user_key() function does: 1. assert(size > 8) (ooops, no assert), 2. returns `Slice(internal_key.data(), internal_key.size() - 8)` * since `internal_key.size()` is unsigned int, this call translates to `Slice(whatever, 1298471928561892576182756)`. Bazinga. Fix: * VersionEdit checks if InternalKey is valid in `VersionEdit::GetInternalKey()`. If it's invalid, returns corruption. Lessons learned: * Always keep in mind that even if you `assert()`, production code will continue execution even if assert fails. * Never `assert` based on DB corruption. Assert only if the code should guarantee that assert can't fail. Test Plan: dumped offending manifest. Before: assert. Now: corruption Reviewers: dhruba, haobo, sdong Reviewed By: dhruba CC: leveldb Differential Revision: https://reviews.facebook.net/D18507
10 years ago
}
void DecodeFrom(const Slice& s) { rep_.assign(s.data(), s.size()); }
Slice Encode() const {
assert(!rep_.empty());
return rep_;
}
Slice user_key() const { return ExtractUserKey(rep_); }
Account memory of FileMetaData in global memory limit (#9924) Summary: **Context/Summary:** As revealed by heap profiling, allocation of `FileMetaData` for [newly created file added to a Version](https://github.com/facebook/rocksdb/pull/9924/files#diff-a6aa385940793f95a2c5b39cc670bd440c4547fa54fd44622f756382d5e47e43R774) can consume significant heap memory. This PR is to account that toward our global memory limit based on block cache capacity. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9924 Test Plan: - Previous `make check` verified there are only 2 places where the memory of the allocated `FileMetaData` can be released - New unit test `TEST_P(ChargeFileMetadataTestWithParam, Basic)` - db bench (CPU cost of `charge_file_metadata` in write and compact) - **write micros/op: -0.24%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 (remove this option for pre-PR) -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` - **compact micros/op -0.87%** : `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_file_metadata=1 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 -numdistinct=1000 && ./db_bench -benchmarks=compact -db=$TEST_TMPDIR -use_existing_db=1 -charge_file_metadata=1 -disable_auto_compactions=1 | egrep 'compact'` table 1 - write #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | -0.3633711465 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | 0.5289363078 80 | 3.87828 | 0.119007 | 3.86791 | 0.115674 | **-0.2673865734** 160 | 3.87677 | 0.162231 | 3.86739 | 0.16663 | **-0.2419539978** table 2 - compact #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 2,399,650.00 | 96,375.80 | 2,359,537.00 | 53,243.60 | -1.67 20 | 2,410,480.00 | 89,988.00 | 2,433,580.00 | 91,121.20 | 0.96 40 | 2.41E+06 | 121811 | 2.39E+06 | 131525 | **-0.96** 80 | 2.40E+06 | 134503 | 2.39E+06 | 108799 | **-0.78** - stress test: `python3 tools/db_crashtest.py blackbox --charge_file_metadata=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36055583 Pulled By: hx235 fbshipit-source-id: b60eab94707103cb1322cf815f05810ef0232625
2 years ago
size_t size() const { return rep_.size(); }
Miscellaneous performance improvements Summary: I was investigating performance issues in the SstFileWriter and found all of the following: - The SstFileWriter::Add() function created a local InternalKey every time it was called generating a allocation and free each time. Changed to have an InternalKey member variable that can be reset with the new InternalKey::Set() function. - In SstFileWriter::Add() the smallest_key and largest_key values were assigned the result of a ToString() call, but it is simpler to just assign them directly from the user's key. - The Slice class had no move constructor so each time one was returned from a function a new one had to be allocated, the old data copied to the new, and the old one was freed. I added the move constructor which also required a copy constructor and assignment operator. - The BlockBuilder::CurrentSizeEstimate() function calculates the current estimate size, but was being called 2 or 3 times for each key added. I changed the class to maintain a running estimate (equal to the original calculation) so that the function can return an already calculated value. - The code in BlockBuilder::Add() that calculated the shared bytes between the last key and the new key duplicated what Slice::difference_offset does, so I replaced it with the standard function. - BlockBuilder::Add() had code to copy just the changed portion into the last key value (and asserted that it now matched the new key). It is more efficient just to copy the whole new key over. - Moved this same code up into the 'if (use_delta_encoding_)' since the last key value is only needed when delta encoding is on. - FlushBlockBySizePolicy::BlockAlmostFull calculated a standard deviation value each time it was called, but this information would only change if block_size of block_size_deviation changed, so I created a member variable to hold the value to avoid the calculation each time. - Each PutVarint??() function has a buffer and calls std::string::append(). Two or three calls in a row could share a buffer and a single call to std::string::append(). Some of these will be helpful outside of the SstFileWriter. I'm not 100% the addition of the move constructor is appropriate as I wonder why this wasn't done before - maybe because of compiler compatibility? I tried it on gcc 4.8 and 4.9. Test Plan: The changes should not affect the results so the existing tests should all still work and no new tests were added. The value of the changes was seen by manually testing the SstFileWriter class through MyRocks and adding timing code to identify problem areas. Reviewers: sdong, IslamAbdelRahman Reviewed By: IslamAbdelRahman Subscribers: andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D59607
8 years ago
void Set(const Slice& _user_key, SequenceNumber s, ValueType t) {
SetFrom(ParsedInternalKey(_user_key, s, t));
}
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
void Set(const Slice& _user_key_with_ts, SequenceNumber s, ValueType t,
const Slice& ts) {
ParsedInternalKey pik = ParsedInternalKey(_user_key_with_ts, s, t);
// Should not call pik.SetTimestamp() directly as it overwrites the buffer
// containing _user_key.
SetFrom(pik, ts);
}
void SetFrom(const ParsedInternalKey& p) {
rep_.clear();
AppendInternalKey(&rep_, p);
}
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
void SetFrom(const ParsedInternalKey& p, const Slice& ts) {
rep_.clear();
AppendInternalKeyWithDifferentTimestamp(&rep_, p, ts);
}
void Clear() { rep_.clear(); }
// The underlying representation.
// Intended only to be used together with ConvertFromUserKey().
std::string* rep() { return &rep_; }
// Assuming that *rep() contains a user key, this method makes internal key
// out of it in-place. This saves a memcpy compared to Set()/SetFrom().
void ConvertFromUserKey(SequenceNumber s, ValueType t) {
AppendInternalKeyFooter(&rep_, s, t);
}
std::string DebugString(bool hex) const;
};
inline int InternalKeyComparator::Compare(const InternalKey& a,
const InternalKey& b) const {
return Compare(a.Encode(), b.Encode());
}
inline Status ParseInternalKey(const Slice& internal_key,
ParsedInternalKey* result, bool log_err_key) {
const size_t n = internal_key.size();
if (n < kNumInternalBytes) {
return Status::Corruption("Corrupted Key: Internal Key too small. Size=" +
std::to_string(n) + ". ");
}
uint64_t num = DecodeFixed64(internal_key.data() + n - kNumInternalBytes);
unsigned char c = num & 0xff;
result->sequence = num >> 8;
result->type = static_cast<ValueType>(c);
assert(result->type <= ValueType::kMaxValue);
result->user_key = Slice(internal_key.data(), n - kNumInternalBytes);
if (IsExtendedValueType(result->type)) {
return Status::OK();
} else {
return Status::Corruption("Corrupted Key",
result->DebugString(log_err_key, true));
}
}
// Update the sequence number in the internal key.
// Guarantees not to invalidate ikey.data().
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
inline void UpdateInternalKey(std::string* ikey, uint64_t seq, ValueType t) {
size_t ikey_sz = ikey->size();
assert(ikey_sz >= kNumInternalBytes);
uint64_t newval = (seq << 8) | t;
// Note: Since C++11, strings are guaranteed to be stored contiguously and
// string::operator[]() is guaranteed not to change ikey.data().
EncodeFixed64(&(*ikey)[ikey_sz - kNumInternalBytes], newval);
}
// Get the sequence number from the internal key
inline uint64_t GetInternalKeySeqno(const Slice& internal_key) {
const size_t n = internal_key.size();
assert(n >= kNumInternalBytes);
uint64_t num = DecodeFixed64(internal_key.data() + n - kNumInternalBytes);
return num >> 8;
}
// The class to store keys in an efficient way. It allows:
// 1. Users can either copy the key into it, or have it point to an unowned
// address.
// 2. For copied key, a short inline buffer is kept to reduce memory
// allocation for smaller keys.
// 3. It tracks user key or internal key, and allow conversion between them.
class IterKey {
public:
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
IterKey()
: buf_(space_),
key_(buf_),
key_size_(0),
buf_size_(sizeof(space_)),
is_user_key_(true) {}
// No copying allowed
IterKey(const IterKey&) = delete;
void operator=(const IterKey&) = delete;
~IterKey() { ResetBuffer(); }
// The bool will be picked up by the next calls to SetKey
void SetIsUserKey(bool is_user_key) { is_user_key_ = is_user_key; }
// Returns the key in whichever format that was provided to KeyIter
// If user-defined timestamp is enabled, then timestamp is included in the
// return result.
Slice GetKey() const { return Slice(key_, key_size_); }
Slice GetInternalKey() const {
assert(!IsUserKey());
return Slice(key_, key_size_);
}
// If user-defined timestamp is enabled, then timestamp is included in the
// return result of GetUserKey();
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
Slice GetUserKey() const {
if (IsUserKey()) {
return Slice(key_, key_size_);
} else {
assert(key_size_ >= kNumInternalBytes);
return Slice(key_, key_size_ - kNumInternalBytes);
}
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
}
size_t Size() const { return key_size_; }
void Clear() { key_size_ = 0; }
// Append "non_shared_data" to its back, from "shared_len"
// This function is used in Block::Iter::ParseNextKey
// shared_len: bytes in [0, shard_len-1] would be remained
// non_shared_data: data to be append, its length must be >= non_shared_len
void TrimAppend(const size_t shared_len, const char* non_shared_data,
const size_t non_shared_len) {
assert(shared_len <= key_size_);
size_t total_size = shared_len + non_shared_len;
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
if (IsKeyPinned() /* key is not in buf_ */) {
// Copy the key from external memory to buf_ (copy shared_len bytes)
EnlargeBufferIfNeeded(total_size);
memcpy(buf_, key_, shared_len);
} else if (total_size > buf_size_) {
// Need to allocate space, delete previous space
char* p = new char[total_size];
memcpy(p, key_, shared_len);
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
if (buf_ != space_) {
delete[] buf_;
}
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
buf_ = p;
buf_size_ = total_size;
}
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
memcpy(buf_ + shared_len, non_shared_data, non_shared_len);
key_ = buf_;
key_size_ = total_size;
}
Slice SetKey(const Slice& key, bool copy = true) {
// is_user_key_ expected to be set already via SetIsUserKey
return SetKeyImpl(key, copy);
}
// If user-defined timestamp is enabled, then `key` includes timestamp.
// TODO(yanqin) this is also used to set prefix, which do not include
// timestamp. Should be handled.
Slice SetUserKey(const Slice& key, bool copy = true) {
is_user_key_ = true;
return SetKeyImpl(key, copy);
}
Slice SetInternalKey(const Slice& key, bool copy = true) {
is_user_key_ = false;
return SetKeyImpl(key, copy);
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
}
// Copies the content of key, updates the reference to the user key in ikey
// and returns a Slice referencing the new copy.
Slice SetInternalKey(const Slice& key, ParsedInternalKey* ikey) {
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
size_t key_n = key.size();
assert(key_n >= kNumInternalBytes);
SetInternalKey(key);
ikey->user_key = Slice(key_, key_n - kNumInternalBytes);
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
return Slice(key_, key_n);
}
// Copy the key into IterKey own buf_
void OwnKey() {
assert(IsKeyPinned() == true);
Reserve(key_size_);
memcpy(buf_, key_, key_size_);
key_ = buf_;
}
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
// Update the sequence number in the internal key. Guarantees not to
// invalidate slices to the key (and the user key).
Allow compaction iterator to perform garbage collection (#7556) Summary: Add a threshold timestamp, full_history_ts_low_ of type `std::string*` to `CompactionIterator`, so that RocksDB can also perform garbage collection during compaction. * If full_history_ts_low_ is nullptr, then compaction iterator does not perform GC, preserving all timestamp history for all keys. Compaction iterator will treat user key with different timestamps as different user keys. * If full_history_ts_low_ is not nullptr, then compaction iterator performs GC. GC will look at keys older than `*full_history_ts_low_` and determine their eligibility based on factors including snapshots. Current rules of GC: * If an internal key is in the same snapshot as a previous counterpart with the same user key, and this key is eligible for GC, and the key is not single-delete or merge operand, then this key can be dropped. Note that the previous internal key cannot be a merge operand either. * If a tombstone is the most recent one in the earliest snapshot and it is eligible for GC, and keyNotExistsBeyondLevel() is true, then this tombstone can be dropped. * If a tombstone is the most recent one in a snapshot and it is eligible for GC, and the compaction is at bottommost level, then all other older internal keys of the same user key must also be eligible for GC, thus can be dropped * Single-delete, delete-range and merge are not currently supported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7556 Test Plan: make check Reviewed By: ltamasi Differential Revision: D24507728 Pulled By: riversand963 fbshipit-source-id: 3c09c7301f41eed76dfcf4d1527e68cf6e0a8bb3
4 years ago
void UpdateInternalKey(uint64_t seq, ValueType t, const Slice* ts = nullptr) {
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
assert(!IsKeyPinned());
assert(key_size_ >= kNumInternalBytes);
Allow compaction iterator to perform garbage collection (#7556) Summary: Add a threshold timestamp, full_history_ts_low_ of type `std::string*` to `CompactionIterator`, so that RocksDB can also perform garbage collection during compaction. * If full_history_ts_low_ is nullptr, then compaction iterator does not perform GC, preserving all timestamp history for all keys. Compaction iterator will treat user key with different timestamps as different user keys. * If full_history_ts_low_ is not nullptr, then compaction iterator performs GC. GC will look at keys older than `*full_history_ts_low_` and determine their eligibility based on factors including snapshots. Current rules of GC: * If an internal key is in the same snapshot as a previous counterpart with the same user key, and this key is eligible for GC, and the key is not single-delete or merge operand, then this key can be dropped. Note that the previous internal key cannot be a merge operand either. * If a tombstone is the most recent one in the earliest snapshot and it is eligible for GC, and keyNotExistsBeyondLevel() is true, then this tombstone can be dropped. * If a tombstone is the most recent one in a snapshot and it is eligible for GC, and the compaction is at bottommost level, then all other older internal keys of the same user key must also be eligible for GC, thus can be dropped * Single-delete, delete-range and merge are not currently supported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7556 Test Plan: make check Reviewed By: ltamasi Differential Revision: D24507728 Pulled By: riversand963 fbshipit-source-id: 3c09c7301f41eed76dfcf4d1527e68cf6e0a8bb3
4 years ago
if (ts) {
assert(key_size_ >= kNumInternalBytes + ts->size());
memcpy(&buf_[key_size_ - kNumInternalBytes - ts->size()], ts->data(),
ts->size());
}
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
uint64_t newval = (seq << 8) | t;
EncodeFixed64(&buf_[key_size_ - kNumInternalBytes], newval);
}
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
bool IsKeyPinned() const { return (key_ != buf_); }
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` is provided, user_key should not contain timestamp,
// and `ts` is appended after user_key.
// TODO: more efficient storage for timestamp.
void SetInternalKey(const Slice& key_prefix, const Slice& user_key,
SequenceNumber s,
Iterator with timestamp (#6255) Summary: Preliminary support for iterator with user timestamp. Current implementation does not consider merge operator and reverse iterator. Auto compaction is also disabled in unit tests. Create an iterator with timestamp. ``` ... read_opts.timestamp = &ts; auto* iter = db->NewIterator(read_opts); // target is key without timestamp. for (iter->Seek(target); iter->Valid(); iter->Next()) {} for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {} delete iter; read_opts.timestamp = &ts1; // lower_bound and upper_bound are without timestamp. read_opts.iterate_lower_bound = &lower_bound; read_opts.iterate_upper_bound = &upper_bound; auto* iter1 = db->NewIterator(read_opts); // Do Seek or SeekToFirst() delete iter1; ``` Test plan (dev server) ``` $make check ``` Simple benchmarking (dev server) 1. The overhead introduced by this PR even when timestamp is disabled. key size: 16 bytes value size: 100 bytes Entries: 1000000 Data reside in main memory, and try to stress iterator. Repeated three times on master and this PR. - Seek without next ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 ``` master: 159047.0 ops/sec this PR: 158922.3 ops/sec (2% drop in throughput) - Seek and next 10 times ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 -seek_nexts=10 ``` master: 109539.3 ops/sec this PR: 107519.7 ops/sec (2% drop in throughput) Pull Request resolved: https://github.com/facebook/rocksdb/pull/6255 Differential Revision: D19438227 Pulled By: riversand963 fbshipit-source-id: b66b4979486f8474619f4aa6bdd88598870b0746
4 years ago
ValueType value_type = kValueTypeForSeek,
const Slice* ts = nullptr) {
size_t psize = key_prefix.size();
size_t usize = user_key.size();
Iterator with timestamp (#6255) Summary: Preliminary support for iterator with user timestamp. Current implementation does not consider merge operator and reverse iterator. Auto compaction is also disabled in unit tests. Create an iterator with timestamp. ``` ... read_opts.timestamp = &ts; auto* iter = db->NewIterator(read_opts); // target is key without timestamp. for (iter->Seek(target); iter->Valid(); iter->Next()) {} for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {} delete iter; read_opts.timestamp = &ts1; // lower_bound and upper_bound are without timestamp. read_opts.iterate_lower_bound = &lower_bound; read_opts.iterate_upper_bound = &upper_bound; auto* iter1 = db->NewIterator(read_opts); // Do Seek or SeekToFirst() delete iter1; ``` Test plan (dev server) ``` $make check ``` Simple benchmarking (dev server) 1. The overhead introduced by this PR even when timestamp is disabled. key size: 16 bytes value size: 100 bytes Entries: 1000000 Data reside in main memory, and try to stress iterator. Repeated three times on master and this PR. - Seek without next ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 ``` master: 159047.0 ops/sec this PR: 158922.3 ops/sec (2% drop in throughput) - Seek and next 10 times ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 -seek_nexts=10 ``` master: 109539.3 ops/sec this PR: 107519.7 ops/sec (2% drop in throughput) Pull Request resolved: https://github.com/facebook/rocksdb/pull/6255 Differential Revision: D19438227 Pulled By: riversand963 fbshipit-source-id: b66b4979486f8474619f4aa6bdd88598870b0746
4 years ago
size_t ts_sz = (ts != nullptr ? ts->size() : 0);
EnlargeBufferIfNeeded(psize + usize + sizeof(uint64_t) + ts_sz);
if (psize > 0) {
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
memcpy(buf_, key_prefix.data(), psize);
}
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
memcpy(buf_ + psize, user_key.data(), usize);
Iterator with timestamp (#6255) Summary: Preliminary support for iterator with user timestamp. Current implementation does not consider merge operator and reverse iterator. Auto compaction is also disabled in unit tests. Create an iterator with timestamp. ``` ... read_opts.timestamp = &ts; auto* iter = db->NewIterator(read_opts); // target is key without timestamp. for (iter->Seek(target); iter->Valid(); iter->Next()) {} for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {} delete iter; read_opts.timestamp = &ts1; // lower_bound and upper_bound are without timestamp. read_opts.iterate_lower_bound = &lower_bound; read_opts.iterate_upper_bound = &upper_bound; auto* iter1 = db->NewIterator(read_opts); // Do Seek or SeekToFirst() delete iter1; ``` Test plan (dev server) ``` $make check ``` Simple benchmarking (dev server) 1. The overhead introduced by this PR even when timestamp is disabled. key size: 16 bytes value size: 100 bytes Entries: 1000000 Data reside in main memory, and try to stress iterator. Repeated three times on master and this PR. - Seek without next ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 ``` master: 159047.0 ops/sec this PR: 158922.3 ops/sec (2% drop in throughput) - Seek and next 10 times ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 -seek_nexts=10 ``` master: 109539.3 ops/sec this PR: 107519.7 ops/sec (2% drop in throughput) Pull Request resolved: https://github.com/facebook/rocksdb/pull/6255 Differential Revision: D19438227 Pulled By: riversand963 fbshipit-source-id: b66b4979486f8474619f4aa6bdd88598870b0746
4 years ago
if (ts) {
memcpy(buf_ + psize + usize, ts->data(), ts_sz);
}
EncodeFixed64(buf_ + usize + psize + ts_sz,
PackSequenceAndType(s, value_type));
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
key_ = buf_;
Iterator with timestamp (#6255) Summary: Preliminary support for iterator with user timestamp. Current implementation does not consider merge operator and reverse iterator. Auto compaction is also disabled in unit tests. Create an iterator with timestamp. ``` ... read_opts.timestamp = &ts; auto* iter = db->NewIterator(read_opts); // target is key without timestamp. for (iter->Seek(target); iter->Valid(); iter->Next()) {} for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {} delete iter; read_opts.timestamp = &ts1; // lower_bound and upper_bound are without timestamp. read_opts.iterate_lower_bound = &lower_bound; read_opts.iterate_upper_bound = &upper_bound; auto* iter1 = db->NewIterator(read_opts); // Do Seek or SeekToFirst() delete iter1; ``` Test plan (dev server) ``` $make check ``` Simple benchmarking (dev server) 1. The overhead introduced by this PR even when timestamp is disabled. key size: 16 bytes value size: 100 bytes Entries: 1000000 Data reside in main memory, and try to stress iterator. Repeated three times on master and this PR. - Seek without next ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 ``` master: 159047.0 ops/sec this PR: 158922.3 ops/sec (2% drop in throughput) - Seek and next 10 times ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 -seek_nexts=10 ``` master: 109539.3 ops/sec this PR: 107519.7 ops/sec (2% drop in throughput) Pull Request resolved: https://github.com/facebook/rocksdb/pull/6255 Differential Revision: D19438227 Pulled By: riversand963 fbshipit-source-id: b66b4979486f8474619f4aa6bdd88598870b0746
4 years ago
key_size_ = psize + usize + sizeof(uint64_t) + ts_sz;
is_user_key_ = false;
}
void SetInternalKey(const Slice& user_key, SequenceNumber s,
Iterator with timestamp (#6255) Summary: Preliminary support for iterator with user timestamp. Current implementation does not consider merge operator and reverse iterator. Auto compaction is also disabled in unit tests. Create an iterator with timestamp. ``` ... read_opts.timestamp = &ts; auto* iter = db->NewIterator(read_opts); // target is key without timestamp. for (iter->Seek(target); iter->Valid(); iter->Next()) {} for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {} delete iter; read_opts.timestamp = &ts1; // lower_bound and upper_bound are without timestamp. read_opts.iterate_lower_bound = &lower_bound; read_opts.iterate_upper_bound = &upper_bound; auto* iter1 = db->NewIterator(read_opts); // Do Seek or SeekToFirst() delete iter1; ``` Test plan (dev server) ``` $make check ``` Simple benchmarking (dev server) 1. The overhead introduced by this PR even when timestamp is disabled. key size: 16 bytes value size: 100 bytes Entries: 1000000 Data reside in main memory, and try to stress iterator. Repeated three times on master and this PR. - Seek without next ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 ``` master: 159047.0 ops/sec this PR: 158922.3 ops/sec (2% drop in throughput) - Seek and next 10 times ``` ./db_bench -db=/dev/shm/rocksdbtest-1000 -benchmarks=fillseq,seekrandom -enable_pipelined_write=false -disable_wal=true -format_version=3 -seek_nexts=10 ``` master: 109539.3 ops/sec this PR: 107519.7 ops/sec (2% drop in throughput) Pull Request resolved: https://github.com/facebook/rocksdb/pull/6255 Differential Revision: D19438227 Pulled By: riversand963 fbshipit-source-id: b66b4979486f8474619f4aa6bdd88598870b0746
4 years ago
ValueType value_type = kValueTypeForSeek,
const Slice* ts = nullptr) {
SetInternalKey(Slice(), user_key, s, value_type, ts);
}
void Reserve(size_t size) {
EnlargeBufferIfNeeded(size);
key_size_ = size;
}
void SetInternalKey(const ParsedInternalKey& parsed_key) {
SetInternalKey(Slice(), parsed_key);
}
void SetInternalKey(const Slice& key_prefix,
const ParsedInternalKey& parsed_key_suffix) {
SetInternalKey(key_prefix, parsed_key_suffix.user_key,
parsed_key_suffix.sequence, parsed_key_suffix.type);
}
void EncodeLengthPrefixedKey(const Slice& key) {
auto size = key.size();
EnlargeBufferIfNeeded(size + static_cast<size_t>(VarintLength(size)));
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
char* ptr = EncodeVarint32(buf_, static_cast<uint32_t>(size));
memcpy(ptr, key.data(), size);
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
key_ = buf_;
is_user_key_ = true;
}
bool IsUserKey() const { return is_user_key_; }
private:
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
char* buf_;
const char* key_;
size_t key_size_;
size_t buf_size_;
char space_[32]; // Avoid allocation for short keys
bool is_user_key_;
Slice SetKeyImpl(const Slice& key, bool copy) {
size_t size = key.size();
if (copy) {
// Copy key to buf_
EnlargeBufferIfNeeded(size);
memcpy(buf_, key.data(), size);
key_ = buf_;
} else {
// Update key_ to point to external memory
key_ = key.data();
}
key_size_ = size;
return Slice(key_, key_size_);
}
void ResetBuffer() {
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
if (buf_ != space_) {
delete[] buf_;
buf_ = space_;
}
buf_size_ = sizeof(space_);
key_size_ = 0;
}
// Enlarge the buffer size if needed based on key_size.
// By default, static allocated buffer is used. Once there is a key
// larger than the static allocated buffer, another buffer is dynamically
// allocated, until a larger key buffer is requested. In that case, we
// reallocate buffer and delete the old one.
void EnlargeBufferIfNeeded(size_t key_size) {
// If size is smaller than buffer size, continue using current buffer,
// or the static allocated one, as default
if (key_size > buf_size_) {
EnlargeBuffer(key_size);
}
}
void EnlargeBuffer(size_t key_size);
};
// Convert from a SliceTransform of user keys, to a SliceTransform of
// internal keys.
class InternalKeySliceTransform : public SliceTransform {
public:
explicit InternalKeySliceTransform(const SliceTransform* transform)
: transform_(transform) {}
virtual const char* Name() const override { return transform_->Name(); }
virtual Slice Transform(const Slice& src) const override {
auto user_key = ExtractUserKey(src);
return transform_->Transform(user_key);
}
virtual bool InDomain(const Slice& src) const override {
auto user_key = ExtractUserKey(src);
return transform_->InDomain(user_key);
}
virtual bool InRange(const Slice& dst) const override {
auto user_key = ExtractUserKey(dst);
return transform_->InRange(user_key);
}
const SliceTransform* user_prefix_extractor() const { return transform_; }
private:
// Like comparator, InternalKeySliceTransform will not take care of the
// deletion of transform_
const SliceTransform* const transform_;
};
// Read the key of a record from a write batch.
// if this record represent the default column family then cf_record
// must be passed as false, otherwise it must be passed as true.
extern bool ReadKeyFromWriteBatchEntry(Slice* input, Slice* key,
bool cf_record);
// Read record from a write batch piece from input.
// tag, column_family, key, value and blob are return values. Callers own the
// slice they point to.
// Tag is defined as ValueType.
// input will be advanced to after the record.
// If user-defined timestamp is enabled for a column family, then the `key`
// resulting from this call will include timestamp.
extern Status ReadRecordFromWriteBatch(Slice* input, char* tag,
uint32_t* column_family, Slice* key,
Slice* value, Slice* blob, Slice* xid);
// When user call DeleteRange() to delete a range of keys,
// we will store a serialized RangeTombstone in MemTable and SST.
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
// the struct here is an easy-understood form
// start/end_key_ is the start/end user key of the range to be deleted
struct RangeTombstone {
Slice start_key_;
Slice end_key_;
SequenceNumber 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
// TODO: we should optimize the storage here when user-defined timestamp
// is NOT enabled: they currently take up (16 + 32 + 32) bytes per tombstone.
Slice ts_;
std::string pinned_start_key_;
std::string pinned_end_key_;
RangeTombstone() = default;
RangeTombstone(Slice sk, Slice ek, SequenceNumber sn)
: start_key_(sk), end_key_(ek), seq_(sn) {}
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
// User-defined timestamp is enabled, `sk` and `ek` should be user key
// with timestamp, `ts` will replace the timestamps in `sk` and
// `ek`.
RangeTombstone(Slice sk, Slice ek, SequenceNumber sn, Slice ts)
: seq_(sn), ts_(ts) {
assert(!ts.empty());
pinned_start_key_.reserve(sk.size());
pinned_start_key_.append(sk.data(), sk.size() - ts.size());
pinned_start_key_.append(ts.data(), ts.size());
pinned_end_key_.reserve(ek.size());
pinned_end_key_.append(ek.data(), ek.size() - ts.size());
pinned_end_key_.append(ts.data(), ts.size());
start_key_ = pinned_start_key_;
end_key_ = pinned_end_key_;
}
RangeTombstone(ParsedInternalKey parsed_key, Slice value) {
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
start_key_ = parsed_key.user_key;
seq_ = parsed_key.sequence;
end_key_ = value;
}
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
// be careful to use Serialize(), allocates new memory
std::pair<InternalKey, Slice> Serialize() const {
auto key = InternalKey(start_key_, seq_, kTypeRangeDeletion);
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
return std::make_pair(std::move(key), end_key_);
}
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
// be careful to use SerializeKey(), allocates new memory
InternalKey SerializeKey() const {
return InternalKey(start_key_, seq_, kTypeRangeDeletion);
}
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
// The tombstone end-key is exclusive, so we generate an internal-key here
// which has a similar property. Using kMaxSequenceNumber guarantees that
// the returned internal-key will compare less than any other internal-key
// with the same user-key. This in turn guarantees that the serialized
// end-key for a tombstone such as [a-b] will compare less than the key "b".
//
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
// be careful to use SerializeEndKey(), allocates new memory
InternalKey SerializeEndKey() const {
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_.empty()) {
static constexpr char kTsMax[] = "\xff\xff\xff\xff\xff\xff\xff\xff\xff";
if (ts_.size() <= strlen(kTsMax)) {
return InternalKey(end_key_, kMaxSequenceNumber, kTypeRangeDeletion,
Slice(kTsMax, ts_.size()));
} else {
return InternalKey(end_key_, kMaxSequenceNumber, kTypeRangeDeletion,
std::string(ts_.size(), '\xff'));
}
}
return InternalKey(end_key_, kMaxSequenceNumber, kTypeRangeDeletion);
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
}
};
inline int InternalKeyComparator::Compare(const Slice& akey,
const Slice& bkey) const {
// Order by:
// increasing user key (according to user-supplied comparator)
// decreasing sequence number
// decreasing type (though sequence# should be enough to disambiguate)
int r = user_comparator_.Compare(ExtractUserKey(akey), ExtractUserKey(bkey));
if (r == 0) {
const uint64_t anum =
DecodeFixed64(akey.data() + akey.size() - kNumInternalBytes);
const uint64_t bnum =
DecodeFixed64(bkey.data() + bkey.size() - kNumInternalBytes);
if (anum > bnum) {
r = -1;
} else if (anum < bnum) {
r = +1;
}
}
return r;
}
inline int InternalKeyComparator::CompareKeySeq(const Slice& akey,
const Slice& bkey) const {
// Order by:
// increasing user key (according to user-supplied comparator)
// decreasing sequence number
int r = user_comparator_.Compare(ExtractUserKey(akey), ExtractUserKey(bkey));
if (r == 0) {
// Shift the number to exclude the last byte which contains the value type
const uint64_t anum =
DecodeFixed64(akey.data() + akey.size() - kNumInternalBytes) >> 8;
const uint64_t bnum =
DecodeFixed64(bkey.data() + bkey.size() - kNumInternalBytes) >> 8;
if (anum > bnum) {
r = -1;
} else if (anum < bnum) {
r = +1;
}
}
return r;
}
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
inline int InternalKeyComparator::Compare(const Slice& a,
SequenceNumber a_global_seqno,
const Slice& b,
SequenceNumber b_global_seqno) const {
int r = user_comparator_.Compare(ExtractUserKey(a), ExtractUserKey(b));
if (r == 0) {
uint64_t a_footer, b_footer;
if (a_global_seqno == kDisableGlobalSequenceNumber) {
a_footer = ExtractInternalKeyFooter(a);
} else {
a_footer = PackSequenceAndType(a_global_seqno, ExtractValueType(a));
}
if (b_global_seqno == kDisableGlobalSequenceNumber) {
b_footer = ExtractInternalKeyFooter(b);
} else {
b_footer = PackSequenceAndType(b_global_seqno, ExtractValueType(b));
}
if (a_footer > b_footer) {
r = -1;
} else if (a_footer < b_footer) {
r = +1;
}
}
return r;
}
// Wrap InternalKeyComparator as a comparator class for ParsedInternalKey.
struct ParsedInternalKeyComparator {
explicit ParsedInternalKeyComparator(const InternalKeyComparator* c)
: cmp(c) {}
bool operator()(const ParsedInternalKey& a,
const ParsedInternalKey& b) const {
return cmp->Compare(a, b) < 0;
}
const InternalKeyComparator* cmp;
};
} // namespace ROCKSDB_NAMESPACE