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

705 lines
26 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "db/table_cache.h"
#include "db/dbformat.h"
Use only "local" range tombstones during Get (#4449) Summary: Previously, range tombstones were accumulated from every level, which was necessary if a range tombstone in a higher level covered a key in a lower level. However, RangeDelAggregator::AddTombstones's complexity is based on the number of tombstones that are currently stored in it, which is wasteful in the Get case, where we only need to know the highest sequence number of range tombstones that cover the key from higher levels, and compute the highest covering sequence number at the current level. This change introduces this optimization, and removes the use of RangeDelAggregator from the Get path. In the benchmark results, the following command was used to initialize the database: ``` ./db_bench -db=/dev/shm/5k-rts -use_existing_db=false -benchmarks=filluniquerandom -write_buffer_size=1048576 -compression_type=lz4 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -value_size=112 -key_size=16 -block_size=4096 -level_compaction_dynamic_level_bytes=true -num=5000000 -max_background_jobs=12 -benchmark_write_rate_limit=20971520 -range_tombstone_width=100 -writes_per_range_tombstone=100 -max_num_range_tombstones=50000 -bloom_bits=8 ``` ...and the following command was used to measure read throughput: ``` ./db_bench -db=/dev/shm/5k-rts/ -use_existing_db=true -benchmarks=readrandom -disable_auto_compactions=true -num=5000000 -reads=100000 -threads=32 ``` The filluniquerandom command was only run once, and the resulting database was used to measure read performance before and after the PR. Both binaries were compiled with `DEBUG_LEVEL=0`. Readrandom results before PR: ``` readrandom : 4.544 micros/op 220090 ops/sec; 16.9 MB/s (63103 of 100000 found) ``` Readrandom results after PR: ``` readrandom : 11.147 micros/op 89707 ops/sec; 6.9 MB/s (63103 of 100000 found) ``` So it's actually slower right now, but this PR paves the way for future optimizations (see #4493). ---- Pull Request resolved: https://github.com/facebook/rocksdb/pull/4449 Differential Revision: D10370575 Pulled By: abhimadan fbshipit-source-id: 9a2e152be1ef36969055c0e9eb4beb0d96c11f4d
6 years ago
#include "db/range_tombstone_fragmenter.h"
#include "db/snapshot_impl.h"
#include "db/version_edit.h"
#include "file/file_util.h"
#include "file/filename.h"
#include "file/random_access_file_reader.h"
#include "monitoring/perf_context_imp.h"
#include "rocksdb/statistics.h"
#include "table/block_based/block_based_table_reader.h"
#include "table/get_context.h"
#include "table/internal_iterator.h"
#include "table/iterator_wrapper.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/multiget_context.h"
#include "table/table_builder.h"
#include "table/table_reader.h"
#include "test_util/sync_point.h"
#include "util/cast_util.h"
#include "util/coding.h"
#include "util/stop_watch.h"
namespace ROCKSDB_NAMESPACE {
namespace {
template <class T>
static void DeleteEntry(const Slice& /*key*/, void* value) {
T* typed_value = reinterpret_cast<T*>(value);
delete typed_value;
}
static void UnrefEntry(void* arg1, void* arg2) {
Cache* cache = reinterpret_cast<Cache*>(arg1);
Cache::Handle* h = reinterpret_cast<Cache::Handle*>(arg2);
cache->Release(h);
}
static Slice GetSliceForFileNumber(const uint64_t* file_number) {
return Slice(reinterpret_cast<const char*>(file_number),
sizeof(*file_number));
}
#ifndef ROCKSDB_LITE
void AppendVarint64(IterKey* key, uint64_t v) {
char buf[10];
auto ptr = EncodeVarint64(buf, v);
key->TrimAppend(key->Size(), buf, ptr - buf);
}
#endif // ROCKSDB_LITE
} // namespace
const int kLoadConcurency = 128;
TableCache::TableCache(const ImmutableCFOptions& ioptions,
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* const cache,
BlockCacheTracer* const block_cache_tracer,
const std::shared_ptr<IOTracer>& io_tracer)
: ioptions_(ioptions),
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
file_options_(file_options),
cache_(cache),
immortal_tables_(false),
block_cache_tracer_(block_cache_tracer),
Integrity protection for live updates to WriteBatch (#7748) Summary: This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.). The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer. When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7748 Test Plan: - an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught - add to stress/crash test to verify it works in variety of configs/operations without intentional corruption - [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc. Reviewed By: pdillinger Differential Revision: D25754492 Pulled By: ajkr fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
4 years ago
loader_mutex_(kLoadConcurency, kGetSliceNPHash64UnseededFnPtr),
io_tracer_(io_tracer) {
if (ioptions_.row_cache) {
// If the same cache is shared by multiple instances, we need to
// disambiguate its entries.
PutVarint64(&row_cache_id_, ioptions_.row_cache->NewId());
}
}
TableCache::~TableCache() {
}
TableReader* TableCache::GetTableReaderFromHandle(Cache::Handle* handle) {
return reinterpret_cast<TableReader*>(cache_->Value(handle));
}
void TableCache::ReleaseHandle(Cache::Handle* handle) {
cache_->Release(handle);
}
Status TableCache::GetTableReader(
const ReadOptions& ro, const FileOptions& file_options,
const InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
bool sequential_mode, bool record_read_stats, HistogramImpl* file_read_hist,
std::unique_ptr<TableReader>* table_reader,
const SliceTransform* prefix_extractor, bool skip_filters, int level,
bool prefetch_index_and_filter_in_cache,
size_t max_file_size_for_l0_meta_pin) {
std::string fname =
TableFileName(ioptions_.cf_paths, fd.GetNumber(), fd.GetPathId());
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
std::unique_ptr<FSRandomAccessFile> file;
FileOptions fopts = file_options;
Status s = PrepareIOFromReadOptions(ro, ioptions_.clock, fopts.io_options);
if (s.ok()) {
s = ioptions_.fs->NewRandomAccessFile(fname, fopts, &file, nullptr);
}
RecordTick(ioptions_.stats, NO_FILE_OPENS);
if (s.IsPathNotFound()) {
fname = Rocks2LevelTableFileName(fname);
s = PrepareIOFromReadOptions(ro, ioptions_.clock, fopts.io_options);
if (s.ok()) {
s = ioptions_.fs->NewRandomAccessFile(fname, file_options, &file,
nullptr);
}
RecordTick(ioptions_.stats, NO_FILE_OPENS);
}
if (s.ok()) {
if (!sequential_mode && ioptions_.advise_random_on_open) {
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
file->Hint(FSRandomAccessFile::kRandom);
}
StopWatch sw(ioptions_.clock, ioptions_.stats, TABLE_OPEN_IO_MICROS);
std::unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(
std::move(file), fname, ioptions_.clock, io_tracer_,
record_read_stats ? ioptions_.stats : nullptr, SST_READ_MICROS,
file_read_hist, ioptions_.rate_limiter.get(), ioptions_.listeners));
s = ioptions_.table_factory->NewTableReader(
ro,
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
TableReaderOptions(ioptions_, prefix_extractor, file_options,
internal_comparator, skip_filters, immortal_tables_,
false /* force_direct_prefetch */, level,
fd.largest_seqno, block_cache_tracer_,
max_file_size_for_l0_meta_pin),
std::move(file_reader), fd.GetFileSize(), table_reader,
prefetch_index_and_filter_in_cache);
TEST_SYNC_POINT("TableCache::GetTableReader:0");
}
return s;
}
void TableCache::EraseHandle(const FileDescriptor& fd, Cache::Handle* handle) {
ReleaseHandle(handle);
uint64_t number = fd.GetNumber();
Slice key = GetSliceForFileNumber(&number);
cache_->Erase(key);
}
Status TableCache::FindTable(const ReadOptions& ro,
const FileOptions& file_options,
const InternalKeyComparator& internal_comparator,
const FileDescriptor& fd, Cache::Handle** handle,
const SliceTransform* prefix_extractor,
const bool no_io, bool record_read_stats,
HistogramImpl* file_read_hist, bool skip_filters,
int level, bool prefetch_index_and_filter_in_cache,
size_t max_file_size_for_l0_meta_pin) {
PERF_TIMER_GUARD_WITH_CLOCK(find_table_nanos, ioptions_.clock);
uint64_t number = fd.GetNumber();
Slice key = GetSliceForFileNumber(&number);
*handle = cache_->Lookup(key);
TEST_SYNC_POINT_CALLBACK("TableCache::FindTable:0",
const_cast<bool*>(&no_io));
if (*handle == nullptr) {
if (no_io) {
return Status::Incomplete("Table not found in table_cache, no_io is set");
}
MutexLock load_lock(loader_mutex_.get(key));
// We check the cache again under loading mutex
*handle = cache_->Lookup(key);
if (*handle != nullptr) {
return Status::OK();
}
std::unique_ptr<TableReader> table_reader;
Status s = GetTableReader(
ro, file_options, internal_comparator, fd, false /* sequential mode */,
record_read_stats, file_read_hist, &table_reader, prefix_extractor,
skip_filters, level, prefetch_index_and_filter_in_cache,
max_file_size_for_l0_meta_pin);
if (!s.ok()) {
assert(table_reader == nullptr);
RecordTick(ioptions_.stats, NO_FILE_ERRORS);
// We do not cache error results so that if the error is transient,
// or somebody repairs the file, we recover automatically.
} else {
s = cache_->Insert(key, table_reader.get(), 1, &DeleteEntry<TableReader>,
handle);
if (s.ok()) {
// Release ownership of table reader.
table_reader.release();
}
}
return s;
}
return Status::OK();
}
InternalIterator* TableCache::NewIterator(
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 ReadOptions& options, const FileOptions& file_options,
const InternalKeyComparator& icomparator, const FileMetaData& file_meta,
RangeDelAggregator* range_del_agg, const SliceTransform* prefix_extractor,
TableReader** table_reader_ptr, HistogramImpl* file_read_hist,
TableReaderCaller caller, Arena* arena, bool skip_filters, int level,
size_t max_file_size_for_l0_meta_pin,
const InternalKey* smallest_compaction_key,
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
const InternalKey* largest_compaction_key, bool allow_unprepared_value) {
PERF_TIMER_GUARD(new_table_iterator_nanos);
Status s;
TableReader* table_reader = nullptr;
Cache::Handle* handle = nullptr;
if (table_reader_ptr != nullptr) {
*table_reader_ptr = nullptr;
}
bool for_compaction = caller == TableReaderCaller::kCompaction;
auto& fd = file_meta.fd;
table_reader = fd.table_reader;
if (table_reader == nullptr) {
s = FindTable(
options, file_options, icomparator, fd, &handle, prefix_extractor,
options.read_tier == kBlockCacheTier /* no_io */,
!for_compaction /* record_read_stats */, file_read_hist, skip_filters,
level, true /* prefetch_index_and_filter_in_cache */,
max_file_size_for_l0_meta_pin);
if (s.ok()) {
table_reader = GetTableReaderFromHandle(handle);
}
}
InternalIterator* result = nullptr;
if (s.ok()) {
expose a hook to skip tables during iteration Summary: As discussed on the mailing list (["Skipping entire SSTs while iterating"](https://groups.google.com/forum/#!topic/rocksdb/ujHCJVLrHlU)), this patch adds a `table_filter` to `ReadOptions` that allows specifying a callback to be executed during iteration before each table in the database is scanned. The callback is passed the table's properties; the table is scanned iff the callback returns true. This can be used in conjunction with a `TablePropertiesCollector` to dramatically speed up scans by skipping tables that are known to contain irrelevant data for the scan at hand. We're using this [downstream in CockroachDB](https://github.com/cockroachdb/cockroach/blob/master/pkg/storage/engine/db.cc#L2009-L2022) already. With this feature, under ideal conditions, we can reduce the time of an incremental backup in from hours to seconds. FYI, the first commit in this PR fixes a segfault that I unfortunately have not figured out how to reproduce outside of CockroachDB. I'm hoping you accept it on the grounds that it is not correct to return 8-byte aligned memory from a call to `malloc` on some 64-bit platforms; one correct approach is to infer the necessary alignment from `std::max_align_t`, as done here. As noted in the first commit message, the bug is tickled by having a`std::function` in `struct ReadOptions`. That is, the following patch alone is enough to cause RocksDB to segfault when run from CockroachDB on Darwin. ```diff --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -1546,6 +1546,13 @@ struct ReadOptions { // Default: false bool ignore_range_deletions; + // A callback to determine whether relevant keys for this scan exist in a + // given table based on the table's properties. The callback is passed the + // properties of each table during iteration. If the callback returns false, + // the table will not be scanned. + // Default: empty (every table will be scanned) + std::function<bool(const TableProperties&)> table_filter; + ReadOptions(); ReadOptions(bool cksum, bool cache); }; ``` /cc danhhz Closes https://github.com/facebook/rocksdb/pull/2265 Differential Revision: D5054262 Pulled By: yiwu-arbug fbshipit-source-id: dd6b28f2bba6cb8466250d8c5c542d3c92785476
7 years ago
if (options.table_filter &&
!options.table_filter(*table_reader->GetTableProperties())) {
result = NewEmptyInternalIterator<Slice>(arena);
expose a hook to skip tables during iteration Summary: As discussed on the mailing list (["Skipping entire SSTs while iterating"](https://groups.google.com/forum/#!topic/rocksdb/ujHCJVLrHlU)), this patch adds a `table_filter` to `ReadOptions` that allows specifying a callback to be executed during iteration before each table in the database is scanned. The callback is passed the table's properties; the table is scanned iff the callback returns true. This can be used in conjunction with a `TablePropertiesCollector` to dramatically speed up scans by skipping tables that are known to contain irrelevant data for the scan at hand. We're using this [downstream in CockroachDB](https://github.com/cockroachdb/cockroach/blob/master/pkg/storage/engine/db.cc#L2009-L2022) already. With this feature, under ideal conditions, we can reduce the time of an incremental backup in from hours to seconds. FYI, the first commit in this PR fixes a segfault that I unfortunately have not figured out how to reproduce outside of CockroachDB. I'm hoping you accept it on the grounds that it is not correct to return 8-byte aligned memory from a call to `malloc` on some 64-bit platforms; one correct approach is to infer the necessary alignment from `std::max_align_t`, as done here. As noted in the first commit message, the bug is tickled by having a`std::function` in `struct ReadOptions`. That is, the following patch alone is enough to cause RocksDB to segfault when run from CockroachDB on Darwin. ```diff --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -1546,6 +1546,13 @@ struct ReadOptions { // Default: false bool ignore_range_deletions; + // A callback to determine whether relevant keys for this scan exist in a + // given table based on the table's properties. The callback is passed the + // properties of each table during iteration. If the callback returns false, + // the table will not be scanned. + // Default: empty (every table will be scanned) + std::function<bool(const TableProperties&)> table_filter; + ReadOptions(); ReadOptions(bool cksum, bool cache); }; ``` /cc danhhz Closes https://github.com/facebook/rocksdb/pull/2265 Differential Revision: D5054262 Pulled By: yiwu-arbug fbshipit-source-id: dd6b28f2bba6cb8466250d8c5c542d3c92785476
7 years ago
} else {
result = table_reader->NewIterator(options, prefix_extractor, arena,
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
skip_filters, caller,
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
file_options.compaction_readahead_size,
allow_unprepared_value);
expose a hook to skip tables during iteration Summary: As discussed on the mailing list (["Skipping entire SSTs while iterating"](https://groups.google.com/forum/#!topic/rocksdb/ujHCJVLrHlU)), this patch adds a `table_filter` to `ReadOptions` that allows specifying a callback to be executed during iteration before each table in the database is scanned. The callback is passed the table's properties; the table is scanned iff the callback returns true. This can be used in conjunction with a `TablePropertiesCollector` to dramatically speed up scans by skipping tables that are known to contain irrelevant data for the scan at hand. We're using this [downstream in CockroachDB](https://github.com/cockroachdb/cockroach/blob/master/pkg/storage/engine/db.cc#L2009-L2022) already. With this feature, under ideal conditions, we can reduce the time of an incremental backup in from hours to seconds. FYI, the first commit in this PR fixes a segfault that I unfortunately have not figured out how to reproduce outside of CockroachDB. I'm hoping you accept it on the grounds that it is not correct to return 8-byte aligned memory from a call to `malloc` on some 64-bit platforms; one correct approach is to infer the necessary alignment from `std::max_align_t`, as done here. As noted in the first commit message, the bug is tickled by having a`std::function` in `struct ReadOptions`. That is, the following patch alone is enough to cause RocksDB to segfault when run from CockroachDB on Darwin. ```diff --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -1546,6 +1546,13 @@ struct ReadOptions { // Default: false bool ignore_range_deletions; + // A callback to determine whether relevant keys for this scan exist in a + // given table based on the table's properties. The callback is passed the + // properties of each table during iteration. If the callback returns false, + // the table will not be scanned. + // Default: empty (every table will be scanned) + std::function<bool(const TableProperties&)> table_filter; + ReadOptions(); ReadOptions(bool cksum, bool cache); }; ``` /cc danhhz Closes https://github.com/facebook/rocksdb/pull/2265 Differential Revision: D5054262 Pulled By: yiwu-arbug fbshipit-source-id: dd6b28f2bba6cb8466250d8c5c542d3c92785476
7 years ago
}
if (handle != nullptr) {
result->RegisterCleanup(&UnrefEntry, cache_, handle);
handle = nullptr; // prevent from releasing below
}
if (for_compaction) {
table_reader->SetupForCompaction();
}
if (table_reader_ptr != nullptr) {
*table_reader_ptr = table_reader;
}
}
if (s.ok() && range_del_agg != nullptr && !options.ignore_range_deletions) {
Recommit "Avoid adding tombstones of the same file to RangeDelAggregator multiple times" Summary: The origin commit #3635 will hurt performance for users who aren't using range deletions, because unneeded std::set operations, so it was reverted by commit 44653c7b7aabe821e671946e732dda7ae6b43d1b. (see #3672) To fix this, move the set to and add a check in , i.e., file will be added only if is non-nullptr. The db_bench command which find the performance regression: > ./db_bench --benchmarks=fillrandom,seekrandomwhilewriting --threads=1 --num=1000000 --reads=150000 --key_size=66 > --value_size=1262 --statistics=0 --compression_ratio=0.5 --histogram=1 --seek_nexts=1 --stats_per_interval=1 > --stats_interval_seconds=600 --max_background_flushes=4 --num_multi_db=1 --max_background_compactions=16 --seed=1522388277 > -write_buffer_size=1048576 --level0_file_num_compaction_trigger=10000 --compression_type=none Before and after the modification, I re-run this command on the machine, the results of are as follows: **fillrandom** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 5.92 | 8.57 | 19.63 | 980.97 | 12196.00 | after commit | 5.91 | 8.55 | 19.34 | 965.56 | 13513.56 | **seekrandomwhilewriting** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 1418.62 | 1867.01 | 3823.28 | 4980.99 | 9240.00 | after commit | 1450.54 | 1880.61 | 3962.87 | 5429.60 | 7542.86 | Closes https://github.com/facebook/rocksdb/pull/3800 Differential Revision: D7874245 Pulled By: ajkr fbshipit-source-id: 2e8bec781b3f7399246babd66395c88619534a17
7 years ago
if (range_del_agg->AddFile(fd.GetNumber())) {
std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
static_cast<FragmentedRangeTombstoneIterator*>(
table_reader->NewRangeTombstoneIterator(options)));
Recommit "Avoid adding tombstones of the same file to RangeDelAggregator multiple times" Summary: The origin commit #3635 will hurt performance for users who aren't using range deletions, because unneeded std::set operations, so it was reverted by commit 44653c7b7aabe821e671946e732dda7ae6b43d1b. (see #3672) To fix this, move the set to and add a check in , i.e., file will be added only if is non-nullptr. The db_bench command which find the performance regression: > ./db_bench --benchmarks=fillrandom,seekrandomwhilewriting --threads=1 --num=1000000 --reads=150000 --key_size=66 > --value_size=1262 --statistics=0 --compression_ratio=0.5 --histogram=1 --seek_nexts=1 --stats_per_interval=1 > --stats_interval_seconds=600 --max_background_flushes=4 --num_multi_db=1 --max_background_compactions=16 --seed=1522388277 > -write_buffer_size=1048576 --level0_file_num_compaction_trigger=10000 --compression_type=none Before and after the modification, I re-run this command on the machine, the results of are as follows: **fillrandom** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 5.92 | 8.57 | 19.63 | 980.97 | 12196.00 | after commit | 5.91 | 8.55 | 19.34 | 965.56 | 13513.56 | **seekrandomwhilewriting** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 1418.62 | 1867.01 | 3823.28 | 4980.99 | 9240.00 | after commit | 1450.54 | 1880.61 | 3962.87 | 5429.60 | 7542.86 | Closes https://github.com/facebook/rocksdb/pull/3800 Differential Revision: D7874245 Pulled By: ajkr fbshipit-source-id: 2e8bec781b3f7399246babd66395c88619534a17
7 years ago
if (range_del_iter != nullptr) {
s = range_del_iter->status();
}
if (s.ok()) {
const InternalKey* smallest = &file_meta.smallest;
const InternalKey* largest = &file_meta.largest;
if (smallest_compaction_key != nullptr) {
smallest = smallest_compaction_key;
}
if (largest_compaction_key != nullptr) {
largest = largest_compaction_key;
}
range_del_agg->AddTombstones(std::move(range_del_iter), smallest,
largest);
Recommit "Avoid adding tombstones of the same file to RangeDelAggregator multiple times" Summary: The origin commit #3635 will hurt performance for users who aren't using range deletions, because unneeded std::set operations, so it was reverted by commit 44653c7b7aabe821e671946e732dda7ae6b43d1b. (see #3672) To fix this, move the set to and add a check in , i.e., file will be added only if is non-nullptr. The db_bench command which find the performance regression: > ./db_bench --benchmarks=fillrandom,seekrandomwhilewriting --threads=1 --num=1000000 --reads=150000 --key_size=66 > --value_size=1262 --statistics=0 --compression_ratio=0.5 --histogram=1 --seek_nexts=1 --stats_per_interval=1 > --stats_interval_seconds=600 --max_background_flushes=4 --num_multi_db=1 --max_background_compactions=16 --seed=1522388277 > -write_buffer_size=1048576 --level0_file_num_compaction_trigger=10000 --compression_type=none Before and after the modification, I re-run this command on the machine, the results of are as follows: **fillrandom** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 5.92 | 8.57 | 19.63 | 980.97 | 12196.00 | after commit | 5.91 | 8.55 | 19.34 | 965.56 | 13513.56 | **seekrandomwhilewriting** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 1418.62 | 1867.01 | 3823.28 | 4980.99 | 9240.00 | after commit | 1450.54 | 1880.61 | 3962.87 | 5429.60 | 7542.86 | Closes https://github.com/facebook/rocksdb/pull/3800 Differential Revision: D7874245 Pulled By: ajkr fbshipit-source-id: 2e8bec781b3f7399246babd66395c88619534a17
7 years ago
}
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
}
}
if (handle != nullptr) {
ReleaseHandle(handle);
}
if (!s.ok()) {
assert(result == nullptr);
result = NewErrorInternalIterator<Slice>(s, arena);
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
}
return result;
}
Status TableCache::GetRangeTombstoneIterator(
const ReadOptions& options,
const InternalKeyComparator& internal_comparator,
const FileMetaData& file_meta,
std::unique_ptr<FragmentedRangeTombstoneIterator>* out_iter) {
const FileDescriptor& fd = file_meta.fd;
Status s;
TableReader* t = fd.table_reader;
Cache::Handle* handle = nullptr;
if (t == nullptr) {
s = FindTable(options, file_options_, internal_comparator, fd, &handle);
if (s.ok()) {
t = GetTableReaderFromHandle(handle);
}
}
if (s.ok()) {
out_iter->reset(t->NewRangeTombstoneIterator(options));
assert(out_iter);
}
return s;
}
#ifndef ROCKSDB_LITE
void TableCache::CreateRowCacheKeyPrefix(const ReadOptions& options,
const FileDescriptor& fd,
const Slice& internal_key,
GetContext* get_context,
IterKey& row_cache_key) {
uint64_t fd_number = fd.GetNumber();
// We use the user key as cache key instead of the internal key,
// otherwise the whole cache would be invalidated every time the
// sequence key increases. However, to support caching snapshot
// reads, we append the sequence number (incremented by 1 to
// distinguish from 0) only in this case.
// If the snapshot is larger than the largest seqno in the file,
// all data should be exposed to the snapshot, so we treat it
// the same as there is no snapshot. The exception is that if
// a seq-checking callback is registered, some internal keys
// may still be filtered out.
uint64_t seq_no = 0;
// Maybe we can include the whole file ifsnapshot == fd.largest_seqno.
if (options.snapshot != nullptr &&
(get_context->has_callback() ||
static_cast_with_check<const SnapshotImpl>(options.snapshot)
->GetSequenceNumber() <= fd.largest_seqno)) {
// We should consider to use options.snapshot->GetSequenceNumber()
// instead of GetInternalKeySeqno(k), which will make the code
// easier to understand.
seq_no = 1 + GetInternalKeySeqno(internal_key);
}
// Compute row cache key.
row_cache_key.TrimAppend(row_cache_key.Size(), row_cache_id_.data(),
row_cache_id_.size());
AppendVarint64(&row_cache_key, fd_number);
AppendVarint64(&row_cache_key, seq_no);
}
bool TableCache::GetFromRowCache(const Slice& user_key, IterKey& row_cache_key,
size_t prefix_size, GetContext* get_context) {
bool found = false;
row_cache_key.TrimAppend(prefix_size, user_key.data(), user_key.size());
if (auto row_handle =
ioptions_.row_cache->Lookup(row_cache_key.GetUserKey())) {
// Cleanable routine to release the cache entry
Cleanable value_pinner;
auto release_cache_entry_func = [](void* cache_to_clean,
void* cache_handle) {
((Cache*)cache_to_clean)->Release((Cache::Handle*)cache_handle);
};
auto found_row_cache_entry =
static_cast<const std::string*>(ioptions_.row_cache->Value(row_handle));
// If it comes here value is located on the cache.
// found_row_cache_entry points to the value on cache,
// and value_pinner has cleanup procedure for the cached entry.
// After replayGetContextLog() returns, get_context.pinnable_slice_
// will point to cache entry buffer (or a copy based on that) and
// cleanup routine under value_pinner will be delegated to
// get_context.pinnable_slice_. Cache entry is released when
// get_context.pinnable_slice_ is reset.
value_pinner.RegisterCleanup(release_cache_entry_func,
ioptions_.row_cache.get(), row_handle);
replayGetContextLog(*found_row_cache_entry, user_key, get_context,
&value_pinner);
RecordTick(ioptions_.stats, ROW_CACHE_HIT);
found = true;
} else {
RecordTick(ioptions_.stats, ROW_CACHE_MISS);
}
return found;
}
#endif // ROCKSDB_LITE
Status TableCache::Get(const ReadOptions& options,
const InternalKeyComparator& internal_comparator,
const FileMetaData& file_meta, const Slice& k,
GetContext* get_context,
const SliceTransform* prefix_extractor,
HistogramImpl* file_read_hist, bool skip_filters,
int level, size_t max_file_size_for_l0_meta_pin) {
auto& fd = file_meta.fd;
std::string* row_cache_entry = nullptr;
bool done = false;
#ifndef ROCKSDB_LITE
IterKey row_cache_key;
std::string row_cache_entry_buffer;
// Check row cache if enabled. Since row cache does not currently store
// sequence numbers, we cannot use it if we need to fetch the sequence.
if (ioptions_.row_cache && !get_context->NeedToReadSequence()) {
auto user_key = ExtractUserKey(k);
CreateRowCacheKeyPrefix(options, fd, k, get_context, row_cache_key);
done = GetFromRowCache(user_key, row_cache_key, row_cache_key.Size(),
get_context);
if (!done) {
row_cache_entry = &row_cache_entry_buffer;
}
}
#endif // ROCKSDB_LITE
Status s;
TableReader* t = fd.table_reader;
Cache::Handle* handle = nullptr;
if (!done) {
assert(s.ok());
if (t == nullptr) {
s = FindTable(options, file_options_, internal_comparator, fd, &handle,
prefix_extractor,
options.read_tier == kBlockCacheTier /* no_io */,
true /* record_read_stats */, file_read_hist, skip_filters,
level, true /* prefetch_index_and_filter_in_cache */,
max_file_size_for_l0_meta_pin);
if (s.ok()) {
t = GetTableReaderFromHandle(handle);
}
}
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 =
get_context->max_covering_tombstone_seq();
if (s.ok() && max_covering_tombstone_seq != nullptr &&
!options.ignore_range_deletions) {
std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
t->NewRangeTombstoneIterator(options));
if (range_del_iter != nullptr) {
*max_covering_tombstone_seq = std::max(
*max_covering_tombstone_seq,
range_del_iter->MaxCoveringTombstoneSeqnum(ExtractUserKey(k)));
}
}
if (s.ok()) {
get_context->SetReplayLog(row_cache_entry); // nullptr if no cache.
s = t->Get(options, k, get_context, prefix_extractor, skip_filters);
get_context->SetReplayLog(nullptr);
} else if (options.read_tier == kBlockCacheTier && s.IsIncomplete()) {
// Couldn't find Table in cache but treat as kFound if no_io set
get_context->MarkKeyMayExist();
s = Status::OK();
done = true;
}
}
#ifndef ROCKSDB_LITE
// Put the replay log in row cache only if something was found.
if (!done && s.ok() && row_cache_entry && !row_cache_entry->empty()) {
size_t charge =
row_cache_key.Size() + row_cache_entry->size() + sizeof(std::string);
void* row_ptr = new std::string(std::move(*row_cache_entry));
// If row cache is full, it's OK to continue.
ioptions_.row_cache
->Insert(row_cache_key.GetUserKey(), row_ptr, charge,
&DeleteEntry<std::string>)
.PermitUncheckedError();
}
#endif // ROCKSDB_LITE
if (handle != nullptr) {
ReleaseHandle(handle);
}
return s;
}
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
// Batched version of TableCache::MultiGet.
Status TableCache::MultiGet(const ReadOptions& options,
const InternalKeyComparator& internal_comparator,
const FileMetaData& file_meta,
const MultiGetContext::Range* mget_range,
const SliceTransform* prefix_extractor,
HistogramImpl* file_read_hist, bool skip_filters,
int level) {
auto& fd = file_meta.fd;
Status s;
TableReader* t = fd.table_reader;
Cache::Handle* handle = nullptr;
MultiGetRange table_range(*mget_range, mget_range->begin(),
mget_range->end());
#ifndef ROCKSDB_LITE
autovector<std::string, MultiGetContext::MAX_BATCH_SIZE> row_cache_entries;
IterKey row_cache_key;
size_t row_cache_key_prefix_size = 0;
KeyContext& first_key = *table_range.begin();
bool lookup_row_cache =
ioptions_.row_cache && !first_key.get_context->NeedToReadSequence();
// Check row cache if enabled. Since row cache does not currently store
// sequence numbers, we cannot use it if we need to fetch the sequence.
if (lookup_row_cache) {
GetContext* first_context = first_key.get_context;
CreateRowCacheKeyPrefix(options, fd, first_key.ikey, first_context,
row_cache_key);
row_cache_key_prefix_size = row_cache_key.Size();
for (auto miter = table_range.begin(); miter != table_range.end();
++miter) {
const Slice& user_key = miter->ukey_with_ts;
GetContext* get_context = miter->get_context;
if (GetFromRowCache(user_key, row_cache_key, row_cache_key_prefix_size,
get_context)) {
table_range.SkipKey(miter);
} else {
row_cache_entries.emplace_back();
get_context->SetReplayLog(&(row_cache_entries.back()));
}
}
}
#endif // ROCKSDB_LITE
// Check that table_range is not empty. Its possible all keys may have been
// found in the row cache and thus the range may now be empty
if (s.ok() && !table_range.empty()) {
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
if (t == nullptr) {
s = FindTable(
options, file_options_, internal_comparator, fd, &handle,
prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */,
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
true /* record_read_stats */, file_read_hist, skip_filters, level);
TEST_SYNC_POINT_CALLBACK("TableCache::MultiGet:FindTable", &s);
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
if (s.ok()) {
t = GetTableReaderFromHandle(handle);
assert(t);
}
}
if (s.ok() && !options.ignore_range_deletions) {
std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
t->NewRangeTombstoneIterator(options));
if (range_del_iter != nullptr) {
for (auto iter = table_range.begin(); iter != table_range.end();
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
++iter) {
SequenceNumber* max_covering_tombstone_seq =
iter->get_context->max_covering_tombstone_seq();
*max_covering_tombstone_seq = std::max(
*max_covering_tombstone_seq,
range_del_iter->MaxCoveringTombstoneSeqnum(iter->ukey_with_ts));
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
}
}
}
if (s.ok()) {
t->MultiGet(options, &table_range, prefix_extractor, skip_filters);
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
} else if (options.read_tier == kBlockCacheTier && s.IsIncomplete()) {
for (auto iter = table_range.begin(); iter != table_range.end(); ++iter) {
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
Status* status = iter->s;
if (status->IsIncomplete()) {
// Couldn't find Table in cache but treat as kFound if no_io set
iter->get_context->MarkKeyMayExist();
s = Status::OK();
}
}
}
}
#ifndef ROCKSDB_LITE
if (lookup_row_cache) {
size_t row_idx = 0;
for (auto miter = table_range.begin(); miter != table_range.end();
++miter) {
std::string& row_cache_entry = row_cache_entries[row_idx++];
const Slice& user_key = miter->ukey_with_ts;
;
GetContext* get_context = miter->get_context;
get_context->SetReplayLog(nullptr);
// Compute row cache key.
row_cache_key.TrimAppend(row_cache_key_prefix_size, user_key.data(),
user_key.size());
// Put the replay log in row cache only if something was found.
if (s.ok() && !row_cache_entry.empty()) {
size_t charge =
row_cache_key.Size() + row_cache_entry.size() + sizeof(std::string);
void* row_ptr = new std::string(std::move(row_cache_entry));
// If row cache is full, it's OK.
ioptions_.row_cache
->Insert(row_cache_key.GetUserKey(), row_ptr, charge,
&DeleteEntry<std::string>)
.PermitUncheckedError();
}
}
}
#endif // ROCKSDB_LITE
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
if (handle != nullptr) {
ReleaseHandle(handle);
}
return s;
}
Status TableCache::GetTableProperties(
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 InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
std::shared_ptr<const TableProperties>* properties,
const SliceTransform* prefix_extractor, bool no_io) {
auto table_reader = fd.table_reader;
// table already been pre-loaded?
if (table_reader) {
*properties = table_reader->GetTableProperties();
return Status::OK();
}
Cache::Handle* table_handle = nullptr;
Status s = FindTable(ReadOptions(), file_options, internal_comparator, fd,
&table_handle, prefix_extractor, no_io);
if (!s.ok()) {
return s;
}
assert(table_handle);
auto table = GetTableReaderFromHandle(table_handle);
*properties = table->GetTableProperties();
ReleaseHandle(table_handle);
return s;
}
size_t TableCache::GetMemoryUsageByTableReader(
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 InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
const SliceTransform* prefix_extractor) {
auto table_reader = fd.table_reader;
// table already been pre-loaded?
if (table_reader) {
return table_reader->ApproximateMemoryUsage();
}
Cache::Handle* table_handle = nullptr;
Status s = FindTable(ReadOptions(), file_options, internal_comparator, fd,
&table_handle, prefix_extractor, true);
if (!s.ok()) {
return 0;
}
assert(table_handle);
auto table = GetTableReaderFromHandle(table_handle);
auto ret = table->ApproximateMemoryUsage();
ReleaseHandle(table_handle);
return ret;
}
void TableCache::Evict(Cache* cache, uint64_t file_number) {
cache->Erase(GetSliceForFileNumber(&file_number));
}
uint64_t TableCache::ApproximateOffsetOf(
const Slice& key, const FileDescriptor& fd, TableReaderCaller caller,
const InternalKeyComparator& internal_comparator,
const SliceTransform* prefix_extractor) {
uint64_t result = 0;
TableReader* table_reader = fd.table_reader;
Cache::Handle* table_handle = nullptr;
if (table_reader == nullptr) {
const bool for_compaction = (caller == TableReaderCaller::kCompaction);
Status s = FindTable(ReadOptions(), file_options_, internal_comparator, fd,
&table_handle, prefix_extractor, false /* no_io */,
!for_compaction /* record_read_stats */);
if (s.ok()) {
table_reader = GetTableReaderFromHandle(table_handle);
}
}
if (table_reader != nullptr) {
result = table_reader->ApproximateOffsetOf(key, caller);
}
if (table_handle != nullptr) {
ReleaseHandle(table_handle);
}
return result;
}
uint64_t TableCache::ApproximateSize(
const Slice& start, const Slice& end, const FileDescriptor& fd,
TableReaderCaller caller, const InternalKeyComparator& internal_comparator,
const SliceTransform* prefix_extractor) {
uint64_t result = 0;
TableReader* table_reader = fd.table_reader;
Cache::Handle* table_handle = nullptr;
if (table_reader == nullptr) {
const bool for_compaction = (caller == TableReaderCaller::kCompaction);
Status s = FindTable(ReadOptions(), file_options_, internal_comparator, fd,
&table_handle, prefix_extractor, false /* no_io */,
!for_compaction /* record_read_stats */);
if (s.ok()) {
table_reader = GetTableReaderFromHandle(table_handle);
}
}
if (table_reader != nullptr) {
result = table_reader->ApproximateSize(start, end, caller);
}
if (table_handle != nullptr) {
ReleaseHandle(table_handle);
}
return result;
}
} // namespace ROCKSDB_NAMESPACE