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

1683 lines
66 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.
//
// The representation of a DBImpl consists of a set of Versions. The
// newest version is called "current". Older versions may be kept
// around to provide a consistent view to live iterators.
//
Add blob files to VersionStorageInfo/VersionBuilder (#6597) Summary: The patch adds a couple of classes to represent metadata about blob files: `SharedBlobFileMetaData` contains the information elements that are immutable (once the blob file is closed), e.g. blob file number, total number and size of blob files, checksum method/value, while `BlobFileMetaData` contains attributes that can vary across versions like the amount of garbage in the file. There is a single `SharedBlobFileMetaData` for each blob file, which is jointly owned by the `BlobFileMetaData` objects that point to it; `BlobFileMetaData` objects, in turn, are owned by `Version`s and can also be shared if the (immutable _and_ mutable) state of the blob file is the same in two versions. In addition, the patch adds the blob file metadata to `VersionStorageInfo`, and extends `VersionBuilder` so that it can apply blob file related `VersionEdit`s (i.e. those containing `BlobFileAddition`s and/or `BlobFileGarbage`), and save blob file metadata to a new `VersionStorageInfo`. Consistency checks are also extended to ensure that table files point to blob files that are part of the `Version`, and that all blob files that are part of any given `Version` have at least some _non_-garbage data in them. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6597 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D20656803 Pulled By: ltamasi fbshipit-source-id: f1f74d135045b3b42d0146f03ee576ef0a4bfd80
5 years ago
// Each Version keeps track of a set of table files per level, as well as a
// set of blob files. The entire set of versions is maintained in a
// VersionSet.
//
// Version,VersionSet are thread-compatible, but require external
// synchronization on all accesses.
#pragma once
#include <atomic>
#include <deque>
#include <limits>
#include <map>
#include <memory>
#include <optional>
#include <set>
#include <string>
#include <unordered_set>
#include <utility>
#include <vector>
#include "cache/cache_helpers.h"
Add blob files to VersionStorageInfo/VersionBuilder (#6597) Summary: The patch adds a couple of classes to represent metadata about blob files: `SharedBlobFileMetaData` contains the information elements that are immutable (once the blob file is closed), e.g. blob file number, total number and size of blob files, checksum method/value, while `BlobFileMetaData` contains attributes that can vary across versions like the amount of garbage in the file. There is a single `SharedBlobFileMetaData` for each blob file, which is jointly owned by the `BlobFileMetaData` objects that point to it; `BlobFileMetaData` objects, in turn, are owned by `Version`s and can also be shared if the (immutable _and_ mutable) state of the blob file is the same in two versions. In addition, the patch adds the blob file metadata to `VersionStorageInfo`, and extends `VersionBuilder` so that it can apply blob file related `VersionEdit`s (i.e. those containing `BlobFileAddition`s and/or `BlobFileGarbage`), and save blob file metadata to a new `VersionStorageInfo`. Consistency checks are also extended to ensure that table files point to blob files that are part of the `Version`, and that all blob files that are part of any given `Version` have at least some _non_-garbage data in them. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6597 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D20656803 Pulled By: ltamasi fbshipit-source-id: f1f74d135045b3b42d0146f03ee576ef0a4bfd80
5 years ago
#include "db/blob/blob_file_meta.h"
#include "db/column_family.h"
#include "db/compaction/compaction.h"
#include "db/compaction/compaction_picker.h"
#include "db/dbformat.h"
#include "db/file_indexer.h"
#include "db/log_reader.h"
#include "db/range_del_aggregator.h"
#include "db/read_callback.h"
#include "db/table_cache.h"
#include "db/version_builder.h"
#include "db/version_edit.h"
#include "db/write_controller.h"
#include "env/file_system_tracer.h"
MultiGet async IO across multiple levels (#10535) Summary: This PR exploits parallelism in MultiGet across levels. It applies only to the coroutine version of MultiGet. Previously, MultiGet file reads from SST files in the same level were parallelized. With this PR, MultiGet batches with keys distributed across multiple levels are read in parallel. This is accomplished by splitting the keys not present in a level (determined by bloom filtering) into a separate batch, and processing the new batch in parallel with the original batch. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10535 Test Plan: 1. Ensure existing MultiGet unit tests pass, updating them as necessary 2. New unit tests - TODO 3. Run stress test - TODO No noticeable regression (<1%) without async IO - Without PR: `multireadrandom : 7.261 micros/op 1101724 ops/sec 60.007 seconds 66110936 operations; 571.6 MB/s (8168992 of 8168992 found)` With PR: `multireadrandom : 7.305 micros/op 1095167 ops/sec 60.007 seconds 65717936 operations; 568.2 MB/s (8271992 of 8271992 found)` For a fully cached DB, but with async IO option on, no regression observed (<1%) - Without PR: `multireadrandom : 5.201 micros/op 1538027 ops/sec 60.005 seconds 92288936 operations; 797.9 MB/s (11540992 of 11540992 found) ` With PR: `multireadrandom : 5.249 micros/op 1524097 ops/sec 60.005 seconds 91452936 operations; 790.7 MB/s (11649992 of 11649992 found) ` Reviewed By: akankshamahajan15 Differential Revision: D38774009 Pulled By: anand1976 fbshipit-source-id: c955e259749f1c091590ade73105b3ee46cd0007
2 years ago
#if USE_COROUTINES
#include "folly/experimental/coro/BlockingWait.h"
#include "folly/experimental/coro/Collect.h"
#endif
#include "monitoring/instrumented_mutex.h"
#include "options/db_options.h"
#include "port/port.h"
#include "rocksdb/env.h"
#include "rocksdb/file_checksum.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
6 years ago
#include "table/get_context.h"
#include "table/multiget_context.h"
#include "trace_replay/block_cache_tracer.h"
MultiGet async IO across multiple levels (#10535) Summary: This PR exploits parallelism in MultiGet across levels. It applies only to the coroutine version of MultiGet. Previously, MultiGet file reads from SST files in the same level were parallelized. With this PR, MultiGet batches with keys distributed across multiple levels are read in parallel. This is accomplished by splitting the keys not present in a level (determined by bloom filtering) into a separate batch, and processing the new batch in parallel with the original batch. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10535 Test Plan: 1. Ensure existing MultiGet unit tests pass, updating them as necessary 2. New unit tests - TODO 3. Run stress test - TODO No noticeable regression (<1%) without async IO - Without PR: `multireadrandom : 7.261 micros/op 1101724 ops/sec 60.007 seconds 66110936 operations; 571.6 MB/s (8168992 of 8168992 found)` With PR: `multireadrandom : 7.305 micros/op 1095167 ops/sec 60.007 seconds 65717936 operations; 568.2 MB/s (8271992 of 8271992 found)` For a fully cached DB, but with async IO option on, no regression observed (<1%) - Without PR: `multireadrandom : 5.201 micros/op 1538027 ops/sec 60.005 seconds 92288936 operations; 797.9 MB/s (11540992 of 11540992 found) ` With PR: `multireadrandom : 5.249 micros/op 1524097 ops/sec 60.005 seconds 91452936 operations; 790.7 MB/s (11649992 of 11649992 found) ` Reviewed By: akankshamahajan15 Differential Revision: D38774009 Pulled By: anand1976 fbshipit-source-id: c955e259749f1c091590ade73105b3ee46cd0007
2 years ago
#include "util/autovector.h"
Multi file concurrency in MultiGet using coroutines and async IO (#9968) Summary: This PR implements a coroutine version of batched MultiGet in order to concurrently read from multiple SST files in a level using async IO, thus reducing the latency of the MultiGet. The API from the user perspective is still synchronous and single threaded, with the RocksDB part of the processing happening in the context of the caller's thread. In Version::MultiGet, the decision is made whether to call synchronous or coroutine code. A good way to review this PR is to review the first 4 commits in order - de773b3, 70c2f70, 10b50e1, and 377a597 - before reviewing the rest. TODO: 1. Figure out how to build it in CircleCI (requires some dependencies to be installed) 2. Do some stress testing with coroutines enabled No regression in synchronous MultiGet between this branch and main - ``` ./db_bench -use_existing_db=true --db=/data/mysql/rocksdb/prefix_scan -benchmarks="readseq,multireadrandom" -key_size=32 -value_size=512 -num=5000000 -batch_size=64 -multiread_batched=true -use_direct_reads=false -duration=60 -ops_between_duration_checks=1 -readonly=true -adaptive_readahead=true -threads=16 -cache_size=10485760000 -async_io=false -multiread_stride=40000 -statistics ``` Branch - ```multireadrandom : 4.025 micros/op 3975111 ops/sec 60.001 seconds 238509056 operations; 2062.3 MB/s (14767808 of 14767808 found)``` Main - ```multireadrandom : 3.987 micros/op 4013216 ops/sec 60.001 seconds 240795392 operations; 2082.1 MB/s (15231040 of 15231040 found)``` More benchmarks in various scenarios are given below. The measurements were taken with ```async_io=false``` (no coroutines) and ```async_io=true``` (use coroutines). For an IO bound workload (with every key requiring an IO), the coroutines version shows a clear benefit, being ~2.6X faster. For CPU bound workloads, the coroutines version has ~6-15% higher CPU utilization, depending on how many keys overlap an SST file. 1. Single thread IO bound workload on remote storage with sparse MultiGet batch keys (~1 key overlap/file) - No coroutines - ```multireadrandom : 831.774 micros/op 1202 ops/sec 60.001 seconds 72136 operations; 0.6 MB/s (72136 of 72136 found)``` Using coroutines - ```multireadrandom : 318.742 micros/op 3137 ops/sec 60.003 seconds 188248 operations; 1.6 MB/s (188248 of 188248 found)``` 2. Single thread CPU bound workload (all data cached) with ~1 key overlap/file - No coroutines - ```multireadrandom : 4.127 micros/op 242322 ops/sec 60.000 seconds 14539384 operations; 125.7 MB/s (14539384 of 14539384 found)``` Using coroutines - ```multireadrandom : 4.741 micros/op 210935 ops/sec 60.000 seconds 12656176 operations; 109.4 MB/s (12656176 of 12656176 found)``` 3. Single thread CPU bound workload with ~2 key overlap/file - No coroutines - ```multireadrandom : 3.717 micros/op 269000 ops/sec 60.000 seconds 16140024 operations; 139.6 MB/s (16140024 of 16140024 found)``` Using coroutines - ```multireadrandom : 4.146 micros/op 241204 ops/sec 60.000 seconds 14472296 operations; 125.1 MB/s (14472296 of 14472296 found)``` 4. CPU bound multi-threaded (16 threads) with ~4 key overlap/file - No coroutines - ```multireadrandom : 4.534 micros/op 3528792 ops/sec 60.000 seconds 211728728 operations; 1830.7 MB/s (12737024 of 12737024 found) ``` Using coroutines - ```multireadrandom : 4.872 micros/op 3283812 ops/sec 60.000 seconds 197030096 operations; 1703.6 MB/s (12548032 of 12548032 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9968 Reviewed By: akankshamahajan15 Differential Revision: D36348563 Pulled By: anand1976 fbshipit-source-id: c0ce85a505fd26ebfbb09786cbd7f25202038696
3 years ago
#include "util/coro_utils.h"
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
3 years ago
#include "util/hash_containers.h"
namespace ROCKSDB_NAMESPACE {
namespace log {
class Writer;
}
Integrated blob garbage collection: relocate blobs (#7694) Summary: The patch adds basic garbage collection support to the integrated BlobDB implementation. Valid blobs residing in the oldest blob files are relocated as they are encountered during compaction. The threshold that determines which blob files qualify is computed based on the configuration option `blob_garbage_collection_age_cutoff`, which was introduced in https://github.com/facebook/rocksdb/issues/7661 . Once a blob is retrieved for the purposes of relocation, it passes through the same logic that extracts large values to blob files in general. This means that if, for instance, the size threshold for key-value separation (`min_blob_size`) got changed or writing blob files got disabled altogether, it is possible for the value to be moved back into the LSM tree. In particular, one way to re-inline all blob values if needed would be to perform a full manual compaction with `enable_blob_files` set to `false`, `enable_blob_garbage_collection` set to `true`, and `blob_file_garbage_collection_age_cutoff` set to `1.0`. Some TODOs that I plan to address in separate PRs: 1) We'll have to measure the amount of new garbage in each blob file and log `BlobFileGarbage` entries as part of the compaction job's `VersionEdit`. (For the time being, blob files are cleaned up solely based on the `oldest_blob_file_number` relationships.) 2) When compression is used for blobs, the compression type hasn't changed, and the blob still qualifies for being written to a blob file, we can simply copy the compressed blob to the new file instead of going through decompression and compression. 3) We need to update the formula for computing write amplification to account for the amount of data read from blob files as part of GC. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7694 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D25069663 Pulled By: ltamasi fbshipit-source-id: bdfa8feb09afcf5bca3b4eba2ba72ce2f15cd06a
4 years ago
class BlobIndex;
class Compaction;
class LogBuffer;
class LookupKey;
class MemTable;
class Version;
class VersionSet;
class WriteBufferManager;
class MergeContext;
class ColumnFamilySet;
class MergeIteratorBuilder;
class SystemClock;
class ManifestTailer;
MultiGet async IO across multiple levels (#10535) Summary: This PR exploits parallelism in MultiGet across levels. It applies only to the coroutine version of MultiGet. Previously, MultiGet file reads from SST files in the same level were parallelized. With this PR, MultiGet batches with keys distributed across multiple levels are read in parallel. This is accomplished by splitting the keys not present in a level (determined by bloom filtering) into a separate batch, and processing the new batch in parallel with the original batch. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10535 Test Plan: 1. Ensure existing MultiGet unit tests pass, updating them as necessary 2. New unit tests - TODO 3. Run stress test - TODO No noticeable regression (<1%) without async IO - Without PR: `multireadrandom : 7.261 micros/op 1101724 ops/sec 60.007 seconds 66110936 operations; 571.6 MB/s (8168992 of 8168992 found)` With PR: `multireadrandom : 7.305 micros/op 1095167 ops/sec 60.007 seconds 65717936 operations; 568.2 MB/s (8271992 of 8271992 found)` For a fully cached DB, but with async IO option on, no regression observed (<1%) - Without PR: `multireadrandom : 5.201 micros/op 1538027 ops/sec 60.005 seconds 92288936 operations; 797.9 MB/s (11540992 of 11540992 found) ` With PR: `multireadrandom : 5.249 micros/op 1524097 ops/sec 60.005 seconds 91452936 operations; 790.7 MB/s (11649992 of 11649992 found) ` Reviewed By: akankshamahajan15 Differential Revision: D38774009 Pulled By: anand1976 fbshipit-source-id: c955e259749f1c091590ade73105b3ee46cd0007
2 years ago
class FilePickerMultiGet;
// VersionEdit is always supposed to be valid and it is used to point at
// entries in Manifest. Ideally it should not be used as a container to
// carry around few of its fields as function params because it can cause
// readers to think it's a valid entry from Manifest. To avoid that confusion
// introducing VersionEditParams to simply carry around multiple VersionEdit
// params. It need not point to a valid record in Manifest.
using VersionEditParams = VersionEdit;
create compressed_levels_ in Version, allocate its space using arena. Make Version::Get, Version::FindFile faster Summary: Define CompressedFileMetaData that just contains fd, smallest_slice, largest_slice. Create compressed_levels_ in Version, the space is allocated using arena Thus increase the file meta data locality, speed up "Get" and "FindFile" benchmark with in-memory tmpfs, could have 4% improvement under "random read" and 2% improvement under "read while writing" benchmark command: ./db_bench --db=/mnt/db/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --block_size=4096 --cache_size=17179869184 --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=33554432 --max_bytes_for_level_base=1073741824 --disable_wal=0 --sync=0 --disable_data_sync=1 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_grandparent_overlap_factor=10 --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 --perf_level=0 --benchmarks=readwhilewriting,readwhilewriting,readwhilewriting --use_existing_db=1 --num=52428800 --threads=1 —writes_per_second=81920 Read Random: From 1.8363 ms/op, improve to 1.7587 ms/op. Read while writing: From 2.985 ms/op, improve to 2.924 ms/op. Test Plan: make all check Reviewers: ljin, haobo, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, igor Differential Revision: https://reviews.facebook.net/D19419
11 years ago
// Return the smallest index i such that file_level.files[i]->largest >= key.
// Return file_level.num_files if there is no such file.
// REQUIRES: "file_level.files" contains a sorted list of
// non-overlapping files.
extern int FindFile(const InternalKeyComparator& icmp,
const LevelFilesBrief& file_level, const Slice& key);
create compressed_levels_ in Version, allocate its space using arena. Make Version::Get, Version::FindFile faster Summary: Define CompressedFileMetaData that just contains fd, smallest_slice, largest_slice. Create compressed_levels_ in Version, the space is allocated using arena Thus increase the file meta data locality, speed up "Get" and "FindFile" benchmark with in-memory tmpfs, could have 4% improvement under "random read" and 2% improvement under "read while writing" benchmark command: ./db_bench --db=/mnt/db/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --block_size=4096 --cache_size=17179869184 --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=33554432 --max_bytes_for_level_base=1073741824 --disable_wal=0 --sync=0 --disable_data_sync=1 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_grandparent_overlap_factor=10 --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 --perf_level=0 --benchmarks=readwhilewriting,readwhilewriting,readwhilewriting --use_existing_db=1 --num=52428800 --threads=1 —writes_per_second=81920 Read Random: From 1.8363 ms/op, improve to 1.7587 ms/op. Read while writing: From 2.985 ms/op, improve to 2.924 ms/op. Test Plan: make all check Reviewers: ljin, haobo, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, igor Differential Revision: https://reviews.facebook.net/D19419
11 years ago
// Returns true iff some file in "files" overlaps the user key range
// [*smallest,*largest].
// smallest==nullptr represents a key smaller than all keys in the DB.
// largest==nullptr represents a key largest than all keys in the DB.
create compressed_levels_ in Version, allocate its space using arena. Make Version::Get, Version::FindFile faster Summary: Define CompressedFileMetaData that just contains fd, smallest_slice, largest_slice. Create compressed_levels_ in Version, the space is allocated using arena Thus increase the file meta data locality, speed up "Get" and "FindFile" benchmark with in-memory tmpfs, could have 4% improvement under "random read" and 2% improvement under "read while writing" benchmark command: ./db_bench --db=/mnt/db/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --block_size=4096 --cache_size=17179869184 --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=33554432 --max_bytes_for_level_base=1073741824 --disable_wal=0 --sync=0 --disable_data_sync=1 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_grandparent_overlap_factor=10 --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 --perf_level=0 --benchmarks=readwhilewriting,readwhilewriting,readwhilewriting --use_existing_db=1 --num=52428800 --threads=1 —writes_per_second=81920 Read Random: From 1.8363 ms/op, improve to 1.7587 ms/op. Read while writing: From 2.985 ms/op, improve to 2.924 ms/op. Test Plan: make all check Reviewers: ljin, haobo, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, igor Differential Revision: https://reviews.facebook.net/D19419
11 years ago
// REQUIRES: If disjoint_sorted_files, file_level.files[]
// contains disjoint ranges in sorted order.
extern bool SomeFileOverlapsRange(const InternalKeyComparator& icmp,
bool disjoint_sorted_files,
const LevelFilesBrief& file_level,
const Slice* smallest_user_key,
const Slice* largest_user_key);
// Generate LevelFilesBrief from vector<FdWithKeyRange*>
// Would copy smallest_key and largest_key data to sequential memory
// arena: Arena used to allocate the memory
extern void DoGenerateLevelFilesBrief(LevelFilesBrief* file_level,
const std::vector<FileMetaData*>& files,
Arena* arena);
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
enum EpochNumberRequirement {
kMightMissing,
kMustPresent,
};
// Information of the storage associated with each Version, including number of
// levels of LSM tree, files information at each level, files marked for
Add blob files to VersionStorageInfo/VersionBuilder (#6597) Summary: The patch adds a couple of classes to represent metadata about blob files: `SharedBlobFileMetaData` contains the information elements that are immutable (once the blob file is closed), e.g. blob file number, total number and size of blob files, checksum method/value, while `BlobFileMetaData` contains attributes that can vary across versions like the amount of garbage in the file. There is a single `SharedBlobFileMetaData` for each blob file, which is jointly owned by the `BlobFileMetaData` objects that point to it; `BlobFileMetaData` objects, in turn, are owned by `Version`s and can also be shared if the (immutable _and_ mutable) state of the blob file is the same in two versions. In addition, the patch adds the blob file metadata to `VersionStorageInfo`, and extends `VersionBuilder` so that it can apply blob file related `VersionEdit`s (i.e. those containing `BlobFileAddition`s and/or `BlobFileGarbage`), and save blob file metadata to a new `VersionStorageInfo`. Consistency checks are also extended to ensure that table files point to blob files that are part of the `Version`, and that all blob files that are part of any given `Version` have at least some _non_-garbage data in them. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6597 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D20656803 Pulled By: ltamasi fbshipit-source-id: f1f74d135045b3b42d0146f03ee576ef0a4bfd80
5 years ago
// compaction, blob files, etc.
class VersionStorageInfo {
public:
VersionStorageInfo(const InternalKeyComparator* internal_comparator,
const Comparator* user_comparator, int num_levels,
CompactionStyle compaction_style,
VersionStorageInfo* src_vstorage,
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
bool _force_consistency_checks,
EpochNumberRequirement epoch_number_requirement =
EpochNumberRequirement::kMustPresent);
// No copying allowed
VersionStorageInfo(const VersionStorageInfo&) = delete;
void operator=(const VersionStorageInfo&) = delete;
~VersionStorageInfo();
void Reserve(int level, size_t size) { files_[level].reserve(size); }
void AddFile(int level, FileMetaData* f);
Add basic kRoundRobin compaction policy (#10107) Summary: Add `kRoundRobin` as a compaction priority. The implementation is as follows. - Define a cursor as the smallest Internal key in the successor of the selected file. Add `vector<InternalKey> compact_cursor_` into `VersionStorageInfo` where each element (`InternalKey`) in `compact_cursor_` represents a cursor. In round-robin compaction policy, we just need to select the first file (assuming files are sorted) and also has the smallest InternalKey larger than/equal to the cursor. After a file is chosen, we create a new `Fsize` vector which puts the selected file is placed at the first position in `temp`, the next cursor is then updated as the smallest InternalKey in successor of the selected file (the above logic is implemented in `SortFileByRoundRobin`). - After a compaction succeeds, typically `InstallCompactionResults()`, we choose the next cursor for the input level and save it to `edit`. When calling `LogAndApply`, we save the next cursor with its level into some local variable and finally apply the change to `vstorage` in `SaveTo` function. - Cursors are persist pair by pair (<level, InternalKey>) in `EncodeTo` so that they can be reconstructed when reopening. An empty cursor will not be encoded to MANIFEST Pull Request resolved: https://github.com/facebook/rocksdb/pull/10107 Test Plan: add unit test (`CompactionPriRoundRobin`) in `compaction_picker_test`, add `kRoundRobin` priority in `CompactionPriTest` from `db_compaction_test`, and add `PersistRoundRobinCompactCursor` in `db_compaction_test` Reviewed By: ajkr Differential Revision: D37316037 Pulled By: littlepig2013 fbshipit-source-id: 9f481748190ace416079139044e00df2968fb1ee
3 years ago
// Resize/Initialize the space for compact_cursor_
void ResizeCompactCursors(int level) {
compact_cursor_.resize(level, InternalKey());
}
const std::vector<InternalKey>& GetCompactCursors() const {
return compact_cursor_;
}
// REQUIRES: ResizeCompactCursors has been called
void AddCursorForOneLevel(int level,
const InternalKey& smallest_uncompacted_key) {
compact_cursor_[level] = smallest_uncompacted_key;
}
// REQUIRES: lock is held
Support subcmpct using reserved resources for round-robin priority (#10341) Summary: Earlier implementation of round-robin priority can only pick one file at a time and disallows parallel compactions within the same level. In this PR, round-robin compaction policy will expand towards more input files with respecting some additional constraints, which are summarized as follows: * Constraint 1: We can only pick consecutive files - Constraint 1a: When a file is being compacted (or some input files are being compacted after expanding), we cannot choose it and have to stop choosing more files - Constraint 1b: When we reach the last file (with the largest keys), we cannot choose more files (the next file will be the first one with small keys) * Constraint 2: We should ensure the total compaction bytes (including the overlapped files from the next level) is no more than `mutable_cf_options_.max_compaction_bytes` * Constraint 3: We try our best to pick as many files as possible so that the post-compaction level size can be just less than `MaxBytesForLevel(start_level_)` * Constraint 4: If trivial move is allowed, we reuse the logic of `TryNonL0TrivialMove()` instead of expanding files with Constraint 3 More details can be found in `LevelCompactionBuilder::SetupOtherFilesWithRoundRobinExpansion()`. The above optimization accelerates the process of moving the compaction cursor, in which the write-amp can be further reduced. While a large compaction may lead to high write stall, we break this large compaction into several subcompactions **regardless of** the `max_subcompactions` limit. The number of subcompactions for round-robin compaction priority is determined through the following steps: * Step 1: Initialized against `max_output_file_limit`, the number of input files in the start level, and also the range size limit `ranges.size()` * Step 2: Call `AcquireSubcompactionResources()`when max subcompactions is not sufficient, but we may or may not obtain desired resources, additional number of resources is stored in `extra_num_subcompaction_threads_reserved_`). Subcompaction limit is changed and update `num_planned_subcompactions` with `GetSubcompactionLimit()` * Step 3: Call `ShrinkSubcompactionResources()` to ensure extra resources can be released (extra resources may exist for round-robin compaction when the number of actual number of subcompactions is less than the number of planned subcompactions) More details can be found in `CompactionJob::AcquireSubcompactionResources()`,`CompactionJob::ShrinkSubcompactionResources()`, and `CompactionJob::ReleaseSubcompactionResources()`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10341 Test Plan: Add `CompactionPriMultipleFilesRoundRobin[1-3]` unit test in `compaction_picker_test.cc` and `RoundRobinSubcompactionsAgainstResources.SubcompactionsUsingResources/[0-4]`, `RoundRobinSubcompactionsAgainstPressureToken.PressureTokenTest/[0-1]` in `db_compaction_test.cc` Reviewed By: ajkr, hx235 Differential Revision: D37792644 Pulled By: littlepig2013 fbshipit-source-id: 7fecb7c4ffd97b34bbf6e3b760b2c35a772a0657
2 years ago
// Update the compact cursor and advance the file index using increment
// so that it can point to the next cursor (increment means the number of
// input files in this level of the last compaction)
const InternalKey& GetNextCompactCursor(int level, size_t increment) {
int cmp_idx = next_file_to_compact_by_size_[level] + (int)increment;
assert(cmp_idx <= (int)files_by_compaction_pri_[level].size());
Add basic kRoundRobin compaction policy (#10107) Summary: Add `kRoundRobin` as a compaction priority. The implementation is as follows. - Define a cursor as the smallest Internal key in the successor of the selected file. Add `vector<InternalKey> compact_cursor_` into `VersionStorageInfo` where each element (`InternalKey`) in `compact_cursor_` represents a cursor. In round-robin compaction policy, we just need to select the first file (assuming files are sorted) and also has the smallest InternalKey larger than/equal to the cursor. After a file is chosen, we create a new `Fsize` vector which puts the selected file is placed at the first position in `temp`, the next cursor is then updated as the smallest InternalKey in successor of the selected file (the above logic is implemented in `SortFileByRoundRobin`). - After a compaction succeeds, typically `InstallCompactionResults()`, we choose the next cursor for the input level and save it to `edit`. When calling `LogAndApply`, we save the next cursor with its level into some local variable and finally apply the change to `vstorage` in `SaveTo` function. - Cursors are persist pair by pair (<level, InternalKey>) in `EncodeTo` so that they can be reconstructed when reopening. An empty cursor will not be encoded to MANIFEST Pull Request resolved: https://github.com/facebook/rocksdb/pull/10107 Test Plan: add unit test (`CompactionPriRoundRobin`) in `compaction_picker_test`, add `kRoundRobin` priority in `CompactionPriTest` from `db_compaction_test`, and add `PersistRoundRobinCompactCursor` in `db_compaction_test` Reviewed By: ajkr Differential Revision: D37316037 Pulled By: littlepig2013 fbshipit-source-id: 9f481748190ace416079139044e00df2968fb1ee
3 years ago
// TODO(zichen): may need to update next_file_to_compact_by_size_
// for parallel compaction.
InternalKey new_cursor;
if (cmp_idx >= (int)files_by_compaction_pri_[level].size()) {
cmp_idx = 0;
}
// TODO(zichen): rethink if this strategy gives us some good guarantee
return files_[level][files_by_compaction_pri_[level][cmp_idx]]->smallest;
}
Use a sorted vector instead of a map to store blob file metadata (#9526) Summary: The patch replaces `std::map` with a sorted `std::vector` for `VersionStorageInfo::blob_files_` and preallocates the space for the `vector` before saving the `BlobFileMetaData` into the new `VersionStorageInfo` in `VersionBuilder::Rep::SaveBlobFilesTo`. These changes reduce the time the DB mutex is held while saving new `Version`s, and using a sorted `vector` also makes lookups faster thanks to better memory locality. In addition, the patch introduces helper methods `VersionStorageInfo::GetBlobFileMetaData` and `VersionStorageInfo::GetBlobFileMetaDataLB` that can be used by clients to perform lookups in the `vector`, and does some general cleanup in the parts of code where blob file metadata are used. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9526 Test Plan: Ran `make check` and the crash test script for a while. Performance was tested using a load-optimized benchmark (`fillseq` with vector memtable, no WAL) and small file sizes so that a significant number of files are produced: ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --memtablerep=skip_list --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --min_level_to_compress=3 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --allow_concurrent_memtable_write=false --disable_wal=1 --enable_blob_files=1 --blob_file_size=16777216 --min_blob_size=0 --blob_compression_type=lz4 --enable_blob_garbage_collection=1 --seed=<some value> ``` Final statistics before the patch: ``` Cumulative writes: 0 writes, 700M keys, 0 commit groups, 0.0 writes per commit group, ingest: 284.62 GB, 121.27 MB/s Interval writes: 0 writes, 334K keys, 0 commit groups, 0.0 writes per commit group, ingest: 139.28 MB, 72.46 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 760M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.66 GB, 131.52 MB/s Interval writes: 0 writes, 445K keys, 0 commit groups, 0.0 writes per commit group, ingest: 185.35 MB, 93.15 MB/s ``` Total time to complete the benchmark is 2611 seconds with the patch, down from 2986 secs. Reviewed By: riversand963 Differential Revision: D34082728 Pulled By: ltamasi fbshipit-source-id: fc598abf676dce436734d06bb9d2d99a26a004fc
3 years ago
void ReserveBlob(size_t size) { blob_files_.reserve(size); }
Add blob files to VersionStorageInfo/VersionBuilder (#6597) Summary: The patch adds a couple of classes to represent metadata about blob files: `SharedBlobFileMetaData` contains the information elements that are immutable (once the blob file is closed), e.g. blob file number, total number and size of blob files, checksum method/value, while `BlobFileMetaData` contains attributes that can vary across versions like the amount of garbage in the file. There is a single `SharedBlobFileMetaData` for each blob file, which is jointly owned by the `BlobFileMetaData` objects that point to it; `BlobFileMetaData` objects, in turn, are owned by `Version`s and can also be shared if the (immutable _and_ mutable) state of the blob file is the same in two versions. In addition, the patch adds the blob file metadata to `VersionStorageInfo`, and extends `VersionBuilder` so that it can apply blob file related `VersionEdit`s (i.e. those containing `BlobFileAddition`s and/or `BlobFileGarbage`), and save blob file metadata to a new `VersionStorageInfo`. Consistency checks are also extended to ensure that table files point to blob files that are part of the `Version`, and that all blob files that are part of any given `Version` have at least some _non_-garbage data in them. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6597 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D20656803 Pulled By: ltamasi fbshipit-source-id: f1f74d135045b3b42d0146f03ee576ef0a4bfd80
5 years ago
void AddBlobFile(std::shared_ptr<BlobFileMetaData> blob_file_meta);
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
void PrepareForVersionAppend(const ImmutableOptions& immutable_options,
const MutableCFOptions& mutable_cf_options);
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: PrepareForVersionAppend has been called
void SetFinalized();
// Update the accumulated stats from a file-meta.
void UpdateAccumulatedStats(FileMetaData* file_meta);
// Decrease the current stat from a to-be-deleted file-meta
void RemoveCurrentStats(FileMetaData* file_meta);
// Updates internal structures that keep track of compaction scores
// We use compaction scores to figure out which compaction to do next
// REQUIRES: db_mutex held!!
// TODO find a better way to pass compaction_options_fifo.
void ComputeCompactionScore(const ImmutableOptions& immutable_options,
const MutableCFOptions& mutable_cf_options);
// Estimate est_comp_needed_bytes_
void EstimateCompactionBytesNeeded(
const MutableCFOptions& mutable_cf_options);
// This computes files_marked_for_compaction_ and is called by
// ComputeCompactionScore()
void ComputeFilesMarkedForCompaction();
// This computes ttl_expired_files_ and is called by
// ComputeCompactionScore()
void ComputeExpiredTtlFiles(const ImmutableOptions& ioptions,
const uint64_t ttl);
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
6 years ago
// This computes files_marked_for_periodic_compaction_ and is called by
// ComputeCompactionScore()
void ComputeFilesMarkedForPeriodicCompaction(
const ImmutableOptions& ioptions,
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
6 years ago
const uint64_t periodic_compaction_seconds);
single-file bottom-level compaction when snapshot released Summary: When snapshots are held for a long time, files may reach the bottom level containing overwritten/deleted keys. We previously had no mechanism to trigger compaction on such files. This particularly impacted DBs that write to different parts of the keyspace over time, as such files would never be naturally compacted due to second-last level files moving down. This PR introduces a mechanism for bottommost files to be recompacted upon releasing all snapshots that prevent them from dropping their deleted/overwritten keys. - Changed `CompactionPicker` to compact files in `BottommostFilesMarkedForCompaction()`. These are the last choice when picking. Each file will be compacted alone and output to the same level in which it originated. The goal of this type of compaction is to rewrite the data excluding deleted/overwritten keys. - Changed `ReleaseSnapshot()` to recompute the bottom files marked for compaction when the oldest existing snapshot changes, and schedule a compaction if needed. We cache the value that oldest existing snapshot needs to exceed in order for another file to be marked in `bottommost_files_mark_threshold_`, which allows us to avoid recomputing marked files for most snapshot releases. - Changed `VersionStorageInfo` to track the list of bottommost files, which is recomputed every time the version changes by `UpdateBottommostFiles()`. The list of marked bottommost files is first computed in `ComputeBottommostFilesMarkedForCompaction()` when the version changes, but may also be recomputed when `ReleaseSnapshot()` is called. - Extracted core logic of `Compaction::IsBottommostLevel()` into `VersionStorageInfo::RangeMightExistAfterSortedRun()` since logic to check whether a file is bottommost is now necessary outside of compaction. Closes https://github.com/facebook/rocksdb/pull/3009 Differential Revision: D6062044 Pulled By: ajkr fbshipit-source-id: 123d201cf140715a7d5928e8b3cb4f9cd9f7ad21
7 years ago
// This computes bottommost_files_marked_for_compaction_ and is called by
// ComputeCompactionScore() or UpdateOldestSnapshot().
//
// Among bottommost files (assumes they've already been computed), marks the
// ones that have keys that would be eliminated if recompacted, according to
// the seqnum of the oldest existing snapshot. Must be called every time
// oldest snapshot changes as that is when bottom-level files can become
// eligible for compaction.
//
// REQUIRES: DB mutex held
void ComputeBottommostFilesMarkedForCompaction();
Make it possible to force the garbage collection of the oldest blob files (#8994) Summary: The current BlobDB garbage collection logic works by relocating the valid blobs from the oldest blob files as they are encountered during compaction, and cleaning up blob files once they contain nothing but garbage. However, with sufficiently skewed workloads, it is theoretically possible to end up in a situation when few or no compactions get scheduled for the SST files that contain references to the oldest blob files, which can lead to increased space amp due to the lack of GC. In order to efficiently handle such workloads, the patch adds a new BlobDB configuration option called `blob_garbage_collection_force_threshold`, which signals to BlobDB to schedule targeted compactions for the SST files that keep alive the oldest batch of blob files if the overall ratio of garbage in the given blob files meets the threshold *and* all the given blob files are eligible for GC based on `blob_garbage_collection_age_cutoff`. (For example, if the new option is set to 0.9, targeted compactions will get scheduled if the sum of garbage bytes meets or exceeds 90% of the sum of total bytes in the oldest blob files, assuming all affected blob files are below the age-based cutoff.) The net result of these targeted compactions is that the valid blobs in the oldest blob files are relocated and the oldest blob files themselves cleaned up (since *all* SST files that rely on them get compacted away). These targeted compactions are similar to periodic compactions in the sense that they force certain SST files that otherwise would not get picked up to undergo compaction and also in the sense that instead of merging files from multiple levels, they target a single file. (Note: such compactions might still include neighboring files from the same level due to the need of having a "clean cut" boundary but they never include any files from any other level.) This functionality is currently only supported with the leveled compaction style and is inactive by default (since the default value is set to 1.0, i.e. 100%). Pull Request resolved: https://github.com/facebook/rocksdb/pull/8994 Test Plan: Ran `make check` and tested using `db_bench` and the stress/crash tests. Reviewed By: riversand963 Differential Revision: D31489850 Pulled By: ltamasi fbshipit-source-id: 44057d511726a0e2a03c5d9313d7511b3f0c4eab
3 years ago
// This computes files_marked_for_forced_blob_gc_ and is called by
// ComputeCompactionScore()
//
// REQUIRES: DB mutex held
void ComputeFilesMarkedForForcedBlobGC(
double blob_garbage_collection_age_cutoff,
double blob_garbage_collection_force_threshold);
bool level0_non_overlapping() const { return level0_non_overlapping_; }
Allowing L0 -> L1 trivial move on sorted data Summary: This diff updates the logic of how we do trivial move, now trivial move can run on any number of files in input level as long as they are not overlapping The conditions for trivial move have been updated Introduced conditions: - Trivial move cannot happen if we have a compaction filter (except if the compaction is not manual) - Input level files cannot be overlapping Removed conditions: - Trivial move only run when the compaction is not manual - Input level should can contain only 1 file More context on what tests failed because of Trivial move ``` DBTest.CompactionsGenerateMultipleFiles This test is expecting compaction on a file in L0 to generate multiple files in L1, this test will fail with trivial move because we end up with one file in L1 ``` ``` DBTest.NoSpaceCompactRange This test expect compaction to fail when we force environment to report running out of space, of course this is not valid in trivial move situation because trivial move does not need any extra space, and did not check for that ``` ``` DBTest.DropWrites Similar to DBTest.NoSpaceCompactRange ``` ``` DBTest.DeleteObsoleteFilesPendingOutputs This test expect that a file in L2 is deleted after it's moved to L3, this is not valid with trivial move because although the file was moved it is now used by L3 ``` ``` CuckooTableDBTest.CompactionIntoMultipleFiles Same as DBTest.CompactionsGenerateMultipleFiles ``` This diff is based on a work by @sdong https://reviews.facebook.net/D34149 Test Plan: make -j64 check Reviewers: rven, sdong, igor Reviewed By: igor Subscribers: yhchiang, ott, march, dhruba, sdong Differential Revision: https://reviews.facebook.net/D34797
10 years ago
single-file bottom-level compaction when snapshot released Summary: When snapshots are held for a long time, files may reach the bottom level containing overwritten/deleted keys. We previously had no mechanism to trigger compaction on such files. This particularly impacted DBs that write to different parts of the keyspace over time, as such files would never be naturally compacted due to second-last level files moving down. This PR introduces a mechanism for bottommost files to be recompacted upon releasing all snapshots that prevent them from dropping their deleted/overwritten keys. - Changed `CompactionPicker` to compact files in `BottommostFilesMarkedForCompaction()`. These are the last choice when picking. Each file will be compacted alone and output to the same level in which it originated. The goal of this type of compaction is to rewrite the data excluding deleted/overwritten keys. - Changed `ReleaseSnapshot()` to recompute the bottom files marked for compaction when the oldest existing snapshot changes, and schedule a compaction if needed. We cache the value that oldest existing snapshot needs to exceed in order for another file to be marked in `bottommost_files_mark_threshold_`, which allows us to avoid recomputing marked files for most snapshot releases. - Changed `VersionStorageInfo` to track the list of bottommost files, which is recomputed every time the version changes by `UpdateBottommostFiles()`. The list of marked bottommost files is first computed in `ComputeBottommostFilesMarkedForCompaction()` when the version changes, but may also be recomputed when `ReleaseSnapshot()` is called. - Extracted core logic of `Compaction::IsBottommostLevel()` into `VersionStorageInfo::RangeMightExistAfterSortedRun()` since logic to check whether a file is bottommost is now necessary outside of compaction. Closes https://github.com/facebook/rocksdb/pull/3009 Differential Revision: D6062044 Pulled By: ajkr fbshipit-source-id: 123d201cf140715a7d5928e8b3cb4f9cd9f7ad21
7 years ago
// Updates the oldest snapshot and related internal state, like the bottommost
// files marked for compaction.
// REQUIRES: DB mutex held
void UpdateOldestSnapshot(SequenceNumber oldest_snapshot_seqnum);
int MaxInputLevel() const;
Introduce bottom-pri thread pool for large universal compactions Summary: When we had a single thread pool for compactions, a thread could be busy for a long time (minutes) executing a compaction involving the bottom level. In multi-instance setups, the entire thread pool could be consumed by such bottom-level compactions. Then, top-level compactions (e.g., a few L0 files) would be blocked for a long time ("head-of-line blocking"). Such top-level compactions are critical to prevent compaction stalls as they can quickly reduce number of L0 files / sorted runs. This diff introduces a bottom-priority queue for universal compactions including the bottom level. This alleviates the head-of-line blocking situation for fast, top-level compactions. - Added `Env::Priority::BOTTOM` thread pool. This feature is only enabled if user explicitly configures it to have a positive number of threads. - Changed `ThreadPoolImpl`'s default thread limit from one to zero. This change is invisible to users as we call `IncBackgroundThreadsIfNeeded` on the low-pri/high-pri pools during `DB::Open` with values of at least one. It is necessary, though, for bottom-pri to start with zero threads so the feature is disabled by default. - Separated `ManualCompaction` into two parts in `PrepickedCompaction`. `PrepickedCompaction` is used for any compaction that's picked outside of its execution thread, either manual or automatic. - Forward universal compactions involving last level to the bottom pool (worker thread's entry point is `BGWorkBottomCompaction`). - Track `bg_bottom_compaction_scheduled_` so we can wait for bottom-level compactions to finish. We don't count them against the background jobs limits. So users of this feature will get an extra compaction for free. Closes https://github.com/facebook/rocksdb/pull/2580 Differential Revision: D5422916 Pulled By: ajkr fbshipit-source-id: a74bd11f1ea4933df3739b16808bb21fcd512333
7 years ago
int MaxOutputLevel(bool allow_ingest_behind) const;
// Return level number that has idx'th highest score
int CompactionScoreLevel(int idx) const { return compaction_level_[idx]; }
// Return idx'th highest score
double CompactionScore(int idx) const { return compaction_score_[idx]; }
void GetOverlappingInputs(
int level, const InternalKey* begin, // nullptr means before all keys
const InternalKey* end, // nullptr means after all keys
std::vector<FileMetaData*>* inputs,
int hint_index = -1, // index of overlap file
int* file_index = nullptr, // return index of overlap file
bool expand_range = true, // if set, returns files which overlap the
// range and overlap each other. If false,
// then just files intersecting the range
InternalKey** next_smallest = nullptr) // if non-null, returns the
const; // smallest key of next file not included
void GetCleanInputsWithinInterval(
int level, const InternalKey* begin, // nullptr means before all keys
const InternalKey* end, // nullptr means after all keys
std::vector<FileMetaData*>* inputs,
int hint_index = -1, // index of overlap file
int* file_index = nullptr) // return index of overlap file
const;
void GetOverlappingInputsRangeBinarySearch(
int level, // level > 0
const InternalKey* begin, // nullptr means before all keys
const InternalKey* end, // nullptr means after all keys
std::vector<FileMetaData*>* inputs,
int hint_index, // index of overlap file
int* file_index, // return index of overlap file
bool within_interval = false, // if set, force the inputs within interval
InternalKey** next_smallest = nullptr) // if non-null, returns the
const; // smallest key of next file not included
// Returns true iff some file in the specified level overlaps
// some part of [*smallest_user_key,*largest_user_key].
// smallest_user_key==NULL represents a key smaller than all keys in the DB.
// largest_user_key==NULL represents a key largest than all keys in the DB.
bool OverlapInLevel(int level, const Slice* smallest_user_key,
const Slice* largest_user_key);
[RocksDB] [MergeOperator] The new Merge Interface! Uses merge sequences. Summary: Here are the major changes to the Merge Interface. It has been expanded to handle cases where the MergeOperator is not associative. It does so by stacking up merge operations while scanning through the key history (i.e.: during Get() or Compaction), until a valid Put/Delete/end-of-history is encountered; it then applies all of the merge operations in the correct sequence starting with the base/sentinel value. I have also introduced an "AssociativeMerge" function which allows the user to take advantage of associative merge operations (such as in the case of counters). The implementation will always attempt to merge the operations/operands themselves together when they are encountered, and will resort to the "stacking" method if and only if the "associative-merge" fails. This implementation is conjectured to allow MergeOperator to handle the general case, while still providing the user with the ability to take advantage of certain efficiencies in their own merge-operator / data-structure. NOTE: This is a preliminary diff. This must still go through a lot of review, revision, and testing. Feedback welcome! Test Plan: -This is a preliminary diff. I have only just begun testing/debugging it. -I will be testing this with the existing MergeOperator use-cases and unit-tests (counters, string-append, and redis-lists) -I will be "desk-checking" and walking through the code with the help gdb. -I will find a way of stress-testing the new interface / implementation using db_bench, db_test, merge_test, and/or db_stress. -I will ensure that my tests cover all cases: Get-Memtable, Get-Immutable-Memtable, Get-from-Disk, Iterator-Range-Scan, Flush-Memtable-to-L0, Compaction-L0-L1, Compaction-Ln-L(n+1), Put/Delete found, Put/Delete not-found, end-of-history, end-of-file, etc. -A lot of feedback from the reviewers. Reviewers: haobo, dhruba, zshao, emayanke Reviewed By: haobo CC: leveldb Differential Revision: https://reviews.facebook.net/D11499
12 years ago
// Returns true iff the first or last file in inputs contains
// an overlapping user key to the file "just outside" of it (i.e.
// just after the last file, or just before the first file)
// REQUIRES: "*inputs" is a sorted list of non-overlapping files
bool HasOverlappingUserKey(const std::vector<FileMetaData*>* inputs,
int level);
int num_levels() const { return num_levels_; }
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: PrepareForVersionAppend has been called
int num_non_empty_levels() const {
assert(finalized_);
return num_non_empty_levels_;
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: PrepareForVersionAppend has been called
// This may or may not return number of level files. It is to keep backward
// compatible behavior in universal compaction.
int l0_delay_trigger_count() const { return l0_delay_trigger_count_; }
void set_l0_delay_trigger_count(int v) { l0_delay_trigger_count_ = v; }
Use a sorted vector instead of a map to store blob file metadata (#9526) Summary: The patch replaces `std::map` with a sorted `std::vector` for `VersionStorageInfo::blob_files_` and preallocates the space for the `vector` before saving the `BlobFileMetaData` into the new `VersionStorageInfo` in `VersionBuilder::Rep::SaveBlobFilesTo`. These changes reduce the time the DB mutex is held while saving new `Version`s, and using a sorted `vector` also makes lookups faster thanks to better memory locality. In addition, the patch introduces helper methods `VersionStorageInfo::GetBlobFileMetaData` and `VersionStorageInfo::GetBlobFileMetaDataLB` that can be used by clients to perform lookups in the `vector`, and does some general cleanup in the parts of code where blob file metadata are used. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9526 Test Plan: Ran `make check` and the crash test script for a while. Performance was tested using a load-optimized benchmark (`fillseq` with vector memtable, no WAL) and small file sizes so that a significant number of files are produced: ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --memtablerep=skip_list --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --min_level_to_compress=3 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --allow_concurrent_memtable_write=false --disable_wal=1 --enable_blob_files=1 --blob_file_size=16777216 --min_blob_size=0 --blob_compression_type=lz4 --enable_blob_garbage_collection=1 --seed=<some value> ``` Final statistics before the patch: ``` Cumulative writes: 0 writes, 700M keys, 0 commit groups, 0.0 writes per commit group, ingest: 284.62 GB, 121.27 MB/s Interval writes: 0 writes, 334K keys, 0 commit groups, 0.0 writes per commit group, ingest: 139.28 MB, 72.46 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 760M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.66 GB, 131.52 MB/s Interval writes: 0 writes, 445K keys, 0 commit groups, 0.0 writes per commit group, ingest: 185.35 MB, 93.15 MB/s ``` Total time to complete the benchmark is 2611 seconds with the patch, down from 2986 secs. Reviewed By: riversand963 Differential Revision: D34082728 Pulled By: ltamasi fbshipit-source-id: fc598abf676dce436734d06bb9d2d99a26a004fc
3 years ago
// REQUIRES: This version has been saved (see VersionBuilder::SaveTo)
int NumLevelFiles(int level) const {
assert(finalized_);
return static_cast<int>(files_[level].size());
}
// Return the combined file size of all files at the specified level.
uint64_t NumLevelBytes(int level) const;
Use a sorted vector instead of a map to store blob file metadata (#9526) Summary: The patch replaces `std::map` with a sorted `std::vector` for `VersionStorageInfo::blob_files_` and preallocates the space for the `vector` before saving the `BlobFileMetaData` into the new `VersionStorageInfo` in `VersionBuilder::Rep::SaveBlobFilesTo`. These changes reduce the time the DB mutex is held while saving new `Version`s, and using a sorted `vector` also makes lookups faster thanks to better memory locality. In addition, the patch introduces helper methods `VersionStorageInfo::GetBlobFileMetaData` and `VersionStorageInfo::GetBlobFileMetaDataLB` that can be used by clients to perform lookups in the `vector`, and does some general cleanup in the parts of code where blob file metadata are used. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9526 Test Plan: Ran `make check` and the crash test script for a while. Performance was tested using a load-optimized benchmark (`fillseq` with vector memtable, no WAL) and small file sizes so that a significant number of files are produced: ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --memtablerep=skip_list --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --min_level_to_compress=3 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --allow_concurrent_memtable_write=false --disable_wal=1 --enable_blob_files=1 --blob_file_size=16777216 --min_blob_size=0 --blob_compression_type=lz4 --enable_blob_garbage_collection=1 --seed=<some value> ``` Final statistics before the patch: ``` Cumulative writes: 0 writes, 700M keys, 0 commit groups, 0.0 writes per commit group, ingest: 284.62 GB, 121.27 MB/s Interval writes: 0 writes, 334K keys, 0 commit groups, 0.0 writes per commit group, ingest: 139.28 MB, 72.46 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 760M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.66 GB, 131.52 MB/s Interval writes: 0 writes, 445K keys, 0 commit groups, 0.0 writes per commit group, ingest: 185.35 MB, 93.15 MB/s ``` Total time to complete the benchmark is 2611 seconds with the patch, down from 2986 secs. Reviewed By: riversand963 Differential Revision: D34082728 Pulled By: ltamasi fbshipit-source-id: fc598abf676dce436734d06bb9d2d99a26a004fc
3 years ago
// REQUIRES: This version has been saved (see VersionBuilder::SaveTo)
const std::vector<FileMetaData*>& LevelFiles(int level) const {
return files_[level];
}
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
bool HasMissingEpochNumber() const;
uint64_t GetMaxEpochNumberOfFiles() const;
EpochNumberRequirement GetEpochNumberRequirement() const {
return epoch_number_requirement_;
}
void SetEpochNumberRequirement(
EpochNumberRequirement epoch_number_requirement) {
epoch_number_requirement_ = epoch_number_requirement;
}
void RecoverEpochNumbers(ColumnFamilyData* cfd);
class FileLocation {
public:
FileLocation() = default;
FileLocation(int level, size_t position)
: level_(level), position_(position) {}
int GetLevel() const { return level_; }
size_t GetPosition() const { return position_; }
bool IsValid() const { return level_ >= 0; }
bool operator==(const FileLocation& rhs) const {
return level_ == rhs.level_ && position_ == rhs.position_;
}
bool operator!=(const FileLocation& rhs) const { return !(*this == rhs); }
static FileLocation Invalid() { return FileLocation(); }
private:
int level_ = -1;
size_t position_ = 0;
};
Mitigate the overhead of building the hash of file locations (#9504) Summary: The patch builds on the refactoring done in https://github.com/facebook/rocksdb/issues/9494 and improves the performance of building the hash of file locations in `VersionStorageInfo` in two ways. First, the hash building is moved from `AddFile` (which is called under the DB mutex) to a separate post-processing step done as part of `PrepareForVersionAppend` (during which the mutex is *not* held). Second, the space necessary for the hash is preallocated to prevent costly reallocation/rehashing operations. These changes mitigate the overhead of the file location hash, which can be significant with certain workloads where the baseline CPU usage is low (see https://github.com/facebook/rocksdb/issues/9351, which is a workload where keys are sorted, WAL is turned off, the vector memtable implementation is used, and there are lots of small SST files). Fixes https://github.com/facebook/rocksdb/issues/9351 Pull Request resolved: https://github.com/facebook/rocksdb/pull/9504 Test Plan: `make check` ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --disable_wal=1 --seed=<some_seed> ``` Final statistics before this patch: ``` Cumulative writes: 0 writes, 697M keys, 0 commit groups, 0.0 writes per commit group, ingest: 283.25 GB, 241.08 MB/s Interval writes: 0 writes, 1264K keys, 0 commit groups, 0.0 writes per commit group, ingest: 525.69 MB, 176.67 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 759M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.57 GB, 262.63 MB/s Interval writes: 0 writes, 1555K keys, 0 commit groups, 0.0 writes per commit group, ingest: 646.61 MB, 215.11 MB/s ``` Reviewed By: riversand963 Differential Revision: D34014734 Pulled By: ltamasi fbshipit-source-id: acb2703677451d5ccaa7e9d950844b33d240695b
3 years ago
// REQUIRES: PrepareForVersionAppend has been called
FileLocation GetFileLocation(uint64_t file_number) const {
const auto it = file_locations_.find(file_number);
if (it == file_locations_.end()) {
return FileLocation::Invalid();
}
assert(it->second.GetLevel() < num_levels_);
assert(it->second.GetPosition() < files_[it->second.GetLevel()].size());
assert(files_[it->second.GetLevel()][it->second.GetPosition()]);
assert(files_[it->second.GetLevel()][it->second.GetPosition()]
->fd.GetNumber() == file_number);
return it->second;
}
Mitigate the overhead of building the hash of file locations (#9504) Summary: The patch builds on the refactoring done in https://github.com/facebook/rocksdb/issues/9494 and improves the performance of building the hash of file locations in `VersionStorageInfo` in two ways. First, the hash building is moved from `AddFile` (which is called under the DB mutex) to a separate post-processing step done as part of `PrepareForVersionAppend` (during which the mutex is *not* held). Second, the space necessary for the hash is preallocated to prevent costly reallocation/rehashing operations. These changes mitigate the overhead of the file location hash, which can be significant with certain workloads where the baseline CPU usage is low (see https://github.com/facebook/rocksdb/issues/9351, which is a workload where keys are sorted, WAL is turned off, the vector memtable implementation is used, and there are lots of small SST files). Fixes https://github.com/facebook/rocksdb/issues/9351 Pull Request resolved: https://github.com/facebook/rocksdb/pull/9504 Test Plan: `make check` ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --disable_wal=1 --seed=<some_seed> ``` Final statistics before this patch: ``` Cumulative writes: 0 writes, 697M keys, 0 commit groups, 0.0 writes per commit group, ingest: 283.25 GB, 241.08 MB/s Interval writes: 0 writes, 1264K keys, 0 commit groups, 0.0 writes per commit group, ingest: 525.69 MB, 176.67 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 759M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.57 GB, 262.63 MB/s Interval writes: 0 writes, 1555K keys, 0 commit groups, 0.0 writes per commit group, ingest: 646.61 MB, 215.11 MB/s ``` Reviewed By: riversand963 Differential Revision: D34014734 Pulled By: ltamasi fbshipit-source-id: acb2703677451d5ccaa7e9d950844b33d240695b
3 years ago
// REQUIRES: PrepareForVersionAppend has been called
FileMetaData* GetFileMetaDataByNumber(uint64_t file_number) const {
auto location = GetFileLocation(file_number);
if (!location.IsValid()) {
return nullptr;
}
return files_[location.GetLevel()][location.GetPosition()];
}
Use a sorted vector instead of a map to store blob file metadata (#9526) Summary: The patch replaces `std::map` with a sorted `std::vector` for `VersionStorageInfo::blob_files_` and preallocates the space for the `vector` before saving the `BlobFileMetaData` into the new `VersionStorageInfo` in `VersionBuilder::Rep::SaveBlobFilesTo`. These changes reduce the time the DB mutex is held while saving new `Version`s, and using a sorted `vector` also makes lookups faster thanks to better memory locality. In addition, the patch introduces helper methods `VersionStorageInfo::GetBlobFileMetaData` and `VersionStorageInfo::GetBlobFileMetaDataLB` that can be used by clients to perform lookups in the `vector`, and does some general cleanup in the parts of code where blob file metadata are used. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9526 Test Plan: Ran `make check` and the crash test script for a while. Performance was tested using a load-optimized benchmark (`fillseq` with vector memtable, no WAL) and small file sizes so that a significant number of files are produced: ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --memtablerep=skip_list --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --min_level_to_compress=3 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --allow_concurrent_memtable_write=false --disable_wal=1 --enable_blob_files=1 --blob_file_size=16777216 --min_blob_size=0 --blob_compression_type=lz4 --enable_blob_garbage_collection=1 --seed=<some value> ``` Final statistics before the patch: ``` Cumulative writes: 0 writes, 700M keys, 0 commit groups, 0.0 writes per commit group, ingest: 284.62 GB, 121.27 MB/s Interval writes: 0 writes, 334K keys, 0 commit groups, 0.0 writes per commit group, ingest: 139.28 MB, 72.46 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 760M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.66 GB, 131.52 MB/s Interval writes: 0 writes, 445K keys, 0 commit groups, 0.0 writes per commit group, ingest: 185.35 MB, 93.15 MB/s ``` Total time to complete the benchmark is 2611 seconds with the patch, down from 2986 secs. Reviewed By: riversand963 Differential Revision: D34082728 Pulled By: ltamasi fbshipit-source-id: fc598abf676dce436734d06bb9d2d99a26a004fc
3 years ago
// REQUIRES: This version has been saved (see VersionBuilder::SaveTo)
using BlobFiles = std::vector<std::shared_ptr<BlobFileMetaData>>;
Add blob files to VersionStorageInfo/VersionBuilder (#6597) Summary: The patch adds a couple of classes to represent metadata about blob files: `SharedBlobFileMetaData` contains the information elements that are immutable (once the blob file is closed), e.g. blob file number, total number and size of blob files, checksum method/value, while `BlobFileMetaData` contains attributes that can vary across versions like the amount of garbage in the file. There is a single `SharedBlobFileMetaData` for each blob file, which is jointly owned by the `BlobFileMetaData` objects that point to it; `BlobFileMetaData` objects, in turn, are owned by `Version`s and can also be shared if the (immutable _and_ mutable) state of the blob file is the same in two versions. In addition, the patch adds the blob file metadata to `VersionStorageInfo`, and extends `VersionBuilder` so that it can apply blob file related `VersionEdit`s (i.e. those containing `BlobFileAddition`s and/or `BlobFileGarbage`), and save blob file metadata to a new `VersionStorageInfo`. Consistency checks are also extended to ensure that table files point to blob files that are part of the `Version`, and that all blob files that are part of any given `Version` have at least some _non_-garbage data in them. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6597 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D20656803 Pulled By: ltamasi fbshipit-source-id: f1f74d135045b3b42d0146f03ee576ef0a4bfd80
5 years ago
const BlobFiles& GetBlobFiles() const { return blob_files_; }
Use a sorted vector instead of a map to store blob file metadata (#9526) Summary: The patch replaces `std::map` with a sorted `std::vector` for `VersionStorageInfo::blob_files_` and preallocates the space for the `vector` before saving the `BlobFileMetaData` into the new `VersionStorageInfo` in `VersionBuilder::Rep::SaveBlobFilesTo`. These changes reduce the time the DB mutex is held while saving new `Version`s, and using a sorted `vector` also makes lookups faster thanks to better memory locality. In addition, the patch introduces helper methods `VersionStorageInfo::GetBlobFileMetaData` and `VersionStorageInfo::GetBlobFileMetaDataLB` that can be used by clients to perform lookups in the `vector`, and does some general cleanup in the parts of code where blob file metadata are used. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9526 Test Plan: Ran `make check` and the crash test script for a while. Performance was tested using a load-optimized benchmark (`fillseq` with vector memtable, no WAL) and small file sizes so that a significant number of files are produced: ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --memtablerep=skip_list --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --min_level_to_compress=3 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --allow_concurrent_memtable_write=false --disable_wal=1 --enable_blob_files=1 --blob_file_size=16777216 --min_blob_size=0 --blob_compression_type=lz4 --enable_blob_garbage_collection=1 --seed=<some value> ``` Final statistics before the patch: ``` Cumulative writes: 0 writes, 700M keys, 0 commit groups, 0.0 writes per commit group, ingest: 284.62 GB, 121.27 MB/s Interval writes: 0 writes, 334K keys, 0 commit groups, 0.0 writes per commit group, ingest: 139.28 MB, 72.46 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 760M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.66 GB, 131.52 MB/s Interval writes: 0 writes, 445K keys, 0 commit groups, 0.0 writes per commit group, ingest: 185.35 MB, 93.15 MB/s ``` Total time to complete the benchmark is 2611 seconds with the patch, down from 2986 secs. Reviewed By: riversand963 Differential Revision: D34082728 Pulled By: ltamasi fbshipit-source-id: fc598abf676dce436734d06bb9d2d99a26a004fc
3 years ago
// REQUIRES: This version has been saved (see VersionBuilder::SaveTo)
BlobFiles::const_iterator GetBlobFileMetaDataLB(
uint64_t blob_file_number) const;
// REQUIRES: This version has been saved (see VersionBuilder::SaveTo)
std::shared_ptr<BlobFileMetaData> GetBlobFileMetaData(
uint64_t blob_file_number) const {
const auto it = GetBlobFileMetaDataLB(blob_file_number);
assert(it == blob_files_.end() || *it);
if (it != blob_files_.end() &&
(*it)->GetBlobFileNumber() == blob_file_number) {
return *it;
}
return std::shared_ptr<BlobFileMetaData>();
}
// REQUIRES: This version has been saved (see VersionBuilder::SaveTo)
struct BlobStats {
uint64_t total_file_size = 0;
uint64_t total_garbage_size = 0;
double space_amp = 0.0;
};
BlobStats GetBlobStats() const {
uint64_t total_file_size = 0;
uint64_t total_garbage_size = 0;
Use a sorted vector instead of a map to store blob file metadata (#9526) Summary: The patch replaces `std::map` with a sorted `std::vector` for `VersionStorageInfo::blob_files_` and preallocates the space for the `vector` before saving the `BlobFileMetaData` into the new `VersionStorageInfo` in `VersionBuilder::Rep::SaveBlobFilesTo`. These changes reduce the time the DB mutex is held while saving new `Version`s, and using a sorted `vector` also makes lookups faster thanks to better memory locality. In addition, the patch introduces helper methods `VersionStorageInfo::GetBlobFileMetaData` and `VersionStorageInfo::GetBlobFileMetaDataLB` that can be used by clients to perform lookups in the `vector`, and does some general cleanup in the parts of code where blob file metadata are used. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9526 Test Plan: Ran `make check` and the crash test script for a while. Performance was tested using a load-optimized benchmark (`fillseq` with vector memtable, no WAL) and small file sizes so that a significant number of files are produced: ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --memtablerep=skip_list --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --min_level_to_compress=3 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --allow_concurrent_memtable_write=false --disable_wal=1 --enable_blob_files=1 --blob_file_size=16777216 --min_blob_size=0 --blob_compression_type=lz4 --enable_blob_garbage_collection=1 --seed=<some value> ``` Final statistics before the patch: ``` Cumulative writes: 0 writes, 700M keys, 0 commit groups, 0.0 writes per commit group, ingest: 284.62 GB, 121.27 MB/s Interval writes: 0 writes, 334K keys, 0 commit groups, 0.0 writes per commit group, ingest: 139.28 MB, 72.46 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 760M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.66 GB, 131.52 MB/s Interval writes: 0 writes, 445K keys, 0 commit groups, 0.0 writes per commit group, ingest: 185.35 MB, 93.15 MB/s ``` Total time to complete the benchmark is 2611 seconds with the patch, down from 2986 secs. Reviewed By: riversand963 Differential Revision: D34082728 Pulled By: ltamasi fbshipit-source-id: fc598abf676dce436734d06bb9d2d99a26a004fc
3 years ago
for (const auto& meta : blob_files_) {
assert(meta);
total_file_size += meta->GetBlobFileSize();
total_garbage_size += meta->GetGarbageBlobBytes();
}
double space_amp = 0.0;
if (total_file_size > total_garbage_size) {
space_amp = static_cast<double>(total_file_size) /
(total_file_size - total_garbage_size);
}
return BlobStats{total_file_size, total_garbage_size, space_amp};
}
const ROCKSDB_NAMESPACE::LevelFilesBrief& LevelFilesBrief(int level) const {
assert(level < static_cast<int>(level_files_brief_.size()));
return level_files_brief_[level];
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: PrepareForVersionAppend has been called
const std::vector<int>& FilesByCompactionPri(int level) const {
assert(finalized_);
return files_by_compaction_pri_[level];
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: ComputeCompactionScore has been called
// REQUIRES: DB mutex held during access
const autovector<std::pair<int, FileMetaData*>>& FilesMarkedForCompaction()
const {
assert(finalized_);
return files_marked_for_compaction_;
}
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
void TEST_AddFileMarkedForCompaction(int level, FileMetaData* f) {
f->marked_for_compaction = true;
files_marked_for_compaction_.emplace_back(level, f);
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: ComputeCompactionScore has been called
// REQUIRES: DB mutex held during access
const autovector<std::pair<int, FileMetaData*>>& ExpiredTtlFiles() const {
assert(finalized_);
return expired_ttl_files_;
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: ComputeCompactionScore has been called
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
6 years ago
// REQUIRES: DB mutex held during access
const autovector<std::pair<int, FileMetaData*>>&
FilesMarkedForPeriodicCompaction() const {
assert(finalized_);
return files_marked_for_periodic_compaction_;
}
void TEST_AddFileMarkedForPeriodicCompaction(int level, FileMetaData* f) {
files_marked_for_periodic_compaction_.emplace_back(level, f);
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: ComputeCompactionScore has been called
single-file bottom-level compaction when snapshot released Summary: When snapshots are held for a long time, files may reach the bottom level containing overwritten/deleted keys. We previously had no mechanism to trigger compaction on such files. This particularly impacted DBs that write to different parts of the keyspace over time, as such files would never be naturally compacted due to second-last level files moving down. This PR introduces a mechanism for bottommost files to be recompacted upon releasing all snapshots that prevent them from dropping their deleted/overwritten keys. - Changed `CompactionPicker` to compact files in `BottommostFilesMarkedForCompaction()`. These are the last choice when picking. Each file will be compacted alone and output to the same level in which it originated. The goal of this type of compaction is to rewrite the data excluding deleted/overwritten keys. - Changed `ReleaseSnapshot()` to recompute the bottom files marked for compaction when the oldest existing snapshot changes, and schedule a compaction if needed. We cache the value that oldest existing snapshot needs to exceed in order for another file to be marked in `bottommost_files_mark_threshold_`, which allows us to avoid recomputing marked files for most snapshot releases. - Changed `VersionStorageInfo` to track the list of bottommost files, which is recomputed every time the version changes by `UpdateBottommostFiles()`. The list of marked bottommost files is first computed in `ComputeBottommostFilesMarkedForCompaction()` when the version changes, but may also be recomputed when `ReleaseSnapshot()` is called. - Extracted core logic of `Compaction::IsBottommostLevel()` into `VersionStorageInfo::RangeMightExistAfterSortedRun()` since logic to check whether a file is bottommost is now necessary outside of compaction. Closes https://github.com/facebook/rocksdb/pull/3009 Differential Revision: D6062044 Pulled By: ajkr fbshipit-source-id: 123d201cf140715a7d5928e8b3cb4f9cd9f7ad21
7 years ago
// REQUIRES: DB mutex held during access
const autovector<std::pair<int, FileMetaData*>>&
BottommostFilesMarkedForCompaction() const {
assert(finalized_);
return bottommost_files_marked_for_compaction_;
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: ComputeCompactionScore has been called
Make it possible to force the garbage collection of the oldest blob files (#8994) Summary: The current BlobDB garbage collection logic works by relocating the valid blobs from the oldest blob files as they are encountered during compaction, and cleaning up blob files once they contain nothing but garbage. However, with sufficiently skewed workloads, it is theoretically possible to end up in a situation when few or no compactions get scheduled for the SST files that contain references to the oldest blob files, which can lead to increased space amp due to the lack of GC. In order to efficiently handle such workloads, the patch adds a new BlobDB configuration option called `blob_garbage_collection_force_threshold`, which signals to BlobDB to schedule targeted compactions for the SST files that keep alive the oldest batch of blob files if the overall ratio of garbage in the given blob files meets the threshold *and* all the given blob files are eligible for GC based on `blob_garbage_collection_age_cutoff`. (For example, if the new option is set to 0.9, targeted compactions will get scheduled if the sum of garbage bytes meets or exceeds 90% of the sum of total bytes in the oldest blob files, assuming all affected blob files are below the age-based cutoff.) The net result of these targeted compactions is that the valid blobs in the oldest blob files are relocated and the oldest blob files themselves cleaned up (since *all* SST files that rely on them get compacted away). These targeted compactions are similar to periodic compactions in the sense that they force certain SST files that otherwise would not get picked up to undergo compaction and also in the sense that instead of merging files from multiple levels, they target a single file. (Note: such compactions might still include neighboring files from the same level due to the need of having a "clean cut" boundary but they never include any files from any other level.) This functionality is currently only supported with the leveled compaction style and is inactive by default (since the default value is set to 1.0, i.e. 100%). Pull Request resolved: https://github.com/facebook/rocksdb/pull/8994 Test Plan: Ran `make check` and tested using `db_bench` and the stress/crash tests. Reviewed By: riversand963 Differential Revision: D31489850 Pulled By: ltamasi fbshipit-source-id: 44057d511726a0e2a03c5d9313d7511b3f0c4eab
3 years ago
// REQUIRES: DB mutex held during access
const autovector<std::pair<int, FileMetaData*>>& FilesMarkedForForcedBlobGC()
const {
assert(finalized_);
return files_marked_for_forced_blob_gc_;
}
int base_level() const { return base_level_; }
double level_multiplier() const { return level_multiplier_; }
// REQUIRES: lock is held
// Set the index that is used to offset into files_by_compaction_pri_ to find
// the next compaction candidate file.
void SetNextCompactionIndex(int level, int index) {
next_file_to_compact_by_size_[level] = index;
}
// REQUIRES: lock is held
int NextCompactionIndex(int level) const {
return next_file_to_compact_by_size_[level];
}
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// REQUIRES: PrepareForVersionAppend has been called
const FileIndexer& file_indexer() const {
assert(finalized_);
return file_indexer_;
}
// Only the first few entries of files_by_compaction_pri_ are sorted.
// There is no need to sort all the files because it is likely
// that on a running system, we need to look at only the first
// few largest files because a new version is created every few
// seconds/minutes (because of concurrent compactions).
static const size_t kNumberFilesToSort = 50;
// Return a human-readable short (single-line) summary of the number
// of files per level. Uses *scratch as backing store.
struct LevelSummaryStorage {
char buffer[1000];
};
struct FileSummaryStorage {
char buffer[3000];
};
const char* LevelSummary(LevelSummaryStorage* scratch) const;
// Return a human-readable short (single-line) summary of files
// in a specified level. Uses *scratch as backing store.
const char* LevelFileSummary(FileSummaryStorage* scratch, int level) const;
// Return the maximum overlapping data (in bytes) at next level for any
// file at a level >= 1.
uint64_t MaxNextLevelOverlappingBytes();
// Return a human readable string that describes this version's contents.
std::string DebugString(bool hex = false) const;
uint64_t GetAverageValueSize() const {
if (accumulated_num_non_deletions_ == 0) {
return 0;
}
assert(accumulated_raw_key_size_ + accumulated_raw_value_size_ > 0);
assert(accumulated_file_size_ > 0);
return accumulated_raw_value_size_ / accumulated_num_non_deletions_ *
accumulated_file_size_ /
(accumulated_raw_key_size_ + accumulated_raw_value_size_);
}
uint64_t GetEstimatedActiveKeys() const;
double GetEstimatedCompressionRatioAtLevel(int level) const;
// re-initializes the index that is used to offset into
// files_by_compaction_pri_
// to find the next compaction candidate file.
void ResetNextCompactionIndex(int level) {
next_file_to_compact_by_size_[level] = 0;
}
const InternalKeyComparator* InternalComparator() const {
return internal_comparator_;
}
// Returns maximum total bytes of data on a given level.
uint64_t MaxBytesForLevel(int level) const;
// Returns an estimate of the amount of live data in bytes.
uint64_t EstimateLiveDataSize() const;
uint64_t estimated_compaction_needed_bytes() const {
return estimated_compaction_needed_bytes_;
}
void TEST_set_estimated_compaction_needed_bytes(uint64_t v) {
estimated_compaction_needed_bytes_ = v;
}
bool force_consistency_checks() const { return force_consistency_checks_; }
SequenceNumber bottommost_files_mark_threshold() const {
return bottommost_files_mark_threshold_;
}
single-file bottom-level compaction when snapshot released Summary: When snapshots are held for a long time, files may reach the bottom level containing overwritten/deleted keys. We previously had no mechanism to trigger compaction on such files. This particularly impacted DBs that write to different parts of the keyspace over time, as such files would never be naturally compacted due to second-last level files moving down. This PR introduces a mechanism for bottommost files to be recompacted upon releasing all snapshots that prevent them from dropping their deleted/overwritten keys. - Changed `CompactionPicker` to compact files in `BottommostFilesMarkedForCompaction()`. These are the last choice when picking. Each file will be compacted alone and output to the same level in which it originated. The goal of this type of compaction is to rewrite the data excluding deleted/overwritten keys. - Changed `ReleaseSnapshot()` to recompute the bottom files marked for compaction when the oldest existing snapshot changes, and schedule a compaction if needed. We cache the value that oldest existing snapshot needs to exceed in order for another file to be marked in `bottommost_files_mark_threshold_`, which allows us to avoid recomputing marked files for most snapshot releases. - Changed `VersionStorageInfo` to track the list of bottommost files, which is recomputed every time the version changes by `UpdateBottommostFiles()`. The list of marked bottommost files is first computed in `ComputeBottommostFilesMarkedForCompaction()` when the version changes, but may also be recomputed when `ReleaseSnapshot()` is called. - Extracted core logic of `Compaction::IsBottommostLevel()` into `VersionStorageInfo::RangeMightExistAfterSortedRun()` since logic to check whether a file is bottommost is now necessary outside of compaction. Closes https://github.com/facebook/rocksdb/pull/3009 Differential Revision: D6062044 Pulled By: ajkr fbshipit-source-id: 123d201cf140715a7d5928e8b3cb4f9cd9f7ad21
7 years ago
// Returns whether any key in [`smallest_key`, `largest_key`] could appear in
// an older L0 file than `last_l0_idx` or in a greater level than `last_level`
//
// @param last_level Level after which we check for overlap
// @param last_l0_idx If `last_level == 0`, index of L0 file after which we
// check for overlap; otherwise, must be -1
bool RangeMightExistAfterSortedRun(const Slice& smallest_user_key,
const Slice& largest_user_key,
int last_level, int last_l0_idx);
single-file bottom-level compaction when snapshot released Summary: When snapshots are held for a long time, files may reach the bottom level containing overwritten/deleted keys. We previously had no mechanism to trigger compaction on such files. This particularly impacted DBs that write to different parts of the keyspace over time, as such files would never be naturally compacted due to second-last level files moving down. This PR introduces a mechanism for bottommost files to be recompacted upon releasing all snapshots that prevent them from dropping their deleted/overwritten keys. - Changed `CompactionPicker` to compact files in `BottommostFilesMarkedForCompaction()`. These are the last choice when picking. Each file will be compacted alone and output to the same level in which it originated. The goal of this type of compaction is to rewrite the data excluding deleted/overwritten keys. - Changed `ReleaseSnapshot()` to recompute the bottom files marked for compaction when the oldest existing snapshot changes, and schedule a compaction if needed. We cache the value that oldest existing snapshot needs to exceed in order for another file to be marked in `bottommost_files_mark_threshold_`, which allows us to avoid recomputing marked files for most snapshot releases. - Changed `VersionStorageInfo` to track the list of bottommost files, which is recomputed every time the version changes by `UpdateBottommostFiles()`. The list of marked bottommost files is first computed in `ComputeBottommostFilesMarkedForCompaction()` when the version changes, but may also be recomputed when `ReleaseSnapshot()` is called. - Extracted core logic of `Compaction::IsBottommostLevel()` into `VersionStorageInfo::RangeMightExistAfterSortedRun()` since logic to check whether a file is bottommost is now necessary outside of compaction. Closes https://github.com/facebook/rocksdb/pull/3009 Differential Revision: D6062044 Pulled By: ajkr fbshipit-source-id: 123d201cf140715a7d5928e8b3cb4f9cd9f7ad21
7 years ago
private:
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
void ComputeCompensatedSizes();
void UpdateNumNonEmptyLevels();
void CalculateBaseBytes(const ImmutableOptions& ioptions,
const MutableCFOptions& options);
void UpdateFilesByCompactionPri(const ImmutableOptions& immutable_options,
const MutableCFOptions& mutable_cf_options);
void GenerateFileIndexer() {
file_indexer_.UpdateIndex(&arena_, num_non_empty_levels_, files_);
}
void GenerateLevelFilesBrief();
void GenerateLevel0NonOverlapping();
void GenerateBottommostFiles();
Mitigate the overhead of building the hash of file locations (#9504) Summary: The patch builds on the refactoring done in https://github.com/facebook/rocksdb/issues/9494 and improves the performance of building the hash of file locations in `VersionStorageInfo` in two ways. First, the hash building is moved from `AddFile` (which is called under the DB mutex) to a separate post-processing step done as part of `PrepareForVersionAppend` (during which the mutex is *not* held). Second, the space necessary for the hash is preallocated to prevent costly reallocation/rehashing operations. These changes mitigate the overhead of the file location hash, which can be significant with certain workloads where the baseline CPU usage is low (see https://github.com/facebook/rocksdb/issues/9351, which is a workload where keys are sorted, WAL is turned off, the vector memtable implementation is used, and there are lots of small SST files). Fixes https://github.com/facebook/rocksdb/issues/9351 Pull Request resolved: https://github.com/facebook/rocksdb/pull/9504 Test Plan: `make check` ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --disable_wal=1 --seed=<some_seed> ``` Final statistics before this patch: ``` Cumulative writes: 0 writes, 697M keys, 0 commit groups, 0.0 writes per commit group, ingest: 283.25 GB, 241.08 MB/s Interval writes: 0 writes, 1264K keys, 0 commit groups, 0.0 writes per commit group, ingest: 525.69 MB, 176.67 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 759M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.57 GB, 262.63 MB/s Interval writes: 0 writes, 1555K keys, 0 commit groups, 0.0 writes per commit group, ingest: 646.61 MB, 215.11 MB/s ``` Reviewed By: riversand963 Differential Revision: D34014734 Pulled By: ltamasi fbshipit-source-id: acb2703677451d5ccaa7e9d950844b33d240695b
3 years ago
void GenerateFileLocationIndex();
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
const InternalKeyComparator* internal_comparator_;
const Comparator* user_comparator_;
int num_levels_; // Number of levels
int num_non_empty_levels_; // Number of levels. Any level larger than it
// is guaranteed to be empty.
// Per-level max bytes
std::vector<uint64_t> level_max_bytes_;
// A short brief metadata of files per level
autovector<ROCKSDB_NAMESPACE::LevelFilesBrief> level_files_brief_;
FileIndexer file_indexer_;
Arena arena_; // Used to allocate space for file_levels_
CompactionStyle compaction_style_;
// List of files per level, files in each level are arranged
// in increasing order of keys
std::vector<FileMetaData*>* files_;
// Map of all table files in version. Maps file number to (level, position on
// level).
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
3 years ago
using FileLocations = UnorderedMap<uint64_t, FileLocation>;
FileLocations file_locations_;
Use a sorted vector instead of a map to store blob file metadata (#9526) Summary: The patch replaces `std::map` with a sorted `std::vector` for `VersionStorageInfo::blob_files_` and preallocates the space for the `vector` before saving the `BlobFileMetaData` into the new `VersionStorageInfo` in `VersionBuilder::Rep::SaveBlobFilesTo`. These changes reduce the time the DB mutex is held while saving new `Version`s, and using a sorted `vector` also makes lookups faster thanks to better memory locality. In addition, the patch introduces helper methods `VersionStorageInfo::GetBlobFileMetaData` and `VersionStorageInfo::GetBlobFileMetaDataLB` that can be used by clients to perform lookups in the `vector`, and does some general cleanup in the parts of code where blob file metadata are used. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9526 Test Plan: Ran `make check` and the crash test script for a while. Performance was tested using a load-optimized benchmark (`fillseq` with vector memtable, no WAL) and small file sizes so that a significant number of files are produced: ``` numactl --interleave=all ./db_bench --benchmarks=fillseq --allow_concurrent_memtable_write=false --level0_file_num_compaction_trigger=4 --level0_slowdown_writes_trigger=20 --level0_stop_writes_trigger=30 --max_background_jobs=8 --max_write_buffer_number=8 --db=/data/ltamasi-dbbench --wal_dir=/data/ltamasi-dbbench --num=800000000 --num_levels=8 --key_size=20 --value_size=400 --block_size=8192 --cache_size=51539607552 --cache_numshardbits=6 --compression_max_dict_bytes=0 --compression_ratio=0.5 --compression_type=lz4 --bytes_per_sync=8388608 --cache_index_and_filter_blocks=1 --cache_high_pri_pool_ratio=0.5 --benchmark_write_rate_limit=0 --write_buffer_size=16777216 --target_file_size_base=16777216 --max_bytes_for_level_base=67108864 --verify_checksum=1 --delete_obsolete_files_period_micros=62914560 --max_bytes_for_level_multiplier=8 --statistics=0 --stats_per_interval=1 --stats_interval_seconds=20 --histogram=1 --memtablerep=skip_list --bloom_bits=10 --open_files=-1 --subcompactions=1 --compaction_style=0 --min_level_to_compress=3 --level_compaction_dynamic_level_bytes=true --pin_l0_filter_and_index_blocks_in_cache=1 --soft_pending_compaction_bytes_limit=167503724544 --hard_pending_compaction_bytes_limit=335007449088 --min_level_to_compress=0 --use_existing_db=0 --sync=0 --threads=1 --memtablerep=vector --allow_concurrent_memtable_write=false --disable_wal=1 --enable_blob_files=1 --blob_file_size=16777216 --min_blob_size=0 --blob_compression_type=lz4 --enable_blob_garbage_collection=1 --seed=<some value> ``` Final statistics before the patch: ``` Cumulative writes: 0 writes, 700M keys, 0 commit groups, 0.0 writes per commit group, ingest: 284.62 GB, 121.27 MB/s Interval writes: 0 writes, 334K keys, 0 commit groups, 0.0 writes per commit group, ingest: 139.28 MB, 72.46 MB/s ``` With the patch: ``` Cumulative writes: 0 writes, 760M keys, 0 commit groups, 0.0 writes per commit group, ingest: 308.66 GB, 131.52 MB/s Interval writes: 0 writes, 445K keys, 0 commit groups, 0.0 writes per commit group, ingest: 185.35 MB, 93.15 MB/s ``` Total time to complete the benchmark is 2611 seconds with the patch, down from 2986 secs. Reviewed By: riversand963 Differential Revision: D34082728 Pulled By: ltamasi fbshipit-source-id: fc598abf676dce436734d06bb9d2d99a26a004fc
3 years ago
// Vector of blob files in version sorted by blob file number.
Add blob files to VersionStorageInfo/VersionBuilder (#6597) Summary: The patch adds a couple of classes to represent metadata about blob files: `SharedBlobFileMetaData` contains the information elements that are immutable (once the blob file is closed), e.g. blob file number, total number and size of blob files, checksum method/value, while `BlobFileMetaData` contains attributes that can vary across versions like the amount of garbage in the file. There is a single `SharedBlobFileMetaData` for each blob file, which is jointly owned by the `BlobFileMetaData` objects that point to it; `BlobFileMetaData` objects, in turn, are owned by `Version`s and can also be shared if the (immutable _and_ mutable) state of the blob file is the same in two versions. In addition, the patch adds the blob file metadata to `VersionStorageInfo`, and extends `VersionBuilder` so that it can apply blob file related `VersionEdit`s (i.e. those containing `BlobFileAddition`s and/or `BlobFileGarbage`), and save blob file metadata to a new `VersionStorageInfo`. Consistency checks are also extended to ensure that table files point to blob files that are part of the `Version`, and that all blob files that are part of any given `Version` have at least some _non_-garbage data in them. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6597 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D20656803 Pulled By: ltamasi fbshipit-source-id: f1f74d135045b3b42d0146f03ee576ef0a4bfd80
5 years ago
BlobFiles blob_files_;
// Level that L0 data should be compacted to. All levels < base_level_ should
// be empty. -1 if it is not level-compaction so it's not applicable.
int base_level_;
double level_multiplier_;
// A list for the same set of files that are stored in files_,
// but files in each level are now sorted based on file
// size. The file with the largest size is at the front.
// This vector stores the index of the file from files_.
std::vector<std::vector<int>> files_by_compaction_pri_;
Allowing L0 -> L1 trivial move on sorted data Summary: This diff updates the logic of how we do trivial move, now trivial move can run on any number of files in input level as long as they are not overlapping The conditions for trivial move have been updated Introduced conditions: - Trivial move cannot happen if we have a compaction filter (except if the compaction is not manual) - Input level files cannot be overlapping Removed conditions: - Trivial move only run when the compaction is not manual - Input level should can contain only 1 file More context on what tests failed because of Trivial move ``` DBTest.CompactionsGenerateMultipleFiles This test is expecting compaction on a file in L0 to generate multiple files in L1, this test will fail with trivial move because we end up with one file in L1 ``` ``` DBTest.NoSpaceCompactRange This test expect compaction to fail when we force environment to report running out of space, of course this is not valid in trivial move situation because trivial move does not need any extra space, and did not check for that ``` ``` DBTest.DropWrites Similar to DBTest.NoSpaceCompactRange ``` ``` DBTest.DeleteObsoleteFilesPendingOutputs This test expect that a file in L2 is deleted after it's moved to L3, this is not valid with trivial move because although the file was moved it is now used by L3 ``` ``` CuckooTableDBTest.CompactionIntoMultipleFiles Same as DBTest.CompactionsGenerateMultipleFiles ``` This diff is based on a work by @sdong https://reviews.facebook.net/D34149 Test Plan: make -j64 check Reviewers: rven, sdong, igor Reviewed By: igor Subscribers: yhchiang, ott, march, dhruba, sdong Differential Revision: https://reviews.facebook.net/D34797
10 years ago
// If true, means that files in L0 have keys with non overlapping ranges
bool level0_non_overlapping_;
// An index into files_by_compaction_pri_ that specifies the first
// file that is not yet compacted
std::vector<int> next_file_to_compact_by_size_;
// Only the first few entries of files_by_compaction_pri_ are sorted.
// There is no need to sort all the files because it is likely
// that on a running system, we need to look at only the first
// few largest files because a new version is created every few
// seconds/minutes (because of concurrent compactions).
static const size_t number_of_files_to_sort_ = 50;
// This vector contains list of files marked for compaction and also not
// currently being compacted. It is protected by DB mutex. It is calculated in
// ComputeCompactionScore()
autovector<std::pair<int, FileMetaData*>> files_marked_for_compaction_;
autovector<std::pair<int, FileMetaData*>> expired_ttl_files_;
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
6 years ago
autovector<std::pair<int, FileMetaData*>>
files_marked_for_periodic_compaction_;
single-file bottom-level compaction when snapshot released Summary: When snapshots are held for a long time, files may reach the bottom level containing overwritten/deleted keys. We previously had no mechanism to trigger compaction on such files. This particularly impacted DBs that write to different parts of the keyspace over time, as such files would never be naturally compacted due to second-last level files moving down. This PR introduces a mechanism for bottommost files to be recompacted upon releasing all snapshots that prevent them from dropping their deleted/overwritten keys. - Changed `CompactionPicker` to compact files in `BottommostFilesMarkedForCompaction()`. These are the last choice when picking. Each file will be compacted alone and output to the same level in which it originated. The goal of this type of compaction is to rewrite the data excluding deleted/overwritten keys. - Changed `ReleaseSnapshot()` to recompute the bottom files marked for compaction when the oldest existing snapshot changes, and schedule a compaction if needed. We cache the value that oldest existing snapshot needs to exceed in order for another file to be marked in `bottommost_files_mark_threshold_`, which allows us to avoid recomputing marked files for most snapshot releases. - Changed `VersionStorageInfo` to track the list of bottommost files, which is recomputed every time the version changes by `UpdateBottommostFiles()`. The list of marked bottommost files is first computed in `ComputeBottommostFilesMarkedForCompaction()` when the version changes, but may also be recomputed when `ReleaseSnapshot()` is called. - Extracted core logic of `Compaction::IsBottommostLevel()` into `VersionStorageInfo::RangeMightExistAfterSortedRun()` since logic to check whether a file is bottommost is now necessary outside of compaction. Closes https://github.com/facebook/rocksdb/pull/3009 Differential Revision: D6062044 Pulled By: ajkr fbshipit-source-id: 123d201cf140715a7d5928e8b3cb4f9cd9f7ad21
7 years ago
// These files are considered bottommost because none of their keys can exist
// at lower levels. They are not necessarily all in the same level. The marked
// ones are eligible for compaction because they contain duplicate key
// versions that are no longer protected by snapshot. These variables are
// protected by DB mutex and are calculated in `GenerateBottommostFiles()` and
// `ComputeBottommostFilesMarkedForCompaction()`.
autovector<std::pair<int, FileMetaData*>> bottommost_files_;
autovector<std::pair<int, FileMetaData*>>
bottommost_files_marked_for_compaction_;
Make it possible to force the garbage collection of the oldest blob files (#8994) Summary: The current BlobDB garbage collection logic works by relocating the valid blobs from the oldest blob files as they are encountered during compaction, and cleaning up blob files once they contain nothing but garbage. However, with sufficiently skewed workloads, it is theoretically possible to end up in a situation when few or no compactions get scheduled for the SST files that contain references to the oldest blob files, which can lead to increased space amp due to the lack of GC. In order to efficiently handle such workloads, the patch adds a new BlobDB configuration option called `blob_garbage_collection_force_threshold`, which signals to BlobDB to schedule targeted compactions for the SST files that keep alive the oldest batch of blob files if the overall ratio of garbage in the given blob files meets the threshold *and* all the given blob files are eligible for GC based on `blob_garbage_collection_age_cutoff`. (For example, if the new option is set to 0.9, targeted compactions will get scheduled if the sum of garbage bytes meets or exceeds 90% of the sum of total bytes in the oldest blob files, assuming all affected blob files are below the age-based cutoff.) The net result of these targeted compactions is that the valid blobs in the oldest blob files are relocated and the oldest blob files themselves cleaned up (since *all* SST files that rely on them get compacted away). These targeted compactions are similar to periodic compactions in the sense that they force certain SST files that otherwise would not get picked up to undergo compaction and also in the sense that instead of merging files from multiple levels, they target a single file. (Note: such compactions might still include neighboring files from the same level due to the need of having a "clean cut" boundary but they never include any files from any other level.) This functionality is currently only supported with the leveled compaction style and is inactive by default (since the default value is set to 1.0, i.e. 100%). Pull Request resolved: https://github.com/facebook/rocksdb/pull/8994 Test Plan: Ran `make check` and tested using `db_bench` and the stress/crash tests. Reviewed By: riversand963 Differential Revision: D31489850 Pulled By: ltamasi fbshipit-source-id: 44057d511726a0e2a03c5d9313d7511b3f0c4eab
3 years ago
autovector<std::pair<int, FileMetaData*>> files_marked_for_forced_blob_gc_;
single-file bottom-level compaction when snapshot released Summary: When snapshots are held for a long time, files may reach the bottom level containing overwritten/deleted keys. We previously had no mechanism to trigger compaction on such files. This particularly impacted DBs that write to different parts of the keyspace over time, as such files would never be naturally compacted due to second-last level files moving down. This PR introduces a mechanism for bottommost files to be recompacted upon releasing all snapshots that prevent them from dropping their deleted/overwritten keys. - Changed `CompactionPicker` to compact files in `BottommostFilesMarkedForCompaction()`. These are the last choice when picking. Each file will be compacted alone and output to the same level in which it originated. The goal of this type of compaction is to rewrite the data excluding deleted/overwritten keys. - Changed `ReleaseSnapshot()` to recompute the bottom files marked for compaction when the oldest existing snapshot changes, and schedule a compaction if needed. We cache the value that oldest existing snapshot needs to exceed in order for another file to be marked in `bottommost_files_mark_threshold_`, which allows us to avoid recomputing marked files for most snapshot releases. - Changed `VersionStorageInfo` to track the list of bottommost files, which is recomputed every time the version changes by `UpdateBottommostFiles()`. The list of marked bottommost files is first computed in `ComputeBottommostFilesMarkedForCompaction()` when the version changes, but may also be recomputed when `ReleaseSnapshot()` is called. - Extracted core logic of `Compaction::IsBottommostLevel()` into `VersionStorageInfo::RangeMightExistAfterSortedRun()` since logic to check whether a file is bottommost is now necessary outside of compaction. Closes https://github.com/facebook/rocksdb/pull/3009 Differential Revision: D6062044 Pulled By: ajkr fbshipit-source-id: 123d201cf140715a7d5928e8b3cb4f9cd9f7ad21
7 years ago
// Threshold for needing to mark another bottommost file. Maintain it so we
// can quickly check when releasing a snapshot whether more bottommost files
// became eligible for compaction. It's defined as the min of the max nonzero
// seqnums of unmarked bottommost files.
SequenceNumber bottommost_files_mark_threshold_ = kMaxSequenceNumber;
// Monotonically increases as we release old snapshots. Zero indicates no
// snapshots have been released yet. When no snapshots remain we set it to the
// current seqnum, which needs to be protected as a snapshot can still be
// created that references it.
SequenceNumber oldest_snapshot_seqnum_ = 0;
// Level that should be compacted next and its compaction score.
// Score < 1 means compaction is not strictly needed. These fields
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// are initialized by ComputeCompactionScore.
// The most critical level to be compacted is listed first
// These are used to pick the best compaction level
std::vector<double> compaction_score_;
std::vector<int> compaction_level_;
int l0_delay_trigger_count_ = 0; // Count used to trigger slow down and stop
// for number of L0 files.
Add basic kRoundRobin compaction policy (#10107) Summary: Add `kRoundRobin` as a compaction priority. The implementation is as follows. - Define a cursor as the smallest Internal key in the successor of the selected file. Add `vector<InternalKey> compact_cursor_` into `VersionStorageInfo` where each element (`InternalKey`) in `compact_cursor_` represents a cursor. In round-robin compaction policy, we just need to select the first file (assuming files are sorted) and also has the smallest InternalKey larger than/equal to the cursor. After a file is chosen, we create a new `Fsize` vector which puts the selected file is placed at the first position in `temp`, the next cursor is then updated as the smallest InternalKey in successor of the selected file (the above logic is implemented in `SortFileByRoundRobin`). - After a compaction succeeds, typically `InstallCompactionResults()`, we choose the next cursor for the input level and save it to `edit`. When calling `LogAndApply`, we save the next cursor with its level into some local variable and finally apply the change to `vstorage` in `SaveTo` function. - Cursors are persist pair by pair (<level, InternalKey>) in `EncodeTo` so that they can be reconstructed when reopening. An empty cursor will not be encoded to MANIFEST Pull Request resolved: https://github.com/facebook/rocksdb/pull/10107 Test Plan: add unit test (`CompactionPriRoundRobin`) in `compaction_picker_test`, add `kRoundRobin` priority in `CompactionPriTest` from `db_compaction_test`, and add `PersistRoundRobinCompactCursor` in `db_compaction_test` Reviewed By: ajkr Differential Revision: D37316037 Pulled By: littlepig2013 fbshipit-source-id: 9f481748190ace416079139044e00df2968fb1ee
3 years ago
// Compact cursors for round-robin compactions in each level
std::vector<InternalKey> compact_cursor_;
// the following are the sampled temporary stats.
// the current accumulated size of sampled files.
uint64_t accumulated_file_size_;
// the current accumulated size of all raw keys based on the sampled files.
uint64_t accumulated_raw_key_size_;
// the current accumulated size of all raw keys based on the sampled files.
uint64_t accumulated_raw_value_size_;
// total number of non-deletion entries
uint64_t accumulated_num_non_deletions_;
// total number of deletion entries
uint64_t accumulated_num_deletions_;
// current number of non_deletion entries
uint64_t current_num_non_deletions_;
// current number of deletion entries
uint64_t current_num_deletions_;
// current number of file samples
uint64_t current_num_samples_;
// Estimated bytes needed to be compacted until all levels' size is down to
// target sizes.
uint64_t estimated_compaction_needed_bytes_;
bool finalized_;
// If set to true, we will run consistency checks even if RocksDB
// is compiled in release mode
bool force_consistency_checks_;
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
EpochNumberRequirement epoch_number_requirement_;
friend class Version;
friend class VersionSet;
};
struct ObsoleteFileInfo {
FileMetaData* metadata;
std::string path;
// If true, the FileMataData should be destroyed but the file should
// not be deleted. This is because another FileMetaData still references
// the file, usually because the file is trivial moved so two FileMetadata
// is managing the file.
bool only_delete_metadata = false;
ObsoleteFileInfo() noexcept
: metadata(nullptr), only_delete_metadata(false) {}
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
3 years ago
ObsoleteFileInfo(FileMetaData* f, const std::string& file_path,
std::shared_ptr<CacheReservationManager>
file_metadata_cache_res_mgr_arg = nullptr)
: metadata(f),
path(file_path),
only_delete_metadata(false),
file_metadata_cache_res_mgr(file_metadata_cache_res_mgr_arg) {}
ObsoleteFileInfo(const ObsoleteFileInfo&) = delete;
ObsoleteFileInfo& operator=(const ObsoleteFileInfo&) = delete;
ObsoleteFileInfo(ObsoleteFileInfo&& rhs) noexcept : ObsoleteFileInfo() {
*this = std::move(rhs);
}
ObsoleteFileInfo& operator=(ObsoleteFileInfo&& rhs) noexcept {
path = std::move(rhs.path);
metadata = rhs.metadata;
rhs.metadata = nullptr;
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
3 years ago
file_metadata_cache_res_mgr = rhs.file_metadata_cache_res_mgr;
rhs.file_metadata_cache_res_mgr = nullptr;
return *this;
}
void DeleteMetadata() {
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
3 years ago
if (file_metadata_cache_res_mgr) {
Status s = file_metadata_cache_res_mgr->UpdateCacheReservation(
metadata->ApproximateMemoryUsage(), false /* increase */);
s.PermitUncheckedError();
}
delete metadata;
metadata = nullptr;
}
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
3 years ago
private:
std::shared_ptr<CacheReservationManager> file_metadata_cache_res_mgr;
};
class ObsoleteBlobFileInfo {
public:
ObsoleteBlobFileInfo(uint64_t blob_file_number, std::string path)
: blob_file_number_(blob_file_number), path_(std::move(path)) {}
uint64_t GetBlobFileNumber() const { return blob_file_number_; }
const std::string& GetPath() const { return path_; }
private:
uint64_t blob_file_number_;
std::string path_;
};
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
6 years ago
using MultiGetRange = MultiGetContext::Range;
Add blob files to VersionStorageInfo/VersionBuilder (#6597) Summary: The patch adds a couple of classes to represent metadata about blob files: `SharedBlobFileMetaData` contains the information elements that are immutable (once the blob file is closed), e.g. blob file number, total number and size of blob files, checksum method/value, while `BlobFileMetaData` contains attributes that can vary across versions like the amount of garbage in the file. There is a single `SharedBlobFileMetaData` for each blob file, which is jointly owned by the `BlobFileMetaData` objects that point to it; `BlobFileMetaData` objects, in turn, are owned by `Version`s and can also be shared if the (immutable _and_ mutable) state of the blob file is the same in two versions. In addition, the patch adds the blob file metadata to `VersionStorageInfo`, and extends `VersionBuilder` so that it can apply blob file related `VersionEdit`s (i.e. those containing `BlobFileAddition`s and/or `BlobFileGarbage`), and save blob file metadata to a new `VersionStorageInfo`. Consistency checks are also extended to ensure that table files point to blob files that are part of the `Version`, and that all blob files that are part of any given `Version` have at least some _non_-garbage data in them. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6597 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D20656803 Pulled By: ltamasi fbshipit-source-id: f1f74d135045b3b42d0146f03ee576ef0a4bfd80
5 years ago
// A column family's version consists of the table and blob files owned by
// the column family at a certain point in time.
class Version {
public:
// Append to *iters a sequence of iterators that will
// yield the contents of this Version when merged together.
// @param read_options Must outlive any iterator built by
// `merger_iter_builder`.
void AddIterators(const ReadOptions& read_options,
const FileOptions& soptions,
MergeIteratorBuilder* merger_iter_builder,
Properly report IO errors when IndexType::kBinarySearchWithFirstKey is used (#6621) Summary: Context: Index type `kBinarySearchWithFirstKey` added the ability for sst file iterator to sometimes report a key from index without reading the corresponding data block. This is useful when sst blocks are cut at some meaningful boundaries (e.g. one block per key prefix), and many seeks land between blocks (e.g. for each prefix, the ranges of keys in different sst files are nearly disjoint, so a typical seek needs to read a data block from only one file even if all files have the prefix). But this added a new error condition, which rocksdb code was really not equipped to deal with: `InternalIterator::value()` may fail with an IO error or Status::Incomplete, but it's just a method returning a Slice, with no way to report error instead. Before this PR, this type of error wasn't handled at all (an empty slice was returned), and kBinarySearchWithFirstKey implementation was considered a prototype. Now that we (LogDevice) have experimented with kBinarySearchWithFirstKey for a while and confirmed that it's really useful, this PR is adding the missing error handling. It's a pretty inconvenient situation implementation-wise. The error needs to be reported from InternalIterator when trying to access value. But there are ~700 call sites of `InternalIterator::value()`, most of which either can't hit the error condition (because the iterator is reading from memtable or from index or something) or wouldn't benefit from the deferred loading of the value (e.g. compaction iterator that reads all values anyway). Adding error handling to all these call sites would needlessly bloat the code. So instead I made the deferred value loading optional: only the call sites that may use deferred loading have to call the new method `PrepareValue()` before calling `value()`. The feature is enabled with a new bool argument `allow_unprepared_value` to a bunch of methods that create iterators (it wouldn't make sense to put it in ReadOptions because it's completely internal to iterators, with virtually no user-visible effect). Lmk if you have better ideas. Note that the deferred value loading only happens for *internal* iterators. The user-visible iterator (DBIter) always prepares the value before returning from Seek/Next/etc. We could go further and add an API to defer that value loading too, but that's most likely not useful for LogDevice, so it doesn't seem worth the complexity for now. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6621 Test Plan: make -j5 check . Will also deploy to some logdevice test clusters and look at stats. Reviewed By: siying Differential Revision: D20786930 Pulled By: al13n321 fbshipit-source-id: 6da77d918bad3780522e918f17f4d5513d3e99ee
5 years ago
bool allow_unprepared_value);
// @param read_options Must outlive any iterator built by
// `merger_iter_builder`.
void AddIteratorsForLevel(const ReadOptions& read_options,
const FileOptions& soptions,
MergeIteratorBuilder* merger_iter_builder,
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
int level, bool allow_unprepared_value);
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
Status OverlapWithLevelIterator(const ReadOptions&, const FileOptions&,
const Slice& smallest_user_key,
const Slice& largest_user_key, int level,
bool* overlap);
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
// Lookup the value for key or get all merge operands for key.
// If do_merge = true (default) then lookup value for key.
// Behavior if do_merge = true:
// If found, store it in *value and
// return OK. Else return a non-OK status.
// Uses *operands to store merge_operator operations to apply later.
//
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
// If the ReadOptions.read_tier is set to do a read-only fetch, then
// *value_found will be set to false if it cannot be determined whether
// this value exists without doing IO.
//
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
// If the key is Deleted, *status will be set to NotFound and
// *key_exists will be set to true.
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
// If no key was found, *status will be set to NotFound and
// *key_exists will be set to false.
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
// If seq is non-null, *seq will be set to the sequence number found
// for the key if a key was found.
// Behavior if do_merge = false
// If the key has any merge operands then store them in
// merge_context.operands_list and don't merge the operands
// REQUIRES: lock is not held
Fix PinSelf() read-after-free in DB::GetMergeOperands() (#9507) Summary: **Context:** Running the new test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` prior to this fix surfaces the read-after-free bug of PinSef() as below: ``` READ of size 8 at 0x60400002529d thread T0 https://github.com/facebook/rocksdb/issues/5 0x7f199a in rocksdb::PinnableSlice::PinSelf(rocksdb::Slice const&) include/rocksdb/slice.h:171 https://github.com/facebook/rocksdb/issues/6 0x7f199a in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1919 https://github.com/facebook/rocksdb/issues/7 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 freed by thread T0 here: https://github.com/facebook/rocksdb/issues/3 0x1191399 in rocksdb::cache_entry_roles_detail::RegisteredDeleter<rocksdb::Block, (rocksdb::CacheEntryRole)0>::Delete(rocksdb::Slice const&, void*) cache/cache_entry_roles.h:99 https://github.com/facebook/rocksdb/issues/4 0x719348 in rocksdb::LRUHandle::Free() cache/lru_cache.h:205 https://github.com/facebook/rocksdb/issues/5 0x71047f in rocksdb::LRUCacheShard::Release(rocksdb::Cache::Handle*, bool) cache/lru_cache.cc:547 https://github.com/facebook/rocksdb/issues/6 0xa78f0a in rocksdb::Cleanable::DoCleanup() include/rocksdb/cleanable.h:60 https://github.com/facebook/rocksdb/issues/7 0xa78f0a in rocksdb::Cleanable::Reset() include/rocksdb/cleanable.h:38 https://github.com/facebook/rocksdb/issues/8 0xa78f0a in rocksdb::PinnedIteratorsManager::ReleasePinnedData() db/pinned_iterators_manager.h:71 https://github.com/facebook/rocksdb/issues/9 0xd0c21b in rocksdb::PinnedIteratorsManager::~PinnedIteratorsManager() db/pinned_iterators_manager.h:24 https://github.com/facebook/rocksdb/issues/10 0xd0c21b in rocksdb::Version::Get(rocksdb::ReadOptions const&, rocksdb::LookupKey const&, rocksdb::PinnableSlice*, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >*, rocksdb::Status*, rocksdb::MergeContext*, unsigned long*, bool*, bool*, unsigned long*, rocksdb::ReadCallback*, bool*, bool) db/pinned_iterators_manager.h:22 https://github.com/facebook/rocksdb/issues/11 0x7f0fdf in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1886 https://github.com/facebook/rocksdb/issues/12 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 previously allocated by thread T0 here: https://github.com/facebook/rocksdb/issues/1 0x1239896 in rocksdb::AllocateBlock(unsigned long, **rocksdb::MemoryAllocator*)** memory/memory_allocator.h:35 https://github.com/facebook/rocksdb/issues/2 0x1239896 in rocksdb::BlockFetcher::CopyBufferToHeapBuf() table/block_fetcher.cc:171 https://github.com/facebook/rocksdb/issues/3 0x1239896 in rocksdb::BlockFetcher::GetBlockContents() table/block_fetcher.cc:206 https://github.com/facebook/rocksdb/issues/4 0x122eae5 in rocksdb::BlockFetcher::ReadBlockContents() table/block_fetcher.cc:325 https://github.com/facebook/rocksdb/issues/5 0x11b1f45 in rocksdb::Status rocksdb::BlockBasedTable::MaybeReadBlockAndLoadToCache<rocksdb::Block>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, bool, rocksdb::CachableEntry<rocksdb::Block>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::BlockContents*) const table/block_based/block_based_table_reader.cc:1503 ``` Here is the analysis: - We have [PinnedIteratorsManager](https://github.com/facebook/rocksdb/blob/6.28.fb/db/version_set.cc#L1980) with `Cleanable` capability in our `Version::Get()` path. It's responsible for managing the life-time of pinned iterator and invoking registered cleanup functions during its own destruction. - For example in case above, the merge operands's clean-up gets associated with this manger in [GetContext::push_operand](https://github.com/facebook/rocksdb/blob/6.28.fb/table/get_context.cc#L405). During PinnedIteratorsManager's [destruction](https://github.com/facebook/rocksdb/blob/6.28.fb/db/pinned_iterators_manager.h#L67), the release function associated with those merge operand data is invoked. **And that's what we see in "freed by thread T955 here" in ASAN.** - Bug 🐛: `PinnedIteratorsManager` is local to `Version::Get()` while the data of merge operands need to outlive `Version::Get` and stay till they get [PinSelf()](https://github.com/facebook/rocksdb/blob/6.28.fb/db/db_impl/db_impl.cc#L1905), **which is the read-after-free in ASAN.** - This bug is likely to be an overlook of `PinnedIteratorsManager` when developing the API `DB::GetMergeOperands` cuz the current logic works fine with the existing case of getting the *merged value* where the operands do not need to live that long. - This bug was not surfaced much (even in its unit test) due to the release function associated with the merge operands (which are actually blocks put in cache as you can see in `BlockBasedTable::MaybeReadBlockAndLoadToCache` **in "previously allocated by" in ASAN report**) is a cache entry deleter. The deleter will call `Cache::Release()` which, for LRU cache, won't immediately deallocate the block based on LRU policy [unless the cache is full or being instructed to force erase](https://github.com/facebook/rocksdb/blob/6.28.fb/cache/lru_cache.cc#L521-L531) - `DBMergeOperandTest.MergeOperandReadAfterFreeBug` makes the cache extremely small to force cache full. **Summary:** - Fix the bug by align `PinnedIteratorsManager`'s lifetime with the merge operands Pull Request resolved: https://github.com/facebook/rocksdb/pull/9507 Test Plan: - New test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` - db bench on read path - Setup (LSM tree with several levels, cache the whole db to avoid read IO, warm cache with readseq to avoid read IO): `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1``TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="readrandom" -num=1000000 -cache_size=100000000 ` - Actual command run (run 20-run for 20 times and then average the 20-run's average micros/op) - `for j in {1..20}; do (for i in {1..20}; do rm -rf /dev/shm/rocksdb/ && TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq,readrandom" -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1 | egrep 'readrandom'; done > rr_output_pre.txt && (awk '{sum+=$3; sum_sqrt+=$3^2}END{print sum/20, sqrt(sum_sqrt/20-(sum/20)^2)}' rr_output_pre.txt) >> rr_output_pre_2.txt); done` - **Result: Pre-change: 3.79193 micros/op; Post-change: 3.79528 micros/op (+0.09%)** (pre-change)sorted avg micros/op of each 20-run | std of micros/op of each 20-run | (post-change) sorted avg micros/op of each 20-run | std of micros/op of each 20-run -- | -- | -- | -- 3.58355 | 0.265209 | 3.48715 | 0.382076 3.58845 | 0.519927 | 3.5832 | 0.382726 3.66415 | 0.452097 | 3.677 | 0.563831 3.68495 | 0.430897 | 3.68405 | 0.495355 3.70295 | 0.482893 | 3.68465 | 0.431438 3.719 | 0.463806 | 3.71945 | 0.457157 3.7393 | 0.453423 | 3.72795 | 0.538604 3.7806 | 0.527613 | 3.75075 | 0.444509 3.7817 | 0.426704 | 3.7683 | 0.468065 3.809 | 0.381033 | 3.8086 | 0.557378 3.80985 | 0.466011 | 3.81805 | 0.524833 3.8165 | 0.500351 | 3.83405 | 0.529339 3.8479 | 0.430326 | 3.86285 | 0.44831 3.85125 | 0.434108 | 3.8717 | 0.544098 3.8556 | 0.524602 | 3.895 | 0.411679 3.8656 | 0.476383 | 3.90965 | 0.566636 3.8911 | 0.488477 | 3.92735 | 0.608038 3.898 | 0.493978 | 3.9439 | 0.524511 3.97235 | 0.515008 | 3.9623 | 0.477416 3.9768 | 0.519993 | 3.98965 | 0.521481 - CI Reviewed By: ajkr Differential Revision: D34030519 Pulled By: hx235 fbshipit-source-id: a99ac585c11704c5ed93af033cb29ba0a7b16ae8
3 years ago
// REQUIRES: pinned_iters_mgr != nullptr
void Get(const ReadOptions&, const LookupKey& key, PinnableSlice* value,
Add support for wide-column point lookups (#10540) Summary: The patch adds a new API `GetEntity` that can be used to perform wide-column point lookups. It also extends the `Get` code path and the `MemTable` / `MemTableList` and `Version` / `GetContext` logic accordingly so that wide-column entities can be served from both memtables and SSTs. If the result of a lookup is a wide-column entity (`kTypeWideColumnEntity`), it is passed to the application in deserialized form; if it is a plain old key-value (`kTypeValue`), it is presented as a wide-column entity with a single default (anonymous) column. (In contrast, regular `Get` returns plain old key-values as-is, and returns the value of the default column for wide-column entities, see https://github.com/facebook/rocksdb/issues/10483 .) The result of `GetEntity` is a self-contained `PinnableWideColumns` object. `PinnableWideColumns` contains a `PinnableSlice`, which either stores the underlying data in its own buffer or holds on to a cache handle. It also contains a `WideColumns` instance, which indexes the contents of the `PinnableSlice`, so applications can access the values of columns efficiently. There are several pieces of functionality which are currently not supported for wide-column entities: there is currently no `MultiGetEntity` or wide-column iterator; also, `Merge` and `GetMergeOperands` are not supported, and there is no `GetEntity` implementation for read-only and secondary instances. We plan to implement these in future PRs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10540 Test Plan: `make check` Reviewed By: akankshamahajan15 Differential Revision: D38847474 Pulled By: ltamasi fbshipit-source-id: 42311a34ccdfe88b3775e847a5e2a5296e002b5b
2 years ago
PinnableWideColumns* columns, std::string* timestamp, Status* status,
MergeContext* merge_context,
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
SequenceNumber* max_covering_tombstone_seq,
Fix PinSelf() read-after-free in DB::GetMergeOperands() (#9507) Summary: **Context:** Running the new test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` prior to this fix surfaces the read-after-free bug of PinSef() as below: ``` READ of size 8 at 0x60400002529d thread T0 https://github.com/facebook/rocksdb/issues/5 0x7f199a in rocksdb::PinnableSlice::PinSelf(rocksdb::Slice const&) include/rocksdb/slice.h:171 https://github.com/facebook/rocksdb/issues/6 0x7f199a in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1919 https://github.com/facebook/rocksdb/issues/7 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 freed by thread T0 here: https://github.com/facebook/rocksdb/issues/3 0x1191399 in rocksdb::cache_entry_roles_detail::RegisteredDeleter<rocksdb::Block, (rocksdb::CacheEntryRole)0>::Delete(rocksdb::Slice const&, void*) cache/cache_entry_roles.h:99 https://github.com/facebook/rocksdb/issues/4 0x719348 in rocksdb::LRUHandle::Free() cache/lru_cache.h:205 https://github.com/facebook/rocksdb/issues/5 0x71047f in rocksdb::LRUCacheShard::Release(rocksdb::Cache::Handle*, bool) cache/lru_cache.cc:547 https://github.com/facebook/rocksdb/issues/6 0xa78f0a in rocksdb::Cleanable::DoCleanup() include/rocksdb/cleanable.h:60 https://github.com/facebook/rocksdb/issues/7 0xa78f0a in rocksdb::Cleanable::Reset() include/rocksdb/cleanable.h:38 https://github.com/facebook/rocksdb/issues/8 0xa78f0a in rocksdb::PinnedIteratorsManager::ReleasePinnedData() db/pinned_iterators_manager.h:71 https://github.com/facebook/rocksdb/issues/9 0xd0c21b in rocksdb::PinnedIteratorsManager::~PinnedIteratorsManager() db/pinned_iterators_manager.h:24 https://github.com/facebook/rocksdb/issues/10 0xd0c21b in rocksdb::Version::Get(rocksdb::ReadOptions const&, rocksdb::LookupKey const&, rocksdb::PinnableSlice*, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >*, rocksdb::Status*, rocksdb::MergeContext*, unsigned long*, bool*, bool*, unsigned long*, rocksdb::ReadCallback*, bool*, bool) db/pinned_iterators_manager.h:22 https://github.com/facebook/rocksdb/issues/11 0x7f0fdf in rocksdb::DBImpl::GetImpl(rocksdb::ReadOptions const&, rocksdb::Slice const&, rocksdb::DBImpl::GetImplOptions&) db/db_impl/db_impl.cc:1886 https://github.com/facebook/rocksdb/issues/12 0x540d63 in rocksdb::DBImpl::GetMergeOperands(rocksdb::ReadOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::PinnableSlice*, rocksdb::GetMergeOperandsOptions*, int*) db/db_impl/db_impl.h:203 previously allocated by thread T0 here: https://github.com/facebook/rocksdb/issues/1 0x1239896 in rocksdb::AllocateBlock(unsigned long, **rocksdb::MemoryAllocator*)** memory/memory_allocator.h:35 https://github.com/facebook/rocksdb/issues/2 0x1239896 in rocksdb::BlockFetcher::CopyBufferToHeapBuf() table/block_fetcher.cc:171 https://github.com/facebook/rocksdb/issues/3 0x1239896 in rocksdb::BlockFetcher::GetBlockContents() table/block_fetcher.cc:206 https://github.com/facebook/rocksdb/issues/4 0x122eae5 in rocksdb::BlockFetcher::ReadBlockContents() table/block_fetcher.cc:325 https://github.com/facebook/rocksdb/issues/5 0x11b1f45 in rocksdb::Status rocksdb::BlockBasedTable::MaybeReadBlockAndLoadToCache<rocksdb::Block>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, bool, rocksdb::CachableEntry<rocksdb::Block>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::BlockContents*) const table/block_based/block_based_table_reader.cc:1503 ``` Here is the analysis: - We have [PinnedIteratorsManager](https://github.com/facebook/rocksdb/blob/6.28.fb/db/version_set.cc#L1980) with `Cleanable` capability in our `Version::Get()` path. It's responsible for managing the life-time of pinned iterator and invoking registered cleanup functions during its own destruction. - For example in case above, the merge operands's clean-up gets associated with this manger in [GetContext::push_operand](https://github.com/facebook/rocksdb/blob/6.28.fb/table/get_context.cc#L405). During PinnedIteratorsManager's [destruction](https://github.com/facebook/rocksdb/blob/6.28.fb/db/pinned_iterators_manager.h#L67), the release function associated with those merge operand data is invoked. **And that's what we see in "freed by thread T955 here" in ASAN.** - Bug 🐛: `PinnedIteratorsManager` is local to `Version::Get()` while the data of merge operands need to outlive `Version::Get` and stay till they get [PinSelf()](https://github.com/facebook/rocksdb/blob/6.28.fb/db/db_impl/db_impl.cc#L1905), **which is the read-after-free in ASAN.** - This bug is likely to be an overlook of `PinnedIteratorsManager` when developing the API `DB::GetMergeOperands` cuz the current logic works fine with the existing case of getting the *merged value* where the operands do not need to live that long. - This bug was not surfaced much (even in its unit test) due to the release function associated with the merge operands (which are actually blocks put in cache as you can see in `BlockBasedTable::MaybeReadBlockAndLoadToCache` **in "previously allocated by" in ASAN report**) is a cache entry deleter. The deleter will call `Cache::Release()` which, for LRU cache, won't immediately deallocate the block based on LRU policy [unless the cache is full or being instructed to force erase](https://github.com/facebook/rocksdb/blob/6.28.fb/cache/lru_cache.cc#L521-L531) - `DBMergeOperandTest.MergeOperandReadAfterFreeBug` makes the cache extremely small to force cache full. **Summary:** - Fix the bug by align `PinnedIteratorsManager`'s lifetime with the merge operands Pull Request resolved: https://github.com/facebook/rocksdb/pull/9507 Test Plan: - New test `DBMergeOperandTest.MergeOperandReadAfterFreeBug` - db bench on read path - Setup (LSM tree with several levels, cache the whole db to avoid read IO, warm cache with readseq to avoid read IO): `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1``TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="readrandom" -num=1000000 -cache_size=100000000 ` - Actual command run (run 20-run for 20 times and then average the 20-run's average micros/op) - `for j in {1..20}; do (for i in {1..20}; do rm -rf /dev/shm/rocksdb/ && TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks="fillrandom,readseq,readrandom" -num=1000000 -cache_size=100000000 -write_buffer_size=10000 -statistics=1 -max_bytes_for_level_base=10000 -level0_file_num_compaction_trigger=1 | egrep 'readrandom'; done > rr_output_pre.txt && (awk '{sum+=$3; sum_sqrt+=$3^2}END{print sum/20, sqrt(sum_sqrt/20-(sum/20)^2)}' rr_output_pre.txt) >> rr_output_pre_2.txt); done` - **Result: Pre-change: 3.79193 micros/op; Post-change: 3.79528 micros/op (+0.09%)** (pre-change)sorted avg micros/op of each 20-run | std of micros/op of each 20-run | (post-change) sorted avg micros/op of each 20-run | std of micros/op of each 20-run -- | -- | -- | -- 3.58355 | 0.265209 | 3.48715 | 0.382076 3.58845 | 0.519927 | 3.5832 | 0.382726 3.66415 | 0.452097 | 3.677 | 0.563831 3.68495 | 0.430897 | 3.68405 | 0.495355 3.70295 | 0.482893 | 3.68465 | 0.431438 3.719 | 0.463806 | 3.71945 | 0.457157 3.7393 | 0.453423 | 3.72795 | 0.538604 3.7806 | 0.527613 | 3.75075 | 0.444509 3.7817 | 0.426704 | 3.7683 | 0.468065 3.809 | 0.381033 | 3.8086 | 0.557378 3.80985 | 0.466011 | 3.81805 | 0.524833 3.8165 | 0.500351 | 3.83405 | 0.529339 3.8479 | 0.430326 | 3.86285 | 0.44831 3.85125 | 0.434108 | 3.8717 | 0.544098 3.8556 | 0.524602 | 3.895 | 0.411679 3.8656 | 0.476383 | 3.90965 | 0.566636 3.8911 | 0.488477 | 3.92735 | 0.608038 3.898 | 0.493978 | 3.9439 | 0.524511 3.97235 | 0.515008 | 3.9623 | 0.477416 3.9768 | 0.519993 | 3.98965 | 0.521481 - CI Reviewed By: ajkr Differential Revision: D34030519 Pulled By: hx235 fbshipit-source-id: a99ac585c11704c5ed93af033cb29ba0a7b16ae8
3 years ago
PinnedIteratorsManager* pinned_iters_mgr,
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
bool* value_found = nullptr, bool* key_exists = nullptr,
SequenceNumber* seq = nullptr, ReadCallback* callback = nullptr,
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
bool* is_blob = nullptr, bool do_merge = true);
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
6 years ago
void MultiGet(const ReadOptions&, MultiGetRange* range,
ReadCallback* callback = nullptr);
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
6 years ago
// Interprets blob_index_slice as a blob reference, and (assuming the
// corresponding blob file is part of this Version) retrieves the blob and
// saves it in *value.
// REQUIRES: blob_index_slice stores an encoded blob reference
Integrated blob garbage collection: relocate blobs (#7694) Summary: The patch adds basic garbage collection support to the integrated BlobDB implementation. Valid blobs residing in the oldest blob files are relocated as they are encountered during compaction. The threshold that determines which blob files qualify is computed based on the configuration option `blob_garbage_collection_age_cutoff`, which was introduced in https://github.com/facebook/rocksdb/issues/7661 . Once a blob is retrieved for the purposes of relocation, it passes through the same logic that extracts large values to blob files in general. This means that if, for instance, the size threshold for key-value separation (`min_blob_size`) got changed or writing blob files got disabled altogether, it is possible for the value to be moved back into the LSM tree. In particular, one way to re-inline all blob values if needed would be to perform a full manual compaction with `enable_blob_files` set to `false`, `enable_blob_garbage_collection` set to `true`, and `blob_file_garbage_collection_age_cutoff` set to `1.0`. Some TODOs that I plan to address in separate PRs: 1) We'll have to measure the amount of new garbage in each blob file and log `BlobFileGarbage` entries as part of the compaction job's `VersionEdit`. (For the time being, blob files are cleaned up solely based on the `oldest_blob_file_number` relationships.) 2) When compression is used for blobs, the compression type hasn't changed, and the blob still qualifies for being written to a blob file, we can simply copy the compressed blob to the new file instead of going through decompression and compression. 3) We need to update the formula for computing write amplification to account for the amount of data read from blob files as part of GC. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7694 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D25069663 Pulled By: ltamasi fbshipit-source-id: bdfa8feb09afcf5bca3b4eba2ba72ce2f15cd06a
4 years ago
Status GetBlob(const ReadOptions& read_options, const Slice& user_key,
const Slice& blob_index_slice,
FilePrefetchBuffer* prefetch_buffer, PinnableSlice* value,
uint64_t* bytes_read) const;
Integrated blob garbage collection: relocate blobs (#7694) Summary: The patch adds basic garbage collection support to the integrated BlobDB implementation. Valid blobs residing in the oldest blob files are relocated as they are encountered during compaction. The threshold that determines which blob files qualify is computed based on the configuration option `blob_garbage_collection_age_cutoff`, which was introduced in https://github.com/facebook/rocksdb/issues/7661 . Once a blob is retrieved for the purposes of relocation, it passes through the same logic that extracts large values to blob files in general. This means that if, for instance, the size threshold for key-value separation (`min_blob_size`) got changed or writing blob files got disabled altogether, it is possible for the value to be moved back into the LSM tree. In particular, one way to re-inline all blob values if needed would be to perform a full manual compaction with `enable_blob_files` set to `false`, `enable_blob_garbage_collection` set to `true`, and `blob_file_garbage_collection_age_cutoff` set to `1.0`. Some TODOs that I plan to address in separate PRs: 1) We'll have to measure the amount of new garbage in each blob file and log `BlobFileGarbage` entries as part of the compaction job's `VersionEdit`. (For the time being, blob files are cleaned up solely based on the `oldest_blob_file_number` relationships.) 2) When compression is used for blobs, the compression type hasn't changed, and the blob still qualifies for being written to a blob file, we can simply copy the compressed blob to the new file instead of going through decompression and compression. 3) We need to update the formula for computing write amplification to account for the amount of data read from blob files as part of GC. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7694 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D25069663 Pulled By: ltamasi fbshipit-source-id: bdfa8feb09afcf5bca3b4eba2ba72ce2f15cd06a
4 years ago
// Retrieves a blob using a blob reference and saves it in *value,
// assuming the corresponding blob file is part of this Version.
Integrated blob garbage collection: relocate blobs (#7694) Summary: The patch adds basic garbage collection support to the integrated BlobDB implementation. Valid blobs residing in the oldest blob files are relocated as they are encountered during compaction. The threshold that determines which blob files qualify is computed based on the configuration option `blob_garbage_collection_age_cutoff`, which was introduced in https://github.com/facebook/rocksdb/issues/7661 . Once a blob is retrieved for the purposes of relocation, it passes through the same logic that extracts large values to blob files in general. This means that if, for instance, the size threshold for key-value separation (`min_blob_size`) got changed or writing blob files got disabled altogether, it is possible for the value to be moved back into the LSM tree. In particular, one way to re-inline all blob values if needed would be to perform a full manual compaction with `enable_blob_files` set to `false`, `enable_blob_garbage_collection` set to `true`, and `blob_file_garbage_collection_age_cutoff` set to `1.0`. Some TODOs that I plan to address in separate PRs: 1) We'll have to measure the amount of new garbage in each blob file and log `BlobFileGarbage` entries as part of the compaction job's `VersionEdit`. (For the time being, blob files are cleaned up solely based on the `oldest_blob_file_number` relationships.) 2) When compression is used for blobs, the compression type hasn't changed, and the blob still qualifies for being written to a blob file, we can simply copy the compressed blob to the new file instead of going through decompression and compression. 3) We need to update the formula for computing write amplification to account for the amount of data read from blob files as part of GC. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7694 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D25069663 Pulled By: ltamasi fbshipit-source-id: bdfa8feb09afcf5bca3b4eba2ba72ce2f15cd06a
4 years ago
Status GetBlob(const ReadOptions& read_options, const Slice& user_key,
const BlobIndex& blob_index,
FilePrefetchBuffer* prefetch_buffer, PinnableSlice* value,
uint64_t* bytes_read) const;
Integrated blob garbage collection: relocate blobs (#7694) Summary: The patch adds basic garbage collection support to the integrated BlobDB implementation. Valid blobs residing in the oldest blob files are relocated as they are encountered during compaction. The threshold that determines which blob files qualify is computed based on the configuration option `blob_garbage_collection_age_cutoff`, which was introduced in https://github.com/facebook/rocksdb/issues/7661 . Once a blob is retrieved for the purposes of relocation, it passes through the same logic that extracts large values to blob files in general. This means that if, for instance, the size threshold for key-value separation (`min_blob_size`) got changed or writing blob files got disabled altogether, it is possible for the value to be moved back into the LSM tree. In particular, one way to re-inline all blob values if needed would be to perform a full manual compaction with `enable_blob_files` set to `false`, `enable_blob_garbage_collection` set to `true`, and `blob_file_garbage_collection_age_cutoff` set to `1.0`. Some TODOs that I plan to address in separate PRs: 1) We'll have to measure the amount of new garbage in each blob file and log `BlobFileGarbage` entries as part of the compaction job's `VersionEdit`. (For the time being, blob files are cleaned up solely based on the `oldest_blob_file_number` relationships.) 2) When compression is used for blobs, the compression type hasn't changed, and the blob still qualifies for being written to a blob file, we can simply copy the compressed blob to the new file instead of going through decompression and compression. 3) We need to update the formula for computing write amplification to account for the amount of data read from blob files as part of GC. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7694 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D25069663 Pulled By: ltamasi fbshipit-source-id: bdfa8feb09afcf5bca3b4eba2ba72ce2f15cd06a
4 years ago
using BlobReadContext =
std::pair<BlobIndex, std::reference_wrapper<const KeyContext>>;
using BlobReadContexts = std::vector<BlobReadContext>;
void MultiGetBlob(const ReadOptions& read_options, MultiGetRange& range,
std::unordered_map<uint64_t, BlobReadContexts>& blob_ctxs);
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
// Loads some stats information from files (if update_stats is set) and
// populates derived data structures. Call without mutex held. It needs to be
// called before appending the version to the version set.
void PrepareAppend(const MutableCFOptions& mutable_cf_options,
bool update_stats);
// Reference count management (so Versions do not disappear out from
// under live iterators)
void Ref();
// Decrease reference count. Delete the object if no reference left
// and return true. Otherwise, return false.
bool Unref();
// Add all files listed in the current version to *live_table_files and
// *live_blob_files.
void AddLiveFiles(std::vector<uint64_t>* live_table_files,
std::vector<uint64_t>* live_blob_files) const;
// Remove live files that are in the delete candidate lists.
void RemoveLiveFiles(
std::vector<ObsoleteFileInfo>& sst_delete_candidates,
std::vector<ObsoleteBlobFileInfo>& blob_delete_candidates) const;
// Return a human readable string that describes this version's contents.
std::string DebugString(bool hex = false, bool print_stats = false) const;
// Returns the version number of this version
uint64_t GetVersionNumber() const { return version_number_; }
// REQUIRES: lock is held
// On success, "tp" will contains the table properties of the file
// specified in "file_meta". If the file name of "file_meta" is
// known ahead, passing it by a non-null "fname" can save a
// file-name conversion.
Status GetTableProperties(std::shared_ptr<const TableProperties>* tp,
const FileMetaData* file_meta,
const std::string* fname = nullptr) const;
// REQUIRES: lock is held
// On success, *props will be populated with all SSTables' table properties.
// The keys of `props` are the sst file name, the values of `props` are the
// tables' properties, represented as std::shared_ptr.
Status GetPropertiesOfAllTables(TablePropertiesCollection* props);
Status GetPropertiesOfAllTables(TablePropertiesCollection* props, int level);
Status GetPropertiesOfTablesInRange(const Range* range, std::size_t n,
TablePropertiesCollection* props) const;
// Print summary of range delete tombstones in SST files into out_str,
// with maximum max_entries_to_print entries printed out.
Status TablesRangeTombstoneSummary(int max_entries_to_print,
std::string* out_str);
// REQUIRES: lock is held
// On success, "tp" will contains the aggregated table property among
// the table properties of all sst files in this version.
Status GetAggregatedTableProperties(
std::shared_ptr<const TableProperties>* tp, int level = -1);
uint64_t GetEstimatedActiveKeys() {
return storage_info_.GetEstimatedActiveKeys();
}
size_t GetMemoryUsageByTableReaders();
ColumnFamilyData* cfd() const { return cfd_; }
// Return the next Version in the linked list.
Version* Next() const { return next_; }
int TEST_refs() const { return refs_; }
VersionStorageInfo* storage_info() { return &storage_info_; }
const VersionStorageInfo* storage_info() const { return &storage_info_; }
VersionSet* version_set() { return vset_; }
void GetColumnFamilyMetaData(ColumnFamilyMetaData* cf_meta);
uint64_t GetSstFilesSize();
// Retrieves the file_creation_time of the oldest file in the DB.
// Prerequisite for this API is max_open_files = -1
void GetCreationTimeOfOldestFile(uint64_t* creation_time);
const MutableCFOptions& GetMutableCFOptions() { return mutable_cf_options_; }
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
InternalIterator* TEST_GetLevelIterator(
const ReadOptions& read_options, MergeIteratorBuilder* merge_iter_builder,
int level, bool allow_unprepared_value);
private:
Env* env_;
SystemClock* clock_;
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
friend class ReactiveVersionSet;
friend class VersionSet;
friend class VersionEditHandler;
friend class VersionEditHandlerPointInTime;
const InternalKeyComparator* internal_comparator() const {
return storage_info_.internal_comparator_;
}
const Comparator* user_comparator() const {
return storage_info_.user_comparator_;
}
// Returns true if the filter blocks in the specified level will not be
// checked during read operations. In certain cases (trivial move or preload),
// the filter block may already be cached, but we still do not access it such
// that it eventually expires from the cache.
bool IsFilterSkipped(int level, bool is_file_last_in_level = false);
// The helper function of UpdateAccumulatedStats, which may fill the missing
// fields of file_meta from its associated TableProperties.
// Returns true if it does initialize FileMetaData.
bool MaybeInitializeFileMetaData(FileMetaData* file_meta);
// Update the accumulated stats associated with the current version.
// This accumulated stats will be used in compaction.
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
void UpdateAccumulatedStats();
Multi file concurrency in MultiGet using coroutines and async IO (#9968) Summary: This PR implements a coroutine version of batched MultiGet in order to concurrently read from multiple SST files in a level using async IO, thus reducing the latency of the MultiGet. The API from the user perspective is still synchronous and single threaded, with the RocksDB part of the processing happening in the context of the caller's thread. In Version::MultiGet, the decision is made whether to call synchronous or coroutine code. A good way to review this PR is to review the first 4 commits in order - de773b3, 70c2f70, 10b50e1, and 377a597 - before reviewing the rest. TODO: 1. Figure out how to build it in CircleCI (requires some dependencies to be installed) 2. Do some stress testing with coroutines enabled No regression in synchronous MultiGet between this branch and main - ``` ./db_bench -use_existing_db=true --db=/data/mysql/rocksdb/prefix_scan -benchmarks="readseq,multireadrandom" -key_size=32 -value_size=512 -num=5000000 -batch_size=64 -multiread_batched=true -use_direct_reads=false -duration=60 -ops_between_duration_checks=1 -readonly=true -adaptive_readahead=true -threads=16 -cache_size=10485760000 -async_io=false -multiread_stride=40000 -statistics ``` Branch - ```multireadrandom : 4.025 micros/op 3975111 ops/sec 60.001 seconds 238509056 operations; 2062.3 MB/s (14767808 of 14767808 found)``` Main - ```multireadrandom : 3.987 micros/op 4013216 ops/sec 60.001 seconds 240795392 operations; 2082.1 MB/s (15231040 of 15231040 found)``` More benchmarks in various scenarios are given below. The measurements were taken with ```async_io=false``` (no coroutines) and ```async_io=true``` (use coroutines). For an IO bound workload (with every key requiring an IO), the coroutines version shows a clear benefit, being ~2.6X faster. For CPU bound workloads, the coroutines version has ~6-15% higher CPU utilization, depending on how many keys overlap an SST file. 1. Single thread IO bound workload on remote storage with sparse MultiGet batch keys (~1 key overlap/file) - No coroutines - ```multireadrandom : 831.774 micros/op 1202 ops/sec 60.001 seconds 72136 operations; 0.6 MB/s (72136 of 72136 found)``` Using coroutines - ```multireadrandom : 318.742 micros/op 3137 ops/sec 60.003 seconds 188248 operations; 1.6 MB/s (188248 of 188248 found)``` 2. Single thread CPU bound workload (all data cached) with ~1 key overlap/file - No coroutines - ```multireadrandom : 4.127 micros/op 242322 ops/sec 60.000 seconds 14539384 operations; 125.7 MB/s (14539384 of 14539384 found)``` Using coroutines - ```multireadrandom : 4.741 micros/op 210935 ops/sec 60.000 seconds 12656176 operations; 109.4 MB/s (12656176 of 12656176 found)``` 3. Single thread CPU bound workload with ~2 key overlap/file - No coroutines - ```multireadrandom : 3.717 micros/op 269000 ops/sec 60.000 seconds 16140024 operations; 139.6 MB/s (16140024 of 16140024 found)``` Using coroutines - ```multireadrandom : 4.146 micros/op 241204 ops/sec 60.000 seconds 14472296 operations; 125.1 MB/s (14472296 of 14472296 found)``` 4. CPU bound multi-threaded (16 threads) with ~4 key overlap/file - No coroutines - ```multireadrandom : 4.534 micros/op 3528792 ops/sec 60.000 seconds 211728728 operations; 1830.7 MB/s (12737024 of 12737024 found) ``` Using coroutines - ```multireadrandom : 4.872 micros/op 3283812 ops/sec 60.000 seconds 197030096 operations; 1703.6 MB/s (12548032 of 12548032 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9968 Reviewed By: akankshamahajan15 Differential Revision: D36348563 Pulled By: anand1976 fbshipit-source-id: c0ce85a505fd26ebfbb09786cbd7f25202038696
3 years ago
DECLARE_SYNC_AND_ASYNC(
/* ret_type */ Status, /* func_name */ MultiGetFromSST,
const ReadOptions& read_options, MultiGetRange file_range,
int hit_file_level, bool skip_filters, bool skip_range_deletions,
FdWithKeyRange* f,
std::unordered_map<uint64_t, BlobReadContexts>& blob_ctxs,
Cache::Handle* table_handle, uint64_t& num_filter_read,
uint64_t& num_index_read, uint64_t& num_sst_read);
Multi file concurrency in MultiGet using coroutines and async IO (#9968) Summary: This PR implements a coroutine version of batched MultiGet in order to concurrently read from multiple SST files in a level using async IO, thus reducing the latency of the MultiGet. The API from the user perspective is still synchronous and single threaded, with the RocksDB part of the processing happening in the context of the caller's thread. In Version::MultiGet, the decision is made whether to call synchronous or coroutine code. A good way to review this PR is to review the first 4 commits in order - de773b3, 70c2f70, 10b50e1, and 377a597 - before reviewing the rest. TODO: 1. Figure out how to build it in CircleCI (requires some dependencies to be installed) 2. Do some stress testing with coroutines enabled No regression in synchronous MultiGet between this branch and main - ``` ./db_bench -use_existing_db=true --db=/data/mysql/rocksdb/prefix_scan -benchmarks="readseq,multireadrandom" -key_size=32 -value_size=512 -num=5000000 -batch_size=64 -multiread_batched=true -use_direct_reads=false -duration=60 -ops_between_duration_checks=1 -readonly=true -adaptive_readahead=true -threads=16 -cache_size=10485760000 -async_io=false -multiread_stride=40000 -statistics ``` Branch - ```multireadrandom : 4.025 micros/op 3975111 ops/sec 60.001 seconds 238509056 operations; 2062.3 MB/s (14767808 of 14767808 found)``` Main - ```multireadrandom : 3.987 micros/op 4013216 ops/sec 60.001 seconds 240795392 operations; 2082.1 MB/s (15231040 of 15231040 found)``` More benchmarks in various scenarios are given below. The measurements were taken with ```async_io=false``` (no coroutines) and ```async_io=true``` (use coroutines). For an IO bound workload (with every key requiring an IO), the coroutines version shows a clear benefit, being ~2.6X faster. For CPU bound workloads, the coroutines version has ~6-15% higher CPU utilization, depending on how many keys overlap an SST file. 1. Single thread IO bound workload on remote storage with sparse MultiGet batch keys (~1 key overlap/file) - No coroutines - ```multireadrandom : 831.774 micros/op 1202 ops/sec 60.001 seconds 72136 operations; 0.6 MB/s (72136 of 72136 found)``` Using coroutines - ```multireadrandom : 318.742 micros/op 3137 ops/sec 60.003 seconds 188248 operations; 1.6 MB/s (188248 of 188248 found)``` 2. Single thread CPU bound workload (all data cached) with ~1 key overlap/file - No coroutines - ```multireadrandom : 4.127 micros/op 242322 ops/sec 60.000 seconds 14539384 operations; 125.7 MB/s (14539384 of 14539384 found)``` Using coroutines - ```multireadrandom : 4.741 micros/op 210935 ops/sec 60.000 seconds 12656176 operations; 109.4 MB/s (12656176 of 12656176 found)``` 3. Single thread CPU bound workload with ~2 key overlap/file - No coroutines - ```multireadrandom : 3.717 micros/op 269000 ops/sec 60.000 seconds 16140024 operations; 139.6 MB/s (16140024 of 16140024 found)``` Using coroutines - ```multireadrandom : 4.146 micros/op 241204 ops/sec 60.000 seconds 14472296 operations; 125.1 MB/s (14472296 of 14472296 found)``` 4. CPU bound multi-threaded (16 threads) with ~4 key overlap/file - No coroutines - ```multireadrandom : 4.534 micros/op 3528792 ops/sec 60.000 seconds 211728728 operations; 1830.7 MB/s (12737024 of 12737024 found) ``` Using coroutines - ```multireadrandom : 4.872 micros/op 3283812 ops/sec 60.000 seconds 197030096 operations; 1703.6 MB/s (12548032 of 12548032 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9968 Reviewed By: akankshamahajan15 Differential Revision: D36348563 Pulled By: anand1976 fbshipit-source-id: c0ce85a505fd26ebfbb09786cbd7f25202038696
3 years ago
MultiGet async IO across multiple levels (#10535) Summary: This PR exploits parallelism in MultiGet across levels. It applies only to the coroutine version of MultiGet. Previously, MultiGet file reads from SST files in the same level were parallelized. With this PR, MultiGet batches with keys distributed across multiple levels are read in parallel. This is accomplished by splitting the keys not present in a level (determined by bloom filtering) into a separate batch, and processing the new batch in parallel with the original batch. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10535 Test Plan: 1. Ensure existing MultiGet unit tests pass, updating them as necessary 2. New unit tests - TODO 3. Run stress test - TODO No noticeable regression (<1%) without async IO - Without PR: `multireadrandom : 7.261 micros/op 1101724 ops/sec 60.007 seconds 66110936 operations; 571.6 MB/s (8168992 of 8168992 found)` With PR: `multireadrandom : 7.305 micros/op 1095167 ops/sec 60.007 seconds 65717936 operations; 568.2 MB/s (8271992 of 8271992 found)` For a fully cached DB, but with async IO option on, no regression observed (<1%) - Without PR: `multireadrandom : 5.201 micros/op 1538027 ops/sec 60.005 seconds 92288936 operations; 797.9 MB/s (11540992 of 11540992 found) ` With PR: `multireadrandom : 5.249 micros/op 1524097 ops/sec 60.005 seconds 91452936 operations; 790.7 MB/s (11649992 of 11649992 found) ` Reviewed By: akankshamahajan15 Differential Revision: D38774009 Pulled By: anand1976 fbshipit-source-id: c955e259749f1c091590ade73105b3ee46cd0007
2 years ago
#ifdef USE_COROUTINES
// MultiGet using async IO to read data blocks from SST files in parallel
// within and across levels
Status MultiGetAsync(
const ReadOptions& options, MultiGetRange* range,
std::unordered_map<uint64_t, BlobReadContexts>* blob_ctxs);
// A helper function to lookup a batch of keys in a single level. It will
// queue coroutine tasks to mget_tasks. It may also split the input batch
// by creating a new batch with keys definitely not in this level and
// enqueuing it to to_process.
Status ProcessBatch(
const ReadOptions& read_options, FilePickerMultiGet* batch,
std::vector<folly::coro::Task<Status>>& mget_tasks,
std::unordered_map<uint64_t, BlobReadContexts>* blob_ctxs,
autovector<FilePickerMultiGet, 4>& batches, std::deque<size_t>& waiting,
std::deque<size_t>& to_process, unsigned int& num_tasks_queued,
std::unordered_map<int, std::tuple<uint64_t, uint64_t, uint64_t>>&
mget_stats);
MultiGet async IO across multiple levels (#10535) Summary: This PR exploits parallelism in MultiGet across levels. It applies only to the coroutine version of MultiGet. Previously, MultiGet file reads from SST files in the same level were parallelized. With this PR, MultiGet batches with keys distributed across multiple levels are read in parallel. This is accomplished by splitting the keys not present in a level (determined by bloom filtering) into a separate batch, and processing the new batch in parallel with the original batch. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10535 Test Plan: 1. Ensure existing MultiGet unit tests pass, updating them as necessary 2. New unit tests - TODO 3. Run stress test - TODO No noticeable regression (<1%) without async IO - Without PR: `multireadrandom : 7.261 micros/op 1101724 ops/sec 60.007 seconds 66110936 operations; 571.6 MB/s (8168992 of 8168992 found)` With PR: `multireadrandom : 7.305 micros/op 1095167 ops/sec 60.007 seconds 65717936 operations; 568.2 MB/s (8271992 of 8271992 found)` For a fully cached DB, but with async IO option on, no regression observed (<1%) - Without PR: `multireadrandom : 5.201 micros/op 1538027 ops/sec 60.005 seconds 92288936 operations; 797.9 MB/s (11540992 of 11540992 found) ` With PR: `multireadrandom : 5.249 micros/op 1524097 ops/sec 60.005 seconds 91452936 operations; 790.7 MB/s (11649992 of 11649992 found) ` Reviewed By: akankshamahajan15 Differential Revision: D38774009 Pulled By: anand1976 fbshipit-source-id: c955e259749f1c091590ade73105b3ee46cd0007
2 years ago
#endif
ColumnFamilyData* cfd_; // ColumnFamilyData to which this Version belongs
Logger* info_log_;
Statistics* db_statistics_;
TableCache* table_cache_;
BlobSource* blob_source_;
const MergeOperator* merge_operator_;
create compressed_levels_ in Version, allocate its space using arena. Make Version::Get, Version::FindFile faster Summary: Define CompressedFileMetaData that just contains fd, smallest_slice, largest_slice. Create compressed_levels_ in Version, the space is allocated using arena Thus increase the file meta data locality, speed up "Get" and "FindFile" benchmark with in-memory tmpfs, could have 4% improvement under "random read" and 2% improvement under "read while writing" benchmark command: ./db_bench --db=/mnt/db/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --block_size=4096 --cache_size=17179869184 --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=33554432 --max_bytes_for_level_base=1073741824 --disable_wal=0 --sync=0 --disable_data_sync=1 --verify_checksum=1 --delete_obsolete_files_period_micros=314572800 --max_grandparent_overlap_factor=10 --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 --perf_level=0 --benchmarks=readwhilewriting,readwhilewriting,readwhilewriting --use_existing_db=1 --num=52428800 --threads=1 —writes_per_second=81920 Read Random: From 1.8363 ms/op, improve to 1.7587 ms/op. Read while writing: From 2.985 ms/op, improve to 2.924 ms/op. Test Plan: make all check Reviewers: ljin, haobo, yhchiang, sdong Reviewed By: sdong Subscribers: dhruba, igor Differential Revision: https://reviews.facebook.net/D19419
11 years ago
VersionStorageInfo storage_info_;
VersionSet* vset_; // VersionSet to which this Version belongs
Version* next_; // Next version in linked list
Version* prev_; // Previous version in linked list
int refs_; // Number of live refs to this version
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
const FileOptions file_options_;
const MutableCFOptions mutable_cf_options_;
// Cached value to avoid recomputing it on every read.
const size_t max_file_size_for_l0_meta_pin_;
// A version number that uniquely represents this version. This is
// used for debugging and logging purposes only.
uint64_t version_number_;
std::shared_ptr<IOTracer> io_tracer_;
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
Version(ColumnFamilyData* cfd, VersionSet* vset, const FileOptions& file_opt,
MutableCFOptions mutable_cf_options,
const std::shared_ptr<IOTracer>& io_tracer,
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
uint64_t version_number = 0,
EpochNumberRequirement epoch_number_requirement =
EpochNumberRequirement::kMustPresent);
~Version();
// No copying allowed
Version(const Version&) = delete;
void operator=(const Version&) = delete;
};
class BaseReferencedVersionBuilder;
class AtomicGroupReadBuffer {
public:
AtomicGroupReadBuffer() = default;
Status AddEdit(VersionEdit* edit);
void Clear();
bool IsFull() const;
bool IsEmpty() const;
uint64_t TEST_read_edits_in_atomic_group() const {
return read_edits_in_atomic_group_;
}
std::vector<VersionEdit>& replay_buffer() { return replay_buffer_; }
private:
uint64_t read_edits_in_atomic_group_ = 0;
std::vector<VersionEdit> replay_buffer_;
};
// VersionSet is the collection of versions of all the column families of the
// database. Each database owns one VersionSet. A VersionSet has access to all
// column families via ColumnFamilySet, i.e. set of the column families.
class VersionSet {
public:
VersionSet(const std::string& dbname, const ImmutableDBOptions* db_options,
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
const FileOptions& file_options, Cache* table_cache,
WriteBufferManager* write_buffer_manager,
WriteController* write_controller,
BlockCacheTracer* const block_cache_tracer,
const std::shared_ptr<IOTracer>& io_tracer,
const std::string& db_id, const std::string& db_session_id);
// No copying allowed
VersionSet(const VersionSet&) = delete;
void operator=(const VersionSet&) = delete;
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
virtual ~VersionSet();
Status LogAndApplyToDefaultColumnFamily(
VersionEdit* edit, InstrumentedMutex* mu,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
FSDirectory* dir_contains_current_file, bool new_descriptor_log = false,
const ColumnFamilyOptions* column_family_options = nullptr) {
ColumnFamilyData* default_cf = GetColumnFamilySet()->GetDefault();
const MutableCFOptions* cf_options =
default_cf->GetLatestMutableCFOptions();
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
return LogAndApply(default_cf, *cf_options, edit, mu,
dir_contains_current_file, new_descriptor_log,
column_family_options);
}
// Apply *edit to the current version to form a new descriptor that
// is both saved to persistent state and installed as the new
// current version. Will release *mu while actually writing to the file.
// column_family_options has to be set if edit is column family add
// REQUIRES: *mu is held on entry.
// REQUIRES: no other thread concurrently calls LogAndApply()
Status LogAndApply(
ColumnFamilyData* column_family_data,
const MutableCFOptions& mutable_cf_options, VersionEdit* edit,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
InstrumentedMutex* mu, FSDirectory* dir_contains_current_file,
bool new_descriptor_log = false,
group multiple batch of flush into one manifest file (one call to LogAndApply) Summary: Currently, if several flush outputs are committed together, we issue each manifest write per batch (1 batch = 1 flush = 1 sst file = 1+ continuous memtables). Each manifest write requires one fsync and one fsync to parent directory. In some cases, it becomes the bottleneck of write. We should batch them and write in one manifest write when possible. Test Plan: ` ./db_bench -benchmarks="fillseq" -max_write_buffer_number=16 -max_background_flushes=16 -disable_auto_compactions=true -min_write_buffer_number_to_merge=1 -write_buffer_size=65536 -level0_stop_writes_trigger=10000 -level0_slowdown_writes_trigger=10000` **Before** ``` Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 4.9 Date: Fri Jul 1 15:38:17 2016 CPU: 32 * Intel(R) Xeon(R) CPU E5-2660 0 @ 2.20GHz CPUCache: 20480 KB Keys: 16 bytes each Values: 100 bytes each (50 bytes after compression) Entries: 1000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 110.6 MB (estimated) FileSize: 62.9 MB (estimated) Write rate: 0 bytes/second Compression: Snappy Memtablerep: skip_list Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags DB path: [/tmp/rocksdbtest-112628/dbbench] fillseq : 166.277 micros/op 6014 ops/sec; 0.7 MB/s ``` **After** ``` Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 4.9 Date: Fri Jul 1 15:35:05 2016 CPU: 32 * Intel(R) Xeon(R) CPU E5-2660 0 @ 2.20GHz CPUCache: 20480 KB Keys: 16 bytes each Values: 100 bytes each (50 bytes after compression) Entries: 1000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 110.6 MB (estimated) FileSize: 62.9 MB (estimated) Write rate: 0 bytes/second Compression: Snappy Memtablerep: skip_list Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags DB path: [/tmp/rocksdbtest-112628/dbbench] fillseq : 52.328 micros/op 19110 ops/sec; 2.1 MB/s ``` Reviewers: andrewkr, IslamAbdelRahman, yhchiang, sdong Reviewed By: sdong Subscribers: igor, andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D60075
9 years ago
const ColumnFamilyOptions* column_family_options = nullptr) {
autovector<ColumnFamilyData*> cfds;
cfds.emplace_back(column_family_data);
autovector<const MutableCFOptions*> mutable_cf_options_list;
mutable_cf_options_list.emplace_back(&mutable_cf_options);
autovector<autovector<VersionEdit*>> edit_lists;
autovector<VersionEdit*> edit_list;
edit_list.emplace_back(edit);
edit_lists.emplace_back(edit_list);
return LogAndApply(cfds, mutable_cf_options_list, edit_lists, mu,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
dir_contains_current_file, new_descriptor_log,
column_family_options);
group multiple batch of flush into one manifest file (one call to LogAndApply) Summary: Currently, if several flush outputs are committed together, we issue each manifest write per batch (1 batch = 1 flush = 1 sst file = 1+ continuous memtables). Each manifest write requires one fsync and one fsync to parent directory. In some cases, it becomes the bottleneck of write. We should batch them and write in one manifest write when possible. Test Plan: ` ./db_bench -benchmarks="fillseq" -max_write_buffer_number=16 -max_background_flushes=16 -disable_auto_compactions=true -min_write_buffer_number_to_merge=1 -write_buffer_size=65536 -level0_stop_writes_trigger=10000 -level0_slowdown_writes_trigger=10000` **Before** ``` Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 4.9 Date: Fri Jul 1 15:38:17 2016 CPU: 32 * Intel(R) Xeon(R) CPU E5-2660 0 @ 2.20GHz CPUCache: 20480 KB Keys: 16 bytes each Values: 100 bytes each (50 bytes after compression) Entries: 1000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 110.6 MB (estimated) FileSize: 62.9 MB (estimated) Write rate: 0 bytes/second Compression: Snappy Memtablerep: skip_list Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags DB path: [/tmp/rocksdbtest-112628/dbbench] fillseq : 166.277 micros/op 6014 ops/sec; 0.7 MB/s ``` **After** ``` Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 4.9 Date: Fri Jul 1 15:35:05 2016 CPU: 32 * Intel(R) Xeon(R) CPU E5-2660 0 @ 2.20GHz CPUCache: 20480 KB Keys: 16 bytes each Values: 100 bytes each (50 bytes after compression) Entries: 1000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 110.6 MB (estimated) FileSize: 62.9 MB (estimated) Write rate: 0 bytes/second Compression: Snappy Memtablerep: skip_list Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags DB path: [/tmp/rocksdbtest-112628/dbbench] fillseq : 52.328 micros/op 19110 ops/sec; 2.1 MB/s ``` Reviewers: andrewkr, IslamAbdelRahman, yhchiang, sdong Reviewed By: sdong Subscribers: igor, andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D60075
9 years ago
}
// The batch version. If edit_list.size() > 1, caller must ensure that
// no edit in the list column family add or drop
Status LogAndApply(
ColumnFamilyData* column_family_data,
const MutableCFOptions& mutable_cf_options,
const autovector<VersionEdit*>& edit_list, InstrumentedMutex* mu,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
FSDirectory* dir_contains_current_file, bool new_descriptor_log = false,
Perform post-flush updates of memtable list in a callback (#6069) Summary: Currently, the following interleaving of events can lead to SuperVersion containing both immutable memtables as well as the resulting L0. This can cause Get to return incorrect result if there are merge operands. This may also affect other operations such as single deletes. ``` time main_thr bg_flush_thr bg_compact_thr compact_thr set_opts_thr 0 | WriteManifest:0 1 | issue compact 2 | wait 3 | Merge(counter) 4 | issue flush 5 | wait 6 | WriteManifest:1 7 | wake up 8 | write manifest 9 | wake up 10 | Get(counter) 11 | remove imm V ``` The reason behind is that: one bg flush thread's installing new `Version` can be batched and performed by another thread that is the "leader" MANIFEST writer. This bg thread removes the memtables from current super version only after `LogAndApply` returns. After the leader MANIFEST writer signals (releasing mutex) this bg flush thread, it is possible that another thread sees this cf with both memtables (whose data have been flushed to the newest L0) and the L0 before this bg flush thread removes the memtables. To address this issue, each bg flush thread can pass a callback function to `LogAndApply`. The callback is responsible for removing the memtables. Therefore, the leader MANIFEST writer can call this callback and remove the memtables before releasing the mutex. Test plan (devserver) ``` $make merge_test $./merge_test --gtest_filter=MergeTest.MergeWithCompactionAndFlush $make check ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/6069 Reviewed By: cheng-chang Differential Revision: D18790894 Pulled By: riversand963 fbshipit-source-id: e41bd600c0448b4f4b2deb3f7677f95e3076b4ed
4 years ago
const ColumnFamilyOptions* column_family_options = nullptr,
const std::function<void(const Status&)>& manifest_wcb = {}) {
autovector<ColumnFamilyData*> cfds;
cfds.emplace_back(column_family_data);
autovector<const MutableCFOptions*> mutable_cf_options_list;
mutable_cf_options_list.emplace_back(&mutable_cf_options);
autovector<autovector<VersionEdit*>> edit_lists;
edit_lists.emplace_back(edit_list);
return LogAndApply(cfds, mutable_cf_options_list, edit_lists, mu,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
dir_contains_current_file, new_descriptor_log,
column_family_options, {manifest_wcb});
}
// The across-multi-cf batch version. If edit_lists contain more than
// 1 version edits, caller must ensure that no edit in the []list is column
// family manipulation.
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
virtual Status LogAndApply(
const autovector<ColumnFamilyData*>& cfds,
const autovector<const MutableCFOptions*>& mutable_cf_options_list,
const autovector<autovector<VersionEdit*>>& edit_lists,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
InstrumentedMutex* mu, FSDirectory* dir_contains_current_file,
bool new_descriptor_log = false,
Perform post-flush updates of memtable list in a callback (#6069) Summary: Currently, the following interleaving of events can lead to SuperVersion containing both immutable memtables as well as the resulting L0. This can cause Get to return incorrect result if there are merge operands. This may also affect other operations such as single deletes. ``` time main_thr bg_flush_thr bg_compact_thr compact_thr set_opts_thr 0 | WriteManifest:0 1 | issue compact 2 | wait 3 | Merge(counter) 4 | issue flush 5 | wait 6 | WriteManifest:1 7 | wake up 8 | write manifest 9 | wake up 10 | Get(counter) 11 | remove imm V ``` The reason behind is that: one bg flush thread's installing new `Version` can be batched and performed by another thread that is the "leader" MANIFEST writer. This bg thread removes the memtables from current super version only after `LogAndApply` returns. After the leader MANIFEST writer signals (releasing mutex) this bg flush thread, it is possible that another thread sees this cf with both memtables (whose data have been flushed to the newest L0) and the L0 before this bg flush thread removes the memtables. To address this issue, each bg flush thread can pass a callback function to `LogAndApply`. The callback is responsible for removing the memtables. Therefore, the leader MANIFEST writer can call this callback and remove the memtables before releasing the mutex. Test plan (devserver) ``` $make merge_test $./merge_test --gtest_filter=MergeTest.MergeWithCompactionAndFlush $make check ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/6069 Reviewed By: cheng-chang Differential Revision: D18790894 Pulled By: riversand963 fbshipit-source-id: e41bd600c0448b4f4b2deb3f7677f95e3076b4ed
4 years ago
const ColumnFamilyOptions* new_cf_options = nullptr,
const std::vector<std::function<void(const Status&)>>& manifest_wcbs =
{});
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
static Status GetCurrentManifestPath(const std::string& dbname,
FileSystem* fs,
std::string* manifest_filename,
uint64_t* manifest_file_number);
void WakeUpWaitingManifestWriters();
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
// Recover the last saved descriptor from persistent storage.
// If read_only == true, Recover() will not complain if some column families
// are not opened
Status Recover(const std::vector<ColumnFamilyDescriptor>& column_families,
bool read_only = false, std::string* db_id = nullptr,
bool no_error_if_files_missing = false);
Status TryRecover(const std::vector<ColumnFamilyDescriptor>& column_families,
bool read_only,
const std::vector<std::string>& files_in_dbname,
std::string* db_id, bool* has_missing_table_file);
// Try to recover the version set to the most recent consistent state
// recorded in the specified manifest.
Status TryRecoverFromOneManifest(
const std::string& manifest_path,
const std::vector<ColumnFamilyDescriptor>& column_families,
bool read_only, std::string* db_id, bool* has_missing_table_file);
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
// Recover the next epoch number of each CFs and epoch number
// of their files (if missing)
void RecoverEpochNumbers();
// Reads a manifest file and returns a list of column families in
// column_families.
static Status ListColumnFamilies(std::vector<std::string>* column_families,
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
const std::string& dbname, FileSystem* fs);
static Status ListColumnFamiliesFromManifest(
const std::string& manifest_path, FileSystem* fs,
std::vector<std::string>* column_families);
#ifndef ROCKSDB_LITE
// Try to reduce the number of levels. This call is valid when
// only one level from the new max level to the old
// max level containing files.
// The call is static, since number of levels is immutable during
// the lifetime of a RocksDB instance. It reduces number of levels
// in a DB by applying changes to manifest.
// For example, a db currently has 7 levels [0-6], and a call to
// to reduce to 5 [0-4] can only be executed when only one level
// among [4-6] contains files.
static Status ReduceNumberOfLevels(const std::string& dbname,
const Options* options,
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
const FileOptions& file_options,
int new_levels);
// Get the checksum information of all live files
Status GetLiveFilesChecksumInfo(FileChecksumList* checksum_list);
// printf contents (for debugging)
Status DumpManifest(Options& options, std::string& manifestFileName,
Added JSON manifest dump option to ldb command Summary: Added a new flag --json to the ldb manifest_dump command that prints out the version edits as JSON objects for easier reading and parsing of information. Test Plan: **Sample usage: ** ``` ./ldb manifest_dump --json --path=path/to/manifest/file ``` **Sample output:** ``` {"EditNumber": 0, "Comparator": "leveldb.BytewiseComparator", "ColumnFamily": 0} {"EditNumber": 1, "LogNumber": 0, "ColumnFamily": 0} {"EditNumber": 2, "LogNumber": 4, "PrevLogNumber": 0, "NextFileNumber": 7, "LastSeq": 35356, "AddedFiles": [{"Level": 0, "FileNumber": 5, "FileSize": 1949284, "SmallestIKey": "'", "LargestIKey": "'"}], "ColumnFamily": 0} ... {"EditNumber": 13, "PrevLogNumber": 0, "NextFileNumber": 36, "LastSeq": 290994, "DeletedFiles": [{"Level": 0, "FileNumber": 17}, {"Level": 0, "FileNumber": 20}, {"Level": 0, "FileNumber": 22}, {"Level": 0, "FileNumber": 24}, {"Level": 1, "FileNumber": 13}, {"Level": 1, "FileNumber": 14}, {"Level": 1, "FileNumber": 15}, {"Level": 1, "FileNumber": 18}], "AddedFiles": [{"Level": 1, "FileNumber": 25, "FileSize": 2114340, "SmallestIKey": "'", "LargestIKey": "'"}, {"Level": 1, "FileNumber": 26, "FileSize": 2115213, "SmallestIKey": "'", "LargestIKey": "'"}, {"Level": 1, "FileNumber": 27, "FileSize": 2114807, "SmallestIKey": "'", "LargestIKey": "'"}, {"Level": 1, "FileNumber": 30, "FileSize": 2115271, "SmallestIKey": "'", "LargestIKey": "'"}, {"Level": 1, "FileNumber": 31, "FileSize": 2115165, "SmallestIKey": "'", "LargestIKey": "'"}, {"Level": 1, "FileNumber": 32, "FileSize": 2114683, "SmallestIKey": "'", "LargestIKey": "'"}, {"Level": 1, "FileNumber": 35, "FileSize": 1757512, "SmallestIKey": "'", "LargestIKey": "'"}], "ColumnFamily": 0} ... ``` Reviewers: sdong, anthony, yhchiang, igor Reviewed By: igor Subscribers: dhruba Differential Revision: https://reviews.facebook.net/D41727
10 years ago
bool verbose, bool hex = false, bool json = false);
#endif // ROCKSDB_LITE
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
const std::string& DbSessionId() const { return db_session_id_; }
// Return the current manifest file number
uint64_t manifest_file_number() const { return manifest_file_number_; }
uint64_t options_file_number() const { return options_file_number_; }
uint64_t pending_manifest_file_number() const {
return pending_manifest_file_number_;
}
uint64_t current_next_file_number() const { return next_file_number_.load(); }
Fix a race condition in WAL tracking causing DB open failure (#9715) Summary: There is a race condition if WAL tracking in the MANIFEST is enabled in a database that disables 2PC. The race condition is between two background flush threads trying to install flush results to the MANIFEST. Consider an example database with two column families: "default" (cfd0) and "cf1" (cfd1). Initially, both column families have one mutable (active) memtable whose data backed by 6.log. 1. Trigger a manual flush for "cf1", creating a 7.log 2. Insert another key to "default", and trigger flush for "default", creating 8.log 3. BgFlushThread1 finishes writing 9.sst 4. BgFlushThread2 finishes writing 10.sst ``` Time BgFlushThread1 BgFlushThread2 | mutex_.Lock() | precompute min_wal_to_keep as 6 | mutex_.Unlock() | mutex_.Lock() | precompute min_wal_to_keep as 6 | join MANIFEST write queue and mutex_.Unlock() | write to MANIFEST | mutex_.Lock() | cfd1->log_number = 7 | Signal bg_flush_2 and mutex_.Unlock() | wake up and mutex_.Lock() | cfd0->log_number = 8 | FindObsoleteFiles() with job_context->log_number == 7 | mutex_.Unlock() | PurgeObsoleteFiles() deletes 6.log V ``` As shown in the above, BgFlushThread2 thinks that the min wal to keep is 6.log because "cf1" has unflushed data in 6.log (cf1.log_number=6). Similarly, BgThread1 thinks that min wal to keep is also 6.log because "default" has unflushed data (default.log_number=6). No WAL deletion will be written to MANIFEST because 6 is equal to `versions_->wals_.min_wal_number_to_keep`, due to https://github.com/facebook/rocksdb/blob/7.1.fb/db/memtable_list.cc#L513:L514. The bg flush thread that finishes last will perform file purging. `job_context.log_number` will be evaluated as 7, i.e. the min wal that contains unflushed data, causing 6.log to be deleted. However, MANIFEST thinks 6.log should still exist. If you close the db at this point, you won't be able to re-open it if `track_and_verify_wal_in_manifest` is true. We must handle the case of multiple bg flush threads, and it is difficult for one bg flush thread to know the correct min wal number until the other bg flush threads have finished committing to the manifest and updated the `cfd::log_number`. To fix this issue, we rename an existing variable `min_log_number_to_keep_2pc` to `min_log_number_to_keep`, and use it to track WAL file deletion in non-2pc mode as well. This variable is updated only 1) during recovery with mutex held, or 2) in the MANIFEST write thread. `min_log_number_to_keep` means RocksDB will delete WALs below it, although there may be WALs above it which are also obsolete. Formally, we will have [min_wal_to_keep, max_obsolete_wal]. During recovery, we make sure that only WALs above max_obsolete_wal are checked and added back to `alive_log_files_`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9715 Test Plan: ``` make check ``` Also ran stress test below (with asan) to make sure it completes successfully. ``` TEST_TMPDIR=/dev/shm/rocksdb OPT=-g ASAN_OPTIONS=disable_coredump=0 \ CRASH_TEST_EXT_ARGS=--compression_type=zstd SKIP_FORMAT_BUCK_CHECKS=1 \ make J=52 -j52 blackbox_asan_crash_test ``` Reviewed By: ltamasi Differential Revision: D34984412 Pulled By: riversand963 fbshipit-source-id: c7b21a8d84751bb55ea79c9f387103d21b231005
3 years ago
uint64_t min_log_number_to_keep() const {
return min_log_number_to_keep_.load();
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
}
// Allocate and return a new file number
uint64_t NewFileNumber() { return next_file_number_.fetch_add(1); }
// Fetch And Add n new file number
uint64_t FetchAddFileNumber(uint64_t n) {
return next_file_number_.fetch_add(n);
}
// Return the last sequence number.
Fix TSAN build and re-enable the tests (#7386) Summary: Resolve TSAN build warnings and re-enable disabled TSAN tests. Not sure if it's a compiler issue or TSAN check issue. Switching from conditional operator to if-else mitigated the problem. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7386 Test Plan: run TSAN check 10 times in circleci. ``` WARNING: ThreadSanitizer: data race (pid=27735) Atomic write of size 8 at 0x7b54000005e8 by thread T32: #0 __tsan_atomic64_store <null> (db_test+0x4cee95) https://github.com/facebook/rocksdb/issues/1 std::__atomic_base<unsigned long>::store(unsigned long, std::memory_order) /usr/bin/../lib/gcc/x86_64-linux-gnu/5.4.0/../../../../include/c++/5.4.0/bits/atomic_base.h:374:2 (db_test+0x78460e) https://github.com/facebook/rocksdb/issues/2 rocksdb::VersionSet::SetLastSequence(unsigned long) /home/circleci/project/./db/version_set.h:1058:20 (db_test+0x78460e) ... Previous read of size 8 at 0x7b54000005e8 by thread T31: #0 bool rocksdb::DBImpl::MultiCFSnapshot<std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > > >(rocksdb::ReadOptions const&, rocksdb::ReadCallback*, std::function<rocksdb::DBImpl::MultiGetColumnFamilyData* (std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >::iterator&)>&, std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >*, unsigned long*) /home/circleci/project/db/db_impl/db_impl.cc (db_test+0x715087) ``` Reviewed By: ltamasi Differential Revision: D23725226 Pulled By: jay-zhuang fbshipit-source-id: a6d662a5ea68111246cd32ec95f3411a25f76bc6
4 years ago
uint64_t LastSequence() const {
return last_sequence_.load(std::memory_order_acquire);
}
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
8 years ago
// Note: memory_order_acquire must be sufficient.
uint64_t LastAllocatedSequence() const {
return last_allocated_sequence_.load(std::memory_order_seq_cst);
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
8 years ago
}
// Note: memory_order_acquire must be sufficient.
Fix TSAN build and re-enable the tests (#7386) Summary: Resolve TSAN build warnings and re-enable disabled TSAN tests. Not sure if it's a compiler issue or TSAN check issue. Switching from conditional operator to if-else mitigated the problem. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7386 Test Plan: run TSAN check 10 times in circleci. ``` WARNING: ThreadSanitizer: data race (pid=27735) Atomic write of size 8 at 0x7b54000005e8 by thread T32: #0 __tsan_atomic64_store <null> (db_test+0x4cee95) https://github.com/facebook/rocksdb/issues/1 std::__atomic_base<unsigned long>::store(unsigned long, std::memory_order) /usr/bin/../lib/gcc/x86_64-linux-gnu/5.4.0/../../../../include/c++/5.4.0/bits/atomic_base.h:374:2 (db_test+0x78460e) https://github.com/facebook/rocksdb/issues/2 rocksdb::VersionSet::SetLastSequence(unsigned long) /home/circleci/project/./db/version_set.h:1058:20 (db_test+0x78460e) ... Previous read of size 8 at 0x7b54000005e8 by thread T31: #0 bool rocksdb::DBImpl::MultiCFSnapshot<std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > > >(rocksdb::ReadOptions const&, rocksdb::ReadCallback*, std::function<rocksdb::DBImpl::MultiGetColumnFamilyData* (std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >::iterator&)>&, std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >*, unsigned long*) /home/circleci/project/db/db_impl/db_impl.cc (db_test+0x715087) ``` Reviewed By: ltamasi Differential Revision: D23725226 Pulled By: jay-zhuang fbshipit-source-id: a6d662a5ea68111246cd32ec95f3411a25f76bc6
4 years ago
uint64_t LastPublishedSequence() const {
return last_published_sequence_.load(std::memory_order_seq_cst);
}
// Set the last sequence number to s.
Fix TSAN build and re-enable the tests (#7386) Summary: Resolve TSAN build warnings and re-enable disabled TSAN tests. Not sure if it's a compiler issue or TSAN check issue. Switching from conditional operator to if-else mitigated the problem. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7386 Test Plan: run TSAN check 10 times in circleci. ``` WARNING: ThreadSanitizer: data race (pid=27735) Atomic write of size 8 at 0x7b54000005e8 by thread T32: #0 __tsan_atomic64_store <null> (db_test+0x4cee95) https://github.com/facebook/rocksdb/issues/1 std::__atomic_base<unsigned long>::store(unsigned long, std::memory_order) /usr/bin/../lib/gcc/x86_64-linux-gnu/5.4.0/../../../../include/c++/5.4.0/bits/atomic_base.h:374:2 (db_test+0x78460e) https://github.com/facebook/rocksdb/issues/2 rocksdb::VersionSet::SetLastSequence(unsigned long) /home/circleci/project/./db/version_set.h:1058:20 (db_test+0x78460e) ... Previous read of size 8 at 0x7b54000005e8 by thread T31: #0 bool rocksdb::DBImpl::MultiCFSnapshot<std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > > >(rocksdb::ReadOptions const&, rocksdb::ReadCallback*, std::function<rocksdb::DBImpl::MultiGetColumnFamilyData* (std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >::iterator&)>&, std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >*, unsigned long*) /home/circleci/project/db/db_impl/db_impl.cc (db_test+0x715087) ``` Reviewed By: ltamasi Differential Revision: D23725226 Pulled By: jay-zhuang fbshipit-source-id: a6d662a5ea68111246cd32ec95f3411a25f76bc6
4 years ago
void SetLastSequence(uint64_t s) {
assert(s >= last_sequence_);
// Last visible sequence must always be less than last written seq
assert(!db_options_->two_write_queues || s <= last_allocated_sequence_);
last_sequence_.store(s, std::memory_order_release);
}
// Note: memory_order_release must be sufficient
Fix TSAN build and re-enable the tests (#7386) Summary: Resolve TSAN build warnings and re-enable disabled TSAN tests. Not sure if it's a compiler issue or TSAN check issue. Switching from conditional operator to if-else mitigated the problem. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7386 Test Plan: run TSAN check 10 times in circleci. ``` WARNING: ThreadSanitizer: data race (pid=27735) Atomic write of size 8 at 0x7b54000005e8 by thread T32: #0 __tsan_atomic64_store <null> (db_test+0x4cee95) https://github.com/facebook/rocksdb/issues/1 std::__atomic_base<unsigned long>::store(unsigned long, std::memory_order) /usr/bin/../lib/gcc/x86_64-linux-gnu/5.4.0/../../../../include/c++/5.4.0/bits/atomic_base.h:374:2 (db_test+0x78460e) https://github.com/facebook/rocksdb/issues/2 rocksdb::VersionSet::SetLastSequence(unsigned long) /home/circleci/project/./db/version_set.h:1058:20 (db_test+0x78460e) ... Previous read of size 8 at 0x7b54000005e8 by thread T31: #0 bool rocksdb::DBImpl::MultiCFSnapshot<std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > > >(rocksdb::ReadOptions const&, rocksdb::ReadCallback*, std::function<rocksdb::DBImpl::MultiGetColumnFamilyData* (std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >::iterator&)>&, std::unordered_map<unsigned int, rocksdb::DBImpl::MultiGetColumnFamilyData, std::hash<unsigned int>, std::equal_to<unsigned int>, std::allocator<std::pair<unsigned int const, rocksdb::DBImpl::MultiGetColumnFamilyData> > >*, unsigned long*) /home/circleci/project/db/db_impl/db_impl.cc (db_test+0x715087) ``` Reviewed By: ltamasi Differential Revision: D23725226 Pulled By: jay-zhuang fbshipit-source-id: a6d662a5ea68111246cd32ec95f3411a25f76bc6
4 years ago
void SetLastPublishedSequence(uint64_t s) {
assert(s >= last_published_sequence_);
last_published_sequence_.store(s, std::memory_order_seq_cst);
}
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
8 years ago
// Note: memory_order_release must be sufficient
void SetLastAllocatedSequence(uint64_t s) {
assert(s >= last_allocated_sequence_);
last_allocated_sequence_.store(s, std::memory_order_seq_cst);
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
8 years ago
}
// Note: memory_order_release must be sufficient
uint64_t FetchAddLastAllocatedSequence(uint64_t s) {
return last_allocated_sequence_.fetch_add(s, std::memory_order_seq_cst);
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
8 years ago
}
// Mark the specified file number as used.
// REQUIRED: this is only called during single-threaded recovery or repair.
void MarkFileNumberUsed(uint64_t number);
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
// Mark the specified log number as deleted
// REQUIRED: this is only called during single-threaded recovery or repair, or
// from ::LogAndApply where the global mutex is held.
Fix a race condition in WAL tracking causing DB open failure (#9715) Summary: There is a race condition if WAL tracking in the MANIFEST is enabled in a database that disables 2PC. The race condition is between two background flush threads trying to install flush results to the MANIFEST. Consider an example database with two column families: "default" (cfd0) and "cf1" (cfd1). Initially, both column families have one mutable (active) memtable whose data backed by 6.log. 1. Trigger a manual flush for "cf1", creating a 7.log 2. Insert another key to "default", and trigger flush for "default", creating 8.log 3. BgFlushThread1 finishes writing 9.sst 4. BgFlushThread2 finishes writing 10.sst ``` Time BgFlushThread1 BgFlushThread2 | mutex_.Lock() | precompute min_wal_to_keep as 6 | mutex_.Unlock() | mutex_.Lock() | precompute min_wal_to_keep as 6 | join MANIFEST write queue and mutex_.Unlock() | write to MANIFEST | mutex_.Lock() | cfd1->log_number = 7 | Signal bg_flush_2 and mutex_.Unlock() | wake up and mutex_.Lock() | cfd0->log_number = 8 | FindObsoleteFiles() with job_context->log_number == 7 | mutex_.Unlock() | PurgeObsoleteFiles() deletes 6.log V ``` As shown in the above, BgFlushThread2 thinks that the min wal to keep is 6.log because "cf1" has unflushed data in 6.log (cf1.log_number=6). Similarly, BgThread1 thinks that min wal to keep is also 6.log because "default" has unflushed data (default.log_number=6). No WAL deletion will be written to MANIFEST because 6 is equal to `versions_->wals_.min_wal_number_to_keep`, due to https://github.com/facebook/rocksdb/blob/7.1.fb/db/memtable_list.cc#L513:L514. The bg flush thread that finishes last will perform file purging. `job_context.log_number` will be evaluated as 7, i.e. the min wal that contains unflushed data, causing 6.log to be deleted. However, MANIFEST thinks 6.log should still exist. If you close the db at this point, you won't be able to re-open it if `track_and_verify_wal_in_manifest` is true. We must handle the case of multiple bg flush threads, and it is difficult for one bg flush thread to know the correct min wal number until the other bg flush threads have finished committing to the manifest and updated the `cfd::log_number`. To fix this issue, we rename an existing variable `min_log_number_to_keep_2pc` to `min_log_number_to_keep`, and use it to track WAL file deletion in non-2pc mode as well. This variable is updated only 1) during recovery with mutex held, or 2) in the MANIFEST write thread. `min_log_number_to_keep` means RocksDB will delete WALs below it, although there may be WALs above it which are also obsolete. Formally, we will have [min_wal_to_keep, max_obsolete_wal]. During recovery, we make sure that only WALs above max_obsolete_wal are checked and added back to `alive_log_files_`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9715 Test Plan: ``` make check ``` Also ran stress test below (with asan) to make sure it completes successfully. ``` TEST_TMPDIR=/dev/shm/rocksdb OPT=-g ASAN_OPTIONS=disable_coredump=0 \ CRASH_TEST_EXT_ARGS=--compression_type=zstd SKIP_FORMAT_BUCK_CHECKS=1 \ make J=52 -j52 blackbox_asan_crash_test ``` Reviewed By: ltamasi Differential Revision: D34984412 Pulled By: riversand963 fbshipit-source-id: c7b21a8d84751bb55ea79c9f387103d21b231005
3 years ago
void MarkMinLogNumberToKeep(uint64_t number);
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
// Return the log file number for the log file that is currently
// being compacted, or zero if there is no such log file.
uint64_t prev_log_number() const { return prev_log_number_; }
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
// Returns the minimum log number which still has data not flushed to any SST
// file.
// In non-2PC mode, all the log numbers smaller than this number can be safely
Fix a race condition in WAL tracking causing DB open failure (#9715) Summary: There is a race condition if WAL tracking in the MANIFEST is enabled in a database that disables 2PC. The race condition is between two background flush threads trying to install flush results to the MANIFEST. Consider an example database with two column families: "default" (cfd0) and "cf1" (cfd1). Initially, both column families have one mutable (active) memtable whose data backed by 6.log. 1. Trigger a manual flush for "cf1", creating a 7.log 2. Insert another key to "default", and trigger flush for "default", creating 8.log 3. BgFlushThread1 finishes writing 9.sst 4. BgFlushThread2 finishes writing 10.sst ``` Time BgFlushThread1 BgFlushThread2 | mutex_.Lock() | precompute min_wal_to_keep as 6 | mutex_.Unlock() | mutex_.Lock() | precompute min_wal_to_keep as 6 | join MANIFEST write queue and mutex_.Unlock() | write to MANIFEST | mutex_.Lock() | cfd1->log_number = 7 | Signal bg_flush_2 and mutex_.Unlock() | wake up and mutex_.Lock() | cfd0->log_number = 8 | FindObsoleteFiles() with job_context->log_number == 7 | mutex_.Unlock() | PurgeObsoleteFiles() deletes 6.log V ``` As shown in the above, BgFlushThread2 thinks that the min wal to keep is 6.log because "cf1" has unflushed data in 6.log (cf1.log_number=6). Similarly, BgThread1 thinks that min wal to keep is also 6.log because "default" has unflushed data (default.log_number=6). No WAL deletion will be written to MANIFEST because 6 is equal to `versions_->wals_.min_wal_number_to_keep`, due to https://github.com/facebook/rocksdb/blob/7.1.fb/db/memtable_list.cc#L513:L514. The bg flush thread that finishes last will perform file purging. `job_context.log_number` will be evaluated as 7, i.e. the min wal that contains unflushed data, causing 6.log to be deleted. However, MANIFEST thinks 6.log should still exist. If you close the db at this point, you won't be able to re-open it if `track_and_verify_wal_in_manifest` is true. We must handle the case of multiple bg flush threads, and it is difficult for one bg flush thread to know the correct min wal number until the other bg flush threads have finished committing to the manifest and updated the `cfd::log_number`. To fix this issue, we rename an existing variable `min_log_number_to_keep_2pc` to `min_log_number_to_keep`, and use it to track WAL file deletion in non-2pc mode as well. This variable is updated only 1) during recovery with mutex held, or 2) in the MANIFEST write thread. `min_log_number_to_keep` means RocksDB will delete WALs below it, although there may be WALs above it which are also obsolete. Formally, we will have [min_wal_to_keep, max_obsolete_wal]. During recovery, we make sure that only WALs above max_obsolete_wal are checked and added back to `alive_log_files_`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9715 Test Plan: ``` make check ``` Also ran stress test below (with asan) to make sure it completes successfully. ``` TEST_TMPDIR=/dev/shm/rocksdb OPT=-g ASAN_OPTIONS=disable_coredump=0 \ CRASH_TEST_EXT_ARGS=--compression_type=zstd SKIP_FORMAT_BUCK_CHECKS=1 \ make J=52 -j52 blackbox_asan_crash_test ``` Reviewed By: ltamasi Differential Revision: D34984412 Pulled By: riversand963 fbshipit-source-id: c7b21a8d84751bb55ea79c9f387103d21b231005
3 years ago
// deleted, although we still use `min_log_number_to_keep_` to determine when
// to delete a WAL file.
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
uint64_t MinLogNumberWithUnflushedData() const {
return PreComputeMinLogNumberWithUnflushedData(nullptr);
}
Fix a race condition in WAL tracking causing DB open failure (#9715) Summary: There is a race condition if WAL tracking in the MANIFEST is enabled in a database that disables 2PC. The race condition is between two background flush threads trying to install flush results to the MANIFEST. Consider an example database with two column families: "default" (cfd0) and "cf1" (cfd1). Initially, both column families have one mutable (active) memtable whose data backed by 6.log. 1. Trigger a manual flush for "cf1", creating a 7.log 2. Insert another key to "default", and trigger flush for "default", creating 8.log 3. BgFlushThread1 finishes writing 9.sst 4. BgFlushThread2 finishes writing 10.sst ``` Time BgFlushThread1 BgFlushThread2 | mutex_.Lock() | precompute min_wal_to_keep as 6 | mutex_.Unlock() | mutex_.Lock() | precompute min_wal_to_keep as 6 | join MANIFEST write queue and mutex_.Unlock() | write to MANIFEST | mutex_.Lock() | cfd1->log_number = 7 | Signal bg_flush_2 and mutex_.Unlock() | wake up and mutex_.Lock() | cfd0->log_number = 8 | FindObsoleteFiles() with job_context->log_number == 7 | mutex_.Unlock() | PurgeObsoleteFiles() deletes 6.log V ``` As shown in the above, BgFlushThread2 thinks that the min wal to keep is 6.log because "cf1" has unflushed data in 6.log (cf1.log_number=6). Similarly, BgThread1 thinks that min wal to keep is also 6.log because "default" has unflushed data (default.log_number=6). No WAL deletion will be written to MANIFEST because 6 is equal to `versions_->wals_.min_wal_number_to_keep`, due to https://github.com/facebook/rocksdb/blob/7.1.fb/db/memtable_list.cc#L513:L514. The bg flush thread that finishes last will perform file purging. `job_context.log_number` will be evaluated as 7, i.e. the min wal that contains unflushed data, causing 6.log to be deleted. However, MANIFEST thinks 6.log should still exist. If you close the db at this point, you won't be able to re-open it if `track_and_verify_wal_in_manifest` is true. We must handle the case of multiple bg flush threads, and it is difficult for one bg flush thread to know the correct min wal number until the other bg flush threads have finished committing to the manifest and updated the `cfd::log_number`. To fix this issue, we rename an existing variable `min_log_number_to_keep_2pc` to `min_log_number_to_keep`, and use it to track WAL file deletion in non-2pc mode as well. This variable is updated only 1) during recovery with mutex held, or 2) in the MANIFEST write thread. `min_log_number_to_keep` means RocksDB will delete WALs below it, although there may be WALs above it which are also obsolete. Formally, we will have [min_wal_to_keep, max_obsolete_wal]. During recovery, we make sure that only WALs above max_obsolete_wal are checked and added back to `alive_log_files_`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9715 Test Plan: ``` make check ``` Also ran stress test below (with asan) to make sure it completes successfully. ``` TEST_TMPDIR=/dev/shm/rocksdb OPT=-g ASAN_OPTIONS=disable_coredump=0 \ CRASH_TEST_EXT_ARGS=--compression_type=zstd SKIP_FORMAT_BUCK_CHECKS=1 \ make J=52 -j52 blackbox_asan_crash_test ``` Reviewed By: ltamasi Differential Revision: D34984412 Pulled By: riversand963 fbshipit-source-id: c7b21a8d84751bb55ea79c9f387103d21b231005
3 years ago
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
// Returns the minimum log number which still has data not flushed to any SST
Track WAL obsoletion when updating empty CF's log number (#7781) Summary: In the write path, there is an optimization: when a new WAL is created during SwitchMemtable, we update the internal log number of the empty column families to the new WAL. `FindObsoleteFiles` marks a WAL as obsolete if the WAL's log number is less than `VersionSet::MinLogNumberWithUnflushedData`. After updating the empty column families' internal log number, `VersionSet::MinLogNumberWithUnflushedData` might change, so some WALs might become obsolete to be purged from disk. For example, consider there are 3 column families: 0, 1, 2: 1. initially, all the column families' log number is 1; 2. write some data to cf0, and flush cf0, but the flush is pending; 3. now a new WAL 2 is created; 4. write data to cf1 and WAL 2, now cf0's log number is 1, cf1's log number is 2, cf2's log number is 2 (because cf1 and cf2 are empty, so their log numbers will be set to the highest log number); 5. now cf0's flush hasn't finished, flush cf1, a new WAL 3 is created, and cf1's flush finishes, now cf0's log number is 1, cf1's log number is 3, cf2's log number is 3, since WAL 1 still contains data for the unflushed cf0, no WAL can be deleted from disk; 6. now cf0's flush finishes, cf0's log number is 2 (because when cf0 was switching memtable, WAL 3 does not exist yet), cf1's log number is 3, cf2's log number is 3, so WAL 1 can be purged from disk now, but WAL 2 still cannot because `MinLogNumberToKeep()` is 2; 7. write data to cf2 and WAL 3, because cf0 is empty, its log number is updated to 3, so now cf0's log number is 3, cf1's log number is 3, cf2's log number is 3; 8. now if the background threads want to purge obsolete files from disk, WAL 2 can be purged because `MinLogNumberToKeep()` is 3. But there are only two flush results written to MANIFEST: the first is for flushing cf1, and the `MinLogNumberToKeep` is 1, the second is for flushing cf0, and the `MinLogNumberToKeep` is 2. So without this PR, if the DB crashes at this point and try to recover, `WalSet` will still expect WAL 2 to exist. When WAL tracking is enabled, we assume WALs will only become obsolete after a flush result is written to MANIFEST in `MemtableList::TryInstallMemtableFlushResults` (or its atomic flush counterpart). The above situation breaks this assumption. This PR tracks WAL obsoletion if necessary before updating the empty column families' log numbers. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7781 Test Plan: watch existing tests and stress tests to pass. `make -j48 blackbox_crash_test` on devserver Reviewed By: ltamasi Differential Revision: D25631695 Pulled By: cheng-chang fbshipit-source-id: ca7fff967bdb42204b84226063d909893bc0a4ec
4 years ago
// file.
// Empty column families' log number is considered to be
// new_log_number_for_empty_cf.
uint64_t PreComputeMinLogNumberWithUnflushedData(
uint64_t new_log_number_for_empty_cf) const {
uint64_t min_log_num = std::numeric_limits<uint64_t>::max();
Track WAL obsoletion when updating empty CF's log number (#7781) Summary: In the write path, there is an optimization: when a new WAL is created during SwitchMemtable, we update the internal log number of the empty column families to the new WAL. `FindObsoleteFiles` marks a WAL as obsolete if the WAL's log number is less than `VersionSet::MinLogNumberWithUnflushedData`. After updating the empty column families' internal log number, `VersionSet::MinLogNumberWithUnflushedData` might change, so some WALs might become obsolete to be purged from disk. For example, consider there are 3 column families: 0, 1, 2: 1. initially, all the column families' log number is 1; 2. write some data to cf0, and flush cf0, but the flush is pending; 3. now a new WAL 2 is created; 4. write data to cf1 and WAL 2, now cf0's log number is 1, cf1's log number is 2, cf2's log number is 2 (because cf1 and cf2 are empty, so their log numbers will be set to the highest log number); 5. now cf0's flush hasn't finished, flush cf1, a new WAL 3 is created, and cf1's flush finishes, now cf0's log number is 1, cf1's log number is 3, cf2's log number is 3, since WAL 1 still contains data for the unflushed cf0, no WAL can be deleted from disk; 6. now cf0's flush finishes, cf0's log number is 2 (because when cf0 was switching memtable, WAL 3 does not exist yet), cf1's log number is 3, cf2's log number is 3, so WAL 1 can be purged from disk now, but WAL 2 still cannot because `MinLogNumberToKeep()` is 2; 7. write data to cf2 and WAL 3, because cf0 is empty, its log number is updated to 3, so now cf0's log number is 3, cf1's log number is 3, cf2's log number is 3; 8. now if the background threads want to purge obsolete files from disk, WAL 2 can be purged because `MinLogNumberToKeep()` is 3. But there are only two flush results written to MANIFEST: the first is for flushing cf1, and the `MinLogNumberToKeep` is 1, the second is for flushing cf0, and the `MinLogNumberToKeep` is 2. So without this PR, if the DB crashes at this point and try to recover, `WalSet` will still expect WAL 2 to exist. When WAL tracking is enabled, we assume WALs will only become obsolete after a flush result is written to MANIFEST in `MemtableList::TryInstallMemtableFlushResults` (or its atomic flush counterpart). The above situation breaks this assumption. This PR tracks WAL obsoletion if necessary before updating the empty column families' log numbers. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7781 Test Plan: watch existing tests and stress tests to pass. `make -j48 blackbox_crash_test` on devserver Reviewed By: ltamasi Differential Revision: D25631695 Pulled By: cheng-chang fbshipit-source-id: ca7fff967bdb42204b84226063d909893bc0a4ec
4 years ago
for (auto cfd : *column_family_set_) {
// It's safe to ignore dropped column families here:
// cfd->IsDropped() becomes true after the drop is persisted in MANIFEST.
uint64_t num =
cfd->IsEmpty() ? new_log_number_for_empty_cf : cfd->GetLogNumber();
if (min_log_num > num && !cfd->IsDropped()) {
min_log_num = num;
}
}
return min_log_num;
}
// Returns the minimum log number which still has data not flushed to any SST
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
// file, except data from `cfd_to_skip`.
uint64_t PreComputeMinLogNumberWithUnflushedData(
const ColumnFamilyData* cfd_to_skip) const {
uint64_t min_log_num = std::numeric_limits<uint64_t>::max();
for (auto cfd : *column_family_set_) {
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
if (cfd == cfd_to_skip) {
continue;
}
// It's safe to ignore dropped column families here:
// cfd->IsDropped() becomes true after the drop is persisted in MANIFEST.
if (min_log_num > cfd->GetLogNumber() && !cfd->IsDropped()) {
min_log_num = cfd->GetLogNumber();
}
}
return min_log_num;
}
// Returns the minimum log number which still has data not flushed to any SST
// file, except data from `cfds_to_skip`.
uint64_t PreComputeMinLogNumberWithUnflushedData(
const std::unordered_set<const ColumnFamilyData*>& cfds_to_skip) const {
uint64_t min_log_num = std::numeric_limits<uint64_t>::max();
for (auto cfd : *column_family_set_) {
if (cfds_to_skip.count(cfd)) {
continue;
}
// It's safe to ignore dropped column families here:
// cfd->IsDropped() becomes true after the drop is persisted in MANIFEST.
if (min_log_num > cfd->GetLogNumber() && !cfd->IsDropped()) {
min_log_num = cfd->GetLogNumber();
}
}
return min_log_num;
}
// Create an iterator that reads over the compaction inputs for "*c".
// The caller should delete the iterator when no longer needed.
// @param read_options Must outlive the returned iterator.
// @param start, end indicates compaction range
InternalIterator* MakeInputIterator(
const ReadOptions& read_options, const Compaction* c,
RangeDelAggregator* range_del_agg,
const FileOptions& file_options_compactions,
const std::optional<const Slice>& start,
const std::optional<const Slice>& end);
// Add all files listed in any live version to *live_table_files and
// *live_blob_files. Note that these lists may contain duplicates.
void AddLiveFiles(std::vector<uint64_t>* live_table_files,
std::vector<uint64_t>* live_blob_files) const;
// Remove live files that are in the delete candidate lists.
void RemoveLiveFiles(
std::vector<ObsoleteFileInfo>& sst_delete_candidates,
std::vector<ObsoleteBlobFileInfo>& blob_delete_candidates) const;
// Return the approximate size of data to be scanned for range [start, end)
// in levels [start_level, end_level). If end_level == -1 it will search
// through all non-empty levels
uint64_t ApproximateSize(const SizeApproximationOptions& options, Version* v,
const Slice& start, const Slice& end,
int start_level, int end_level,
TableReaderCaller caller);
// Return the size of the current manifest file
uint64_t manifest_file_size() const { return manifest_file_size_; }
Status GetMetadataForFile(uint64_t number, int* filelevel,
FileMetaData** metadata, ColumnFamilyData** cfd);
// This function doesn't support leveldb SST filenames
void GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata);
void AddObsoleteBlobFile(uint64_t blob_file_number, std::string path) {
assert(table_cache_);
table_cache_->Erase(GetSlice(&blob_file_number));
obsolete_blob_files_.emplace_back(blob_file_number, std::move(path));
}
void GetObsoleteFiles(std::vector<ObsoleteFileInfo>* files,
std::vector<ObsoleteBlobFileInfo>* blob_files,
std::vector<std::string>* manifest_filenames,
uint64_t min_pending_output);
ColumnFamilySet* GetColumnFamilySet() { return column_family_set_.get(); }
RefedColumnFamilySet GetRefedColumnFamilySet() {
return RefedColumnFamilySet(GetColumnFamilySet());
}
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
const FileOptions& file_options() { return file_options_; }
void ChangeFileOptions(const MutableDBOptions& new_options) {
file_options_.writable_file_max_buffer_size =
new_options.writable_file_max_buffer_size;
}
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
7 years ago
const ImmutableDBOptions* db_options() const { return db_options_; }
static uint64_t GetNumLiveVersions(Version* dummy_versions);
static uint64_t GetTotalSstFilesSize(Version* dummy_versions);
Add DB properties for BlobDB (#8734) Summary: RocksDB exposes certain internal statistics via the DB property interface. However, there are currently no properties related to BlobDB. For starters, we would like to add the following BlobDB properties: `rocksdb.num-blob-files`: number of blob files in the current Version (kind of like `num-files-at-level` but note this is not per level, since blob files are not part of the LSM tree). `rocksdb.blob-stats`: this could return the total number and size of all blob files, and potentially also the total amount of garbage (in bytes) in the blob files in the current Version. `rocksdb.total-blob-file-size`: the total size of all blob files (as a blob counterpart for `total-sst-file-size`) of all Versions. `rocksdb.live-blob-file-size`: the total size of all blob files in the current Version. `rocksdb.estimate-live-data-size`: this is actually an existing property that we can extend so it considers blob files as well. When it comes to blobs, we actually have an exact value for live bytes. Namely, live bytes can be computed simply as total bytes minus garbage bytes, summed over the entire set of blob files in the Version. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8734 Test Plan: ``` ➜ rocksdb git:(new_feature_blobDB_properties) ./db_blob_basic_test [==========] Running 16 tests from 2 test cases. [----------] Global test environment set-up. [----------] 10 tests from DBBlobBasicTest [ RUN ] DBBlobBasicTest.GetBlob [ OK ] DBBlobBasicTest.GetBlob (12 ms) [ RUN ] DBBlobBasicTest.MultiGetBlobs [ OK ] DBBlobBasicTest.MultiGetBlobs (11 ms) [ RUN ] DBBlobBasicTest.GetBlob_CorruptIndex [ OK ] DBBlobBasicTest.GetBlob_CorruptIndex (10 ms) [ RUN ] DBBlobBasicTest.GetBlob_InlinedTTLIndex [ OK ] DBBlobBasicTest.GetBlob_InlinedTTLIndex (12 ms) [ RUN ] DBBlobBasicTest.GetBlob_IndexWithInvalidFileNumber [ OK ] DBBlobBasicTest.GetBlob_IndexWithInvalidFileNumber (9 ms) [ RUN ] DBBlobBasicTest.GenerateIOTracing [ OK ] DBBlobBasicTest.GenerateIOTracing (11 ms) [ RUN ] DBBlobBasicTest.BestEffortsRecovery_MissingNewestBlobFile [ OK ] DBBlobBasicTest.BestEffortsRecovery_MissingNewestBlobFile (13 ms) [ RUN ] DBBlobBasicTest.GetMergeBlobWithPut [ OK ] DBBlobBasicTest.GetMergeBlobWithPut (11 ms) [ RUN ] DBBlobBasicTest.MultiGetMergeBlobWithPut [ OK ] DBBlobBasicTest.MultiGetMergeBlobWithPut (14 ms) [ RUN ] DBBlobBasicTest.BlobDBProperties [ OK ] DBBlobBasicTest.BlobDBProperties (21 ms) [----------] 10 tests from DBBlobBasicTest (124 ms total) [----------] 6 tests from DBBlobBasicTest/DBBlobBasicIOErrorTest [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/0 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/0 (12 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/1 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.GetBlob_IOError/1 (10 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/0 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/0 (10 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/1 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.MultiGetBlobs_IOError/1 (10 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/0 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/0 (1011 ms) [ RUN ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/1 [ OK ] DBBlobBasicTest/DBBlobBasicIOErrorTest.CompactionFilterReadBlob_IOError/1 (1013 ms) [----------] 6 tests from DBBlobBasicTest/DBBlobBasicIOErrorTest (2066 ms total) [----------] Global test environment tear-down [==========] 16 tests from 2 test cases ran. (2190 ms total) [ PASSED ] 16 tests. ``` Reviewed By: ltamasi Differential Revision: D30690849 Pulled By: Zhiyi-Zhang fbshipit-source-id: a7567319487ad76bd1a2e24bf143afdbbd9e4346
3 years ago
static uint64_t GetTotalBlobFileSize(Version* dummy_versions);
// Get the IO Status returned by written Manifest.
const IOStatus& io_status() const { return io_status_; }
// The returned WalSet needs to be accessed with DB mutex held.
Define WAL related classes to be used in VersionEdit and VersionSet (#7164) Summary: `WalAddition`, `WalDeletion` are defined in `wal_version.h` and used in `VersionEdit`. `WalAddition` is used to represent events of creating a new WAL (no size, just log number), or closing a WAL (with size). `WalDeletion` is used to represent events of deleting or archiving a WAL, it means the WAL is no longer alive (won't be replayed during recovery). `WalSet` is the set of alive WALs kept in `VersionSet`. 1. Why use `WalDeletion` instead of relying on `MinLogNumber` to identify outdated WALs On recovery, we can compute `MinLogNumber()` based on the log numbers kept in MANIFEST, any log with number < MinLogNumber can be ignored. So it seems that we don't need to persist `WalDeletion` to MANIFEST, since we can ignore the WALs based on MinLogNumber. But the `MinLogNumber()` is actually a lower bound, it does not exactly mean that logs starting from MinLogNumber must exist. This is because in a corner case, when a column family is empty and never flushed, its log number is set to the largest log number, but not persisted in MANIFEST. So let's say there are 2 column families, when creating the DB, the first WAL has log number 1, so it's persisted to MANIFEST for both column families. Then CF 0 is empty and never flushed, CF 1 is updated and flushed, so a new WAL with log number 2 is created and persisted to MANIFEST for CF 1. But CF 0's log number in MANIFEST is still 1. So on recovery, MinLogNumber is 1, but since log 1 only contains data for CF 1, and CF 1 is flushed, log 1 might have already been deleted from disk. We can make `MinLogNumber()` be the exactly minimum log number that must exist, by persisting the most recent log number for empty column families that are not flushed. But if there are N such column families, then every time a new WAL is created, we need to add N records to MANIFEST. In current design, a record is persisted to MANIFEST only when WAL is created, closed, or deleted/archived, so the number of WAL related records are bounded to 3x number of WALs. 2. Why keep `WalSet` in `VersionSet` instead of applying the `VersionEdit`s to `VersionStorageInfo` `VersionEdit`s are originally designed to track the addition and deletion of SST files. The SST files are related to column families, each column family has a list of `Version`s, and each `Version` keeps the set of active SST files in `VersionStorageInfo`. But WALs are a concept of DB, they are not bounded to specific column families. So logically it does not make sense to store WALs in a column family's `Version`s. Also, `Version`'s purpose is to keep reference to SST / blob files, so that they are not deleted until there is no version referencing them. But a WAL is deleted regardless of version references. So we keep the WALs in `VersionSet` for the purpose of writing out the DB state's snapshot when creating new MANIFESTs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7164 Test Plan: make version_edit_test && ./version_edit_test make wal_edit_test && ./wal_edit_test Reviewed By: ltamasi Differential Revision: D22677936 Pulled By: cheng-chang fbshipit-source-id: 5a3b6890140e572ffd79eb37e6e4c3c32361a859
4 years ago
const WalSet& GetWalSet() const { return wals_; }
Clean up blob files based on the linked SST set (#7001) Summary: The earlier `VersionBuilder` code only cleaned up blob files that were marked as entirely consisting of garbage using `VersionEdits` with `BlobFileGarbage`. This covers the cases when table files go through regular compaction, where we iterate through the KVs and thus have an opportunity to calculate the amount of garbage (that is, most cases). However, it does not help when table files are simply dropped (e.g. deletion compactions or the `DeleteFile` API). To deal with such cases, the patch adds logic that cleans up all blob files at the head of the list until the first one with linked SSTs is found. (As an example, let's assume we have blob files with numbers 1..10, and the first one with any linked SSTs is number 8. This means that SSTs in the `Version` only rely on blob files with numbers >= 8, and thus 1..7 are no longer needed.) The code change itself is pretty small; however, changing the logic like this necessitated changes to some tests that have been added recently (namely to the ones that use blob files in isolation, i.e. without any table files referring to them). Some of these cases were fixed by bypassing `VersionBuilder` altogether in order to keep the tests simple (which actually makes them more proper unit tests as well), while the `VersionBuilder` unit tests were fixed by adding dummy table files to the test cases as needed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7001 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D22119474 Pulled By: ltamasi fbshipit-source-id: c6547141355667d4291d9661d6518eb741e7b54a
5 years ago
void TEST_CreateAndAppendVersion(ColumnFamilyData* cfd) {
assert(cfd);
const auto& mutable_cf_options = *cfd->GetLatestMutableCFOptions();
Version* const version =
new Version(cfd, this, file_options_, mutable_cf_options, io_tracer_);
Clean up blob files based on the linked SST set (#7001) Summary: The earlier `VersionBuilder` code only cleaned up blob files that were marked as entirely consisting of garbage using `VersionEdits` with `BlobFileGarbage`. This covers the cases when table files go through regular compaction, where we iterate through the KVs and thus have an opportunity to calculate the amount of garbage (that is, most cases). However, it does not help when table files are simply dropped (e.g. deletion compactions or the `DeleteFile` API). To deal with such cases, the patch adds logic that cleans up all blob files at the head of the list until the first one with linked SSTs is found. (As an example, let's assume we have blob files with numbers 1..10, and the first one with any linked SSTs is number 8. This means that SSTs in the `Version` only rely on blob files with numbers >= 8, and thus 1..7 are no longer needed.) The code change itself is pretty small; however, changing the logic like this necessitated changes to some tests that have been added recently (namely to the ones that use blob files in isolation, i.e. without any table files referring to them). Some of these cases were fixed by bypassing `VersionBuilder` altogether in order to keep the tests simple (which actually makes them more proper unit tests as well), while the `VersionBuilder` unit tests were fixed by adding dummy table files to the test cases as needed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7001 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D22119474 Pulled By: ltamasi fbshipit-source-id: c6547141355667d4291d9661d6518eb741e7b54a
5 years ago
constexpr bool update_stats = false;
Clean up VersionStorageInfo a bit (#9494) Summary: The patch does some cleanup in and around `VersionStorageInfo`: * Renames the method `PrepareApply` to `PrepareAppend` in `Version` to make it clear that it is to be called before appending the `Version` to `VersionSet` (via `AppendVersion`), not before applying any `VersionEdit`s. * Introduces a helper method `VersionStorageInfo::PrepareForVersionAppend` (called by `Version::PrepareAppend`) that encapsulates the population of the various derived data structures in `VersionStorageInfo`, and turns the methods computing the derived structures (`UpdateNumNonEmptyLevels`, `CalculateBaseBytes` etc.) into private helpers. * Changes `Version::PrepareAppend` so it only calls `UpdateAccumulatedStats` if the `update_stats` flag is set. (Earlier, this was checked by the callee.) Related to this, it also moves the call to `ComputeCompensatedSizes` to `VersionStorageInfo::PrepareForVersionAppend`. * Updates and cleans up `version_builder_test`, `version_set_test`, and `compaction_picker_test` so `PrepareForVersionAppend` is called anytime a new `VersionStorageInfo` is set up or saved. This cleanup also involves splitting `VersionStorageInfoTest.MaxBytesForLevelDynamic` into multiple smaller test cases. * Fixes up a bunch of comments that were outdated or just plain incorrect. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9494 Test Plan: Ran `make check` and the crash test script for a while. Reviewed By: riversand963 Differential Revision: D33971666 Pulled By: ltamasi fbshipit-source-id: fda52faac7783041126e4f8dec0fe01bdcadf65a
3 years ago
version->PrepareAppend(mutable_cf_options, update_stats);
Clean up blob files based on the linked SST set (#7001) Summary: The earlier `VersionBuilder` code only cleaned up blob files that were marked as entirely consisting of garbage using `VersionEdits` with `BlobFileGarbage`. This covers the cases when table files go through regular compaction, where we iterate through the KVs and thus have an opportunity to calculate the amount of garbage (that is, most cases). However, it does not help when table files are simply dropped (e.g. deletion compactions or the `DeleteFile` API). To deal with such cases, the patch adds logic that cleans up all blob files at the head of the list until the first one with linked SSTs is found. (As an example, let's assume we have blob files with numbers 1..10, and the first one with any linked SSTs is number 8. This means that SSTs in the `Version` only rely on blob files with numbers >= 8, and thus 1..7 are no longer needed.) The code change itself is pretty small; however, changing the logic like this necessitated changes to some tests that have been added recently (namely to the ones that use blob files in isolation, i.e. without any table files referring to them). Some of these cases were fixed by bypassing `VersionBuilder` altogether in order to keep the tests simple (which actually makes them more proper unit tests as well), while the `VersionBuilder` unit tests were fixed by adding dummy table files to the test cases as needed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7001 Test Plan: `make check` Reviewed By: riversand963 Differential Revision: D22119474 Pulled By: ltamasi fbshipit-source-id: c6547141355667d4291d9661d6518eb741e7b54a
5 years ago
AppendVersion(cfd, version);
}
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
protected:
Fail recovery when MANIFEST record checksum mismatch (#6996) Summary: https://github.com/facebook/rocksdb/issues/5411 refactored `VersionSet::Recover` but introduced a bug, explained as follows. Before, once a checksum mismatch happens, `reporter` will set `s` to be non-ok. Therefore, Recover will stop processing the MANIFEST any further. ``` // Correct // Inside Recover LogReporter reporter; reporter.status = &s; log::Reader reader(..., reporter); while (reader.ReadRecord() && s.ok()) { ... } ``` The bug is that, the local variable `s` in `ReadAndRecover` won't be updated by `reporter` while reading the MANIFEST. It is possible that the reader sees a checksum mismatch in a record, but `ReadRecord` retries internally read and finds the next valid record. The mismatched record will be ignored and no error is reported. ``` // Incorrect // Inside Recover LogReporter reporter; reporter.status = &s; log::Reader reader(..., reporter); s = ReadAndRecover(reader, ...); // Inside ReadAndRecover Status s; // Shadows the s in Recover. while (reader.ReadRecord() && s.ok()) { ... } ``` `LogReporter` can use a separate `log_read_status` to track the errors while reading the MANIFEST. RocksDB can process more MANIFEST entries only if `log_read_status.ok()`. Test plan (devserver): make check Pull Request resolved: https://github.com/facebook/rocksdb/pull/6996 Reviewed By: ajkr Differential Revision: D22105746 Pulled By: riversand963 fbshipit-source-id: b22f717a423457a41ca152a242abbb64cf91fc38
5 years ago
using VersionBuilderMap =
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
3 years ago
UnorderedMap<uint32_t, std::unique_ptr<BaseReferencedVersionBuilder>>;
Fail recovery when MANIFEST record checksum mismatch (#6996) Summary: https://github.com/facebook/rocksdb/issues/5411 refactored `VersionSet::Recover` but introduced a bug, explained as follows. Before, once a checksum mismatch happens, `reporter` will set `s` to be non-ok. Therefore, Recover will stop processing the MANIFEST any further. ``` // Correct // Inside Recover LogReporter reporter; reporter.status = &s; log::Reader reader(..., reporter); while (reader.ReadRecord() && s.ok()) { ... } ``` The bug is that, the local variable `s` in `ReadAndRecover` won't be updated by `reporter` while reading the MANIFEST. It is possible that the reader sees a checksum mismatch in a record, but `ReadRecord` retries internally read and finds the next valid record. The mismatched record will be ignored and no error is reported. ``` // Incorrect // Inside Recover LogReporter reporter; reporter.status = &s; log::Reader reader(..., reporter); s = ReadAndRecover(reader, ...); // Inside ReadAndRecover Status s; // Shadows the s in Recover. while (reader.ReadRecord() && s.ok()) { ... } ``` `LogReporter` can use a separate `log_read_status` to track the errors while reading the MANIFEST. RocksDB can process more MANIFEST entries only if `log_read_status.ok()`. Test plan (devserver): make check Pull Request resolved: https://github.com/facebook/rocksdb/pull/6996 Reviewed By: ajkr Differential Revision: D22105746 Pulled By: riversand963 fbshipit-source-id: b22f717a423457a41ca152a242abbb64cf91fc38
5 years ago
struct ManifestWriter;
friend class Version;
friend class VersionEditHandler;
friend class VersionEditHandlerPointInTime;
friend class DumpManifestHandler;
friend class DBImpl;
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
friend class DBImplReadOnly;
struct LogReporter : public log::Reader::Reporter {
Status* status;
virtual void Corruption(size_t /*bytes*/, const Status& s) override {
Fail recovery when MANIFEST record checksum mismatch (#6996) Summary: https://github.com/facebook/rocksdb/issues/5411 refactored `VersionSet::Recover` but introduced a bug, explained as follows. Before, once a checksum mismatch happens, `reporter` will set `s` to be non-ok. Therefore, Recover will stop processing the MANIFEST any further. ``` // Correct // Inside Recover LogReporter reporter; reporter.status = &s; log::Reader reader(..., reporter); while (reader.ReadRecord() && s.ok()) { ... } ``` The bug is that, the local variable `s` in `ReadAndRecover` won't be updated by `reporter` while reading the MANIFEST. It is possible that the reader sees a checksum mismatch in a record, but `ReadRecord` retries internally read and finds the next valid record. The mismatched record will be ignored and no error is reported. ``` // Incorrect // Inside Recover LogReporter reporter; reporter.status = &s; log::Reader reader(..., reporter); s = ReadAndRecover(reader, ...); // Inside ReadAndRecover Status s; // Shadows the s in Recover. while (reader.ReadRecord() && s.ok()) { ... } ``` `LogReporter` can use a separate `log_read_status` to track the errors while reading the MANIFEST. RocksDB can process more MANIFEST entries only if `log_read_status.ok()`. Test plan (devserver): make check Pull Request resolved: https://github.com/facebook/rocksdb/pull/6996 Reviewed By: ajkr Differential Revision: D22105746 Pulled By: riversand963 fbshipit-source-id: b22f717a423457a41ca152a242abbb64cf91fc38
5 years ago
if (status->ok()) {
*status = s;
}
}
};
void Reset();
// Returns approximated offset of a key in a file for a given version.
uint64_t ApproximateOffsetOf(Version* v, const FdWithKeyRange& f,
const Slice& key, TableReaderCaller caller);
// Returns approximated data size between start and end keys in a file
// for a given version.
uint64_t ApproximateSize(Version* v, const FdWithKeyRange& f,
const Slice& start, const Slice& end,
TableReaderCaller caller);
Fix a data race for cfd->log_number_ (#6249) Summary: A thread calling LogAndApply may release db mutex when calling WriteCurrentStateToManifest() which reads cfd->log_number_. Another thread can call SwitchMemtable() and writes to cfd->log_number_. Solution is to cache the cfd->log_number_ before releasing mutex in LogAndApply. Test Plan (on devserver): ``` $COMPILE_WITH_TSAN=1 make db_stress $./db_stress --acquire_snapshot_one_in=10000 --avoid_unnecessary_blocking_io=1 --block_size=16384 --bloom_bits=16 --bottommost_compression_type=zstd --cache_index_and_filter_blocks=1 --cache_size=1048576 --checkpoint_one_in=1000000 --checksum_type=kxxHash --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_ttl=0 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --db=/dev/shm/rocksdb/rocksdb_crashtest_blackbox --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --enable_pipelined_write=0 --flush_one_in=1000000 --format_version=5 --get_live_files_and_wal_files_one_in=1000000 --index_block_restart_interval=5 --index_type=0 --log2_keys_per_lock=22 --long_running_snapshots=0 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=1000000 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=16 --max_write_buffer_number=3 --memtablerep=skip_list --mmap_read=0 --nooverwritepercent=1 --open_files=500000 --ops_per_thread=100000000 --partition_filters=0 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefixpercent=5 --progress_reports=0 --readpercent=45 --recycle_log_file_num=0 --reopen=20 --set_options_one_in=10000 --snapshot_hold_ops=100000 --subcompactions=2 --sync=1 --target_file_size_base=2097152 --target_file_size_multiplier=2 --test_batches_snapshots=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_merge=0 --use_multiget=1 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --write_buffer_size=4194304 --write_dbid_to_manifest=1 --writepercent=35 ``` Then repeat the following multiple times, e.g. 100 after compiling with tsan. ``` $./db_test2 --gtest_filter=DBTest2.SwitchMemtableRaceWithNewManifest ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/6249 Differential Revision: D19235077 Pulled By: riversand963 fbshipit-source-id: 79467b52f48739ce7c27e440caa2447a40653173
5 years ago
struct MutableCFState {
uint64_t log_number;
std::string full_history_ts_low;
explicit MutableCFState() = default;
explicit MutableCFState(uint64_t _log_number, std::string ts_low)
: log_number(_log_number), full_history_ts_low(std::move(ts_low)) {}
Fix a data race for cfd->log_number_ (#6249) Summary: A thread calling LogAndApply may release db mutex when calling WriteCurrentStateToManifest() which reads cfd->log_number_. Another thread can call SwitchMemtable() and writes to cfd->log_number_. Solution is to cache the cfd->log_number_ before releasing mutex in LogAndApply. Test Plan (on devserver): ``` $COMPILE_WITH_TSAN=1 make db_stress $./db_stress --acquire_snapshot_one_in=10000 --avoid_unnecessary_blocking_io=1 --block_size=16384 --bloom_bits=16 --bottommost_compression_type=zstd --cache_index_and_filter_blocks=1 --cache_size=1048576 --checkpoint_one_in=1000000 --checksum_type=kxxHash --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_ttl=0 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --db=/dev/shm/rocksdb/rocksdb_crashtest_blackbox --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --enable_pipelined_write=0 --flush_one_in=1000000 --format_version=5 --get_live_files_and_wal_files_one_in=1000000 --index_block_restart_interval=5 --index_type=0 --log2_keys_per_lock=22 --long_running_snapshots=0 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=1000000 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=16 --max_write_buffer_number=3 --memtablerep=skip_list --mmap_read=0 --nooverwritepercent=1 --open_files=500000 --ops_per_thread=100000000 --partition_filters=0 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefixpercent=5 --progress_reports=0 --readpercent=45 --recycle_log_file_num=0 --reopen=20 --set_options_one_in=10000 --snapshot_hold_ops=100000 --subcompactions=2 --sync=1 --target_file_size_base=2097152 --target_file_size_multiplier=2 --test_batches_snapshots=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_merge=0 --use_multiget=1 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --write_buffer_size=4194304 --write_dbid_to_manifest=1 --writepercent=35 ``` Then repeat the following multiple times, e.g. 100 after compiling with tsan. ``` $./db_test2 --gtest_filter=DBTest2.SwitchMemtableRaceWithNewManifest ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/6249 Differential Revision: D19235077 Pulled By: riversand963 fbshipit-source-id: 79467b52f48739ce7c27e440caa2447a40653173
5 years ago
};
// Save current contents to *log
Fix a data race for cfd->log_number_ (#6249) Summary: A thread calling LogAndApply may release db mutex when calling WriteCurrentStateToManifest() which reads cfd->log_number_. Another thread can call SwitchMemtable() and writes to cfd->log_number_. Solution is to cache the cfd->log_number_ before releasing mutex in LogAndApply. Test Plan (on devserver): ``` $COMPILE_WITH_TSAN=1 make db_stress $./db_stress --acquire_snapshot_one_in=10000 --avoid_unnecessary_blocking_io=1 --block_size=16384 --bloom_bits=16 --bottommost_compression_type=zstd --cache_index_and_filter_blocks=1 --cache_size=1048576 --checkpoint_one_in=1000000 --checksum_type=kxxHash --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_ttl=0 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --db=/dev/shm/rocksdb/rocksdb_crashtest_blackbox --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --enable_pipelined_write=0 --flush_one_in=1000000 --format_version=5 --get_live_files_and_wal_files_one_in=1000000 --index_block_restart_interval=5 --index_type=0 --log2_keys_per_lock=22 --long_running_snapshots=0 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=1000000 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=16 --max_write_buffer_number=3 --memtablerep=skip_list --mmap_read=0 --nooverwritepercent=1 --open_files=500000 --ops_per_thread=100000000 --partition_filters=0 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefixpercent=5 --progress_reports=0 --readpercent=45 --recycle_log_file_num=0 --reopen=20 --set_options_one_in=10000 --snapshot_hold_ops=100000 --subcompactions=2 --sync=1 --target_file_size_base=2097152 --target_file_size_multiplier=2 --test_batches_snapshots=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_merge=0 --use_multiget=1 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --write_buffer_size=4194304 --write_dbid_to_manifest=1 --writepercent=35 ``` Then repeat the following multiple times, e.g. 100 after compiling with tsan. ``` $./db_test2 --gtest_filter=DBTest2.SwitchMemtableRaceWithNewManifest ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/6249 Differential Revision: D19235077 Pulled By: riversand963 fbshipit-source-id: 79467b52f48739ce7c27e440caa2447a40653173
5 years ago
Status WriteCurrentStateToManifest(
const std::unordered_map<uint32_t, MutableCFState>& curr_state,
const VersionEdit& wal_additions, log::Writer* log, IOStatus& io_s);
void AppendVersion(ColumnFamilyData* column_family_data, Version* v);
ColumnFamilyData* CreateColumnFamily(const ColumnFamilyOptions& cf_options,
const VersionEdit* edit);
Status VerifyFileMetadata(ColumnFamilyData* cfd, const std::string& fpath,
int level, const FileMetaData& meta);
// Protected by DB mutex.
Define WAL related classes to be used in VersionEdit and VersionSet (#7164) Summary: `WalAddition`, `WalDeletion` are defined in `wal_version.h` and used in `VersionEdit`. `WalAddition` is used to represent events of creating a new WAL (no size, just log number), or closing a WAL (with size). `WalDeletion` is used to represent events of deleting or archiving a WAL, it means the WAL is no longer alive (won't be replayed during recovery). `WalSet` is the set of alive WALs kept in `VersionSet`. 1. Why use `WalDeletion` instead of relying on `MinLogNumber` to identify outdated WALs On recovery, we can compute `MinLogNumber()` based on the log numbers kept in MANIFEST, any log with number < MinLogNumber can be ignored. So it seems that we don't need to persist `WalDeletion` to MANIFEST, since we can ignore the WALs based on MinLogNumber. But the `MinLogNumber()` is actually a lower bound, it does not exactly mean that logs starting from MinLogNumber must exist. This is because in a corner case, when a column family is empty and never flushed, its log number is set to the largest log number, but not persisted in MANIFEST. So let's say there are 2 column families, when creating the DB, the first WAL has log number 1, so it's persisted to MANIFEST for both column families. Then CF 0 is empty and never flushed, CF 1 is updated and flushed, so a new WAL with log number 2 is created and persisted to MANIFEST for CF 1. But CF 0's log number in MANIFEST is still 1. So on recovery, MinLogNumber is 1, but since log 1 only contains data for CF 1, and CF 1 is flushed, log 1 might have already been deleted from disk. We can make `MinLogNumber()` be the exactly minimum log number that must exist, by persisting the most recent log number for empty column families that are not flushed. But if there are N such column families, then every time a new WAL is created, we need to add N records to MANIFEST. In current design, a record is persisted to MANIFEST only when WAL is created, closed, or deleted/archived, so the number of WAL related records are bounded to 3x number of WALs. 2. Why keep `WalSet` in `VersionSet` instead of applying the `VersionEdit`s to `VersionStorageInfo` `VersionEdit`s are originally designed to track the addition and deletion of SST files. The SST files are related to column families, each column family has a list of `Version`s, and each `Version` keeps the set of active SST files in `VersionStorageInfo`. But WALs are a concept of DB, they are not bounded to specific column families. So logically it does not make sense to store WALs in a column family's `Version`s. Also, `Version`'s purpose is to keep reference to SST / blob files, so that they are not deleted until there is no version referencing them. But a WAL is deleted regardless of version references. So we keep the WALs in `VersionSet` for the purpose of writing out the DB state's snapshot when creating new MANIFESTs. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7164 Test Plan: make version_edit_test && ./version_edit_test make wal_edit_test && ./wal_edit_test Reviewed By: ltamasi Differential Revision: D22677936 Pulled By: cheng-chang fbshipit-source-id: 5a3b6890140e572ffd79eb37e6e4c3c32361a859
4 years ago
WalSet wals_;
std::unique_ptr<ColumnFamilySet> column_family_set_;
Cache* table_cache_;
Env* const env_;
FileSystemPtr const fs_;
SystemClock* const clock_;
const std::string dbname_;
std::string db_id_;
const ImmutableDBOptions* const db_options_;
std::atomic<uint64_t> next_file_number_;
// Any WAL number smaller than this should be ignored during recovery,
Fix a race condition in WAL tracking causing DB open failure (#9715) Summary: There is a race condition if WAL tracking in the MANIFEST is enabled in a database that disables 2PC. The race condition is between two background flush threads trying to install flush results to the MANIFEST. Consider an example database with two column families: "default" (cfd0) and "cf1" (cfd1). Initially, both column families have one mutable (active) memtable whose data backed by 6.log. 1. Trigger a manual flush for "cf1", creating a 7.log 2. Insert another key to "default", and trigger flush for "default", creating 8.log 3. BgFlushThread1 finishes writing 9.sst 4. BgFlushThread2 finishes writing 10.sst ``` Time BgFlushThread1 BgFlushThread2 | mutex_.Lock() | precompute min_wal_to_keep as 6 | mutex_.Unlock() | mutex_.Lock() | precompute min_wal_to_keep as 6 | join MANIFEST write queue and mutex_.Unlock() | write to MANIFEST | mutex_.Lock() | cfd1->log_number = 7 | Signal bg_flush_2 and mutex_.Unlock() | wake up and mutex_.Lock() | cfd0->log_number = 8 | FindObsoleteFiles() with job_context->log_number == 7 | mutex_.Unlock() | PurgeObsoleteFiles() deletes 6.log V ``` As shown in the above, BgFlushThread2 thinks that the min wal to keep is 6.log because "cf1" has unflushed data in 6.log (cf1.log_number=6). Similarly, BgThread1 thinks that min wal to keep is also 6.log because "default" has unflushed data (default.log_number=6). No WAL deletion will be written to MANIFEST because 6 is equal to `versions_->wals_.min_wal_number_to_keep`, due to https://github.com/facebook/rocksdb/blob/7.1.fb/db/memtable_list.cc#L513:L514. The bg flush thread that finishes last will perform file purging. `job_context.log_number` will be evaluated as 7, i.e. the min wal that contains unflushed data, causing 6.log to be deleted. However, MANIFEST thinks 6.log should still exist. If you close the db at this point, you won't be able to re-open it if `track_and_verify_wal_in_manifest` is true. We must handle the case of multiple bg flush threads, and it is difficult for one bg flush thread to know the correct min wal number until the other bg flush threads have finished committing to the manifest and updated the `cfd::log_number`. To fix this issue, we rename an existing variable `min_log_number_to_keep_2pc` to `min_log_number_to_keep`, and use it to track WAL file deletion in non-2pc mode as well. This variable is updated only 1) during recovery with mutex held, or 2) in the MANIFEST write thread. `min_log_number_to_keep` means RocksDB will delete WALs below it, although there may be WALs above it which are also obsolete. Formally, we will have [min_wal_to_keep, max_obsolete_wal]. During recovery, we make sure that only WALs above max_obsolete_wal are checked and added back to `alive_log_files_`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9715 Test Plan: ``` make check ``` Also ran stress test below (with asan) to make sure it completes successfully. ``` TEST_TMPDIR=/dev/shm/rocksdb OPT=-g ASAN_OPTIONS=disable_coredump=0 \ CRASH_TEST_EXT_ARGS=--compression_type=zstd SKIP_FORMAT_BUCK_CHECKS=1 \ make J=52 -j52 blackbox_asan_crash_test ``` Reviewed By: ltamasi Differential Revision: D34984412 Pulled By: riversand963 fbshipit-source-id: c7b21a8d84751bb55ea79c9f387103d21b231005
3 years ago
// and is qualified for being deleted.
std::atomic<uint64_t> min_log_number_to_keep_ = {0};
uint64_t manifest_file_number_;
uint64_t options_file_number_;
Add (Live)FileStorageInfo API (#8968) Summary: New classes FileStorageInfo and LiveFileStorageInfo and 'experimental' function DB::GetLiveFilesStorageInfo, which is intended to largely replace several fragmented DB functions needed to create checkpoints and backups. This function is now used to create checkpoints and backups, because it fixes many (probably not all) of the prior complexities of checkpoint not having atomic access to DB metadata. This also ensures strong functional test coverage of the new API. Specifically, much of the old CheckpointImpl::CreateCustomCheckpoint has been migrated to and updated in DBImpl::GetLiveFilesStorageInfo, with the former now calling the latter. Also, the class FileStorageInfo in metadata.h compatibly replaces BackupFileInfo and serves as a new base class for SstFileMetaData. Some old fields of SstFileMetaData are still provided (for now) but deprecated. Although FileStorageInfo::directory is accurate when using db_paths and/or cf_paths, these have never been supported by Checkpoint nor BackupEngine and still are not. This change does now detect these cases and return NotSupported when appropriate. (More work needed for support.) Somehow this change broke ProgressCallbackDuringBackup, but the progress_callback logic was dubious to begin with because it would call the callback based on copy buffer size, not size actually copied. Logic and test updated to track size actually copied per-thread. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8968 Test Plan: tests updated. DB::GetLiveFilesStorageInfo mostly tested by use in CheckpointImpl. DBTest.SnapshotFiles updated to also test GetLiveFilesStorageInfo, including reading the data after DB close. Added CheckpointTest.CheckpointWithDbPath (NotSupported). Reviewed By: siying Differential Revision: D31242045 Pulled By: pdillinger fbshipit-source-id: b183d1ce9799e220daaefd6b3b5365d98de676c0
3 years ago
uint64_t options_file_size_;
uint64_t pending_manifest_file_number_;
// The last seq visible to reads. It normally indicates the last sequence in
// the memtable but when using two write queues it could also indicate the
// last sequence in the WAL visible to reads.
std::atomic<uint64_t> last_sequence_;
Recover to exact latest seqno of data committed to MANIFEST (#9305) Summary: The LastSequence field in the MANIFEST file is the baseline seqno for a recovered DB. Recovering WAL entries might cause the recovered DB's seqno to advance above this baseline, but the recovered DB will never use a smaller seqno. Before this PR, we were writing the DB's seqno at the time of LogAndApply() as the LastSequence value. This works in the sense that it is a large enough baseline for the recovered DB that it'll never overwrite any records in existing SST files. At the same time, it's arbitrarily larger than what's needed. This behavior comes from LevelDB, where there was no tracking of largest seqno in an SST file. Now we know the largest seqno of newly written SST files, so we can write an exact value in LastSequence that actually reflects the largest seqno in any file referred to by the MANIFEST. This is primarily useful for correctness testing with unsynced data loss, where the recovered DB's seqno needs to indicate what records were recovered. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9305 Test Plan: - https://github.com/facebook/rocksdb/issues/9338 adds crash-recovery correctness testing coverage for WAL disabled use cases - https://github.com/facebook/rocksdb/issues/9357 will extend that testing to cover file ingestion - Added assertion at end of LogAndApply() for `VersionSet::descriptor_last_sequence_` consistency with files - Manually tested upgrade/downgrade compatibility with a custom crash test that randomly picks between a `db_stress` built with and without this PR (for old code it must run with `-disable_wal=0`) Reviewed By: riversand963 Differential Revision: D33182770 Pulled By: ajkr fbshipit-source-id: 0bfafaf685f347cc8cb0e1d62e0186340a738f7d
3 years ago
// The last sequence number of data committed to the descriptor (manifest
// file).
SequenceNumber descriptor_last_sequence_ = 0;
// The last seq that is already allocated. It is applicable only when we have
// two write queues. In that case seq might or might not have appreated in
// memtable but it is expected to appear in the WAL.
// We have last_sequence <= last_allocated_sequence_
std::atomic<uint64_t> last_allocated_sequence_;
// The last allocated sequence that is also published to the readers. This is
// applicable only when last_seq_same_as_publish_seq_ is not set. Otherwise
// last_sequence_ also indicates the last published seq.
// We have last_sequence <= last_published_sequence_ <=
// last_allocated_sequence_
std::atomic<uint64_t> last_published_sequence_;
uint64_t prev_log_number_; // 0 or backing store for memtable being compacted
// Opened lazily
std::unique_ptr<log::Writer> descriptor_log_;
// generates a increasing version number for every new version
uint64_t current_version_number_;
// Queue of writers to the manifest file
std::deque<ManifestWriter*> manifest_writers_;
// Current size of manifest file
uint64_t manifest_file_size_;
std::vector<ObsoleteFileInfo> obsolete_files_;
std::vector<ObsoleteBlobFileInfo> obsolete_blob_files_;
std::vector<std::string> obsolete_manifests_;
// env options for all reads and writes except compactions
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
FileOptions file_options_;
BlockCacheTracer* const block_cache_tracer_;
// Store the IO status when Manifest is written
IOStatus io_status_;
std::shared_ptr<IOTracer> io_tracer_;
std::string db_session_id_;
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
private:
// REQUIRES db mutex at beginning. may release and re-acquire db mutex
Status ProcessManifestWrites(std::deque<ManifestWriter>& writers,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
InstrumentedMutex* mu,
FSDirectory* dir_contains_current_file,
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
bool new_descriptor_log,
const ColumnFamilyOptions* new_cf_options);
Recover to exact latest seqno of data committed to MANIFEST (#9305) Summary: The LastSequence field in the MANIFEST file is the baseline seqno for a recovered DB. Recovering WAL entries might cause the recovered DB's seqno to advance above this baseline, but the recovered DB will never use a smaller seqno. Before this PR, we were writing the DB's seqno at the time of LogAndApply() as the LastSequence value. This works in the sense that it is a large enough baseline for the recovered DB that it'll never overwrite any records in existing SST files. At the same time, it's arbitrarily larger than what's needed. This behavior comes from LevelDB, where there was no tracking of largest seqno in an SST file. Now we know the largest seqno of newly written SST files, so we can write an exact value in LastSequence that actually reflects the largest seqno in any file referred to by the MANIFEST. This is primarily useful for correctness testing with unsynced data loss, where the recovered DB's seqno needs to indicate what records were recovered. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9305 Test Plan: - https://github.com/facebook/rocksdb/issues/9338 adds crash-recovery correctness testing coverage for WAL disabled use cases - https://github.com/facebook/rocksdb/issues/9357 will extend that testing to cover file ingestion - Added assertion at end of LogAndApply() for `VersionSet::descriptor_last_sequence_` consistency with files - Manually tested upgrade/downgrade compatibility with a custom crash test that randomly picks between a `db_stress` built with and without this PR (for old code it must run with `-disable_wal=0`) Reviewed By: riversand963 Differential Revision: D33182770 Pulled By: ajkr fbshipit-source-id: 0bfafaf685f347cc8cb0e1d62e0186340a738f7d
3 years ago
void LogAndApplyCFHelper(VersionEdit* edit,
SequenceNumber* max_last_sequence);
Status LogAndApplyHelper(ColumnFamilyData* cfd, VersionBuilder* b,
Recover to exact latest seqno of data committed to MANIFEST (#9305) Summary: The LastSequence field in the MANIFEST file is the baseline seqno for a recovered DB. Recovering WAL entries might cause the recovered DB's seqno to advance above this baseline, but the recovered DB will never use a smaller seqno. Before this PR, we were writing the DB's seqno at the time of LogAndApply() as the LastSequence value. This works in the sense that it is a large enough baseline for the recovered DB that it'll never overwrite any records in existing SST files. At the same time, it's arbitrarily larger than what's needed. This behavior comes from LevelDB, where there was no tracking of largest seqno in an SST file. Now we know the largest seqno of newly written SST files, so we can write an exact value in LastSequence that actually reflects the largest seqno in any file referred to by the MANIFEST. This is primarily useful for correctness testing with unsynced data loss, where the recovered DB's seqno needs to indicate what records were recovered. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9305 Test Plan: - https://github.com/facebook/rocksdb/issues/9338 adds crash-recovery correctness testing coverage for WAL disabled use cases - https://github.com/facebook/rocksdb/issues/9357 will extend that testing to cover file ingestion - Added assertion at end of LogAndApply() for `VersionSet::descriptor_last_sequence_` consistency with files - Manually tested upgrade/downgrade compatibility with a custom crash test that randomly picks between a `db_stress` built with and without this PR (for old code it must run with `-disable_wal=0`) Reviewed By: riversand963 Differential Revision: D33182770 Pulled By: ajkr fbshipit-source-id: 0bfafaf685f347cc8cb0e1d62e0186340a738f7d
3 years ago
VersionEdit* edit, SequenceNumber* max_last_sequence,
InstrumentedMutex* mu);
};
// ReactiveVersionSet represents a collection of versions of the column
// families of the database. Users of ReactiveVersionSet, e.g. DBImplSecondary,
// need to replay the MANIFEST (description log in older terms) in order to
// reconstruct and install versions.
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
class ReactiveVersionSet : public VersionSet {
public:
ReactiveVersionSet(const std::string& dbname,
const ImmutableDBOptions* _db_options,
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
const FileOptions& _file_options, Cache* table_cache,
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
WriteBufferManager* write_buffer_manager,
WriteController* write_controller,
const std::shared_ptr<IOTracer>& io_tracer);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
~ReactiveVersionSet() override;
Status ReadAndApply(
InstrumentedMutex* mu,
std::unique_ptr<log::FragmentBufferedReader>* manifest_reader,
Status* manifest_read_status,
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
std::unordered_set<ColumnFamilyData*>* cfds_changed);
Status Recover(const std::vector<ColumnFamilyDescriptor>& column_families,
std::unique_ptr<log::FragmentBufferedReader>* manifest_reader,
std::unique_ptr<log::Reader::Reporter>* manifest_reporter,
std::unique_ptr<Status>* manifest_reader_status);
#ifndef NDEBUG
uint64_t TEST_read_edits_in_atomic_group() const;
#endif //! NDEBUG
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
std::vector<VersionEdit>& replay_buffer();
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
protected:
// REQUIRES db mutex
Status ApplyOneVersionEditToBuilder(
VersionEdit& edit, std::unordered_set<ColumnFamilyData*>* cfds_changed,
VersionEdit* version_edit);
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
Status MaybeSwitchManifest(
log::Reader::Reporter* reporter,
std::unique_ptr<log::FragmentBufferedReader>* manifest_reader);
private:
std::unique_ptr<ManifestTailer> manifest_tailer_;
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
using VersionSet::LogAndApply;
using VersionSet::Recover;
Status LogAndApply(
const autovector<ColumnFamilyData*>& /*cfds*/,
const autovector<const MutableCFOptions*>& /*mutable_cf_options_list*/,
const autovector<autovector<VersionEdit*>>& /*edit_lists*/,
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
InstrumentedMutex* /*mu*/, FSDirectory* /*dir_contains_current_file*/,
Perform post-flush updates of memtable list in a callback (#6069) Summary: Currently, the following interleaving of events can lead to SuperVersion containing both immutable memtables as well as the resulting L0. This can cause Get to return incorrect result if there are merge operands. This may also affect other operations such as single deletes. ``` time main_thr bg_flush_thr bg_compact_thr compact_thr set_opts_thr 0 | WriteManifest:0 1 | issue compact 2 | wait 3 | Merge(counter) 4 | issue flush 5 | wait 6 | WriteManifest:1 7 | wake up 8 | write manifest 9 | wake up 10 | Get(counter) 11 | remove imm V ``` The reason behind is that: one bg flush thread's installing new `Version` can be batched and performed by another thread that is the "leader" MANIFEST writer. This bg thread removes the memtables from current super version only after `LogAndApply` returns. After the leader MANIFEST writer signals (releasing mutex) this bg flush thread, it is possible that another thread sees this cf with both memtables (whose data have been flushed to the newest L0) and the L0 before this bg flush thread removes the memtables. To address this issue, each bg flush thread can pass a callback function to `LogAndApply`. The callback is responsible for removing the memtables. Therefore, the leader MANIFEST writer can call this callback and remove the memtables before releasing the mutex. Test plan (devserver) ``` $make merge_test $./merge_test --gtest_filter=MergeTest.MergeWithCompactionAndFlush $make check ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/6069 Reviewed By: cheng-chang Differential Revision: D18790894 Pulled By: riversand963 fbshipit-source-id: e41bd600c0448b4f4b2deb3f7677f95e3076b4ed
4 years ago
bool /*new_descriptor_log*/, const ColumnFamilyOptions* /*new_cf_option*/,
const std::vector<std::function<void(const Status&)>>& /*manifest_wcbs*/)
override {
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
6 years ago
return Status::NotSupported("not supported in reactive mode");
}
// No copy allowed
ReactiveVersionSet(const ReactiveVersionSet&);
ReactiveVersionSet& operator=(const ReactiveVersionSet&);
};
} // namespace ROCKSDB_NAMESPACE