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/table/block_based/block_based_table_reader.cc

3075 lines
115 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 "table/block_based/block_based_table_reader.h"
#include <algorithm>
#include <array>
Always verify SST unique IDs on SST file open (#10532) Summary: Although we've been tracking SST unique IDs in the DB manifest unconditionally, checking has been opt-in and with an extra pass at DB::Open time. This changes the behavior of `verify_sst_unique_id_in_manifest` to check unique ID against manifest every time an SST file is opened through table cache (normal DB operations), replacing the explicit pass over files at DB::Open time. This change also enables the option by default and removes the "EXPERIMENTAL" designation. One possible criticism is that the option no longer ensures the integrity of a DB at Open time. This is far from an all-or-nothing issue. Verifying the IDs of all SST files hardly ensures all the data in the DB is readable. (VerifyChecksum is supposed to do that.) Also, with max_open_files=-1 (default, extremely common), all SST files are opened at DB::Open time anyway. Implementation details: * `VerifySstUniqueIdInManifest()` functions are the extra/explicit pass that is now removed. * Unit tests that manipulate/corrupt table properties have to opt out of this check, because that corrupts the "actual" unique id. (And even for testing we don't currently have a mechanism to set "no unique id" in the in-memory file metadata for new files.) * A lot of other unit test churn relates to (a) default checking on, and (b) checking on SST open even without DB::Open (e.g. on flush) * Use `FileMetaData` for more `TableCache` operations (in place of `FileDescriptor`) so that we have access to the unique_id whenever we might need to open an SST file. **There is the possibility of performance impact because we can no longer use the more localized `fd` part of an `FdWithKeyRange` but instead follow the `file_metadata` pointer. However, this change (possible regression) is only done for `GetMemoryUsageByTableReaders`.** * Removed a completely unnecessary constructor overload of `TableReaderOptions` Possible follow-up: * Verification only happens when opening through table cache. Are there more places where this should happen? * Improve error message when there is a file size mismatch vs. manifest (FIXME added in the appropriate place). * I'm not sure there's a justification for `FileDescriptor` to be distinct from `FileMetaData`. * I'm skeptical that `FdWithKeyRange` really still makes sense for optimizing some data locality by duplicating some data in memory, but I could be wrong. * An unnecessary overload of NewTableReader was recently added, in the public API nonetheless (though unusable there). It should be cleaned up to put most things under `TableReaderOptions`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10532 Test Plan: updated unit tests Performance test showing no significant difference (just noise I think): `./db_bench -benchmarks=readwhilewriting[-X10] -num=3000000 -disable_wal=1 -bloom_bits=8 -write_buffer_size=1000000 -target_file_size_base=1000000` Before: readwhilewriting [AVG 10 runs] : 68702 (± 6932) ops/sec After: readwhilewriting [AVG 10 runs] : 68239 (± 7198) ops/sec Reviewed By: jay-zhuang Differential Revision: D38765551 Pulled By: pdillinger fbshipit-source-id: a827a708155f12344ab2a5c16e7701c7636da4c2
2 years ago
#include <atomic>
Rewrite memory-charging feature's option API (#9926) Summary: **Context:** Previous PR https://github.com/facebook/rocksdb/pull/9748, https://github.com/facebook/rocksdb/pull/9073, https://github.com/facebook/rocksdb/pull/8428 added separate flag for each charged memory area. Such API design is not scalable as we charge more and more memory areas. Also, we foresee an opportunity to consolidate this feature with other cache usage related features such as `cache_index_and_filter_blocks` using `CacheEntryRole`. Therefore we decided to consolidate all these flags with `CacheUsageOptions cache_usage_options` and this PR serves as the first step by consolidating memory-charging related flags. **Summary:** - Replaced old API reference with new ones, including making `kCompressionDictionaryBuildingBuffer` opt-out and added a unit test for that - Added missing db bench/stress test for some memory charging features - Renamed related test suite to indicate they are under the same theme of memory charging - Refactored a commonly used mocked cache component in memory charging related tests to reduce code duplication - Replaced the phrases "memory tracking" / "cache reservation" (other than CacheReservationManager-related ones) with "memory charging" for standard description of this feature. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9926 Test Plan: - New unit test for opt-out `kCompressionDictionaryBuildingBuffer` `TEST_F(ChargeCompressionDictionaryBuildingBufferTest, Basic)` - New unit test for option validation/sanitization `TEST_F(CacheUsageOptionsOverridesTest, SanitizeAndValidateOptions)` - CI - db bench (in case querying new options introduces regression) **+0.5% micros/op**: `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_compression_dictionary_building_buffer=1(remove this for comparison) -compression_max_dict_bytes=10000 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | **-0.3633711465** 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | **0.5289363078** - db_stress: `python3 tools/db_crashtest.py blackbox -charge_compression_dictionary_building_buffer=1 -charge_filter_construction=1 -charge_table_reader=1 -cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36054712 Pulled By: hx235 fbshipit-source-id: d406e90f5e0c5ea4dbcb585a484ad9302d4302af
3 years ago
#include <cstdint>
#include <limits>
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
#include <memory>
#include <string>
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
#include <unordered_set>
#include <utility>
#include <vector>
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
#include "block_cache.h"
Use deleters to label cache entries and collect stats (#8297) Summary: This change gathers and publishes statistics about the kinds of items in block cache. This is especially important for profiling relative usage of cache by index vs. filter vs. data blocks. It works by iterating over the cache during periodic stats dump (InternalStats, stats_dump_period_sec) or on demand when DB::Get(Map)Property(kBlockCacheEntryStats), except that for efficiency and sharing among column families, saved data from the last scan is used when the data is not considered too old. The new information can be seen in info LOG, for example: Block cache LRUCache@0x7fca62229330 capacity: 95.37 MB collections: 8 last_copies: 0 last_secs: 0.00178 secs_since: 0 Block cache entry stats(count,size,portion): DataBlock(7092,28.24 MB,29.6136%) FilterBlock(215,867.90 KB,0.888728%) FilterMetaBlock(2,5.31 KB,0.00544%) IndexBlock(217,180.11 KB,0.184432%) WriteBuffer(1,256.00 KB,0.262144%) Misc(1,0.00 KB,0%) And also through DB::GetProperty and GetMapProperty (here using ldb just for demonstration): $ ./ldb --db=/dev/shm/dbbench/ get_property rocksdb.block-cache-entry-stats rocksdb.block-cache-entry-stats.bytes.data-block: 0 rocksdb.block-cache-entry-stats.bytes.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-meta-block: 0 rocksdb.block-cache-entry-stats.bytes.index-block: 178992 rocksdb.block-cache-entry-stats.bytes.misc: 0 rocksdb.block-cache-entry-stats.bytes.other-block: 0 rocksdb.block-cache-entry-stats.bytes.write-buffer: 0 rocksdb.block-cache-entry-stats.capacity: 8388608 rocksdb.block-cache-entry-stats.count.data-block: 0 rocksdb.block-cache-entry-stats.count.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-meta-block: 0 rocksdb.block-cache-entry-stats.count.index-block: 215 rocksdb.block-cache-entry-stats.count.misc: 1 rocksdb.block-cache-entry-stats.count.other-block: 0 rocksdb.block-cache-entry-stats.count.write-buffer: 0 rocksdb.block-cache-entry-stats.id: LRUCache@0x7f3636661290 rocksdb.block-cache-entry-stats.percent.data-block: 0.000000 rocksdb.block-cache-entry-stats.percent.deprecated-filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-meta-block: 0.000000 rocksdb.block-cache-entry-stats.percent.index-block: 2.133751 rocksdb.block-cache-entry-stats.percent.misc: 0.000000 rocksdb.block-cache-entry-stats.percent.other-block: 0.000000 rocksdb.block-cache-entry-stats.percent.write-buffer: 0.000000 rocksdb.block-cache-entry-stats.secs_for_last_collection: 0.000052 rocksdb.block-cache-entry-stats.secs_since_last_collection: 0 Solution detail - We need some way to flag what kind of blocks each entry belongs to, preferably without changing the Cache API. One of the complications is that Cache is a general interface that could have other users that don't adhere to whichever convention we decide on for keys and values. Or we would pay for an extra field in the Handle that would only be used for this purpose. This change uses a back-door approach, the deleter, to indicate the "role" of a Cache entry (in addition to the value type, implicitly). This has the added benefit of ensuring proper code origin whenever we recognize a particular role for a cache entry; if the entry came from some other part of the code, it will use an unrecognized deleter, which we simply attribute to the "Misc" role. An internal API makes for simple instantiation and automatic registration of Cache deleters for a given value type and "role". Another internal API, CacheEntryStatsCollector, solves the problem of caching the results of a scan and sharing them, to ensure scans are neither excessive nor redundant so as not to harm Cache performance. Because code is added to BlocklikeTraits, it is pulled out of block_based_table_reader.cc into its own file. This is a reformulation of https://github.com/facebook/rocksdb/issues/8276, without the type checking option (could still be added), and with actual stat gathering. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8297 Test Plan: manual testing with db_bench, and a couple of basic unit tests Reviewed By: ltamasi Differential Revision: D28488721 Pulled By: pdillinger fbshipit-source-id: 472f524a9691b5afb107934be2d41d84f2b129fb
4 years ago
#include "cache/cache_entry_roles.h"
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
#include "cache/cache_key.h"
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
#include "db/compaction/compaction_picker.h"
#include "db/dbformat.h"
Introduce FullMergeV2 (eliminate memcpy from merge operators) Summary: This diff update the code to pin the merge operator operands while the merge operation is done, so that we can eliminate the memcpy cost, to do that we need a new public API for FullMerge that replace the std::deque<std::string> with std::vector<Slice> This diff is stacked on top of D56493 and D56511 In this diff we - Update FullMergeV2 arguments to be encapsulated in MergeOperationInput and MergeOperationOutput which will make it easier to add new arguments in the future - Replace std::deque<std::string> with std::vector<Slice> to pass operands - Replace MergeContext std::deque with std::vector (based on a simple benchmark I ran https://gist.github.com/IslamAbdelRahman/78fc86c9ab9f52b1df791e58943fb187) - Allow FullMergeV2 output to be an existing operand ``` [Everything in Memtable | 10K operands | 10 KB each | 1 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=10000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 0.607 micros/op 1648235 ops/sec; 16121.2 MB/s readseq : 0.478 micros/op 2091546 ops/sec; 20457.2 MB/s readseq : 0.252 micros/op 3972081 ops/sec; 38850.5 MB/s readseq : 0.237 micros/op 4218328 ops/sec; 41259.0 MB/s readseq : 0.247 micros/op 4043927 ops/sec; 39553.2 MB/s [master] readseq : 3.935 micros/op 254140 ops/sec; 2485.7 MB/s readseq : 3.722 micros/op 268657 ops/sec; 2627.7 MB/s readseq : 3.149 micros/op 317605 ops/sec; 3106.5 MB/s readseq : 3.125 micros/op 320024 ops/sec; 3130.1 MB/s readseq : 4.075 micros/op 245374 ops/sec; 2400.0 MB/s ``` ``` [Everything in Memtable | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="mergerandom,readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --merge_keys=1000 --num=10000 --disable_auto_compactions --value_size=10240 --write_buffer_size=1000000000 [FullMergeV2] readseq : 3.472 micros/op 288018 ops/sec; 2817.1 MB/s readseq : 2.304 micros/op 434027 ops/sec; 4245.2 MB/s readseq : 1.163 micros/op 859845 ops/sec; 8410.0 MB/s readseq : 1.192 micros/op 838926 ops/sec; 8205.4 MB/s readseq : 1.250 micros/op 800000 ops/sec; 7824.7 MB/s [master] readseq : 24.025 micros/op 41623 ops/sec; 407.1 MB/s readseq : 18.489 micros/op 54086 ops/sec; 529.0 MB/s readseq : 18.693 micros/op 53495 ops/sec; 523.2 MB/s readseq : 23.621 micros/op 42335 ops/sec; 414.1 MB/s readseq : 18.775 micros/op 53262 ops/sec; 521.0 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 1 operand per key] [FullMergeV2] $ DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions readseq : 14.741 micros/op 67837 ops/sec; 663.5 MB/s readseq : 1.029 micros/op 971446 ops/sec; 9501.6 MB/s readseq : 0.974 micros/op 1026229 ops/sec; 10037.4 MB/s readseq : 0.965 micros/op 1036080 ops/sec; 10133.8 MB/s readseq : 0.943 micros/op 1060657 ops/sec; 10374.2 MB/s [master] readseq : 16.735 micros/op 59755 ops/sec; 584.5 MB/s readseq : 3.029 micros/op 330151 ops/sec; 3229.2 MB/s readseq : 3.136 micros/op 318883 ops/sec; 3119.0 MB/s readseq : 3.065 micros/op 326245 ops/sec; 3191.0 MB/s readseq : 3.014 micros/op 331813 ops/sec; 3245.4 MB/s ``` ``` [Everything in Block cache | 10K operands | 10 KB each | 10 operand per key] DEBUG_LEVEL=0 make db_bench -j64 && ./db_bench --benchmarks="readseq,readseq,readseq,readseq,readseq" --merge_operator="max" --num=100000 --db="/dev/shm/merge-random-10-operands-10K-10KB" --cache_size=1000000000 --use_existing_db --disable_auto_compactions [FullMergeV2] readseq : 24.325 micros/op 41109 ops/sec; 402.1 MB/s readseq : 1.470 micros/op 680272 ops/sec; 6653.7 MB/s readseq : 1.231 micros/op 812347 ops/sec; 7945.5 MB/s readseq : 1.091 micros/op 916590 ops/sec; 8965.1 MB/s readseq : 1.109 micros/op 901713 ops/sec; 8819.6 MB/s [master] readseq : 27.257 micros/op 36687 ops/sec; 358.8 MB/s readseq : 4.443 micros/op 225073 ops/sec; 2201.4 MB/s readseq : 5.830 micros/op 171526 ops/sec; 1677.7 MB/s readseq : 4.173 micros/op 239635 ops/sec; 2343.8 MB/s readseq : 4.150 micros/op 240963 ops/sec; 2356.8 MB/s ``` Test Plan: COMPILE_WITH_ASAN=1 make check -j64 Reviewers: yhchiang, andrewkr, sdong Reviewed By: sdong Subscribers: lovro, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D57075
8 years ago
#include "db/pinned_iterators_manager.h"
#include "file/file_prefetch_buffer.h"
#include "file/file_util.h"
#include "file/random_access_file_reader.h"
#include "logging/logging.h"
#include "monitoring/perf_context_imp.h"
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
#include "parsed_full_filter_block.h"
#include "port/lang.h"
#include "rocksdb/cache.h"
#include "rocksdb/comparator.h"
#include "rocksdb/convenience.h"
#include "rocksdb/env.h"
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
#include "rocksdb/file_system.h"
#include "rocksdb/filter_policy.h"
#include "rocksdb/iterator.h"
#include "rocksdb/options.h"
#include "rocksdb/snapshot.h"
#include "rocksdb/statistics.h"
#include "rocksdb/system_clock.h"
#include "rocksdb/table.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/trace_record.h"
#include "table/block_based/binary_search_index_reader.h"
#include "table/block_based/block.h"
#include "table/block_based/block_based_table_factory.h"
#include "table/block_based/block_based_table_iterator.h"
#include "table/block_based/block_prefix_index.h"
Use deleters to label cache entries and collect stats (#8297) Summary: This change gathers and publishes statistics about the kinds of items in block cache. This is especially important for profiling relative usage of cache by index vs. filter vs. data blocks. It works by iterating over the cache during periodic stats dump (InternalStats, stats_dump_period_sec) or on demand when DB::Get(Map)Property(kBlockCacheEntryStats), except that for efficiency and sharing among column families, saved data from the last scan is used when the data is not considered too old. The new information can be seen in info LOG, for example: Block cache LRUCache@0x7fca62229330 capacity: 95.37 MB collections: 8 last_copies: 0 last_secs: 0.00178 secs_since: 0 Block cache entry stats(count,size,portion): DataBlock(7092,28.24 MB,29.6136%) FilterBlock(215,867.90 KB,0.888728%) FilterMetaBlock(2,5.31 KB,0.00544%) IndexBlock(217,180.11 KB,0.184432%) WriteBuffer(1,256.00 KB,0.262144%) Misc(1,0.00 KB,0%) And also through DB::GetProperty and GetMapProperty (here using ldb just for demonstration): $ ./ldb --db=/dev/shm/dbbench/ get_property rocksdb.block-cache-entry-stats rocksdb.block-cache-entry-stats.bytes.data-block: 0 rocksdb.block-cache-entry-stats.bytes.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-block: 0 rocksdb.block-cache-entry-stats.bytes.filter-meta-block: 0 rocksdb.block-cache-entry-stats.bytes.index-block: 178992 rocksdb.block-cache-entry-stats.bytes.misc: 0 rocksdb.block-cache-entry-stats.bytes.other-block: 0 rocksdb.block-cache-entry-stats.bytes.write-buffer: 0 rocksdb.block-cache-entry-stats.capacity: 8388608 rocksdb.block-cache-entry-stats.count.data-block: 0 rocksdb.block-cache-entry-stats.count.deprecated-filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-block: 0 rocksdb.block-cache-entry-stats.count.filter-meta-block: 0 rocksdb.block-cache-entry-stats.count.index-block: 215 rocksdb.block-cache-entry-stats.count.misc: 1 rocksdb.block-cache-entry-stats.count.other-block: 0 rocksdb.block-cache-entry-stats.count.write-buffer: 0 rocksdb.block-cache-entry-stats.id: LRUCache@0x7f3636661290 rocksdb.block-cache-entry-stats.percent.data-block: 0.000000 rocksdb.block-cache-entry-stats.percent.deprecated-filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-block: 0.000000 rocksdb.block-cache-entry-stats.percent.filter-meta-block: 0.000000 rocksdb.block-cache-entry-stats.percent.index-block: 2.133751 rocksdb.block-cache-entry-stats.percent.misc: 0.000000 rocksdb.block-cache-entry-stats.percent.other-block: 0.000000 rocksdb.block-cache-entry-stats.percent.write-buffer: 0.000000 rocksdb.block-cache-entry-stats.secs_for_last_collection: 0.000052 rocksdb.block-cache-entry-stats.secs_since_last_collection: 0 Solution detail - We need some way to flag what kind of blocks each entry belongs to, preferably without changing the Cache API. One of the complications is that Cache is a general interface that could have other users that don't adhere to whichever convention we decide on for keys and values. Or we would pay for an extra field in the Handle that would only be used for this purpose. This change uses a back-door approach, the deleter, to indicate the "role" of a Cache entry (in addition to the value type, implicitly). This has the added benefit of ensuring proper code origin whenever we recognize a particular role for a cache entry; if the entry came from some other part of the code, it will use an unrecognized deleter, which we simply attribute to the "Misc" role. An internal API makes for simple instantiation and automatic registration of Cache deleters for a given value type and "role". Another internal API, CacheEntryStatsCollector, solves the problem of caching the results of a scan and sharing them, to ensure scans are neither excessive nor redundant so as not to harm Cache performance. Because code is added to BlocklikeTraits, it is pulled out of block_based_table_reader.cc into its own file. This is a reformulation of https://github.com/facebook/rocksdb/issues/8276, without the type checking option (could still be added), and with actual stat gathering. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8297 Test Plan: manual testing with db_bench, and a couple of basic unit tests Reviewed By: ltamasi Differential Revision: D28488721 Pulled By: pdillinger fbshipit-source-id: 472f524a9691b5afb107934be2d41d84f2b129fb
4 years ago
#include "table/block_based/block_type.h"
#include "table/block_based/filter_block.h"
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
#include "table/block_based/filter_policy_internal.h"
#include "table/block_based/full_filter_block.h"
#include "table/block_based/hash_index_reader.h"
#include "table/block_based/partitioned_filter_block.h"
#include "table/block_based/partitioned_index_reader.h"
#include "table/block_fetcher.h"
#include "table/format.h"
#include "table/get_context.h"
#include "table/internal_iterator.h"
#include "table/meta_blocks.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/persistent_cache_helper.h"
#include "table/persistent_cache_options.h"
#include "table/sst_file_writer_collectors.h"
#include "table/two_level_iterator.h"
#include "test_util/sync_point.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/stop_watch.h"
#include "util/string_util.h"
Multi file concurrency in MultiGet using coroutines and async IO (#9968) Summary: This PR implements a coroutine version of batched MultiGet in order to concurrently read from multiple SST files in a level using async IO, thus reducing the latency of the MultiGet. The API from the user perspective is still synchronous and single threaded, with the RocksDB part of the processing happening in the context of the caller's thread. In Version::MultiGet, the decision is made whether to call synchronous or coroutine code. A good way to review this PR is to review the first 4 commits in order - de773b3, 70c2f70, 10b50e1, and 377a597 - before reviewing the rest. TODO: 1. Figure out how to build it in CircleCI (requires some dependencies to be installed) 2. Do some stress testing with coroutines enabled No regression in synchronous MultiGet between this branch and main - ``` ./db_bench -use_existing_db=true --db=/data/mysql/rocksdb/prefix_scan -benchmarks="readseq,multireadrandom" -key_size=32 -value_size=512 -num=5000000 -batch_size=64 -multiread_batched=true -use_direct_reads=false -duration=60 -ops_between_duration_checks=1 -readonly=true -adaptive_readahead=true -threads=16 -cache_size=10485760000 -async_io=false -multiread_stride=40000 -statistics ``` Branch - ```multireadrandom : 4.025 micros/op 3975111 ops/sec 60.001 seconds 238509056 operations; 2062.3 MB/s (14767808 of 14767808 found)``` Main - ```multireadrandom : 3.987 micros/op 4013216 ops/sec 60.001 seconds 240795392 operations; 2082.1 MB/s (15231040 of 15231040 found)``` More benchmarks in various scenarios are given below. The measurements were taken with ```async_io=false``` (no coroutines) and ```async_io=true``` (use coroutines). For an IO bound workload (with every key requiring an IO), the coroutines version shows a clear benefit, being ~2.6X faster. For CPU bound workloads, the coroutines version has ~6-15% higher CPU utilization, depending on how many keys overlap an SST file. 1. Single thread IO bound workload on remote storage with sparse MultiGet batch keys (~1 key overlap/file) - No coroutines - ```multireadrandom : 831.774 micros/op 1202 ops/sec 60.001 seconds 72136 operations; 0.6 MB/s (72136 of 72136 found)``` Using coroutines - ```multireadrandom : 318.742 micros/op 3137 ops/sec 60.003 seconds 188248 operations; 1.6 MB/s (188248 of 188248 found)``` 2. Single thread CPU bound workload (all data cached) with ~1 key overlap/file - No coroutines - ```multireadrandom : 4.127 micros/op 242322 ops/sec 60.000 seconds 14539384 operations; 125.7 MB/s (14539384 of 14539384 found)``` Using coroutines - ```multireadrandom : 4.741 micros/op 210935 ops/sec 60.000 seconds 12656176 operations; 109.4 MB/s (12656176 of 12656176 found)``` 3. Single thread CPU bound workload with ~2 key overlap/file - No coroutines - ```multireadrandom : 3.717 micros/op 269000 ops/sec 60.000 seconds 16140024 operations; 139.6 MB/s (16140024 of 16140024 found)``` Using coroutines - ```multireadrandom : 4.146 micros/op 241204 ops/sec 60.000 seconds 14472296 operations; 125.1 MB/s (14472296 of 14472296 found)``` 4. CPU bound multi-threaded (16 threads) with ~4 key overlap/file - No coroutines - ```multireadrandom : 4.534 micros/op 3528792 ops/sec 60.000 seconds 211728728 operations; 1830.7 MB/s (12737024 of 12737024 found) ``` Using coroutines - ```multireadrandom : 4.872 micros/op 3283812 ops/sec 60.000 seconds 197030096 operations; 1703.6 MB/s (12548032 of 12548032 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9968 Reviewed By: akankshamahajan15 Differential Revision: D36348563 Pulled By: anand1976 fbshipit-source-id: c0ce85a505fd26ebfbb09786cbd7f25202038696
3 years ago
namespace ROCKSDB_NAMESPACE {
namespace {
CacheAllocationPtr CopyBufferToHeap(MemoryAllocator* allocator, Slice& buf) {
CacheAllocationPtr heap_buf;
heap_buf = AllocateBlock(buf.size(), allocator);
memcpy(heap_buf.get(), buf.data(), buf.size());
return heap_buf;
}
} // namespace
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
// Explicitly instantiate templates for each "blocklike" type we use (and
// before implicit specialization).
// This makes it possible to keep the template definitions in the .cc file.
#define INSTANTIATE_RETRIEVE_BLOCK(T) \
template Status BlockBasedTable::RetrieveBlock<T>( \
FilePrefetchBuffer * prefetch_buffer, const ReadOptions& ro, \
const BlockHandle& handle, const UncompressionDict& uncompression_dict, \
CachableEntry<T>* out_parsed_block, GetContext* get_context, \
BlockCacheLookupContext* lookup_context, bool for_compaction, \
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
bool use_cache, bool async_read) const;
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
INSTANTIATE_RETRIEVE_BLOCK(ParsedFullFilterBlock);
INSTANTIATE_RETRIEVE_BLOCK(UncompressionDict);
INSTANTIATE_RETRIEVE_BLOCK(Block_kData);
INSTANTIATE_RETRIEVE_BLOCK(Block_kIndex);
INSTANTIATE_RETRIEVE_BLOCK(Block_kFilterPartitionIndex);
INSTANTIATE_RETRIEVE_BLOCK(Block_kRangeDeletion);
INSTANTIATE_RETRIEVE_BLOCK(Block_kMetaIndex);
Multi file concurrency in MultiGet using coroutines and async IO (#9968) Summary: This PR implements a coroutine version of batched MultiGet in order to concurrently read from multiple SST files in a level using async IO, thus reducing the latency of the MultiGet. The API from the user perspective is still synchronous and single threaded, with the RocksDB part of the processing happening in the context of the caller's thread. In Version::MultiGet, the decision is made whether to call synchronous or coroutine code. A good way to review this PR is to review the first 4 commits in order - de773b3, 70c2f70, 10b50e1, and 377a597 - before reviewing the rest. TODO: 1. Figure out how to build it in CircleCI (requires some dependencies to be installed) 2. Do some stress testing with coroutines enabled No regression in synchronous MultiGet between this branch and main - ``` ./db_bench -use_existing_db=true --db=/data/mysql/rocksdb/prefix_scan -benchmarks="readseq,multireadrandom" -key_size=32 -value_size=512 -num=5000000 -batch_size=64 -multiread_batched=true -use_direct_reads=false -duration=60 -ops_between_duration_checks=1 -readonly=true -adaptive_readahead=true -threads=16 -cache_size=10485760000 -async_io=false -multiread_stride=40000 -statistics ``` Branch - ```multireadrandom : 4.025 micros/op 3975111 ops/sec 60.001 seconds 238509056 operations; 2062.3 MB/s (14767808 of 14767808 found)``` Main - ```multireadrandom : 3.987 micros/op 4013216 ops/sec 60.001 seconds 240795392 operations; 2082.1 MB/s (15231040 of 15231040 found)``` More benchmarks in various scenarios are given below. The measurements were taken with ```async_io=false``` (no coroutines) and ```async_io=true``` (use coroutines). For an IO bound workload (with every key requiring an IO), the coroutines version shows a clear benefit, being ~2.6X faster. For CPU bound workloads, the coroutines version has ~6-15% higher CPU utilization, depending on how many keys overlap an SST file. 1. Single thread IO bound workload on remote storage with sparse MultiGet batch keys (~1 key overlap/file) - No coroutines - ```multireadrandom : 831.774 micros/op 1202 ops/sec 60.001 seconds 72136 operations; 0.6 MB/s (72136 of 72136 found)``` Using coroutines - ```multireadrandom : 318.742 micros/op 3137 ops/sec 60.003 seconds 188248 operations; 1.6 MB/s (188248 of 188248 found)``` 2. Single thread CPU bound workload (all data cached) with ~1 key overlap/file - No coroutines - ```multireadrandom : 4.127 micros/op 242322 ops/sec 60.000 seconds 14539384 operations; 125.7 MB/s (14539384 of 14539384 found)``` Using coroutines - ```multireadrandom : 4.741 micros/op 210935 ops/sec 60.000 seconds 12656176 operations; 109.4 MB/s (12656176 of 12656176 found)``` 3. Single thread CPU bound workload with ~2 key overlap/file - No coroutines - ```multireadrandom : 3.717 micros/op 269000 ops/sec 60.000 seconds 16140024 operations; 139.6 MB/s (16140024 of 16140024 found)``` Using coroutines - ```multireadrandom : 4.146 micros/op 241204 ops/sec 60.000 seconds 14472296 operations; 125.1 MB/s (14472296 of 14472296 found)``` 4. CPU bound multi-threaded (16 threads) with ~4 key overlap/file - No coroutines - ```multireadrandom : 4.534 micros/op 3528792 ops/sec 60.000 seconds 211728728 operations; 1830.7 MB/s (12737024 of 12737024 found) ``` Using coroutines - ```multireadrandom : 4.872 micros/op 3283812 ops/sec 60.000 seconds 197030096 operations; 1703.6 MB/s (12548032 of 12548032 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9968 Reviewed By: akankshamahajan15 Differential Revision: D36348563 Pulled By: anand1976 fbshipit-source-id: c0ce85a505fd26ebfbb09786cbd7f25202038696
3 years ago
} // namespace ROCKSDB_NAMESPACE
// Generate the regular and coroutine versions of some methods by
// including block_based_table_reader_sync_and_async.h twice
// Macros in the header will expand differently based on whether
// WITH_COROUTINES or WITHOUT_COROUTINES is defined
// clang-format off
#define WITHOUT_COROUTINES
#include "table/block_based/block_based_table_reader_sync_and_async.h"
#undef WITHOUT_COROUTINES
#define WITH_COROUTINES
#include "table/block_based/block_based_table_reader_sync_and_async.h"
#undef WITH_COROUTINES
Multi file concurrency in MultiGet using coroutines and async IO (#9968) Summary: This PR implements a coroutine version of batched MultiGet in order to concurrently read from multiple SST files in a level using async IO, thus reducing the latency of the MultiGet. The API from the user perspective is still synchronous and single threaded, with the RocksDB part of the processing happening in the context of the caller's thread. In Version::MultiGet, the decision is made whether to call synchronous or coroutine code. A good way to review this PR is to review the first 4 commits in order - de773b3, 70c2f70, 10b50e1, and 377a597 - before reviewing the rest. TODO: 1. Figure out how to build it in CircleCI (requires some dependencies to be installed) 2. Do some stress testing with coroutines enabled No regression in synchronous MultiGet between this branch and main - ``` ./db_bench -use_existing_db=true --db=/data/mysql/rocksdb/prefix_scan -benchmarks="readseq,multireadrandom" -key_size=32 -value_size=512 -num=5000000 -batch_size=64 -multiread_batched=true -use_direct_reads=false -duration=60 -ops_between_duration_checks=1 -readonly=true -adaptive_readahead=true -threads=16 -cache_size=10485760000 -async_io=false -multiread_stride=40000 -statistics ``` Branch - ```multireadrandom : 4.025 micros/op 3975111 ops/sec 60.001 seconds 238509056 operations; 2062.3 MB/s (14767808 of 14767808 found)``` Main - ```multireadrandom : 3.987 micros/op 4013216 ops/sec 60.001 seconds 240795392 operations; 2082.1 MB/s (15231040 of 15231040 found)``` More benchmarks in various scenarios are given below. The measurements were taken with ```async_io=false``` (no coroutines) and ```async_io=true``` (use coroutines). For an IO bound workload (with every key requiring an IO), the coroutines version shows a clear benefit, being ~2.6X faster. For CPU bound workloads, the coroutines version has ~6-15% higher CPU utilization, depending on how many keys overlap an SST file. 1. Single thread IO bound workload on remote storage with sparse MultiGet batch keys (~1 key overlap/file) - No coroutines - ```multireadrandom : 831.774 micros/op 1202 ops/sec 60.001 seconds 72136 operations; 0.6 MB/s (72136 of 72136 found)``` Using coroutines - ```multireadrandom : 318.742 micros/op 3137 ops/sec 60.003 seconds 188248 operations; 1.6 MB/s (188248 of 188248 found)``` 2. Single thread CPU bound workload (all data cached) with ~1 key overlap/file - No coroutines - ```multireadrandom : 4.127 micros/op 242322 ops/sec 60.000 seconds 14539384 operations; 125.7 MB/s (14539384 of 14539384 found)``` Using coroutines - ```multireadrandom : 4.741 micros/op 210935 ops/sec 60.000 seconds 12656176 operations; 109.4 MB/s (12656176 of 12656176 found)``` 3. Single thread CPU bound workload with ~2 key overlap/file - No coroutines - ```multireadrandom : 3.717 micros/op 269000 ops/sec 60.000 seconds 16140024 operations; 139.6 MB/s (16140024 of 16140024 found)``` Using coroutines - ```multireadrandom : 4.146 micros/op 241204 ops/sec 60.000 seconds 14472296 operations; 125.1 MB/s (14472296 of 14472296 found)``` 4. CPU bound multi-threaded (16 threads) with ~4 key overlap/file - No coroutines - ```multireadrandom : 4.534 micros/op 3528792 ops/sec 60.000 seconds 211728728 operations; 1830.7 MB/s (12737024 of 12737024 found) ``` Using coroutines - ```multireadrandom : 4.872 micros/op 3283812 ops/sec 60.000 seconds 197030096 operations; 1703.6 MB/s (12548032 of 12548032 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9968 Reviewed By: akankshamahajan15 Differential Revision: D36348563 Pulled By: anand1976 fbshipit-source-id: c0ce85a505fd26ebfbb09786cbd7f25202038696
3 years ago
// clang-format on
namespace ROCKSDB_NAMESPACE {
extern const uint64_t kBlockBasedTableMagicNumber;
extern const std::string kHashIndexPrefixesBlock;
extern const std::string kHashIndexPrefixesMetadataBlock;
BlockBasedTable::~BlockBasedTable() { delete rep_; }
namespace {
// Read the block identified by "handle" from "file".
// The only relevant option is options.verify_checksums for now.
// On failure return non-OK.
// On success fill *result and return OK - caller owns *result
// @param uncompression_dict Data for presetting the compression library's
// dictionary.
template <typename TBlocklike>
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
Status ReadAndParseBlockFromFile(
RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer,
const Footer& footer, const ReadOptions& options, const BlockHandle& handle,
std::unique_ptr<TBlocklike>* result, const ImmutableOptions& ioptions,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
BlockCreateContext& create_context, bool maybe_compressed,
const UncompressionDict& uncompression_dict,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
const PersistentCacheOptions& cache_options,
MemoryAllocator* memory_allocator, bool for_compaction, bool async_read) {
assert(result);
BlockContents contents;
BlockFetcher block_fetcher(
file, prefetch_buffer, footer, options, handle, &contents, ioptions,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
/*do_uncompress*/ maybe_compressed, maybe_compressed,
TBlocklike::kBlockType, uncompression_dict, cache_options,
memory_allocator, nullptr, for_compaction);
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
Status s;
// If prefetch_buffer is not allocated, it will fallback to synchronous
// reading of block contents.
if (async_read && prefetch_buffer != nullptr) {
s = block_fetcher.ReadAsyncBlockContents();
if (!s.ok()) {
return s;
}
} else {
s = block_fetcher.ReadBlockContents();
}
if (s.ok()) {
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
create_context.Create(result, std::move(contents));
}
return s;
}
Ignore `total_order_seek` in DB::Get (#9427) Summary: Apparently setting total_order_seek=true for DB::Get was intended to allow accurate read semantics if the current prefix extractor doesn't match what was used to generate SST files on disk. But since prefix_extractor was made a mutable option in 5.14.0, we have been able to detect this case and provide the correct semantics regardless of the total_order_seek option. Since that time, the option has only made Get() slower in a reasonably common case: prefix_extractor unchanged and whole_key_filtering=false. So this change primarily removes unnecessary effect of total_order_seek on Get. Also cleans up some related comments. Also adds a -total_order_seek option to db_bench and canonicalizes handling of ReadOptions in db_bench so that command line options have the expected association with library features. (There is potential for change in regression test behavior, but the old behavior is likely indefensible, or some other inconsistency would need to be fixed.) TODO in follow-up work: there should be no reason for Get() to depend on current prefix extractor at all. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9427 Test Plan: Unit tests updated. Performance (using db_bench update) Create DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12 -whole_key_filtering=0` Test with and without `-total_order_seek` on `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=readrandom -num=10000000 -duration=40 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Before this change, total_order_seek=false: 25188 ops/sec Before this change, total_order_seek=true: 1222 ops/sec (~20x slower) After this change, total_order_seek=false: 24570 ops/sec After this change, total_order_seek=true: 25012 ops/sec (indistinguishable) Reviewed By: siying Differential Revision: D33753458 Pulled By: pdillinger fbshipit-source-id: bf892f34907a5e407d9c40bd4d42f0adbcbe0014
3 years ago
// For hash based index, return false if table_properties->prefix_extractor_name
// and prefix_extractor both exist and match, otherwise true.
Fast path for detecting unchanged prefix_extractor (#9407) Summary: Fixes a major performance regression in 6.26, where extra CPU is spent in SliceTransform::AsString when reads involve a prefix_extractor (Get, MultiGet, Seek). Common case performance is now better than 6.25. This change creates a "fast path" for verifying that the current prefix extractor is unchanged and compatible with what was used to generate a table file. This fast path detects the common case by pointer comparison on the current prefix_extractor and a "known good" prefix extractor (if applicable) that is saved at the time the table reader is opened. The "known good" prefix extractor is saved as another shared_ptr copy (in an existing field, however) to ensure the pointer is not recycled. When the prefix_extractor has changed to a different instance but same compatible configuration (rare, odd), performance is still a regression compared to 6.25, but this is likely acceptable because of the oddity of such a case. The performance of incompatible prefix_extractor is essentially unchanged. Also fixed a minor case (ForwardIterator) where a prefix_extractor could be used via a raw pointer after being freed as a shared_ptr, if replaced via SetOptions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9407 Test Plan: ## Performance Populate DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Running head-to-head comparisons simultaneously with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=seekrandom -num=10000000 -duration=20 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Below each is compared by ops/sec vs. baseline which is version 6.25 (multiple baseline runs because of variable machine load) v6.26: 4833 vs. 6698 (<- major regression!) v6.27: 4737 vs. 6397 (still) New: 6704 vs. 6461 (better than baseline in common case) Disabled fastpath: 4843 vs. 6389 (e.g. if prefix extractor instance changes but is still compatible) Changed prefix size (no usable filter) in new: 787 vs. 5927 Changed prefix size (no usable filter) in new & baseline: 773 vs. 784 Reviewed By: mrambacher Differential Revision: D33677812 Pulled By: pdillinger fbshipit-source-id: 571d9711c461fb97f957378a061b7e7dbc4d6a76
3 years ago
inline bool PrefixExtractorChangedHelper(
const TableProperties* table_properties,
const SliceTransform* prefix_extractor) {
// BlockBasedTableOptions::kHashSearch requires prefix_extractor to be set.
// Turn off hash index in prefix_extractor is not set; if prefix_extractor
// is set but prefix_extractor_block is not set, also disable hash index
if (prefix_extractor == nullptr || table_properties == nullptr ||
table_properties->prefix_extractor_name.empty()) {
return true;
}
// prefix_extractor and prefix_extractor_block are both non-empty
if (table_properties->prefix_extractor_name != prefix_extractor->AsString()) {
return true;
} else {
return false;
}
}
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
template <typename TBlocklike>
uint32_t GetBlockNumRestarts(const TBlocklike& block) {
if constexpr (std::is_convertible_v<const TBlocklike&, const Block&>) {
const Block& b = block;
return b.NumRestarts();
} else {
return 0;
}
}
} // namespace
void BlockBasedTable::UpdateCacheHitMetrics(BlockType block_type,
GetContext* get_context,
size_t usage) const {
Statistics* const statistics = rep_->ioptions.stats;
PERF_COUNTER_ADD(block_cache_hit_count, 1);
PERF_COUNTER_BY_LEVEL_ADD(block_cache_hit_count, 1,
static_cast<uint32_t>(rep_->level));
if (get_context) {
++get_context->get_context_stats_.num_cache_hit;
get_context->get_context_stats_.num_cache_bytes_read += usage;
} else {
RecordTick(statistics, BLOCK_CACHE_HIT);
RecordTick(statistics, BLOCK_CACHE_BYTES_READ, usage);
}
switch (block_type) {
case BlockType::kFilter:
case BlockType::kFilterPartitionIndex:
PERF_COUNTER_ADD(block_cache_filter_hit_count, 1);
if (get_context) {
++get_context->get_context_stats_.num_cache_filter_hit;
} else {
RecordTick(statistics, BLOCK_CACHE_FILTER_HIT);
}
break;
case BlockType::kCompressionDictionary:
// TODO: introduce perf counter for compression dictionary hit count
if (get_context) {
++get_context->get_context_stats_.num_cache_compression_dict_hit;
} else {
RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_HIT);
}
break;
case BlockType::kIndex:
PERF_COUNTER_ADD(block_cache_index_hit_count, 1);
if (get_context) {
++get_context->get_context_stats_.num_cache_index_hit;
} else {
RecordTick(statistics, BLOCK_CACHE_INDEX_HIT);
}
break;
default:
// TODO: introduce dedicated tickers/statistics/counters
// for range tombstones
if (get_context) {
++get_context->get_context_stats_.num_cache_data_hit;
} else {
RecordTick(statistics, BLOCK_CACHE_DATA_HIT);
}
break;
}
}
void BlockBasedTable::UpdateCacheMissMetrics(BlockType block_type,
GetContext* get_context) const {
Statistics* const statistics = rep_->ioptions.stats;
// TODO: introduce aggregate (not per-level) block cache miss count
PERF_COUNTER_BY_LEVEL_ADD(block_cache_miss_count, 1,
static_cast<uint32_t>(rep_->level));
if (get_context) {
++get_context->get_context_stats_.num_cache_miss;
} else {
RecordTick(statistics, BLOCK_CACHE_MISS);
}
// TODO: introduce perf counters for misses per block type
switch (block_type) {
case BlockType::kFilter:
case BlockType::kFilterPartitionIndex:
if (get_context) {
++get_context->get_context_stats_.num_cache_filter_miss;
} else {
RecordTick(statistics, BLOCK_CACHE_FILTER_MISS);
}
break;
case BlockType::kCompressionDictionary:
if (get_context) {
++get_context->get_context_stats_.num_cache_compression_dict_miss;
} else {
RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_MISS);
}
break;
case BlockType::kIndex:
if (get_context) {
++get_context->get_context_stats_.num_cache_index_miss;
} else {
RecordTick(statistics, BLOCK_CACHE_INDEX_MISS);
}
break;
default:
// TODO: introduce dedicated tickers/statistics/counters
// for range tombstones
if (get_context) {
++get_context->get_context_stats_.num_cache_data_miss;
} else {
RecordTick(statistics, BLOCK_CACHE_DATA_MISS);
}
break;
}
}
void BlockBasedTable::UpdateCacheInsertionMetrics(
BlockType block_type, GetContext* get_context, size_t usage, bool redundant,
Statistics* const statistics) {
// TODO: introduce perf counters for block cache insertions
if (get_context) {
++get_context->get_context_stats_.num_cache_add;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
++get_context->get_context_stats_.num_cache_add_redundant;
}
get_context->get_context_stats_.num_cache_bytes_write += usage;
} else {
RecordTick(statistics, BLOCK_CACHE_ADD);
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
RecordTick(statistics, BLOCK_CACHE_ADD_REDUNDANT);
}
RecordTick(statistics, BLOCK_CACHE_BYTES_WRITE, usage);
}
switch (block_type) {
case BlockType::kFilter:
case BlockType::kFilterPartitionIndex:
if (get_context) {
++get_context->get_context_stats_.num_cache_filter_add;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
++get_context->get_context_stats_.num_cache_filter_add_redundant;
}
get_context->get_context_stats_.num_cache_filter_bytes_insert += usage;
} else {
RecordTick(statistics, BLOCK_CACHE_FILTER_ADD);
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
RecordTick(statistics, BLOCK_CACHE_FILTER_ADD_REDUNDANT);
}
RecordTick(statistics, BLOCK_CACHE_FILTER_BYTES_INSERT, usage);
}
break;
case BlockType::kCompressionDictionary:
if (get_context) {
++get_context->get_context_stats_.num_cache_compression_dict_add;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
++get_context->get_context_stats_
.num_cache_compression_dict_add_redundant;
}
get_context->get_context_stats_
.num_cache_compression_dict_bytes_insert += usage;
} else {
RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_ADD);
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_ADD_REDUNDANT);
}
RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_BYTES_INSERT,
usage);
}
break;
case BlockType::kIndex:
if (get_context) {
++get_context->get_context_stats_.num_cache_index_add;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
++get_context->get_context_stats_.num_cache_index_add_redundant;
}
get_context->get_context_stats_.num_cache_index_bytes_insert += usage;
} else {
RecordTick(statistics, BLOCK_CACHE_INDEX_ADD);
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
RecordTick(statistics, BLOCK_CACHE_INDEX_ADD_REDUNDANT);
}
RecordTick(statistics, BLOCK_CACHE_INDEX_BYTES_INSERT, usage);
}
break;
default:
// TODO: introduce dedicated tickers/statistics/counters
// for range tombstones
if (get_context) {
++get_context->get_context_stats_.num_cache_data_add;
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
++get_context->get_context_stats_.num_cache_data_add_redundant;
}
get_context->get_context_stats_.num_cache_data_bytes_insert += usage;
} else {
RecordTick(statistics, BLOCK_CACHE_DATA_ADD);
Stats for redundant insertions into block cache (#6681) Summary: Since read threads do not coordinate on loading data into block cache, two threads between Lookup and Insert can end up loading and inserting the same data. This is particularly concerning with cache_index_and_filter_blocks since those are hot and more likely to be race targets if ejected from (or not pre-populated in) the cache. Particularly with moves toward disaggregated / network storage, the cost of redundant retrieval might be high, and we should at least have some hard statistics from which we can estimate impact. Example with full filter thrashing "cliff": $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 ... $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((130 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 14181 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 476 rocksdb.block.cache.data.add COUNT : 12749 rocksdb.block.cache.data.add.redundant COUNT : 18 rocksdb.block.cache.filter.add COUNT : 1003 rocksdb.block.cache.filter.add.redundant COUNT : 217 rocksdb.block.cache.index.add COUNT : 429 rocksdb.block.cache.index.add.redundant COUNT : 241 $ ./db_bench --db=/tmp/rocksdbtest-172704/dbbench --use_existing_db --benchmarks=readrandom,stats --num=200000 --cache_index_and_filter_blocks --cache_size=$((120 * 1024 * 1024)) --bloom_bits=10 --threads=16 -statistics 2>&1 | egrep '^rocksdb.block.cache.(.*add|.*redundant)' | grep -v compress | sort rocksdb.block.cache.add COUNT : 1182223 rocksdb.block.cache.add.failures COUNT : 0 rocksdb.block.cache.add.redundant COUNT : 302728 rocksdb.block.cache.data.add COUNT : 31425 rocksdb.block.cache.data.add.redundant COUNT : 12 rocksdb.block.cache.filter.add COUNT : 795455 rocksdb.block.cache.filter.add.redundant COUNT : 130238 rocksdb.block.cache.index.add COUNT : 355343 rocksdb.block.cache.index.add.redundant COUNT : 172478 Pull Request resolved: https://github.com/facebook/rocksdb/pull/6681 Test Plan: Some manual testing (above) and unit test covering key metrics is included Reviewed By: ltamasi Differential Revision: D21134113 Pulled By: pdillinger fbshipit-source-id: c11497b5f00f4ffdfe919823904e52d0a1a91d87
5 years ago
if (redundant) {
RecordTick(statistics, BLOCK_CACHE_DATA_ADD_REDUNDANT);
}
RecordTick(statistics, BLOCK_CACHE_DATA_BYTES_INSERT, usage);
}
break;
}
}
namespace {
// Return True if table_properties has `user_prop_name` has a `true` value
// or it doesn't contain this property (for backward compatible).
bool IsFeatureSupported(const TableProperties& table_properties,
const std::string& user_prop_name, Logger* info_log) {
auto& props = table_properties.user_collected_properties;
auto pos = props.find(user_prop_name);
// Older version doesn't have this value set. Skip this check.
if (pos != props.end()) {
if (pos->second == kPropFalse) {
return false;
} else if (pos->second != kPropTrue) {
ROCKS_LOG_WARN(info_log, "Property %s has invalidate value %s",
user_prop_name.c_str(), pos->second.c_str());
}
}
return true;
}
// Caller has to ensure seqno is not nullptr.
Status GetGlobalSequenceNumber(const TableProperties& table_properties,
SequenceNumber largest_seqno,
SequenceNumber* seqno) {
const auto& props = table_properties.user_collected_properties;
const auto version_pos = props.find(ExternalSstFilePropertyNames::kVersion);
const auto seqno_pos = props.find(ExternalSstFilePropertyNames::kGlobalSeqno);
*seqno = kDisableGlobalSequenceNumber;
if (version_pos == props.end()) {
if (seqno_pos != props.end()) {
std::array<char, 200> msg_buf;
// This is not an external sst file, global_seqno is not supported.
snprintf(
msg_buf.data(), msg_buf.max_size(),
"A non-external sst file have global seqno property with value %s",
seqno_pos->second.c_str());
return Status::Corruption(msg_buf.data());
}
return Status::OK();
}
uint32_t version = DecodeFixed32(version_pos->second.c_str());
if (version < 2) {
if (seqno_pos != props.end() || version != 1) {
std::array<char, 200> msg_buf;
// This is a v1 external sst file, global_seqno is not supported.
snprintf(msg_buf.data(), msg_buf.max_size(),
"An external sst file with version %u have global seqno "
"property with value %s",
version, seqno_pos->second.c_str());
return Status::Corruption(msg_buf.data());
}
return Status::OK();
}
// Since we have a plan to deprecate global_seqno, we do not return failure
// if seqno_pos == props.end(). We rely on version_pos to detect whether the
// SST is external.
SequenceNumber global_seqno(0);
if (seqno_pos != props.end()) {
global_seqno = DecodeFixed64(seqno_pos->second.c_str());
}
// SstTableReader open table reader with kMaxSequenceNumber as largest_seqno
// to denote it is unknown.
if (largest_seqno < kMaxSequenceNumber) {
if (global_seqno == 0) {
global_seqno = largest_seqno;
}
if (global_seqno != largest_seqno) {
std::array<char, 200> msg_buf;
snprintf(
msg_buf.data(), msg_buf.max_size(),
"An external sst file with version %u have global seqno property "
"with value %s, while largest seqno in the file is %llu",
version, seqno_pos->second.c_str(),
static_cast<unsigned long long>(largest_seqno));
return Status::Corruption(msg_buf.data());
}
}
*seqno = global_seqno;
if (global_seqno > kMaxSequenceNumber) {
std::array<char, 200> msg_buf;
snprintf(msg_buf.data(), msg_buf.max_size(),
"An external sst file with version %u have global seqno property "
"with value %llu, which is greater than kMaxSequenceNumber",
version, static_cast<unsigned long long>(global_seqno));
return Status::Corruption(msg_buf.data());
}
return Status::OK();
}
} // namespace
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
void BlockBasedTable::SetupBaseCacheKey(const TableProperties* properties,
const std::string& cur_db_session_id,
uint64_t cur_file_number,
OffsetableCacheKey* out_base_cache_key,
bool* out_is_stable) {
// Use a stable cache key if sufficient data is in table properties
std::string db_session_id;
uint64_t file_num;
std::string db_id;
if (properties && !properties->db_session_id.empty() &&
properties->orig_file_number > 0) {
// (Newer SST file case)
// We must have both properties to get a stable unique id because
// CreateColumnFamilyWithImport or IngestExternalFiles can change the
// file numbers on a file.
db_session_id = properties->db_session_id;
file_num = properties->orig_file_number;
// Less critical, populated in earlier release than above
db_id = properties->db_id;
if (out_is_stable) {
*out_is_stable = true;
}
} else {
// (Old SST file case)
// We use (unique) cache keys based on current identifiers. These are at
// least stable across table file close and re-open, but not across
// different DBs nor DB close and re-open.
db_session_id = cur_db_session_id;
file_num = cur_file_number;
// Plumbing through the DB ID to here would be annoying, and of limited
// value because of the case of VersionSet::Recover opening some table
// files and later setting the DB ID. So we just rely on uniqueness
// level provided by session ID.
db_id = "unknown";
if (out_is_stable) {
*out_is_stable = false;
}
}
// Too many tests to update to get these working
// assert(file_num > 0);
// assert(!db_session_id.empty());
// assert(!db_id.empty());
// Minimum block size is 5 bytes; therefore we can trim off two lower bits
// from offsets. See GetCacheKey.
Derive cache keys from SST unique IDs (#10394) Summary: ... so that cache keys can be derived from DB manifest data before reading the file from storage--so that every part of the file can potentially go in a persistent cache. See updated comments in cache_key.cc for technical details. Importantly, the new cache key encoding uses some fancy but efficient math to pack data into the cache key without depending on the sizes of the various pieces. This simplifies some existing code creating cache keys, like cache warming before the file size is known. This should provide us an essentially permanent mapping between SST unique IDs and base cache keys, with the ability to "upgrade" SST unique IDs (and thus cache keys) with new SST format_versions. These cache keys are of similar, perhaps indistinguishable quality to the previous generation. Before this change (see "corrected" days between collision): ``` ./cache_bench -stress_cache_key -sck_keep_bits=43 18 collisions after 2 x 90 days, est 10 days between (1.15292e+19 corrected) ``` After this change (keep 43 bits, up through 50, to validate "trajectory" is ok on "corrected" days between collision): ``` 19 collisions after 3 x 90 days, est 14.2105 days between (1.63836e+19 corrected) 16 collisions after 5 x 90 days, est 28.125 days between (1.6213e+19 corrected) 15 collisions after 7 x 90 days, est 42 days between (1.21057e+19 corrected) 15 collisions after 17 x 90 days, est 102 days between (1.46997e+19 corrected) 15 collisions after 49 x 90 days, est 294 days between (2.11849e+19 corrected) 15 collisions after 62 x 90 days, est 372 days between (1.34027e+19 corrected) 15 collisions after 53 x 90 days, est 318 days between (5.72858e+18 corrected) 15 collisions after 309 x 90 days, est 1854 days between (1.66994e+19 corrected) ``` However, the change does modify (probably weaken) the "guaranteed unique" promise from this > SST files generated in a single process are guaranteed to have unique cache keys, unless/until number session ids * max file number = 2**86 to this (see https://github.com/facebook/rocksdb/issues/10388) > With the DB id limitation, we only have nice guaranteed unique cache keys for files generated in a single process until biggest session_id_counter and offset_in_file reach combined 64 bits I don't think this is a practical concern, though. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10394 Test Plan: unit tests updated, see simulation results above Reviewed By: jay-zhuang Differential Revision: D38667529 Pulled By: pdillinger fbshipit-source-id: 49af3fe7f47e5b61162809a78b76c769fd519fba
2 years ago
*out_base_cache_key = OffsetableCacheKey(db_id, db_session_id, file_num);
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
}
CacheKey BlockBasedTable::GetCacheKey(const OffsetableCacheKey& base_cache_key,
const BlockHandle& handle) {
// Minimum block size is 5 bytes; therefore we can trim off two lower bits
// from offet.
return base_cache_key.WithOffset(handle.offset() >> 2);
}
Status BlockBasedTable::Open(
const ReadOptions& read_options, const ImmutableOptions& ioptions,
const EnvOptions& env_options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator,
std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
Block per key-value checksum (#11287) Summary: add option `block_protection_bytes_per_key` and implementation for block per key-value checksum. The main changes are 1. checksum construction and verification in block.cc/h 2. pass the option `block_protection_bytes_per_key` around (mainly for methods defined in table_cache.h) 3. unit tests/crash test updates Tests: * Added unit tests * Crash test: `python3 tools/db_crashtest.py blackbox --simple --block_protection_bytes_per_key=1 --write_buffer_size=1048576` Follow up (maybe as a separate PR): make sure corruption status returned from BlockIters are correctly handled. Performance: Turning on block per KV protection has a non-trivial negative impact on read performance and costs additional memory. For memory, each block includes additional 24 bytes for checksum-related states beside checksum itself. For CPU, I set up a DB of size ~1.2GB with 5M keys (32 bytes key and 200 bytes value) which compacts to ~5 SST files (target file size 256 MB) in L6 without compression. I tested readrandom performance with various block cache size (to mimic various cache hit rates): ``` SETUP make OPTIMIZE_LEVEL="-O3" USE_LTO=1 DEBUG_LEVEL=0 -j32 db_bench ./db_bench -benchmarks=fillseq,compact0,waitforcompaction,compact,waitforcompaction -write_buffer_size=33554432 -level_compaction_dynamic_level_bytes=true -max_background_jobs=8 -target_file_size_base=268435456 --num=5000000 --key_size=32 --value_size=200 --compression_type=none BENCHMARK ./db_bench --use_existing_db -benchmarks=readtocache,readrandom[-X10] --num=5000000 --key_size=32 --disable_auto_compactions --reads=1000000 --block_protection_bytes_per_key=[0|1] --cache_size=$CACHESIZE The readrandom ops/sec looks like the following: Block cache size: 2GB 1.2GB * 0.9 1.2GB * 0.8 1.2GB * 0.5 8MB Main 240805 223604 198176 161653 139040 PR prot_bytes=0 238691 226693 200127 161082 141153 PR prot_bytes=1 214983 193199 178532 137013 108211 prot_bytes=1 vs -10% -15% -10.8% -15% -23% prot_bytes=0 ``` The benchmark has a lot of variance, but there was a 5% to 25% regression in this benchmark with different cache hit rates. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11287 Reviewed By: ajkr Differential Revision: D43970708 Pulled By: cbi42 fbshipit-source-id: ef98d898b71779846fa74212b9ec9e08b7183940
2 years ago
uint8_t block_protection_bytes_per_key,
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
std::unique_ptr<TableReader>* table_reader, uint64_t tail_size,
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
std::shared_ptr<CacheReservationManager> table_reader_cache_res_mgr,
Fast path for detecting unchanged prefix_extractor (#9407) Summary: Fixes a major performance regression in 6.26, where extra CPU is spent in SliceTransform::AsString when reads involve a prefix_extractor (Get, MultiGet, Seek). Common case performance is now better than 6.25. This change creates a "fast path" for verifying that the current prefix extractor is unchanged and compatible with what was used to generate a table file. This fast path detects the common case by pointer comparison on the current prefix_extractor and a "known good" prefix extractor (if applicable) that is saved at the time the table reader is opened. The "known good" prefix extractor is saved as another shared_ptr copy (in an existing field, however) to ensure the pointer is not recycled. When the prefix_extractor has changed to a different instance but same compatible configuration (rare, odd), performance is still a regression compared to 6.25, but this is likely acceptable because of the oddity of such a case. The performance of incompatible prefix_extractor is essentially unchanged. Also fixed a minor case (ForwardIterator) where a prefix_extractor could be used via a raw pointer after being freed as a shared_ptr, if replaced via SetOptions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9407 Test Plan: ## Performance Populate DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Running head-to-head comparisons simultaneously with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=seekrandom -num=10000000 -duration=20 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Below each is compared by ops/sec vs. baseline which is version 6.25 (multiple baseline runs because of variable machine load) v6.26: 4833 vs. 6698 (<- major regression!) v6.27: 4737 vs. 6397 (still) New: 6704 vs. 6461 (better than baseline in common case) Disabled fastpath: 4843 vs. 6389 (e.g. if prefix extractor instance changes but is still compatible) Changed prefix size (no usable filter) in new: 787 vs. 5927 Changed prefix size (no usable filter) in new & baseline: 773 vs. 784 Reviewed By: mrambacher Differential Revision: D33677812 Pulled By: pdillinger fbshipit-source-id: 571d9711c461fb97f957378a061b7e7dbc4d6a76
3 years ago
const std::shared_ptr<const SliceTransform>& prefix_extractor,
const bool prefetch_index_and_filter_in_cache, const bool skip_filters,
const int level, const bool immortal_table,
const SequenceNumber largest_seqno, const bool force_direct_prefetch,
TailPrefetchStats* tail_prefetch_stats,
BlockCacheTracer* const block_cache_tracer,
size_t max_file_size_for_l0_meta_pin, const std::string& cur_db_session_id,
Add support to strip / pad timestamp when creating / reading a block based table (#11495) Summary: Add support to strip timestamp in block based table builder and pad timestamp in block based table reader. On the write path, use the per column family option `AdvancedColumnFamilyOptions.persist_user_defined_timestamps` to indicate whether user-defined timestamps should be stripped for all block based tables created for the column family. On the read path, added a per table `TableReadOption.user_defined_timestamps_persisted` to flag whether the user keys in the table contains user defined timestamps. This patch is mostly passing the related flags down to the block building/parsing level with the exception of handling the `first_internal_key` in `IndexValue`, which is included in the `IndexBuilder` level. The value part of range deletion entries should have a similar handling, I haven't decided where to best fit this piece of logic, I will do it in a follow up. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11495 Test Plan: Existing test `BlockBasedTableReaderTest` is parameterized to run with: 1) different UDT test modes: kNone, kNormal, kStripUserDefinedTimestamp 2) all four index types, when index type is `kTwoLevelIndexSearch`, also enables partitioned filters 3) parallel vs non-parallel compression 4) enable/disable compression dictionary. Also added tests for API `BlockBasedTableReader::NewIterator`. `PartitionedFilterBlockTest` is parameterized to run with different UDT test modes:kNone, kNormal, kStripUserDefinedTimestamp. ``` make all check ./block_based_table_reader_test ./partitioned_filter_block_test ``` Reviewed By: ltamasi Differential Revision: D46344577 Pulled By: jowlyzhang fbshipit-source-id: 93ac8542b19319d1298712b8bed908c8831ba675
1 year ago
uint64_t cur_file_num, UniqueId64x2 expected_unique_id,
const bool user_defined_timestamps_persisted) {
table_reader->reset();
Status s;
Footer footer;
std::unique_ptr<FilePrefetchBuffer> prefetch_buffer;
use verify checksum option in block based table reader Open() (#11099) Summary: ## Description In this issue https://github.com/facebook/rocksdb/issues/11002 we found that when we use rocksdb with the `verify checksum` read_option to false the verification is done anyway By analyzing the code along the stacktrace I saw that at the level of https://github.com/facebook/rocksdb/compare/main...matkt:feature/use-verify-checksum-in-block-based-table-reader?expand=1#diff-57ed8c49db2bdd4db7618646a177397674bbf25beacacecb104070071d30129f we are not keeping all the options and we forget the `verify_checksum` the comment in this class suggests that it should be managed https://github.com/facebook/rocksdb/compare/main...matkt:feature/use-verify-checksum-in-block-based-table-reader?expand=1#diff-57ed8c49db2bdd4db7618646a177397674bbf25beacacecb104070071d30129fL581 <img width="1724" alt="204511641-86ab4b9b-45e5-4a2b-a13d-81fa26435d38" src="https://user-images.githubusercontent.com/26581503/213152802-c46bc1c7-a3a2-4a6f-9bb1-bf92ee93af7a.png"> this PR just adds the line to manage the `verify checksum` ## Tests - Running unit tests - Test without setting `verify checksum` and verifying that we are calling the checksum code - Test by setting `verify checksum` to true and verifying that we are calling the checksum code - Test by setting `verify checksum` to false and verifying that we are **not** calling the checksum code Pull Request resolved: https://github.com/facebook/rocksdb/pull/11099 Reviewed By: cbi42 Differential Revision: D42679881 Pulled By: ajkr fbshipit-source-id: c7dd10768282fd0699f7e1bf397ceb7adbea4ab6
2 years ago
// From read_options, retain deadline, io_timeout, rate_limiter_priority, and
// verify_checksums. In future, we may retain more options.
ReadOptions ro;
ro.deadline = read_options.deadline;
ro.io_timeout = read_options.io_timeout;
Set Read rate limiter priority dynamically and pass it to FS (#9996) Summary: ### Context: Background compactions and flush generate large reads and writes, and can be long running, especially for universal compaction. In some cases, this can impact foreground reads and writes by users. ### Solution User, Flush, and Compaction reads share some code path. For this task, we update the rate_limiter_priority in ReadOptions for code paths (e.g. FindTable (mainly in BlockBasedTable::Open()) and various iterators), and eventually update the rate_limiter_priority in IOOptions for FSRandomAccessFile. **This PR is for the Read path.** The **Read:** dynamic priority for different state are listed as follows: | State | Normal | Delayed | Stalled | | ----- | ------ | ------- | ------- | | Flush (verification read in BuildTable()) | IO_USER | IO_USER | IO_USER | | Compaction | IO_LOW | IO_USER | IO_USER | | User | User provided | User provided | User provided | We will respect the read_options that the user provided and will not set it. The only sst read for Flush is the verification read in BuildTable(). It claims to be "regard as user read". **Details** 1. Set read_options.rate_limiter_priority dynamically: - User: Do not update the read_options. Use the read_options that the user provided. - Compaction: Update read_options in CompactionJob::ProcessKeyValueCompaction(). - Flush: Update read_options in BuildTable(). 2. Pass the rate limiter priority to FSRandomAccessFile functions: - After calling the FindTable(), read_options is passed through GetTableReader(table_cache.cc), BlockBasedTableFactory::NewTableReader(block_based_table_factory.cc), and BlockBasedTable::Open(). The Open() needs some updates for the ReadOptions variable and the updates are also needed for the called functions, including PrefetchTail(), PrepareIOOptions(), ReadFooterFromFile(), ReadMetaIndexblock(), ReadPropertiesBlock(), PrefetchIndexAndFilterBlocks(), and ReadRangeDelBlock(). - In RandomAccessFileReader, the functions to be updated include Read(), MultiRead(), ReadAsync(), and Prefetch(). - Update the downstream functions of NewIndexIterator(), NewDataBlockIterator(), and BlockBasedTableIterator(). ### Test Plans Add unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9996 Reviewed By: anand1976 Differential Revision: D36452483 Pulled By: gitbw95 fbshipit-source-id: 60978204a4f849bb9261cb78d9bc1cb56d6008cf
3 years ago
ro.rate_limiter_priority = read_options.rate_limiter_priority;
use verify checksum option in block based table reader Open() (#11099) Summary: ## Description In this issue https://github.com/facebook/rocksdb/issues/11002 we found that when we use rocksdb with the `verify checksum` read_option to false the verification is done anyway By analyzing the code along the stacktrace I saw that at the level of https://github.com/facebook/rocksdb/compare/main...matkt:feature/use-verify-checksum-in-block-based-table-reader?expand=1#diff-57ed8c49db2bdd4db7618646a177397674bbf25beacacecb104070071d30129f we are not keeping all the options and we forget the `verify_checksum` the comment in this class suggests that it should be managed https://github.com/facebook/rocksdb/compare/main...matkt:feature/use-verify-checksum-in-block-based-table-reader?expand=1#diff-57ed8c49db2bdd4db7618646a177397674bbf25beacacecb104070071d30129fL581 <img width="1724" alt="204511641-86ab4b9b-45e5-4a2b-a13d-81fa26435d38" src="https://user-images.githubusercontent.com/26581503/213152802-c46bc1c7-a3a2-4a6f-9bb1-bf92ee93af7a.png"> this PR just adds the line to manage the `verify checksum` ## Tests - Running unit tests - Test without setting `verify checksum` and verifying that we are calling the checksum code - Test by setting `verify checksum` to true and verifying that we are calling the checksum code - Test by setting `verify checksum` to false and verifying that we are **not** calling the checksum code Pull Request resolved: https://github.com/facebook/rocksdb/pull/11099 Reviewed By: cbi42 Differential Revision: D42679881 Pulled By: ajkr fbshipit-source-id: c7dd10768282fd0699f7e1bf397ceb7adbea4ab6
2 years ago
ro.verify_checksums = read_options.verify_checksums;
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
ro.io_activity = read_options.io_activity;
// prefetch both index and filters, down to all partitions
const bool prefetch_all = prefetch_index_and_filter_in_cache || level == 0;
const bool preload_all = !table_options.cache_index_and_filter_blocks;
if (!ioptions.allow_mmap_reads) {
s = PrefetchTail(ro, file.get(), file_size, force_direct_prefetch,
tail_prefetch_stats, prefetch_all, preload_all,
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
&prefetch_buffer, ioptions.stats, tail_size,
ioptions.logger);
// Return error in prefetch path to users.
if (!s.ok()) {
return s;
}
} else {
// Should not prefetch for mmap mode.
prefetch_buffer.reset(new FilePrefetchBuffer(
0 /* readahead_size */, 0 /* max_readahead_size */, false /* enable */,
true /* track_min_offset */));
}
// Read in the following order:
// 1. Footer
// 2. [metaindex block]
// 3. [meta block: properties]
// 4. [meta block: range deletion tombstone]
// 5. [meta block: compression dictionary]
// 6. [meta block: index]
// 7. [meta block: filter]
IOOptions opts;
s = file->PrepareIOOptions(ro, opts);
if (s.ok()) {
s = ReadFooterFromFile(opts, file.get(), *ioptions.fs,
prefetch_buffer.get(), file_size, &footer,
kBlockBasedTableMagicNumber);
}
if (!s.ok()) {
return s;
}
if (!IsSupportedFormatVersion(footer.format_version())) {
return Status::Corruption(
"Unknown Footer version. Maybe this file was created with newer "
"version of RocksDB?");
}
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
Add support to strip / pad timestamp when creating / reading a block based table (#11495) Summary: Add support to strip timestamp in block based table builder and pad timestamp in block based table reader. On the write path, use the per column family option `AdvancedColumnFamilyOptions.persist_user_defined_timestamps` to indicate whether user-defined timestamps should be stripped for all block based tables created for the column family. On the read path, added a per table `TableReadOption.user_defined_timestamps_persisted` to flag whether the user keys in the table contains user defined timestamps. This patch is mostly passing the related flags down to the block building/parsing level with the exception of handling the `first_internal_key` in `IndexValue`, which is included in the `IndexBuilder` level. The value part of range deletion entries should have a similar handling, I haven't decided where to best fit this piece of logic, I will do it in a follow up. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11495 Test Plan: Existing test `BlockBasedTableReaderTest` is parameterized to run with: 1) different UDT test modes: kNone, kNormal, kStripUserDefinedTimestamp 2) all four index types, when index type is `kTwoLevelIndexSearch`, also enables partitioned filters 3) parallel vs non-parallel compression 4) enable/disable compression dictionary. Also added tests for API `BlockBasedTableReader::NewIterator`. `PartitionedFilterBlockTest` is parameterized to run with different UDT test modes:kNone, kNormal, kStripUserDefinedTimestamp. ``` make all check ./block_based_table_reader_test ./partitioned_filter_block_test ``` Reviewed By: ltamasi Differential Revision: D46344577 Pulled By: jowlyzhang fbshipit-source-id: 93ac8542b19319d1298712b8bed908c8831ba675
1 year ago
Rep* rep = new BlockBasedTable::Rep(
ioptions, env_options, table_options, internal_comparator, skip_filters,
file_size, level, immortal_table, user_defined_timestamps_persisted);
rep->file = std::move(file);
rep->footer = footer;
// For fully portable/stable cache keys, we need to read the properties
// block before setting up cache keys. TODO: consider setting up a bootstrap
// cache key for PersistentCache to use for metaindex and properties blocks.
rep->persistent_cache_options = PersistentCacheOptions();
// Meta-blocks are not dictionary compressed. Explicitly set the dictionary
// handle to null, otherwise it may be seen as uninitialized during the below
// meta-block reads.
rep->compression_dict_handle = BlockHandle::NullBlockHandle();
Block per key-value checksum (#11287) Summary: add option `block_protection_bytes_per_key` and implementation for block per key-value checksum. The main changes are 1. checksum construction and verification in block.cc/h 2. pass the option `block_protection_bytes_per_key` around (mainly for methods defined in table_cache.h) 3. unit tests/crash test updates Tests: * Added unit tests * Crash test: `python3 tools/db_crashtest.py blackbox --simple --block_protection_bytes_per_key=1 --write_buffer_size=1048576` Follow up (maybe as a separate PR): make sure corruption status returned from BlockIters are correctly handled. Performance: Turning on block per KV protection has a non-trivial negative impact on read performance and costs additional memory. For memory, each block includes additional 24 bytes for checksum-related states beside checksum itself. For CPU, I set up a DB of size ~1.2GB with 5M keys (32 bytes key and 200 bytes value) which compacts to ~5 SST files (target file size 256 MB) in L6 without compression. I tested readrandom performance with various block cache size (to mimic various cache hit rates): ``` SETUP make OPTIMIZE_LEVEL="-O3" USE_LTO=1 DEBUG_LEVEL=0 -j32 db_bench ./db_bench -benchmarks=fillseq,compact0,waitforcompaction,compact,waitforcompaction -write_buffer_size=33554432 -level_compaction_dynamic_level_bytes=true -max_background_jobs=8 -target_file_size_base=268435456 --num=5000000 --key_size=32 --value_size=200 --compression_type=none BENCHMARK ./db_bench --use_existing_db -benchmarks=readtocache,readrandom[-X10] --num=5000000 --key_size=32 --disable_auto_compactions --reads=1000000 --block_protection_bytes_per_key=[0|1] --cache_size=$CACHESIZE The readrandom ops/sec looks like the following: Block cache size: 2GB 1.2GB * 0.9 1.2GB * 0.8 1.2GB * 0.5 8MB Main 240805 223604 198176 161653 139040 PR prot_bytes=0 238691 226693 200127 161082 141153 PR prot_bytes=1 214983 193199 178532 137013 108211 prot_bytes=1 vs -10% -15% -10.8% -15% -23% prot_bytes=0 ``` The benchmark has a lot of variance, but there was a 5% to 25% regression in this benchmark with different cache hit rates. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11287 Reviewed By: ajkr Differential Revision: D43970708 Pulled By: cbi42 fbshipit-source-id: ef98d898b71779846fa74212b9ec9e08b7183940
2 years ago
rep->create_context.protection_bytes_per_key = block_protection_bytes_per_key;
// Read metaindex
std::unique_ptr<BlockBasedTable> new_table(
new BlockBasedTable(rep, block_cache_tracer));
std::unique_ptr<Block> metaindex;
std::unique_ptr<InternalIterator> metaindex_iter;
s = new_table->ReadMetaIndexBlock(ro, prefetch_buffer.get(), &metaindex,
&metaindex_iter);
if (!s.ok()) {
return s;
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
// Populates table_properties and some fields that depend on it,
// such as index_type.
s = new_table->ReadPropertiesBlock(ro, prefetch_buffer.get(),
metaindex_iter.get(), largest_seqno);
if (!s.ok()) {
return s;
}
Always verify SST unique IDs on SST file open (#10532) Summary: Although we've been tracking SST unique IDs in the DB manifest unconditionally, checking has been opt-in and with an extra pass at DB::Open time. This changes the behavior of `verify_sst_unique_id_in_manifest` to check unique ID against manifest every time an SST file is opened through table cache (normal DB operations), replacing the explicit pass over files at DB::Open time. This change also enables the option by default and removes the "EXPERIMENTAL" designation. One possible criticism is that the option no longer ensures the integrity of a DB at Open time. This is far from an all-or-nothing issue. Verifying the IDs of all SST files hardly ensures all the data in the DB is readable. (VerifyChecksum is supposed to do that.) Also, with max_open_files=-1 (default, extremely common), all SST files are opened at DB::Open time anyway. Implementation details: * `VerifySstUniqueIdInManifest()` functions are the extra/explicit pass that is now removed. * Unit tests that manipulate/corrupt table properties have to opt out of this check, because that corrupts the "actual" unique id. (And even for testing we don't currently have a mechanism to set "no unique id" in the in-memory file metadata for new files.) * A lot of other unit test churn relates to (a) default checking on, and (b) checking on SST open even without DB::Open (e.g. on flush) * Use `FileMetaData` for more `TableCache` operations (in place of `FileDescriptor`) so that we have access to the unique_id whenever we might need to open an SST file. **There is the possibility of performance impact because we can no longer use the more localized `fd` part of an `FdWithKeyRange` but instead follow the `file_metadata` pointer. However, this change (possible regression) is only done for `GetMemoryUsageByTableReaders`.** * Removed a completely unnecessary constructor overload of `TableReaderOptions` Possible follow-up: * Verification only happens when opening through table cache. Are there more places where this should happen? * Improve error message when there is a file size mismatch vs. manifest (FIXME added in the appropriate place). * I'm not sure there's a justification for `FileDescriptor` to be distinct from `FileMetaData`. * I'm skeptical that `FdWithKeyRange` really still makes sense for optimizing some data locality by duplicating some data in memory, but I could be wrong. * An unnecessary overload of NewTableReader was recently added, in the public API nonetheless (though unusable there). It should be cleaned up to put most things under `TableReaderOptions`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10532 Test Plan: updated unit tests Performance test showing no significant difference (just noise I think): `./db_bench -benchmarks=readwhilewriting[-X10] -num=3000000 -disable_wal=1 -bloom_bits=8 -write_buffer_size=1000000 -target_file_size_base=1000000` Before: readwhilewriting [AVG 10 runs] : 68702 (± 6932) ops/sec After: readwhilewriting [AVG 10 runs] : 68239 (± 7198) ops/sec Reviewed By: jay-zhuang Differential Revision: D38765551 Pulled By: pdillinger fbshipit-source-id: a827a708155f12344ab2a5c16e7701c7636da4c2
2 years ago
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
// Populate BlockCreateContext
bool blocks_definitely_zstd_compressed =
rep->table_properties &&
(rep->table_properties->compression_name ==
CompressionTypeToString(kZSTD) ||
rep->table_properties->compression_name ==
CompressionTypeToString(kZSTDNotFinalCompression));
Block per key-value checksum (#11287) Summary: add option `block_protection_bytes_per_key` and implementation for block per key-value checksum. The main changes are 1. checksum construction and verification in block.cc/h 2. pass the option `block_protection_bytes_per_key` around (mainly for methods defined in table_cache.h) 3. unit tests/crash test updates Tests: * Added unit tests * Crash test: `python3 tools/db_crashtest.py blackbox --simple --block_protection_bytes_per_key=1 --write_buffer_size=1048576` Follow up (maybe as a separate PR): make sure corruption status returned from BlockIters are correctly handled. Performance: Turning on block per KV protection has a non-trivial negative impact on read performance and costs additional memory. For memory, each block includes additional 24 bytes for checksum-related states beside checksum itself. For CPU, I set up a DB of size ~1.2GB with 5M keys (32 bytes key and 200 bytes value) which compacts to ~5 SST files (target file size 256 MB) in L6 without compression. I tested readrandom performance with various block cache size (to mimic various cache hit rates): ``` SETUP make OPTIMIZE_LEVEL="-O3" USE_LTO=1 DEBUG_LEVEL=0 -j32 db_bench ./db_bench -benchmarks=fillseq,compact0,waitforcompaction,compact,waitforcompaction -write_buffer_size=33554432 -level_compaction_dynamic_level_bytes=true -max_background_jobs=8 -target_file_size_base=268435456 --num=5000000 --key_size=32 --value_size=200 --compression_type=none BENCHMARK ./db_bench --use_existing_db -benchmarks=readtocache,readrandom[-X10] --num=5000000 --key_size=32 --disable_auto_compactions --reads=1000000 --block_protection_bytes_per_key=[0|1] --cache_size=$CACHESIZE The readrandom ops/sec looks like the following: Block cache size: 2GB 1.2GB * 0.9 1.2GB * 0.8 1.2GB * 0.5 8MB Main 240805 223604 198176 161653 139040 PR prot_bytes=0 238691 226693 200127 161082 141153 PR prot_bytes=1 214983 193199 178532 137013 108211 prot_bytes=1 vs -10% -15% -10.8% -15% -23% prot_bytes=0 ``` The benchmark has a lot of variance, but there was a 5% to 25% regression in this benchmark with different cache hit rates. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11287 Reviewed By: ajkr Differential Revision: D43970708 Pulled By: cbi42 fbshipit-source-id: ef98d898b71779846fa74212b9ec9e08b7183940
2 years ago
rep->create_context = BlockCreateContext(
&rep->table_options, rep->ioptions.stats,
blocks_definitely_zstd_compressed, block_protection_bytes_per_key,
rep->internal_comparator.user_comparator(), rep->index_value_is_full,
rep->index_has_first_key);
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
Always verify SST unique IDs on SST file open (#10532) Summary: Although we've been tracking SST unique IDs in the DB manifest unconditionally, checking has been opt-in and with an extra pass at DB::Open time. This changes the behavior of `verify_sst_unique_id_in_manifest` to check unique ID against manifest every time an SST file is opened through table cache (normal DB operations), replacing the explicit pass over files at DB::Open time. This change also enables the option by default and removes the "EXPERIMENTAL" designation. One possible criticism is that the option no longer ensures the integrity of a DB at Open time. This is far from an all-or-nothing issue. Verifying the IDs of all SST files hardly ensures all the data in the DB is readable. (VerifyChecksum is supposed to do that.) Also, with max_open_files=-1 (default, extremely common), all SST files are opened at DB::Open time anyway. Implementation details: * `VerifySstUniqueIdInManifest()` functions are the extra/explicit pass that is now removed. * Unit tests that manipulate/corrupt table properties have to opt out of this check, because that corrupts the "actual" unique id. (And even for testing we don't currently have a mechanism to set "no unique id" in the in-memory file metadata for new files.) * A lot of other unit test churn relates to (a) default checking on, and (b) checking on SST open even without DB::Open (e.g. on flush) * Use `FileMetaData` for more `TableCache` operations (in place of `FileDescriptor`) so that we have access to the unique_id whenever we might need to open an SST file. **There is the possibility of performance impact because we can no longer use the more localized `fd` part of an `FdWithKeyRange` but instead follow the `file_metadata` pointer. However, this change (possible regression) is only done for `GetMemoryUsageByTableReaders`.** * Removed a completely unnecessary constructor overload of `TableReaderOptions` Possible follow-up: * Verification only happens when opening through table cache. Are there more places where this should happen? * Improve error message when there is a file size mismatch vs. manifest (FIXME added in the appropriate place). * I'm not sure there's a justification for `FileDescriptor` to be distinct from `FileMetaData`. * I'm skeptical that `FdWithKeyRange` really still makes sense for optimizing some data locality by duplicating some data in memory, but I could be wrong. * An unnecessary overload of NewTableReader was recently added, in the public API nonetheless (though unusable there). It should be cleaned up to put most things under `TableReaderOptions`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10532 Test Plan: updated unit tests Performance test showing no significant difference (just noise I think): `./db_bench -benchmarks=readwhilewriting[-X10] -num=3000000 -disable_wal=1 -bloom_bits=8 -write_buffer_size=1000000 -target_file_size_base=1000000` Before: readwhilewriting [AVG 10 runs] : 68702 (± 6932) ops/sec After: readwhilewriting [AVG 10 runs] : 68239 (± 7198) ops/sec Reviewed By: jay-zhuang Differential Revision: D38765551 Pulled By: pdillinger fbshipit-source-id: a827a708155f12344ab2a5c16e7701c7636da4c2
2 years ago
// Check expected unique id if provided
if (expected_unique_id != kNullUniqueId64x2) {
auto props = rep->table_properties;
if (!props) {
return Status::Corruption("Missing table properties on file " +
std::to_string(cur_file_num) +
" with known unique ID");
}
UniqueId64x2 actual_unique_id{};
s = GetSstInternalUniqueId(props->db_id, props->db_session_id,
props->orig_file_number, &actual_unique_id,
/*force*/ true);
assert(s.ok()); // because force=true
if (expected_unique_id != actual_unique_id) {
return Status::Corruption(
"Mismatch in unique ID on table file " +
std::to_string(cur_file_num) +
". Expected: " + InternalUniqueIdToHumanString(&expected_unique_id) +
" Actual: " + InternalUniqueIdToHumanString(&actual_unique_id));
}
TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::PassedVerifyUniqueId",
&actual_unique_id);
} else {
TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::SkippedVerifyUniqueId",
nullptr);
if (ioptions.verify_sst_unique_id_in_manifest && ioptions.logger) {
// A crude but isolated way of reporting unverified files. This should not
// be an ongoing concern so doesn't deserve a place in Statistics IMHO.
static std::atomic<uint64_t> unverified_count{0};
auto prev_count =
unverified_count.fetch_add(1, std::memory_order_relaxed);
if (prev_count == 0) {
ROCKS_LOG_WARN(
ioptions.logger,
"At least one SST file opened without unique ID to verify: %" PRIu64
".sst",
cur_file_num);
} else if (prev_count % 1000 == 0) {
ROCKS_LOG_WARN(
ioptions.logger,
"Another ~1000 SST files opened without unique ID to verify");
}
}
}
// Set up prefix extracto as needed
bool force_null_table_prefix_extractor = false;
TEST_SYNC_POINT_CALLBACK(
"BlockBasedTable::Open::ForceNullTablePrefixExtractor",
&force_null_table_prefix_extractor);
if (force_null_table_prefix_extractor) {
assert(!rep->table_prefix_extractor);
} else if (!PrefixExtractorChangedHelper(rep->table_properties.get(),
prefix_extractor.get())) {
Fast path for detecting unchanged prefix_extractor (#9407) Summary: Fixes a major performance regression in 6.26, where extra CPU is spent in SliceTransform::AsString when reads involve a prefix_extractor (Get, MultiGet, Seek). Common case performance is now better than 6.25. This change creates a "fast path" for verifying that the current prefix extractor is unchanged and compatible with what was used to generate a table file. This fast path detects the common case by pointer comparison on the current prefix_extractor and a "known good" prefix extractor (if applicable) that is saved at the time the table reader is opened. The "known good" prefix extractor is saved as another shared_ptr copy (in an existing field, however) to ensure the pointer is not recycled. When the prefix_extractor has changed to a different instance but same compatible configuration (rare, odd), performance is still a regression compared to 6.25, but this is likely acceptable because of the oddity of such a case. The performance of incompatible prefix_extractor is essentially unchanged. Also fixed a minor case (ForwardIterator) where a prefix_extractor could be used via a raw pointer after being freed as a shared_ptr, if replaced via SetOptions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9407 Test Plan: ## Performance Populate DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Running head-to-head comparisons simultaneously with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=seekrandom -num=10000000 -duration=20 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Below each is compared by ops/sec vs. baseline which is version 6.25 (multiple baseline runs because of variable machine load) v6.26: 4833 vs. 6698 (<- major regression!) v6.27: 4737 vs. 6397 (still) New: 6704 vs. 6461 (better than baseline in common case) Disabled fastpath: 4843 vs. 6389 (e.g. if prefix extractor instance changes but is still compatible) Changed prefix size (no usable filter) in new: 787 vs. 5927 Changed prefix size (no usable filter) in new & baseline: 773 vs. 784 Reviewed By: mrambacher Differential Revision: D33677812 Pulled By: pdillinger fbshipit-source-id: 571d9711c461fb97f957378a061b7e7dbc4d6a76
3 years ago
// Establish fast path for unchanged prefix_extractor
rep->table_prefix_extractor = prefix_extractor;
} else {
// Current prefix_extractor doesn't match table
if (rep->table_properties) {
//**TODO: If/When the DBOptions has a registry in it, the ConfigOptions
// will need to use it
ConfigOptions config_options;
Status st = SliceTransform::CreateFromString(
config_options, rep->table_properties->prefix_extractor_name,
&(rep->table_prefix_extractor));
if (!st.ok()) {
//**TODO: Should this be error be returned or swallowed?
ROCKS_LOG_ERROR(rep->ioptions.logger,
"Failed to create prefix extractor[%s]: %s",
rep->table_properties->prefix_extractor_name.c_str(),
st.ToString().c_str());
}
}
}
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
// With properties loaded, we can set up portable/stable cache keys
SetupBaseCacheKey(rep->table_properties.get(), cur_db_session_id,
Derive cache keys from SST unique IDs (#10394) Summary: ... so that cache keys can be derived from DB manifest data before reading the file from storage--so that every part of the file can potentially go in a persistent cache. See updated comments in cache_key.cc for technical details. Importantly, the new cache key encoding uses some fancy but efficient math to pack data into the cache key without depending on the sizes of the various pieces. This simplifies some existing code creating cache keys, like cache warming before the file size is known. This should provide us an essentially permanent mapping between SST unique IDs and base cache keys, with the ability to "upgrade" SST unique IDs (and thus cache keys) with new SST format_versions. These cache keys are of similar, perhaps indistinguishable quality to the previous generation. Before this change (see "corrected" days between collision): ``` ./cache_bench -stress_cache_key -sck_keep_bits=43 18 collisions after 2 x 90 days, est 10 days between (1.15292e+19 corrected) ``` After this change (keep 43 bits, up through 50, to validate "trajectory" is ok on "corrected" days between collision): ``` 19 collisions after 3 x 90 days, est 14.2105 days between (1.63836e+19 corrected) 16 collisions after 5 x 90 days, est 28.125 days between (1.6213e+19 corrected) 15 collisions after 7 x 90 days, est 42 days between (1.21057e+19 corrected) 15 collisions after 17 x 90 days, est 102 days between (1.46997e+19 corrected) 15 collisions after 49 x 90 days, est 294 days between (2.11849e+19 corrected) 15 collisions after 62 x 90 days, est 372 days between (1.34027e+19 corrected) 15 collisions after 53 x 90 days, est 318 days between (5.72858e+18 corrected) 15 collisions after 309 x 90 days, est 1854 days between (1.66994e+19 corrected) ``` However, the change does modify (probably weaken) the "guaranteed unique" promise from this > SST files generated in a single process are guaranteed to have unique cache keys, unless/until number session ids * max file number = 2**86 to this (see https://github.com/facebook/rocksdb/issues/10388) > With the DB id limitation, we only have nice guaranteed unique cache keys for files generated in a single process until biggest session_id_counter and offset_in_file reach combined 64 bits I don't think this is a practical concern, though. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10394 Test Plan: unit tests updated, see simulation results above Reviewed By: jay-zhuang Differential Revision: D38667529 Pulled By: pdillinger fbshipit-source-id: 49af3fe7f47e5b61162809a78b76c769fd519fba
2 years ago
cur_file_num, &rep->base_cache_key);
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
rep->persistent_cache_options =
PersistentCacheOptions(rep->table_options.persistent_cache,
rep->base_cache_key, rep->ioptions.stats);
Add support to strip / pad timestamp when creating / reading a block based table (#11495) Summary: Add support to strip timestamp in block based table builder and pad timestamp in block based table reader. On the write path, use the per column family option `AdvancedColumnFamilyOptions.persist_user_defined_timestamps` to indicate whether user-defined timestamps should be stripped for all block based tables created for the column family. On the read path, added a per table `TableReadOption.user_defined_timestamps_persisted` to flag whether the user keys in the table contains user defined timestamps. This patch is mostly passing the related flags down to the block building/parsing level with the exception of handling the `first_internal_key` in `IndexValue`, which is included in the `IndexBuilder` level. The value part of range deletion entries should have a similar handling, I haven't decided where to best fit this piece of logic, I will do it in a follow up. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11495 Test Plan: Existing test `BlockBasedTableReaderTest` is parameterized to run with: 1) different UDT test modes: kNone, kNormal, kStripUserDefinedTimestamp 2) all four index types, when index type is `kTwoLevelIndexSearch`, also enables partitioned filters 3) parallel vs non-parallel compression 4) enable/disable compression dictionary. Also added tests for API `BlockBasedTableReader::NewIterator`. `PartitionedFilterBlockTest` is parameterized to run with different UDT test modes:kNone, kNormal, kStripUserDefinedTimestamp. ``` make all check ./block_based_table_reader_test ./partitioned_filter_block_test ``` Reviewed By: ltamasi Differential Revision: D46344577 Pulled By: jowlyzhang fbshipit-source-id: 93ac8542b19319d1298712b8bed908c8831ba675
1 year ago
// TODO(yuzhangyu): handle range deletion entries for UDT in memtable only.
s = new_table->ReadRangeDelBlock(ro, prefetch_buffer.get(),
metaindex_iter.get(), internal_comparator,
&lookup_context);
if (!s.ok()) {
return s;
}
format_version=6 and context-aware block checksums (#9058) Summary: ## Context checksum All RocksDB checksums currently use 32 bits of checking power, which should be 1 in 4 billion false negative (FN) probability (failing to detect corruption). This is true for random corruptions, and in some cases small corruptions are guaranteed to be detected. But some possible corruptions, such as in storage metadata rather than storage payload data, would have a much higher FN rate. For example: * Data larger than one SST block is replaced by data from elsewhere in the same or another SST file. Especially with block_align=true, the probability of exact block size match is probably around 1 in 100, making the FN probability around that same. Without `block_align=true` the probability of same block start location is probably around 1 in 10,000, for FN probability around 1 in a million. To solve this problem in new format_version=6, we add "context awareness" to block checksum checks. The stored and expected checksum value is modified based on the block's position in the file and which file it is in. The modifications are cleverly chosen so that, for example * blocks within about 4GB of each other are guaranteed to use different context * blocks that are offset by exactly some multiple of 4GiB are guaranteed to use different context * files generated by the same process are guaranteed to use different context for the same offsets, until wrap-around after 2^32 - 1 files Thus, with format_version=6, if a valid SST block and checksum is misplaced, its checksum FN probability should be essentially ideal, 1 in 4B. ## Footer checksum This change also adds checksum protection to the SST footer (with format_version=6), for the first time without relying on whole file checksum. To prevent a corruption of the format_version in the footer (e.g. 6 -> 5) to defeat the footer checksum, we change much of the footer data format including an "extended magic number" in format_version 6 that would be interpreted as empty index and metaindex block handles in older footer versions. We also change the encoding of handles to free up space for other new data in footer. ## More detail: making space in footer In order to keep footer the same size in format_version=6 (avoid change to IO patterns), we have to free up some space for new data. We do this two ways: * Metaindex block handle is encoded down to 4 bytes (from 10) by assuming it immediately precedes the footer, and by assuming it is < 4GB. * Index block handle is moved into metaindex. (I don't know why it was in footer to begin with.) ## Performance In case of small performance penalty, I've made a "pay as you go" optimization to compensate: replace `MutableCFOptions` in BlockBasedTableBuilder::Rep with the only field used in that structure after construction: `prefix_extractor`. This makes the PR an overall performance improvement (results below). Nevertheless I'm seeing essentially no difference going from fv=5 to fv=6, even including that improvement for both. That's based on extreme case table write performance testing, many files with many blocks. This is relatively checksum intensive (small blocks) and salt generation intensive (small files). ``` (for I in `seq 1 100`; do TEST_TMPDIR=/dev/shm/dbbench2 ./db_bench -benchmarks=fillseq -memtablerep=vector -disable_wal=1 -allow_concurrent_memtable_write=false -num=3000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -write_buffer_size=100000 -compression_type=none -block_size=1000; done) 2>&1 | grep micros/op | tee out awk '{ tot += $5; n += 1; } END { print int(1.0 * tot / n) }' < out ``` Each value below is ops/s averaged over 100 runs, run simultaneously with competing configuration for load fairness Before -> after (both fv=5): 483530 -> 483673 (negligible) Re-run 1: 480733 -> 485427 (1.0% faster) Re-run 2: 483821 -> 484541 (0.1% faster) Before (fv=5) -> after (fv=6): 482006 -> 485100 (0.6% faster) Re-run 1: 482212 -> 485075 (0.6% faster) Re-run 2: 483590 -> 484073 (0.1% faster) After fv=5 -> after fv=6: 483878 -> 485542 (0.3% faster) Re-run 1: 485331 -> 483385 (0.4% slower) Re-run 2: 485283 -> 483435 (0.4% slower) Re-run 3: 483647 -> 486109 (0.5% faster) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9058 Test Plan: unit tests included (table_test, db_properties_test, salt in env_test). General DB tests and crash test updated to test new format_version. Also temporarily updated the default format version to 6 and saw some test failures. Almost all were due to an inadvertent additional read in VerifyChecksum to verify the index block checksum, though it's arguably a bug that VerifyChecksum does not appear to (re-)verify the index block checksum, just assuming it was verified in opening the index reader (probably *usually* true but probably not always true). Some other concerns about VerifyChecksum are left in FIXME comments. The only remaining test failure on change of default (in block_fetcher_test) now has a comment about how to upgrade the test. The format compatibility test does not need updating because we have not updated the default format_version. Reviewed By: ajkr, mrambacher Differential Revision: D33100915 Pulled By: pdillinger fbshipit-source-id: 8679e3e572fa580181a737fd6d113ed53c5422ee
1 year ago
rep->verify_checksum_set_on_open = ro.verify_checksums;
s = new_table->PrefetchIndexAndFilterBlocks(
ro, prefetch_buffer.get(), metaindex_iter.get(), new_table.get(),
prefetch_all, table_options, level, file_size,
max_file_size_for_l0_meta_pin, &lookup_context);
if (s.ok()) {
// Update tail prefetch stats
assert(prefetch_buffer.get() != nullptr);
if (tail_prefetch_stats != nullptr) {
assert(prefetch_buffer->min_offset_read() < file_size);
tail_prefetch_stats->RecordEffectiveSize(
static_cast<size_t>(file_size) - prefetch_buffer->min_offset_read());
}
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
}
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
if (s.ok() && table_reader_cache_res_mgr) {
std::size_t mem_usage = new_table->ApproximateMemoryUsage();
s = table_reader_cache_res_mgr->MakeCacheReservation(
mem_usage, &(rep->table_reader_cache_res_handle));
if (s.IsMemoryLimit()) {
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
s = Status::MemoryLimit(
Rewrite memory-charging feature's option API (#9926) Summary: **Context:** Previous PR https://github.com/facebook/rocksdb/pull/9748, https://github.com/facebook/rocksdb/pull/9073, https://github.com/facebook/rocksdb/pull/8428 added separate flag for each charged memory area. Such API design is not scalable as we charge more and more memory areas. Also, we foresee an opportunity to consolidate this feature with other cache usage related features such as `cache_index_and_filter_blocks` using `CacheEntryRole`. Therefore we decided to consolidate all these flags with `CacheUsageOptions cache_usage_options` and this PR serves as the first step by consolidating memory-charging related flags. **Summary:** - Replaced old API reference with new ones, including making `kCompressionDictionaryBuildingBuffer` opt-out and added a unit test for that - Added missing db bench/stress test for some memory charging features - Renamed related test suite to indicate they are under the same theme of memory charging - Refactored a commonly used mocked cache component in memory charging related tests to reduce code duplication - Replaced the phrases "memory tracking" / "cache reservation" (other than CacheReservationManager-related ones) with "memory charging" for standard description of this feature. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9926 Test Plan: - New unit test for opt-out `kCompressionDictionaryBuildingBuffer` `TEST_F(ChargeCompressionDictionaryBuildingBufferTest, Basic)` - New unit test for option validation/sanitization `TEST_F(CacheUsageOptionsOverridesTest, SanitizeAndValidateOptions)` - CI - db bench (in case querying new options introduces regression) **+0.5% micros/op**: `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_compression_dictionary_building_buffer=1(remove this for comparison) -compression_max_dict_bytes=10000 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | **-0.3633711465** 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | **0.5289363078** - db_stress: `python3 tools/db_crashtest.py blackbox -charge_compression_dictionary_building_buffer=1 -charge_filter_construction=1 -charge_table_reader=1 -cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36054712 Pulled By: hx235 fbshipit-source-id: d406e90f5e0c5ea4dbcb585a484ad9302d4302af
3 years ago
"Can't allocate " +
kCacheEntryRoleToCamelString[static_cast<std::uint32_t>(
CacheEntryRole::kBlockBasedTableReader)] +
" due to memory limit based on "
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
"cache capacity for memory allocation");
}
}
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
if (s.ok()) {
*table_reader = std::move(new_table);
}
return s;
}
Status BlockBasedTable::PrefetchTail(
const ReadOptions& ro, RandomAccessFileReader* file, uint64_t file_size,
bool force_direct_prefetch, TailPrefetchStats* tail_prefetch_stats,
const bool prefetch_all, const bool preload_all,
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
std::unique_ptr<FilePrefetchBuffer>* prefetch_buffer, Statistics* stats,
uint64_t tail_size, Logger* const logger) {
assert(tail_size <= file_size);
size_t tail_prefetch_size = 0;
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
if (tail_size != 0) {
tail_prefetch_size = tail_size;
} else {
if (tail_prefetch_stats != nullptr) {
// Multiple threads may get a 0 (no history) when running in parallel,
// but it will get cleared after the first of them finishes.
tail_prefetch_size = tail_prefetch_stats->GetSuggestedPrefetchSize();
}
if (tail_prefetch_size == 0) {
// Before read footer, readahead backwards to prefetch data. Do more
// readahead if we're going to read index/filter.
// TODO: This may incorrectly select small readahead in case partitioned
// index/filter is enabled and top-level partition pinning is enabled.
// That's because we need to issue readahead before we read the
// properties, at which point we don't yet know the index type.
Fix higher read qps during db open caused by pr 11406 (#11516) Summary: **Context:** [PR11406](https://github.com/facebook/rocksdb/pull/11406/) caused more frequent read during db open reading files with no `tail_size` in the manifest as part of the upgrade to 11406. This is due to that PR introduced - [smaller](https://github.com/facebook/rocksdb/pull/11406/files#diff-57ed8c49db2bdd4db7618646a177397674bbf25beacacecb104070071d30129fR833) prefetch tail buffer size compared to pre-11406 for small files (< 52 MB) when `tail_prefetch_stats` infers tail size to be 0 (usually happens when the stats does not have much historical data to infer early on) - more read (up to # of partitioned filter/index) when such small prefetch tail buffer does not contain all the partitioned filter/index needed in CacheDependencies() since the [fallback logic](https://github.com/facebook/rocksdb/pull/11406/files#diff-d98f1a83de24412ad7f3527725dae7e28851c7222622c3cdb832d3cdf24bbf9fR165-R179) that prefetches all partitions at once will be [skipped](url) when such a small prefetch tail buffer is passed in **Summary:** - Revert the fallback prefetch buffer size change to preserve existing behavior fully during upgrading in `BlockBasedTable::PrefetchTail()` - Use passed-in prefetch tail buffer in `CacheDependencies()` only if it has a smaller offset than the the offset of first partition filter/index, that is, at least as good as the existing prefetching behavior Pull Request resolved: https://github.com/facebook/rocksdb/pull/11516 Test Plan: - db bench Create db with small files prior to PR 11406 ``` ./db_bench -db=/tmp/testdb/ --partition_index_and_filters=1 --statistics=1 -benchmarks=fillseq -key_size=3200 -value_size=5 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=true -compression_type=zstd` ``` Read db to see if post-pr has lower read qps (i.e, rocksdb.file.read.db.open.micros count) during db open. ``` ./db_bench -use_direct_reads=1 --file_opening_threads=1 --threads=1 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 --db=/tmp/testdb/ --benchmarks=readrandom --key_size=3200 --value_size=5 --num=100 --disable_auto_compactions=true --compression_type=zstd ``` Pre-PR: ``` rocksdb.file.read.db.open.micros P50 : 3.399023 P95 : 5.924468 P99 : 12.408333 P100 : 29.000000 COUNT : 611 SUM : 2539 ``` Post-PR: ``` rocksdb.file.read.db.open.micros P50 : 593.736842 P95 : 861.605263 P99 : 1212.868421 P100 : 2663.000000 COUNT : 585 SUM : 345349 ``` _Note: To control the starting offset of the prefetch tail buffer easier, I manually override the following to eliminate the effect of alignment_ ``` class PosixRandomAccessFile : public FSRandomAccessFile { virtual size_t GetRequiredBufferAlignment() const override { - return logical_sector_size_; + return 1; } ``` - CI Reviewed By: pdillinger Differential Revision: D46472566 Pulled By: hx235 fbshipit-source-id: 2fe14ac8d489d15b0e08e6f8fe4f46d5f110978e
1 year ago
tail_prefetch_size = prefetch_all || preload_all ? 512 * 1024 : 4 * 1024;
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
ROCKS_LOG_WARN(logger,
"Tail prefetch size %zu is calculated based on heuristics",
tail_prefetch_size);
} else {
ROCKS_LOG_WARN(
logger,
"Tail prefetch size %zu is calculated based on TailPrefetchStats",
tail_prefetch_size);
}
}
size_t prefetch_off;
size_t prefetch_len;
if (file_size < tail_prefetch_size) {
prefetch_off = 0;
prefetch_len = static_cast<size_t>(file_size);
} else {
prefetch_off = static_cast<size_t>(file_size - tail_prefetch_size);
prefetch_len = tail_prefetch_size;
}
Add UT to test BG read qps behavior during upgrade for pr11406 (#11522) Summary: **Context/Summary:** When db is upgrading to adopt [pr11406](https://github.com/facebook/rocksdb/pull/11406/), it's possible for RocksDB to infer a small tail size to prefetch for pre-upgrade files. Such small tail size would have caused 1 file read per index or filter partition if partitioned index or filer is used. This PR provides a UT to show this would not happen. Misc: refactor the related UTs a bit to make this new UT more readable. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11522 Test Plan: - New UT If logic of upgrade is wrong e.g, ``` --- a/table/block_based/partitioned_index_reader.cc +++ b/table/block_based/partitioned_index_reader.cc @@ -166,7 +166,8 @@ Status PartitionIndexReader::CacheDependencies( uint64_t prefetch_len = last_off - prefetch_off; std::unique_ptr<FilePrefetchBuffer> prefetch_buffer; if (tail_prefetch_buffer == nullptr || !tail_prefetch_buffer->Enabled() || - tail_prefetch_buffer->GetPrefetchOffset() > prefetch_off) { + (false && tail_prefetch_buffer->GetPrefetchOffset() > prefetch_off)) { ``` , then the UT will fail like below ``` [ RUN ] PrefetchTailTest/PrefetchTailTest.UpgradeToTailSizeInManifest/0 file/prefetch_test.cc:461: Failure Expected: (db_open_file_read.count) < (num_index_partition), actual: 38 vs 33 Received signal 11 (Segmentation fault) ``` Reviewed By: pdillinger Differential Revision: D46546707 Pulled By: hx235 fbshipit-source-id: 9897b0a975e9055963edac5451fd1cd9d6c45d0e
1 year ago
#ifndef NDEBUG
std::pair<size_t*, size_t*> prefetch_off_len_pair = {&prefetch_off,
&prefetch_len};
TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::TailPrefetchLen",
Add UT to test BG read qps behavior during upgrade for pr11406 (#11522) Summary: **Context/Summary:** When db is upgrading to adopt [pr11406](https://github.com/facebook/rocksdb/pull/11406/), it's possible for RocksDB to infer a small tail size to prefetch for pre-upgrade files. Such small tail size would have caused 1 file read per index or filter partition if partitioned index or filer is used. This PR provides a UT to show this would not happen. Misc: refactor the related UTs a bit to make this new UT more readable. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11522 Test Plan: - New UT If logic of upgrade is wrong e.g, ``` --- a/table/block_based/partitioned_index_reader.cc +++ b/table/block_based/partitioned_index_reader.cc @@ -166,7 +166,8 @@ Status PartitionIndexReader::CacheDependencies( uint64_t prefetch_len = last_off - prefetch_off; std::unique_ptr<FilePrefetchBuffer> prefetch_buffer; if (tail_prefetch_buffer == nullptr || !tail_prefetch_buffer->Enabled() || - tail_prefetch_buffer->GetPrefetchOffset() > prefetch_off) { + (false && tail_prefetch_buffer->GetPrefetchOffset() > prefetch_off)) { ``` , then the UT will fail like below ``` [ RUN ] PrefetchTailTest/PrefetchTailTest.UpgradeToTailSizeInManifest/0 file/prefetch_test.cc:461: Failure Expected: (db_open_file_read.count) < (num_index_partition), actual: 38 vs 33 Received signal 11 (Segmentation fault) ``` Reviewed By: pdillinger Differential Revision: D46546707 Pulled By: hx235 fbshipit-source-id: 9897b0a975e9055963edac5451fd1cd9d6c45d0e
1 year ago
&prefetch_off_len_pair);
#endif // NDEBUG
// Try file system prefetch
if (!file->use_direct_io() && !force_direct_prefetch) {
Set Read rate limiter priority dynamically and pass it to FS (#9996) Summary: ### Context: Background compactions and flush generate large reads and writes, and can be long running, especially for universal compaction. In some cases, this can impact foreground reads and writes by users. ### Solution User, Flush, and Compaction reads share some code path. For this task, we update the rate_limiter_priority in ReadOptions for code paths (e.g. FindTable (mainly in BlockBasedTable::Open()) and various iterators), and eventually update the rate_limiter_priority in IOOptions for FSRandomAccessFile. **This PR is for the Read path.** The **Read:** dynamic priority for different state are listed as follows: | State | Normal | Delayed | Stalled | | ----- | ------ | ------- | ------- | | Flush (verification read in BuildTable()) | IO_USER | IO_USER | IO_USER | | Compaction | IO_LOW | IO_USER | IO_USER | | User | User provided | User provided | User provided | We will respect the read_options that the user provided and will not set it. The only sst read for Flush is the verification read in BuildTable(). It claims to be "regard as user read". **Details** 1. Set read_options.rate_limiter_priority dynamically: - User: Do not update the read_options. Use the read_options that the user provided. - Compaction: Update read_options in CompactionJob::ProcessKeyValueCompaction(). - Flush: Update read_options in BuildTable(). 2. Pass the rate limiter priority to FSRandomAccessFile functions: - After calling the FindTable(), read_options is passed through GetTableReader(table_cache.cc), BlockBasedTableFactory::NewTableReader(block_based_table_factory.cc), and BlockBasedTable::Open(). The Open() needs some updates for the ReadOptions variable and the updates are also needed for the called functions, including PrefetchTail(), PrepareIOOptions(), ReadFooterFromFile(), ReadMetaIndexblock(), ReadPropertiesBlock(), PrefetchIndexAndFilterBlocks(), and ReadRangeDelBlock(). - In RandomAccessFileReader, the functions to be updated include Read(), MultiRead(), ReadAsync(), and Prefetch(). - Update the downstream functions of NewIndexIterator(), NewDataBlockIterator(), and BlockBasedTableIterator(). ### Test Plans Add unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9996 Reviewed By: anand1976 Differential Revision: D36452483 Pulled By: gitbw95 fbshipit-source-id: 60978204a4f849bb9261cb78d9bc1cb56d6008cf
3 years ago
if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority)
.IsNotSupported()) {
prefetch_buffer->reset(new FilePrefetchBuffer(
0 /* readahead_size */, 0 /* max_readahead_size */,
false /* enable */, true /* track_min_offset */));
return Status::OK();
}
}
// Use `FilePrefetchBuffer`
Add new stat rocksdb.table.open.prefetch.tail.read.bytes, rocksdb.table.open.prefetch.tail.{miss|hit} (#11265) Summary: **Context/Summary:** We are adding new stats to measure behavior of prefetched tail size and look up into this buffer The stat collection is done in FilePrefetchBuffer but only for prefetched tail buffer during table open for now using FilePrefetchBuffer enum. It's cleaner than the alternative of implementing in upper-level call places of FilePrefetchBuffer for table open. It also has the benefit of extensible to other types of FilePrefetchBuffer if needed. See db bench for perf regression concern. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11265 Test Plan: **- Piggyback on existing test** **- rocksdb.table.open.prefetch.tail.miss is harder to UT so I manually set prefetch tail read bytes to be small and run db bench.** ``` ./db_bench -db=/tmp/testdb -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 -use_direct_reads=true ``` ``` rocksdb.table.open.prefetch.tail.read.bytes P50 : 4096.000000 P95 : 4096.000000 P99 : 4096.000000 P100 : 4096.000000 COUNT : 225 SUM : 921600 rocksdb.table.open.prefetch.tail.miss COUNT : 91 rocksdb.table.open.prefetch.tail.hit COUNT : 1034 ``` **- No perf regression observed in db_bench** SETUP command: create same db with ~900 files for pre-change/post-change. ``` ./db_bench -db=/tmp/testdb -benchmarks="fillseq" -key_size=32 -value_size=512 -num=500000 -write_buffer_size=655360 -disable_auto_compactions=true -target_file_size_base=16777216 -compression_type=none ``` TEST command 60 runs or til convergence: as suggested by anand1976 and akankshamahajan15, vary `seek_nexts` and `async_io` in testing. ``` ./db_bench -use_existing_db=true -db=/tmp/testdb -statistics=false -cache_size=0 -cache_index_and_filter_blocks=false -benchmarks=seekrandom[-X60] -num=50000 -seek_nexts={10, 500, 1000} -async_io={0|1} -use_direct_reads=true ``` async io = 0, direct io read = true | seek_nexts = 10, 30 runs | seek_nexts = 500, 12 runs | seek_nexts = 1000, 6 runs -- | -- | -- | -- pre-post change | 4776 (± 28) ops/sec; 24.8 (± 0.1) MB/sec | 288 (± 1) ops/sec; 74.8 (± 0.4) MB/sec | 145 (± 4) ops/sec; 75.6 (± 2.2) MB/sec post-change | 4790 (± 32) ops/sec; 24.9 (± 0.2) MB/sec | 288 (± 3) ops/sec; 74.7 (± 0.8) MB/sec | 143 (± 3) ops/sec; 74.5 (± 1.6) MB/sec async io = 1, direct io read = true | seek_nexts = 10, 54 runs | seek_nexts = 500, 6 runs | seek_nexts = 1000, 4 runs -- | -- | -- | -- pre-post change | 3350 (± 36) ops/sec; 17.4 (± 0.2) MB/sec | 264 (± 0) ops/sec; 68.7 (± 0.2) MB/sec | 138 (± 1) ops/sec; 71.8 (± 1.0) MB/sec post-change | 3358 (± 27) ops/sec; 17.4 (± 0.1) MB/sec | 263 (± 2) ops/sec; 68.3 (± 0.8) MB/sec | 139 (± 1) ops/sec; 72.6 (± 0.6) MB/sec Reviewed By: ajkr Differential Revision: D43781467 Pulled By: hx235 fbshipit-source-id: a706a18472a8edb2b952bac3af40eec803537f2a
2 years ago
prefetch_buffer->reset(new FilePrefetchBuffer(
0 /* readahead_size */, 0 /* max_readahead_size */, true /* enable */,
true /* track_min_offset */, false /* implicit_auto_readahead */,
0 /* num_file_reads */, 0 /* num_file_reads_for_auto_readahead */,
nullptr /* fs */, nullptr /* clock */, stats,
FilePrefetchBufferUsage::kTableOpenPrefetchTail));
Set Read rate limiter priority dynamically and pass it to FS (#9996) Summary: ### Context: Background compactions and flush generate large reads and writes, and can be long running, especially for universal compaction. In some cases, this can impact foreground reads and writes by users. ### Solution User, Flush, and Compaction reads share some code path. For this task, we update the rate_limiter_priority in ReadOptions for code paths (e.g. FindTable (mainly in BlockBasedTable::Open()) and various iterators), and eventually update the rate_limiter_priority in IOOptions for FSRandomAccessFile. **This PR is for the Read path.** The **Read:** dynamic priority for different state are listed as follows: | State | Normal | Delayed | Stalled | | ----- | ------ | ------- | ------- | | Flush (verification read in BuildTable()) | IO_USER | IO_USER | IO_USER | | Compaction | IO_LOW | IO_USER | IO_USER | | User | User provided | User provided | User provided | We will respect the read_options that the user provided and will not set it. The only sst read for Flush is the verification read in BuildTable(). It claims to be "regard as user read". **Details** 1. Set read_options.rate_limiter_priority dynamically: - User: Do not update the read_options. Use the read_options that the user provided. - Compaction: Update read_options in CompactionJob::ProcessKeyValueCompaction(). - Flush: Update read_options in BuildTable(). 2. Pass the rate limiter priority to FSRandomAccessFile functions: - After calling the FindTable(), read_options is passed through GetTableReader(table_cache.cc), BlockBasedTableFactory::NewTableReader(block_based_table_factory.cc), and BlockBasedTable::Open(). The Open() needs some updates for the ReadOptions variable and the updates are also needed for the called functions, including PrefetchTail(), PrepareIOOptions(), ReadFooterFromFile(), ReadMetaIndexblock(), ReadPropertiesBlock(), PrefetchIndexAndFilterBlocks(), and ReadRangeDelBlock(). - In RandomAccessFileReader, the functions to be updated include Read(), MultiRead(), ReadAsync(), and Prefetch(). - Update the downstream functions of NewIndexIterator(), NewDataBlockIterator(), and BlockBasedTableIterator(). ### Test Plans Add unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9996 Reviewed By: anand1976 Differential Revision: D36452483 Pulled By: gitbw95 fbshipit-source-id: 60978204a4f849bb9261cb78d9bc1cb56d6008cf
3 years ago
IOOptions opts;
Status s = file->PrepareIOOptions(ro, opts);
if (s.ok()) {
Add rate limiter priority to ReadOptions (#9424) Summary: Users can set the priority for file reads associated with their operation by setting `ReadOptions::rate_limiter_priority` to something other than `Env::IO_TOTAL`. Rate limiting `VerifyChecksum()` and `VerifyFileChecksums()` is the motivation for this PR, so it also includes benchmarks and minor bug fixes to get that working. `RandomAccessFileReader::Read()` already had support for rate limiting compaction reads. I changed that rate limiting to be non-specific to compaction, but rather performed according to the passed in `Env::IOPriority`. Now the compaction read rate limiting is supported by setting `rate_limiter_priority = Env::IO_LOW` on its `ReadOptions`. There is no default value for the new `Env::IOPriority` parameter to `RandomAccessFileReader::Read()`. That means this PR goes through all callers (in some cases multiple layers up the call stack) to find a `ReadOptions` to provide the priority. There are TODOs for cases I believe it would be good to let user control the priority some day (e.g., file footer reads), and no TODO in cases I believe it doesn't matter (e.g., trace file reads). The API doc only lists the missing cases where a file read associated with a provided `ReadOptions` cannot be rate limited. For cases like file ingestion checksum calculation, there is no API to provide `ReadOptions` or `Env::IOPriority`, so I didn't count that as missing. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9424 Test Plan: - new unit tests - new benchmarks on ~50MB database with 1MB/s read rate limit and 100ms refill interval; verified with strace reads are chunked (at 0.1MB per chunk) and spaced roughly 100ms apart. - setup command: `./db_bench -benchmarks=fillrandom,compact -db=/tmp/testdb -target_file_size_base=1048576 -disable_auto_compactions=true -file_checksum=true` - benchmarks command: `strace -ttfe pread64 ./db_bench -benchmarks=verifychecksum,verifyfilechecksums -use_existing_db=true -db=/tmp/testdb -rate_limiter_bytes_per_sec=1048576 -rate_limit_bg_reads=1 -rate_limit_user_ops=true -file_checksum=true` - crash test using IO_USER priority on non-validation reads with https://github.com/facebook/rocksdb/issues/9567 reverted: `python3 tools/db_crashtest.py blackbox --max_key=1000000 --write_buffer_size=524288 --target_file_size_base=524288 --level_compaction_dynamic_level_bytes=true --duration=3600 --rate_limit_bg_reads=true --rate_limit_user_ops=true --rate_limiter_bytes_per_sec=10485760 --interval=10` Reviewed By: hx235 Differential Revision: D33747386 Pulled By: ajkr fbshipit-source-id: a2d985e97912fba8c54763798e04f006ccc56e0c
3 years ago
s = (*prefetch_buffer)
->Prefetch(opts, file, prefetch_off, prefetch_len,
ro.rate_limiter_priority);
}
return s;
}
Status BlockBasedTable::ReadPropertiesBlock(
const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
InternalIterator* meta_iter, const SequenceNumber largest_seqno) {
Status s;
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
BlockHandle handle;
s = FindOptionalMetaBlock(meta_iter, kPropertiesBlockName, &handle);
if (!s.ok()) {
ROCKS_LOG_WARN(rep_->ioptions.logger,
"Error when seeking to properties block from file: %s",
s.ToString().c_str());
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
} else if (!handle.IsNull()) {
s = meta_iter->status();
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
std::unique_ptr<TableProperties> table_properties;
if (s.ok()) {
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
s = ReadTablePropertiesHelper(
ro, handle, rep_->file.get(), prefetch_buffer, rep_->footer,
rep_->ioptions, &table_properties, nullptr /* memory_allocator */);
}
IGNORE_STATUS_IF_ERROR(s);
if (!s.ok()) {
ROCKS_LOG_WARN(rep_->ioptions.logger,
"Encountered error while reading data from properties "
"block %s",
s.ToString().c_str());
} else {
assert(table_properties != nullptr);
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
rep_->table_properties = std::move(table_properties);
rep_->blocks_maybe_compressed =
rep_->table_properties->compression_name !=
CompressionTypeToString(kNoCompression);
}
} else {
ROCKS_LOG_ERROR(rep_->ioptions.logger,
"Cannot find Properties block from file.");
}
// Read the table properties, if provided.
if (rep_->table_properties) {
rep_->whole_key_filtering &=
IsFeatureSupported(*(rep_->table_properties),
BlockBasedTablePropertyNames::kWholeKeyFiltering,
rep_->ioptions.logger);
rep_->prefix_filtering &= IsFeatureSupported(
*(rep_->table_properties),
BlockBasedTablePropertyNames::kPrefixFiltering, rep_->ioptions.logger);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
rep_->index_key_includes_seq =
rep_->table_properties->index_key_is_user_key == 0;
rep_->index_value_is_full =
rep_->table_properties->index_value_is_delta_encoded == 0;
// Update index_type with the true type.
// If table properties don't contain index type, we assume that the table
// is in very old format and has kBinarySearch index type.
auto& props = rep_->table_properties->user_collected_properties;
auto index_type_pos = props.find(BlockBasedTablePropertyNames::kIndexType);
if (index_type_pos != props.end()) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
rep_->index_type = static_cast<BlockBasedTableOptions::IndexType>(
DecodeFixed32(index_type_pos->second.c_str()));
}
auto min_ts_pos = props.find("rocksdb.timestamp_min");
if (min_ts_pos != props.end()) {
rep_->min_timestamp = Slice(min_ts_pos->second);
}
auto max_ts_pos = props.find("rocksdb.timestamp_max");
if (max_ts_pos != props.end()) {
rep_->max_timestamp = Slice(max_ts_pos->second);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
}
rep_->index_has_first_key =
rep_->index_type == BlockBasedTableOptions::kBinarySearchWithFirstKey;
s = GetGlobalSequenceNumber(*(rep_->table_properties), largest_seqno,
&(rep_->global_seqno));
if (!s.ok()) {
ROCKS_LOG_ERROR(rep_->ioptions.logger, "%s", s.ToString().c_str());
}
}
return s;
}
Status BlockBasedTable::ReadRangeDelBlock(
const ReadOptions& read_options, FilePrefetchBuffer* prefetch_buffer,
InternalIterator* meta_iter,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
const InternalKeyComparator& internal_comparator,
BlockCacheLookupContext* lookup_context) {
Status s;
BlockHandle range_del_handle;
s = FindOptionalMetaBlock(meta_iter, kRangeDelBlockName, &range_del_handle);
if (!s.ok()) {
ROCKS_LOG_WARN(
rep_->ioptions.logger,
"Error when seeking to range delete tombstones block from file: %s",
s.ToString().c_str());
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
} else if (!range_del_handle.IsNull()) {
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
Status tmp_status;
std::unique_ptr<InternalIterator> iter(NewDataBlockIterator<DataBlockIter>(
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
read_options, range_del_handle,
/*input_iter=*/nullptr, BlockType::kRangeDeletion,
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
/*get_context=*/nullptr, lookup_context, prefetch_buffer,
/*for_compaction= */ false, /*async_read= */ false, tmp_status));
assert(iter != nullptr);
s = iter->status();
Cache fragmented range tombstones in BlockBasedTableReader (#4493) Summary: This allows tombstone fragmenting to only be performed when the table is opened, and cached for subsequent accesses. On the same DB used in #4449, running `readrandom` results in the following: ``` readrandom : 0.983 micros/op 1017076 ops/sec; 78.3 MB/s (63103 of 100000 found) ``` Now that Get performance in the presence of range tombstones is reasonable, I also compared the performance between a DB with range tombstones, "expanded" range tombstones (several point tombstones that cover the same keys the equivalent range tombstone would cover, a common workaround for DeleteRange), and no range tombstones. The created DBs had 5 million keys each, and DeleteRange was called at regular intervals (depending on the total number of range tombstones being written) after 4.5 million Puts. The table below summarizes the results of a `readwhilewriting` benchmark (in order to provide somewhat more realistic results): ``` Tombstones? | avg micros/op | stddev micros/op | avg ops/s | stddev ops/s ----------------- | ------------- | ---------------- | ------------ | ------------ None | 0.6186 | 0.04637 | 1,625,252.90 | 124,679.41 500 Expanded | 0.6019 | 0.03628 | 1,666,670.40 | 101,142.65 500 Unexpanded | 0.6435 | 0.03994 | 1,559,979.40 | 104,090.52 1k Expanded | 0.6034 | 0.04349 | 1,665,128.10 | 125,144.57 1k Unexpanded | 0.6261 | 0.03093 | 1,600,457.50 | 79,024.94 5k Expanded | 0.6163 | 0.05926 | 1,636,668.80 | 154,888.85 5k Unexpanded | 0.6402 | 0.04002 | 1,567,804.70 | 100,965.55 10k Expanded | 0.6036 | 0.05105 | 1,667,237.70 | 142,830.36 10k Unexpanded | 0.6128 | 0.02598 | 1,634,633.40 | 72,161.82 25k Expanded | 0.6198 | 0.04542 | 1,620,980.50 | 116,662.93 25k Unexpanded | 0.5478 | 0.0362 | 1,833,059.10 | 121,233.81 50k Expanded | 0.5104 | 0.04347 | 1,973,107.90 | 184,073.49 50k Unexpanded | 0.4528 | 0.03387 | 2,219,034.50 | 170,984.32 ``` After a large enough quantity of range tombstones are written, range tombstone Gets can become faster than reading from an equivalent DB with several point tombstones. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4493 Differential Revision: D10842844 Pulled By: abhimadan fbshipit-source-id: a7d44534f8120e6aabb65779d26c6b9df954c509
6 years ago
if (!s.ok()) {
ROCKS_LOG_WARN(
rep_->ioptions.logger,
Cache fragmented range tombstones in BlockBasedTableReader (#4493) Summary: This allows tombstone fragmenting to only be performed when the table is opened, and cached for subsequent accesses. On the same DB used in #4449, running `readrandom` results in the following: ``` readrandom : 0.983 micros/op 1017076 ops/sec; 78.3 MB/s (63103 of 100000 found) ``` Now that Get performance in the presence of range tombstones is reasonable, I also compared the performance between a DB with range tombstones, "expanded" range tombstones (several point tombstones that cover the same keys the equivalent range tombstone would cover, a common workaround for DeleteRange), and no range tombstones. The created DBs had 5 million keys each, and DeleteRange was called at regular intervals (depending on the total number of range tombstones being written) after 4.5 million Puts. The table below summarizes the results of a `readwhilewriting` benchmark (in order to provide somewhat more realistic results): ``` Tombstones? | avg micros/op | stddev micros/op | avg ops/s | stddev ops/s ----------------- | ------------- | ---------------- | ------------ | ------------ None | 0.6186 | 0.04637 | 1,625,252.90 | 124,679.41 500 Expanded | 0.6019 | 0.03628 | 1,666,670.40 | 101,142.65 500 Unexpanded | 0.6435 | 0.03994 | 1,559,979.40 | 104,090.52 1k Expanded | 0.6034 | 0.04349 | 1,665,128.10 | 125,144.57 1k Unexpanded | 0.6261 | 0.03093 | 1,600,457.50 | 79,024.94 5k Expanded | 0.6163 | 0.05926 | 1,636,668.80 | 154,888.85 5k Unexpanded | 0.6402 | 0.04002 | 1,567,804.70 | 100,965.55 10k Expanded | 0.6036 | 0.05105 | 1,667,237.70 | 142,830.36 10k Unexpanded | 0.6128 | 0.02598 | 1,634,633.40 | 72,161.82 25k Expanded | 0.6198 | 0.04542 | 1,620,980.50 | 116,662.93 25k Unexpanded | 0.5478 | 0.0362 | 1,833,059.10 | 121,233.81 50k Expanded | 0.5104 | 0.04347 | 1,973,107.90 | 184,073.49 50k Unexpanded | 0.4528 | 0.03387 | 2,219,034.50 | 170,984.32 ``` After a large enough quantity of range tombstones are written, range tombstone Gets can become faster than reading from an equivalent DB with several point tombstones. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4493 Differential Revision: D10842844 Pulled By: abhimadan fbshipit-source-id: a7d44534f8120e6aabb65779d26c6b9df954c509
6 years ago
"Encountered error while reading data from range del block %s",
s.ToString().c_str());
IGNORE_STATUS_IF_ERROR(s);
} else {
rep_->fragmented_range_dels =
std::make_shared<FragmentedRangeTombstoneList>(std::move(iter),
internal_comparator);
}
}
return s;
}
Status BlockBasedTable::PrefetchIndexAndFilterBlocks(
const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
InternalIterator* meta_iter, BlockBasedTable* new_table, bool prefetch_all,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
const BlockBasedTableOptions& table_options, const int level,
size_t file_size, size_t max_file_size_for_l0_meta_pin,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
BlockCacheLookupContext* lookup_context) {
// Find filter handle and filter type
if (rep_->filter_policy) {
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
auto name = rep_->filter_policy->CompatibilityName();
bool builtin_compatible =
strcmp(name, BuiltinFilterPolicy::kCompatibilityName()) == 0;
for (const auto& [filter_type, prefix] :
{std::make_pair(Rep::FilterType::kFullFilter, kFullFilterBlockPrefix),
std::make_pair(Rep::FilterType::kPartitionedFilter,
kPartitionedFilterBlockPrefix),
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
std::make_pair(Rep::FilterType::kNoFilter,
kObsoleteFilterBlockPrefix)}) {
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
if (builtin_compatible) {
// This code is only here to deal with a hiccup in early 7.0.x where
// there was an unintentional name change in the SST files metadata.
// It should be OK to remove this in the future (late 2022) and just
// have the 'else' code.
// NOTE: the test:: names below are likely not needed but included
// out of caution
static const std::unordered_set<std::string> kBuiltinNameAndAliases = {
BuiltinFilterPolicy::kCompatibilityName(),
test::LegacyBloomFilterPolicy::kClassName(),
test::FastLocalBloomFilterPolicy::kClassName(),
test::Standard128RibbonFilterPolicy::kClassName(),
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
"rocksdb.internal.DeprecatedBlockBasedBloomFilter",
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
BloomFilterPolicy::kClassName(),
RibbonFilterPolicy::kClassName(),
};
// For efficiency, do a prefix seek and see if the first match is
// good.
meta_iter->Seek(prefix);
if (meta_iter->status().ok() && meta_iter->Valid()) {
Slice key = meta_iter->key();
if (key.starts_with(prefix)) {
key.remove_prefix(prefix.size());
if (kBuiltinNameAndAliases.find(key.ToString()) !=
kBuiltinNameAndAliases.end()) {
Slice v = meta_iter->value();
Status s = rep_->filter_handle.DecodeFrom(&v);
if (s.ok()) {
rep_->filter_type = filter_type;
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
if (filter_type == Rep::FilterType::kNoFilter) {
ROCKS_LOG_WARN(rep_->ioptions.logger,
"Detected obsolete filter type in %s. Read "
"performance might suffer until DB is fully "
"re-compacted.",
rep_->file->file_name().c_str());
}
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
break;
}
}
}
}
} else {
std::string filter_block_key = prefix + name;
if (FindMetaBlock(meta_iter, filter_block_key, &rep_->filter_handle)
.ok()) {
rep_->filter_type = filter_type;
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
if (filter_type == Rep::FilterType::kNoFilter) {
ROCKS_LOG_WARN(
rep_->ioptions.logger,
"Detected obsolete filter type in %s. Read performance might "
"suffer until DB is fully re-compacted.",
rep_->file->file_name().c_str());
}
break;
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
}
}
}
}
// Partition filters cannot be enabled without partition indexes
assert(rep_->filter_type != Rep::FilterType::kPartitionedFilter ||
rep_->index_type == BlockBasedTableOptions::kTwoLevelIndexSearch);
// Find compression dictionary handle
Fix a major performance bug in 7.0 re: filter compatibility (#9736) Summary: Bloom filters generated by pre-7.0 releases are not read by 7.0.x releases (and vice-versa) due to changes to FilterPolicy::Name() in https://github.com/facebook/rocksdb/issues/9590. This can severely impact read performance and read I/O on upgrade or downgrade with existing DB, but not data correctness. To fix, we go back using the old, unified name in SST metadata but (for a while anyway) recognize the aliases that could be generated by early 7.0.x releases. This unfortunately requires a public API change to avoid interfering with all the good changes from https://github.com/facebook/rocksdb/issues/9590, but the API change only affects users with custom FilterPolicy, which should be very few. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9736 Test Plan: manual Generate DBs with ``` ./db_bench.7.0 -db=/dev/shm/rocksdb.7.0 -bloom_bits=10 -cache_index_and_filter_blocks=1 -benchmarks=fillrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 ``` and similar. Compare with ``` for IMPL in 6.29 7.0 fixed; do for DB in 6.29 7.0 fixed; do echo "Testing $IMPL on $DB:"; ./db_bench.$IMPL -db=/dev/shm/rocksdb.$DB -use_existing_db -readonly -bloom_bits=10 -benchmarks=readrandom -num=10000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -duration=10 2>&1 | grep micros/op; done; done ``` Results: ``` Testing 6.29 on 6.29: readrandom : 34.381 micros/op 29085 ops/sec; 3.2 MB/s (291999 of 291999 found) Testing 6.29 on 7.0: readrandom : 190.443 micros/op 5249 ops/sec; 0.6 MB/s (52999 of 52999 found) Testing 6.29 on fixed: readrandom : 40.148 micros/op 24907 ops/sec; 2.8 MB/s (249999 of 249999 found) Testing 7.0 on 6.29: readrandom : 229.430 micros/op 4357 ops/sec; 0.5 MB/s (43999 of 43999 found) Testing 7.0 on 7.0: readrandom : 33.348 micros/op 29986 ops/sec; 3.3 MB/s (299999 of 299999 found) Testing 7.0 on fixed: readrandom : 152.734 micros/op 6546 ops/sec; 0.7 MB/s (65999 of 65999 found) Testing fixed on 6.29: readrandom : 32.024 micros/op 31224 ops/sec; 3.5 MB/s (312999 of 312999 found) Testing fixed on 7.0: readrandom : 33.990 micros/op 29390 ops/sec; 3.3 MB/s (294999 of 294999 found) Testing fixed on fixed: readrandom : 28.714 micros/op 34825 ops/sec; 3.9 MB/s (348999 of 348999 found) ``` Just paying attention to order of magnitude of ops/sec (short test durations, lots of noise), it's clear that with the fix we can read <= 6.29 & >= 7.0 at full speed, where neither 6.29 nor 7.0 can on both. And 6.29 release can properly read fixed DB at full speed. Reviewed By: siying, ajkr Differential Revision: D35057844 Pulled By: pdillinger fbshipit-source-id: a46893a6af4bf084375ebe4728066d00eb08f050
3 years ago
Status s = FindOptionalMetaBlock(meta_iter, kCompressionDictBlockName,
&rep_->compression_dict_handle);
if (!s.ok()) {
return s;
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
BlockBasedTableOptions::IndexType index_type = rep_->index_type;
const bool use_cache = table_options.cache_index_and_filter_blocks;
const bool maybe_flushed =
level == 0 && file_size <= max_file_size_for_l0_meta_pin;
std::function<bool(PinningTier, PinningTier)> is_pinned =
[maybe_flushed, &is_pinned](PinningTier pinning_tier,
PinningTier fallback_pinning_tier) {
// Fallback to fallback would lead to infinite recursion. Disallow it.
assert(fallback_pinning_tier != PinningTier::kFallback);
switch (pinning_tier) {
case PinningTier::kFallback:
return is_pinned(fallback_pinning_tier,
PinningTier::kNone /* fallback_pinning_tier */);
case PinningTier::kNone:
return false;
case PinningTier::kFlushedAndSimilar:
return maybe_flushed;
case PinningTier::kAll:
return true;
};
// In GCC, this is needed to suppress `control reaches end of non-void
// function [-Werror=return-type]`.
assert(false);
return false;
};
const bool pin_top_level_index = is_pinned(
table_options.metadata_cache_options.top_level_index_pinning,
table_options.pin_top_level_index_and_filter ? PinningTier::kAll
: PinningTier::kNone);
const bool pin_partition =
is_pinned(table_options.metadata_cache_options.partition_pinning,
table_options.pin_l0_filter_and_index_blocks_in_cache
? PinningTier::kFlushedAndSimilar
: PinningTier::kNone);
const bool pin_unpartitioned =
is_pinned(table_options.metadata_cache_options.unpartitioned_pinning,
table_options.pin_l0_filter_and_index_blocks_in_cache
? PinningTier::kFlushedAndSimilar
: PinningTier::kNone);
// pin the first level of index
const bool pin_index =
index_type == BlockBasedTableOptions::kTwoLevelIndexSearch
? pin_top_level_index
: pin_unpartitioned;
// prefetch the first level of index
// WART: this might be redundant (unnecessary cache hit) if !pin_index,
// depending on prepopulate_block_cache option
const bool prefetch_index = prefetch_all || pin_index;
std::unique_ptr<IndexReader> index_reader;
s = new_table->CreateIndexReader(ro, prefetch_buffer, meta_iter, use_cache,
prefetch_index, pin_index, lookup_context,
&index_reader);
if (!s.ok()) {
return s;
}
rep_->index_reader = std::move(index_reader);
// The partitions of partitioned index are always stored in cache. They
// are hence follow the configuration for pin and prefetch regardless of
// the value of cache_index_and_filter_blocks
if (prefetch_all || pin_partition) {
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
s = rep_->index_reader->CacheDependencies(ro, pin_partition,
prefetch_buffer);
}
if (!s.ok()) {
return s;
}
// pin the first level of filter
const bool pin_filter =
rep_->filter_type == Rep::FilterType::kPartitionedFilter
? pin_top_level_index
: pin_unpartitioned;
// prefetch the first level of filter
// WART: this might be redundant (unnecessary cache hit) if !pin_filter,
// depending on prepopulate_block_cache option
const bool prefetch_filter = prefetch_all || pin_filter;
if (rep_->filter_policy) {
auto filter = new_table->CreateFilterBlockReader(
ro, prefetch_buffer, use_cache, prefetch_filter, pin_filter,
lookup_context);
if (filter) {
// Refer to the comment above about paritioned indexes always being cached
if (prefetch_all || pin_partition) {
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
s = filter->CacheDependencies(ro, pin_partition, prefetch_buffer);
if (!s.ok()) {
return s;
}
}
rep_->filter = std::move(filter);
}
}
if (!rep_->compression_dict_handle.IsNull()) {
std::unique_ptr<UncompressionDictReader> uncompression_dict_reader;
s = UncompressionDictReader::Create(
this, ro, prefetch_buffer, use_cache, prefetch_all || pin_unpartitioned,
pin_unpartitioned, lookup_context, &uncompression_dict_reader);
if (!s.ok()) {
return s;
}
rep_->uncompression_dict_reader = std::move(uncompression_dict_reader);
}
assert(s.ok());
return s;
}
void BlockBasedTable::SetupForCompaction() {
switch (rep_->ioptions.access_hint_on_compaction_start) {
case Options::NONE:
break;
case Options::NORMAL:
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
rep_->file->file()->Hint(FSRandomAccessFile::kNormal);
break;
case Options::SEQUENTIAL:
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
rep_->file->file()->Hint(FSRandomAccessFile::kSequential);
break;
case Options::WILLNEED:
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
rep_->file->file()->Hint(FSRandomAccessFile::kWillNeed);
break;
default:
assert(false);
}
}
std::shared_ptr<const TableProperties> BlockBasedTable::GetTableProperties()
const {
return rep_->table_properties;
}
size_t BlockBasedTable::ApproximateMemoryUsage() const {
size_t usage = 0;
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
if (rep_) {
usage += rep_->ApproximateMemoryUsage();
} else {
return usage;
}
if (rep_->filter) {
usage += rep_->filter->ApproximateMemoryUsage();
}
if (rep_->index_reader) {
usage += rep_->index_reader->ApproximateMemoryUsage();
}
if (rep_->uncompression_dict_reader) {
usage += rep_->uncompression_dict_reader->ApproximateMemoryUsage();
}
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
3 years ago
if (rep_->table_properties) {
usage += rep_->table_properties->ApproximateMemoryUsage();
}
return usage;
}
// Load the meta-index-block from the file. On success, return the loaded
// metaindex
// block and its iterator.
Status BlockBasedTable::ReadMetaIndexBlock(
const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
std::unique_ptr<Block>* metaindex_block,
std::unique_ptr<InternalIterator>* iter) {
// TODO(sanjay): Skip this if footer.metaindex_handle() size indicates
// it is an empty block.
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
std::unique_ptr<Block_kMetaIndex> metaindex;
Status s = ReadAndParseBlockFromFile(
rep_->file.get(), prefetch_buffer, rep_->footer, ro,
rep_->footer.metaindex_handle(), &metaindex, rep_->ioptions,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
rep_->create_context, true /*maybe_compressed*/,
UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
GetMemoryAllocator(rep_->table_options), false /* for_compaction */,
false /* async_read */);
if (!s.ok()) {
ROCKS_LOG_ERROR(rep_->ioptions.logger,
"Encountered error while reading data from properties"
" block %s",
s.ToString().c_str());
return s;
}
*metaindex_block = std::move(metaindex);
// meta block uses bytewise comparator.
iter->reset(metaindex_block->get()->NewMetaIterator());
return Status::OK();
}
template <typename TBlocklike>
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
Cache::Priority BlockBasedTable::GetCachePriority() const {
// Here we treat the legacy name "...index_and_filter_blocks..." to mean all
// metadata blocks that might go into block cache, EXCEPT only those needed
// for the read path (Get, etc.). TableProperties should not be needed on the
// read path (prefix extractor setting is an O(1) size special case that we
// are working not to require from TableProperties), so it is not given
// high-priority treatment if it should go into BlockCache.
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
if constexpr (TBlocklike::kBlockType == BlockType::kData ||
TBlocklike::kBlockType == BlockType::kProperties) {
return Cache::Priority::LOW;
} else if (rep_->table_options
.cache_index_and_filter_blocks_with_high_priority) {
return Cache::Priority::HIGH;
} else {
return Cache::Priority::LOW;
}
}
template <typename TBlocklike>
WithBlocklikeCheck<Status, TBlocklike> BlockBasedTable::GetDataBlockFromCache(
const Slice& cache_key, BlockCacheInterface<TBlocklike> block_cache,
CachableEntry<TBlocklike>* out_parsed_block,
GetContext* get_context) const {
assert(out_parsed_block);
assert(out_parsed_block->IsEmpty());
Status s;
Use new Insert and Lookup APIs in table reader to support secondary cache (#8315) Summary: Secondary cache is implemented to achieve the secondary cache tier for block cache. New Insert and Lookup APIs are introduced in https://github.com/facebook/rocksdb/issues/8271 . To support and use the secondary cache in block based table reader, this PR introduces the corresponding callback functions that will be used in secondary cache, and update the Insert and Lookup APIs accordingly. benchmarking: ./db_bench --benchmarks="fillrandom" -num=1000000 -key_size=32 -value_size=256 -use_direct_io_for_flush_and_compaction=true -db=/tmp/rocks_t/db -partition_index_and_filters=true ./db_bench -db=/tmp/rocks_t/db -use_existing_db=true -benchmarks=readrandom -num=1000000 -key_size=32 -value_size=256 -use_direct_reads=true -cache_size=1073741824 -cache_numshardbits=5 -cache_index_and_filter_blocks=true -read_random_exp_range=17 -statistics -partition_index_and_filters=true -stats_dump_period_sec=30 -reads=50000000 master benchmarking results: readrandom : 3.923 micros/op 254881 ops/sec; 33.4 MB/s (23849796 of 50000000 found) rocksdb.db.get.micros P50 : 2.820992 P95 : 5.636716 P99 : 16.450553 P100 : 8396.000000 COUNT : 50000000 SUM : 179947064 Current PR benchmarking results readrandom : 4.083 micros/op 244925 ops/sec; 32.1 MB/s (23849796 of 50000000 found) rocksdb.db.get.micros P50 : 2.967687 P95 : 5.754916 P99 : 15.665912 P100 : 8213.000000 COUNT : 50000000 SUM : 187250053 About 3.8% throughput reduction. P50: 5.2% increasing, P95, 2.09% increasing, P99 4.77% improvement Pull Request resolved: https://github.com/facebook/rocksdb/pull/8315 Test Plan: added the testing case Reviewed By: anand1976 Differential Revision: D28599774 Pulled By: zhichao-cao fbshipit-source-id: 098c4df0d7327d3a546df7604b2f1602f13044ed
4 years ago
Statistics* statistics = rep_->ioptions.statistics.get();
// Lookup uncompressed cache first
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
if (block_cache) {
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
assert(!cache_key.empty());
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
auto cache_handle = block_cache.LookupFull(
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
cache_key, &rep_->create_context, GetCachePriority<TBlocklike>(),
statistics, rep_->ioptions.lowest_used_cache_tier);
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
// Avoid updating metrics here if the handle is not complete yet. This
// happens with MultiGet and secondary cache. So update the metrics only
// if its a miss, or a hit and value is ready
if (!cache_handle) {
UpdateCacheMissMetrics(TBlocklike::kBlockType, get_context);
} else {
TBlocklike* value = block_cache.Value(cache_handle);
if (value) {
UpdateCacheHitMetrics(TBlocklike::kBlockType, get_context,
block_cache.get()->GetUsage(cache_handle));
}
out_parsed_block->SetCachedValue(value, block_cache.get(), cache_handle);
return s;
}
}
// If not found, search from the compressed block cache.
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
assert(out_parsed_block->IsEmpty());
return s;
}
template <typename TBlocklike>
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
WithBlocklikeCheck<Status, TBlocklike> BlockBasedTable::PutDataBlockToCache(
const Slice& cache_key, BlockCacheInterface<TBlocklike> block_cache,
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
CachableEntry<TBlocklike>* out_parsed_block, BlockContents&& block_contents,
CompressionType block_comp_type,
const UncompressionDict& uncompression_dict,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
MemoryAllocator* memory_allocator, GetContext* get_context) const {
const ImmutableOptions& ioptions = rep_->ioptions;
const uint32_t format_version = rep_->table_options.format_version;
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
assert(out_parsed_block);
assert(out_parsed_block->IsEmpty());
Status s;
Statistics* statistics = ioptions.stats;
std::unique_ptr<TBlocklike> block_holder;
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
if (block_comp_type != kNoCompression) {
// Retrieve the uncompressed contents into a new buffer
BlockContents uncompressed_block_contents;
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
UncompressionContext context(block_comp_type);
UncompressionInfo info(context, uncompression_dict, block_comp_type);
s = UncompressBlockData(info, block_contents.data.data(),
block_contents.data.size(),
&uncompressed_block_contents, format_version,
ioptions, memory_allocator);
if (!s.ok()) {
return s;
}
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
rep_->create_context.Create(&block_holder,
std::move(uncompressed_block_contents));
} else {
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
rep_->create_context.Create(&block_holder, std::move(block_contents));
}
// insert into uncompressed block cache
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
if (block_cache && block_holder->own_bytes()) {
size_t charge = block_holder->ApproximateMemoryUsage();
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
BlockCacheTypedHandle<TBlocklike>* cache_handle = nullptr;
s = block_cache.InsertFull(cache_key, block_holder.get(), charge,
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
&cache_handle, GetCachePriority<TBlocklike>(),
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
rep_->ioptions.lowest_used_cache_tier);
if (s.ok()) {
assert(cache_handle != nullptr);
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
out_parsed_block->SetCachedValue(block_holder.release(),
block_cache.get(), cache_handle);
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
UpdateCacheInsertionMetrics(TBlocklike::kBlockType, get_context, charge,
s.IsOkOverwritten(), rep_->ioptions.stats);
} else {
RecordTick(statistics, BLOCK_CACHE_ADD_FAILURES);
}
} else {
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
out_parsed_block->SetOwnedValue(std::move(block_holder));
}
return s;
}
std::unique_ptr<FilterBlockReader> BlockBasedTable::CreateFilterBlockReader(
const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, bool use_cache,
bool prefetch, bool pin, BlockCacheLookupContext* lookup_context) {
auto& rep = rep_;
auto filter_type = rep->filter_type;
if (filter_type == Rep::FilterType::kNoFilter) {
return std::unique_ptr<FilterBlockReader>();
}
assert(rep->filter_policy);
switch (filter_type) {
case Rep::FilterType::kPartitionedFilter:
return PartitionedFilterBlockReader::Create(
this, ro, prefetch_buffer, use_cache, prefetch, pin, lookup_context);
case Rep::FilterType::kFullFilter:
return FullFilterBlockReader::Create(this, ro, prefetch_buffer, use_cache,
prefetch, pin, lookup_context);
default:
// filter_type is either kNoFilter (exited the function at the first if),
// or it must be covered in this switch block
assert(false);
return std::unique_ptr<FilterBlockReader>();
}
}
// disable_prefix_seek should be set to true when prefix_extractor found in SST
// differs from the one in mutable_cf_options and index type is HashBasedIndex
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
InternalIteratorBase<IndexValue>* BlockBasedTable::NewIndexIterator(
const ReadOptions& read_options, bool disable_prefix_seek,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
IndexBlockIter* input_iter, GetContext* get_context,
BlockCacheLookupContext* lookup_context) const {
assert(rep_ != nullptr);
assert(rep_->index_reader != nullptr);
// We don't return pinned data from index blocks, so no need
// to set `block_contents_pinned`.
return rep_->index_reader->NewIterator(read_options, disable_prefix_seek,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
input_iter, get_context,
lookup_context);
}
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
// TODO?
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
template <>
DataBlockIter* BlockBasedTable::InitBlockIterator<DataBlockIter>(
const Rep* rep, Block* block, BlockType block_type,
DataBlockIter* input_iter, bool block_contents_pinned) {
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
return block->NewDataIterator(rep->internal_comparator.user_comparator(),
rep->get_global_seqno(block_type), input_iter,
Add support to strip / pad timestamp when creating / reading a block based table (#11495) Summary: Add support to strip timestamp in block based table builder and pad timestamp in block based table reader. On the write path, use the per column family option `AdvancedColumnFamilyOptions.persist_user_defined_timestamps` to indicate whether user-defined timestamps should be stripped for all block based tables created for the column family. On the read path, added a per table `TableReadOption.user_defined_timestamps_persisted` to flag whether the user keys in the table contains user defined timestamps. This patch is mostly passing the related flags down to the block building/parsing level with the exception of handling the `first_internal_key` in `IndexValue`, which is included in the `IndexBuilder` level. The value part of range deletion entries should have a similar handling, I haven't decided where to best fit this piece of logic, I will do it in a follow up. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11495 Test Plan: Existing test `BlockBasedTableReaderTest` is parameterized to run with: 1) different UDT test modes: kNone, kNormal, kStripUserDefinedTimestamp 2) all four index types, when index type is `kTwoLevelIndexSearch`, also enables partitioned filters 3) parallel vs non-parallel compression 4) enable/disable compression dictionary. Also added tests for API `BlockBasedTableReader::NewIterator`. `PartitionedFilterBlockTest` is parameterized to run with different UDT test modes:kNone, kNormal, kStripUserDefinedTimestamp. ``` make all check ./block_based_table_reader_test ./partitioned_filter_block_test ``` Reviewed By: ltamasi Differential Revision: D46344577 Pulled By: jowlyzhang fbshipit-source-id: 93ac8542b19319d1298712b8bed908c8831ba675
1 year ago
rep->ioptions.stats, block_contents_pinned,
rep->user_defined_timestamps_persisted);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
}
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
// TODO?
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
template <>
IndexBlockIter* BlockBasedTable::InitBlockIterator<IndexBlockIter>(
const Rep* rep, Block* block, BlockType block_type,
IndexBlockIter* input_iter, bool block_contents_pinned) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
return block->NewIndexIterator(
Separate internal and user key comparators in `BlockIter` (#6944) Summary: Replace `BlockIter::comparator_` and `IndexBlockIter::user_comparator_wrapper_` with a concrete `UserComparatorWrapper` and `InternalKeyComparator`. The motivation for this change was the inconvenience of not knowing the concrete type of `BlockIter::comparator_`, which prevented calling specialized internal key comparison functions to optimize comparison of keys with global seqno applied. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6944 Test Plan: benchmark setup -- single file DBs, in-memory, no compression. "normal_db" created by regular flush; "ingestion_db" created by ingesting a file. Both DBs have same contents. ``` $ TEST_TMPDIR=/dev/shm/normal_db/ ./db_bench -benchmarks=fillrandom,compact -write_buffer_size=10485760000 -disable_auto_compactions=true -compression_type=none -num=1000000 $ ./ldb write_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ --compression_type=no --hex --create_if_missing < <(./sst_dump --command=scan --output_hex --file=/dev/shm/normal_db/dbbench/000007.sst | awk 'began {print "0x" substr($1, 2, length($1) - 2), "==>", "0x" $5} ; /^Sst file format: block-based/ {began=1}') $ ./ldb ingest_extern_sst ./tmp.sst --db=/dev/shm/ingestion_db/dbbench/ ``` benchmark run command: ``` $ TEST_TMPDIR=/dev/shm/$DB/ ./db_bench -benchmarks=seekrandom -seek_nexts=$SEEK_NEXT -use_existing_db=true -cache_index_and_filter_blocks=false -num=1000000 -cache_size=0 -threads=1 -reads=200000000 -mmap_read=1 -verify_checksum=false ``` results: perf improved marginally for ingestion_db and did not change significantly for normal_db: SEEK_NEXT | DB | code | ops/sec | % change -- | -- | -- | -- | -- 0 | normal_db | master | 350880 |   0 | normal_db | PR6944 | 351040 | 0.0 0 | ingestion_db | master | 343255 |   0 | ingestion_db | PR6944 | 349424 | 1.8 10 | normal_db | master | 218711 |   10 | normal_db | PR6944 | 217892 | -0.4 10 | ingestion_db | master | 220334 |   10 | ingestion_db | PR6944 | 226437 | 2.8 Reviewed By: pdillinger Differential Revision: D21924676 Pulled By: ajkr fbshipit-source-id: ea4288a2eefa8112eb6c651a671c1de18c12e538
4 years ago
rep->internal_comparator.user_comparator(),
rep->get_global_seqno(block_type), input_iter, rep->ioptions.stats,
/* total_order_seek */ true, rep->index_has_first_key,
rep->index_key_includes_seq, rep->index_value_is_full,
Add support to strip / pad timestamp when creating / reading a block based table (#11495) Summary: Add support to strip timestamp in block based table builder and pad timestamp in block based table reader. On the write path, use the per column family option `AdvancedColumnFamilyOptions.persist_user_defined_timestamps` to indicate whether user-defined timestamps should be stripped for all block based tables created for the column family. On the read path, added a per table `TableReadOption.user_defined_timestamps_persisted` to flag whether the user keys in the table contains user defined timestamps. This patch is mostly passing the related flags down to the block building/parsing level with the exception of handling the `first_internal_key` in `IndexValue`, which is included in the `IndexBuilder` level. The value part of range deletion entries should have a similar handling, I haven't decided where to best fit this piece of logic, I will do it in a follow up. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11495 Test Plan: Existing test `BlockBasedTableReaderTest` is parameterized to run with: 1) different UDT test modes: kNone, kNormal, kStripUserDefinedTimestamp 2) all four index types, when index type is `kTwoLevelIndexSearch`, also enables partitioned filters 3) parallel vs non-parallel compression 4) enable/disable compression dictionary. Also added tests for API `BlockBasedTableReader::NewIterator`. `PartitionedFilterBlockTest` is parameterized to run with different UDT test modes:kNone, kNormal, kStripUserDefinedTimestamp. ``` make all check ./block_based_table_reader_test ./partitioned_filter_block_test ``` Reviewed By: ltamasi Differential Revision: D46344577 Pulled By: jowlyzhang fbshipit-source-id: 93ac8542b19319d1298712b8bed908c8831ba675
1 year ago
block_contents_pinned, rep->user_defined_timestamps_persisted);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
}
// If contents is nullptr, this function looks up the block caches for the
// data block referenced by handle, and read the block from disk if necessary.
// If contents is non-null, it skips the cache lookup and disk read, since
// the caller has already read it. In both cases, if ro.fill_cache is true,
// it inserts the block into the block cache.
template <typename TBlocklike>
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
WithBlocklikeCheck<Status, TBlocklike>
BlockBasedTable::MaybeReadBlockAndLoadToCache(
FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
const BlockHandle& handle, const UncompressionDict& uncompression_dict,
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
bool for_compaction, CachableEntry<TBlocklike>* out_parsed_block,
GetContext* get_context, BlockCacheLookupContext* lookup_context,
BlockContents* contents, bool async_read) const {
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
assert(out_parsed_block != nullptr);
const bool no_io = (ro.read_tier == kBlockCacheTier);
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
BlockCacheInterface<TBlocklike> block_cache{
rep_->table_options.block_cache.get()};
// First, try to get the block from the cache
//
// If either block cache is enabled, we'll try to read from it.
Status s;
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
CacheKey key_data;
Slice key;
bool is_cache_hit = false;
if (block_cache) {
// create key for block cache
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
key_data = GetCacheKey(rep_->base_cache_key, handle);
key = key_data.AsSlice();
if (!contents) {
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
s = GetDataBlockFromCache(key, block_cache, out_parsed_block,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
get_context);
Parallelize secondary cache lookup in MultiGet (#8405) Summary: Implement the ```WaitAll()``` interface in ```LRUCache``` to allow callers to issue multiple lookups in parallel and wait for all of them to complete. Modify ```MultiGet``` to use this to parallelize the secondary cache lookups in order to reduce the overall latency. A call to ```cache->Lookup()``` returns a handle that has an incomplete value (nullptr), and the caller can call ```cache->IsReady()``` to check whether the lookup is complete, and pass a vector of handles to ```WaitAll``` to wait for completion. If any of the lookups fail, ```MultiGet``` will read the block from the SST file. Another change in this PR is to rename ```SecondaryCacheHandle``` to ```SecondaryCacheResultHandle``` as it more accurately describes the return result of the secondary cache lookup, which is more like a future. Tests: 1. Add unit tests in lru_cache_test 2. Benchmark results with no secondary cache configured Master - ``` readrandom : 41.175 micros/op 388562 ops/sec; 106.7 MB/s (7277999 of 7277999 found) readrandom : 41.217 micros/op 388160 ops/sec; 106.6 MB/s (7274999 of 7274999 found) multireadrandom : 10.309 micros/op 1552082 ops/sec; (28908992 of 28908992 found) multireadrandom : 10.321 micros/op 1550218 ops/sec; (29081984 of 29081984 found) ``` This PR - ``` readrandom : 41.158 micros/op 388723 ops/sec; 106.8 MB/s (7290999 of 7290999 found) readrandom : 41.185 micros/op 388463 ops/sec; 106.7 MB/s (7287999 of 7287999 found) multireadrandom : 10.277 micros/op 1556801 ops/sec; (29346944 of 29346944 found) multireadrandom : 10.253 micros/op 1560539 ops/sec; (29274944 of 29274944 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/8405 Reviewed By: zhichao-cao Differential Revision: D29190509 Pulled By: anand1976 fbshipit-source-id: 6f8eff6246712af8a297cfe22ea0d1c3b2a01bb0
3 years ago
// Value could still be null at this point, so check the cache handle
// and update the read pattern for prefetching
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
if (out_parsed_block->GetValue() || out_parsed_block->GetCacheHandle()) {
// TODO(haoyu): Differentiate cache hit on uncompressed block cache and
// compressed block cache.
is_cache_hit = true;
if (prefetch_buffer) {
// Update the block details so that PrefetchBuffer can use the read
// pattern to determine if reads are sequential or not for
// prefetching. It should also take in account blocks read from cache.
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
3 years ago
prefetch_buffer->UpdateReadPattern(
handle.offset(), BlockSizeWithTrailer(handle),
ro.adaptive_readahead /*decrease_readahead_size*/);
}
}
}
// Can't find the block from the cache. If I/O is allowed, read from the
// file.
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
if (out_parsed_block->GetValue() == nullptr &&
out_parsed_block->GetCacheHandle() == nullptr && !no_io &&
ro.fill_cache) {
Statistics* statistics = rep_->ioptions.stats;
const bool maybe_compressed =
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
TBlocklike::kBlockType != BlockType::kFilter &&
TBlocklike::kBlockType != BlockType::kCompressionDictionary &&
rep_->blocks_maybe_compressed;
const bool do_uncompress = maybe_compressed;
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
CompressionType contents_comp_type;
// Maybe serialized or uncompressed
BlockContents tmp_contents;
if (!contents) {
Histograms histogram = for_compaction ? READ_BLOCK_COMPACTION_MICROS
: READ_BLOCK_GET_MICROS;
StopWatch sw(rep_->ioptions.clock, statistics, histogram);
BlockFetcher block_fetcher(
rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle,
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
&tmp_contents, rep_->ioptions, do_uncompress, maybe_compressed,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
TBlocklike::kBlockType, uncompression_dict,
rep_->persistent_cache_options,
GetMemoryAllocator(rep_->table_options),
/*allocator=*/nullptr);
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
// If prefetch_buffer is not allocated, it will fallback to synchronous
// reading of block contents.
if (async_read && prefetch_buffer != nullptr) {
s = block_fetcher.ReadAsyncBlockContents();
if (!s.ok()) {
return s;
}
} else {
s = block_fetcher.ReadBlockContents();
}
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
contents_comp_type = block_fetcher.get_compression_type();
contents = &tmp_contents;
if (get_context) {
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
switch (TBlocklike::kBlockType) {
case BlockType::kIndex:
++get_context->get_context_stats_.num_index_read;
break;
case BlockType::kFilter:
case BlockType::kFilterPartitionIndex:
++get_context->get_context_stats_.num_filter_read;
break;
default:
break;
}
}
} else {
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
contents_comp_type = GetBlockCompressionType(*contents);
}
if (s.ok()) {
// If filling cache is allowed and a cache is configured, try to put the
// block to the cache.
s = PutDataBlockToCache(
key, block_cache, out_parsed_block, std::move(*contents),
contents_comp_type, uncompression_dict,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
GetMemoryAllocator(rep_->table_options), get_context);
}
}
}
Refactor block cache tracing w/improved MultiGet (#11339) Summary: After https://github.com/facebook/rocksdb/issues/11301, I wasn't sure whether I had regressed block cache tracing with MultiGet. Demo PR https://github.com/facebook/rocksdb/issues/11330 shows the flawed state of tracing MultiGet before my change, and based on the unit test, there was essentially no change in tracing behavior with https://github.com/facebook/rocksdb/issues/11301. This change is to leave that code and behavior better than I found it. This change is not intended to change any production behaviors except when block cache tracing is active, though might improve general read path efficiency by disabling some related tracking when such tracing is disabled. More detail on production code: * Refactoring to consolidate the construction of BlockCacheTraceRecord, and other related functionality, in block-based table reader, though it's somewhat awkward to preserve an optimization to avoid copying Slices into temporary strings in BlockCacheLookupContext. * Accurately track cache hits and misses (etc.) for each data block accessed by a MultiGet(). (Previously reported hits as misses.) * Reduced repeated checking of `block_cache_tracer_` state (by creating lookup_context only when active) for efficiency and to reduce the risk of corner case bugs where tracing is enabled or disabled for different parts of a read op. (See a TODO below) * Improved estimate calculation for num_keys_in_block (see code comment) Possible follow-up: * `XXX:` use_cache=true means double cache query? (possible double-query of block cache when allow_mmap_reads=true) * `TODO:` need more than one lookup_context here to track individual filter and index partition hits and misses * `TODO:` optimize more state checks of `block_cache_tracer_` down to `lookup_context != nullptr` * Pre-existing `XXX:` There appear to be 'break' statements above that bypass this writing of the block cache trace record * Expand test coverage (see below) Pull Request resolved: https://github.com/facebook/rocksdb/pull/11339 Test Plan: * Added a basic unit test for block cache tracing MultiGet, for now just covering one data block with two keys. * Added HitMissCountingCache to independently verify that the actual block cache trace and expected block cache trace also agree with the actual number of cache hits / misses (nothing missing or mislabeled). For now only used with MultiGet test. * Better testing of num_keys_in_block, for now just with MultiGet * Misc improvements to table_test to improve clarity, such as making it clear that certain keys are auto-inserted at the start of every test. Performance test: Testing multireadrandom as in https://github.com/facebook/rocksdb/issues/11301, except averaging over distinct runs rather than [-X30] which doesn't seem to sufficiently reset after each run to work as an independent test run. Base with revert of 11301: 3148926 ops/sec Base: 3019146 ops/sec New: 2999529 ops/sec Possibly a tiny MultiGet CPU regression with this change. We are now always allocating an additional vector for the LookupContexts. I'm still contemplating options to try to correct the regression in https://github.com/facebook/rocksdb/issues/11301. Testing readrandom: Base with revert of 11301: 2311988 Base: 2281726 New: 2299722 Possibly a tiny Get CPU improvement with this change. We are now avoiding some unnecessary LookupContext population. Reviewed By: akankshamahajan15 Differential Revision: D44557845 Pulled By: pdillinger fbshipit-source-id: b841691799d2a48fb59cc8880dc7cbb1e107ae3d
2 years ago
// TODO: optimize so that lookup_context != nullptr implies the others
if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() &&
lookup_context) {
Refactor block cache tracing w/improved MultiGet (#11339) Summary: After https://github.com/facebook/rocksdb/issues/11301, I wasn't sure whether I had regressed block cache tracing with MultiGet. Demo PR https://github.com/facebook/rocksdb/issues/11330 shows the flawed state of tracing MultiGet before my change, and based on the unit test, there was essentially no change in tracing behavior with https://github.com/facebook/rocksdb/issues/11301. This change is to leave that code and behavior better than I found it. This change is not intended to change any production behaviors except when block cache tracing is active, though might improve general read path efficiency by disabling some related tracking when such tracing is disabled. More detail on production code: * Refactoring to consolidate the construction of BlockCacheTraceRecord, and other related functionality, in block-based table reader, though it's somewhat awkward to preserve an optimization to avoid copying Slices into temporary strings in BlockCacheLookupContext. * Accurately track cache hits and misses (etc.) for each data block accessed by a MultiGet(). (Previously reported hits as misses.) * Reduced repeated checking of `block_cache_tracer_` state (by creating lookup_context only when active) for efficiency and to reduce the risk of corner case bugs where tracing is enabled or disabled for different parts of a read op. (See a TODO below) * Improved estimate calculation for num_keys_in_block (see code comment) Possible follow-up: * `XXX:` use_cache=true means double cache query? (possible double-query of block cache when allow_mmap_reads=true) * `TODO:` need more than one lookup_context here to track individual filter and index partition hits and misses * `TODO:` optimize more state checks of `block_cache_tracer_` down to `lookup_context != nullptr` * Pre-existing `XXX:` There appear to be 'break' statements above that bypass this writing of the block cache trace record * Expand test coverage (see below) Pull Request resolved: https://github.com/facebook/rocksdb/pull/11339 Test Plan: * Added a basic unit test for block cache tracing MultiGet, for now just covering one data block with two keys. * Added HitMissCountingCache to independently verify that the actual block cache trace and expected block cache trace also agree with the actual number of cache hits / misses (nothing missing or mislabeled). For now only used with MultiGet test. * Better testing of num_keys_in_block, for now just with MultiGet * Misc improvements to table_test to improve clarity, such as making it clear that certain keys are auto-inserted at the start of every test. Performance test: Testing multireadrandom as in https://github.com/facebook/rocksdb/issues/11301, except averaging over distinct runs rather than [-X30] which doesn't seem to sufficiently reset after each run to work as an independent test run. Base with revert of 11301: 3148926 ops/sec Base: 3019146 ops/sec New: 2999529 ops/sec Possibly a tiny MultiGet CPU regression with this change. We are now always allocating an additional vector for the LookupContexts. I'm still contemplating options to try to correct the regression in https://github.com/facebook/rocksdb/issues/11301. Testing readrandom: Base with revert of 11301: 2311988 Base: 2281726 New: 2299722 Possibly a tiny Get CPU improvement with this change. We are now avoiding some unnecessary LookupContext population. Reviewed By: akankshamahajan15 Differential Revision: D44557845 Pulled By: pdillinger fbshipit-source-id: b841691799d2a48fb59cc8880dc7cbb1e107ae3d
2 years ago
SaveLookupContextOrTraceRecord(
key, is_cache_hit, ro, out_parsed_block->GetValue(), lookup_context);
}
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
assert(s.ok() || out_parsed_block->GetValue() == nullptr);
return s;
}
Refactor block cache tracing w/improved MultiGet (#11339) Summary: After https://github.com/facebook/rocksdb/issues/11301, I wasn't sure whether I had regressed block cache tracing with MultiGet. Demo PR https://github.com/facebook/rocksdb/issues/11330 shows the flawed state of tracing MultiGet before my change, and based on the unit test, there was essentially no change in tracing behavior with https://github.com/facebook/rocksdb/issues/11301. This change is to leave that code and behavior better than I found it. This change is not intended to change any production behaviors except when block cache tracing is active, though might improve general read path efficiency by disabling some related tracking when such tracing is disabled. More detail on production code: * Refactoring to consolidate the construction of BlockCacheTraceRecord, and other related functionality, in block-based table reader, though it's somewhat awkward to preserve an optimization to avoid copying Slices into temporary strings in BlockCacheLookupContext. * Accurately track cache hits and misses (etc.) for each data block accessed by a MultiGet(). (Previously reported hits as misses.) * Reduced repeated checking of `block_cache_tracer_` state (by creating lookup_context only when active) for efficiency and to reduce the risk of corner case bugs where tracing is enabled or disabled for different parts of a read op. (See a TODO below) * Improved estimate calculation for num_keys_in_block (see code comment) Possible follow-up: * `XXX:` use_cache=true means double cache query? (possible double-query of block cache when allow_mmap_reads=true) * `TODO:` need more than one lookup_context here to track individual filter and index partition hits and misses * `TODO:` optimize more state checks of `block_cache_tracer_` down to `lookup_context != nullptr` * Pre-existing `XXX:` There appear to be 'break' statements above that bypass this writing of the block cache trace record * Expand test coverage (see below) Pull Request resolved: https://github.com/facebook/rocksdb/pull/11339 Test Plan: * Added a basic unit test for block cache tracing MultiGet, for now just covering one data block with two keys. * Added HitMissCountingCache to independently verify that the actual block cache trace and expected block cache trace also agree with the actual number of cache hits / misses (nothing missing or mislabeled). For now only used with MultiGet test. * Better testing of num_keys_in_block, for now just with MultiGet * Misc improvements to table_test to improve clarity, such as making it clear that certain keys are auto-inserted at the start of every test. Performance test: Testing multireadrandom as in https://github.com/facebook/rocksdb/issues/11301, except averaging over distinct runs rather than [-X30] which doesn't seem to sufficiently reset after each run to work as an independent test run. Base with revert of 11301: 3148926 ops/sec Base: 3019146 ops/sec New: 2999529 ops/sec Possibly a tiny MultiGet CPU regression with this change. We are now always allocating an additional vector for the LookupContexts. I'm still contemplating options to try to correct the regression in https://github.com/facebook/rocksdb/issues/11301. Testing readrandom: Base with revert of 11301: 2311988 Base: 2281726 New: 2299722 Possibly a tiny Get CPU improvement with this change. We are now avoiding some unnecessary LookupContext population. Reviewed By: akankshamahajan15 Differential Revision: D44557845 Pulled By: pdillinger fbshipit-source-id: b841691799d2a48fb59cc8880dc7cbb1e107ae3d
2 years ago
template <typename TBlocklike>
WithBlocklikeCheck<void, TBlocklike>
BlockBasedTable::SaveLookupContextOrTraceRecord(
const Slice& block_key, bool is_cache_hit, const ReadOptions& ro,
const TBlocklike* parsed_block_value,
BlockCacheLookupContext* lookup_context) const {
assert(lookup_context);
size_t usage = 0;
uint64_t nkeys = 0;
if (parsed_block_value) {
// Approximate the number of keys in the block using restarts.
int interval = rep_->table_options.block_restart_interval;
nkeys = interval * GetBlockNumRestarts(*parsed_block_value);
// On average, the last restart should be just over half utilized.
// Specifically, 1..N should be N/2 + 0.5. For example, 7 -> 4, 8 -> 4.5.
// Use the get_id to alternate between rounding up vs. down.
if (nkeys > 0) {
bool rounding = static_cast<int>(lookup_context->get_id) & 1;
nkeys -= (interval - rounding) / 2;
}
usage = parsed_block_value->ApproximateMemoryUsage();
}
TraceType trace_block_type = TraceType::kTraceMax;
switch (TBlocklike::kBlockType) {
case BlockType::kData:
trace_block_type = TraceType::kBlockTraceDataBlock;
break;
case BlockType::kFilter:
case BlockType::kFilterPartitionIndex:
trace_block_type = TraceType::kBlockTraceFilterBlock;
break;
case BlockType::kCompressionDictionary:
trace_block_type = TraceType::kBlockTraceUncompressionDictBlock;
break;
case BlockType::kRangeDeletion:
trace_block_type = TraceType::kBlockTraceRangeDeletionBlock;
break;
case BlockType::kIndex:
trace_block_type = TraceType::kBlockTraceIndexBlock;
break;
default:
// This cannot happen.
assert(false);
break;
}
const bool no_io = ro.read_tier == kBlockCacheTier;
bool no_insert = no_io || !ro.fill_cache;
if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock(
trace_block_type, lookup_context->caller)) {
// Make a copy of the block key here since it will be logged later.
lookup_context->FillLookupContext(is_cache_hit, no_insert, trace_block_type,
/*block_size=*/usage,
block_key.ToString(), nkeys);
// Defer logging the access to Get() and MultiGet() to trace additional
// information, e.g., referenced_key
} else {
// Avoid making copy of block_key if it doesn't need to be saved in
// BlockCacheLookupContext
lookup_context->FillLookupContext(is_cache_hit, no_insert, trace_block_type,
/*block_size=*/usage,
/*block_key=*/{}, nkeys);
// Fill in default values for irrelevant/unknown fields
FinishTraceRecord(*lookup_context, block_key,
lookup_context->referenced_key,
/*does_referenced_key_exist*/ false,
/*referenced_data_size*/ 0);
}
}
void BlockBasedTable::FinishTraceRecord(
const BlockCacheLookupContext& lookup_context, const Slice& block_key,
const Slice& referenced_key, bool does_referenced_key_exist,
uint64_t referenced_data_size) const {
// Avoid making copy of referenced_key if it doesn't need to be saved in
// BlockCacheLookupContext
BlockCacheTraceRecord access_record(
rep_->ioptions.clock->NowMicros(),
/*block_key=*/"", lookup_context.block_type, lookup_context.block_size,
rep_->cf_id_for_tracing(),
/*cf_name=*/"", rep_->level_for_tracing(), rep_->sst_number_for_tracing(),
lookup_context.caller, lookup_context.is_cache_hit,
lookup_context.no_insert, lookup_context.get_id,
lookup_context.get_from_user_specified_snapshot,
/*referenced_key=*/"", referenced_data_size,
lookup_context.num_keys_in_block, does_referenced_key_exist);
// TODO: Should handle status here?
block_cache_tracer_
->WriteBlockAccess(access_record, block_key, rep_->cf_name_for_tracing(),
referenced_key)
.PermitUncheckedError();
}
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
template <typename TBlocklike /*, auto*/>
WithBlocklikeCheck<Status, TBlocklike> BlockBasedTable::RetrieveBlock(
FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
const BlockHandle& handle, const UncompressionDict& uncompression_dict,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
CachableEntry<TBlocklike>* out_parsed_block, GetContext* get_context,
BlockCacheLookupContext* lookup_context, bool for_compaction,
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
bool use_cache, bool async_read) const {
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
assert(out_parsed_block);
assert(out_parsed_block->IsEmpty());
Status s;
Fix regression affecting partitioned indexes/filters when cache_index_and_filter_blocks is false (#5705) Summary: PR https://github.com/facebook/rocksdb/issues/5298 (and subsequent related patches) unintentionally changed the semantics of cache_index_and_filter_blocks: historically, this option only affected the main index/filter block; with the changes, it affects index/filter partitions as well. This can cause performance issues when cache_index_and_filter_blocks is false since in this case, partitions are neither cached nor preloaded (i.e. they are loaded on demand upon each access). The patch reverts to the earlier behavior, that is, partitions are cached similarly to data blocks regardless of the value of the above option. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5705 Test Plan: make check ./db_bench -benchmarks=fillrandom --statistics --stats_interval_seconds=1 --duration=30 --num=500000000 --bloom_bits=20 --partition_index_and_filters=true --cache_index_and_filter_blocks=false ./db_bench -benchmarks=readrandom --use_existing_db --statistics --stats_interval_seconds=1 --duration=10 --num=500000000 --bloom_bits=20 --partition_index_and_filters=true --cache_index_and_filter_blocks=false --cache_size=8000000000 Relevant statistics from the readrandom benchmark with the old code: rocksdb.block.cache.index.miss COUNT : 0 rocksdb.block.cache.index.hit COUNT : 0 rocksdb.block.cache.index.add COUNT : 0 rocksdb.block.cache.index.bytes.insert COUNT : 0 rocksdb.block.cache.index.bytes.evict COUNT : 0 rocksdb.block.cache.filter.miss COUNT : 0 rocksdb.block.cache.filter.hit COUNT : 0 rocksdb.block.cache.filter.add COUNT : 0 rocksdb.block.cache.filter.bytes.insert COUNT : 0 rocksdb.block.cache.filter.bytes.evict COUNT : 0 With the new code: rocksdb.block.cache.index.miss COUNT : 2500 rocksdb.block.cache.index.hit COUNT : 42696 rocksdb.block.cache.index.add COUNT : 2500 rocksdb.block.cache.index.bytes.insert COUNT : 4050048 rocksdb.block.cache.index.bytes.evict COUNT : 0 rocksdb.block.cache.filter.miss COUNT : 2500 rocksdb.block.cache.filter.hit COUNT : 4550493 rocksdb.block.cache.filter.add COUNT : 2500 rocksdb.block.cache.filter.bytes.insert COUNT : 10331040 rocksdb.block.cache.filter.bytes.evict COUNT : 0 Differential Revision: D16817382 Pulled By: ltamasi fbshipit-source-id: 28a516b0da1f041a03313e0b70b28cf5cf205d00
5 years ago
if (use_cache) {
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
s = MaybeReadBlockAndLoadToCache(
HyperClockCache support for SecondaryCache, with refactoring (#11301) Summary: Internally refactors SecondaryCache integration out of LRUCache specifically and into a wrapper/adapter class that works with various Cache implementations. Notably, this relies on separating the notion of async lookup handles from other cache handles, so that HyperClockCache doesn't have to deal with the problem of allocating handles from the hash table for lookups that might fail anyway, and might be on the same key without support for coalescing. (LRUCache's hash table can incorporate previously allocated handles thanks to its pointer indirection.) Specifically, I'm worried about the case in which hundreds of threads try to access the same block and probing in the hash table degrades to linear search on the pile of entries with the same key. This change is a big step in the direction of supporting stacked SecondaryCaches, but there are obstacles to completing that. Especially, there is no SecondaryCache hook for evictions to pass from one to the next. It has been proposed that evictions be transmitted simply as the persisted data (as in SaveToCallback), but given the current structure provided by the CacheItemHelpers, that would require an extra copy of the block data, because there's intentionally no way to ask for a contiguous Slice of the data (to allow for flexibility in storage). `AsyncLookupHandle` and the re-worked `WaitAll()` should be essentially prepared for stacked SecondaryCaches, but several "TODO with stacked secondaries" issues remain in various places. It could be argued that the stacking instead be done as a SecondaryCache adapter that wraps two (or more) SecondaryCaches, but at least with the current API that would require an extra heap allocation on SecondaryCache Lookup for a wrapper SecondaryCacheResultHandle that can transfer a Lookup between secondaries. We could also consider trying to unify the Cache and SecondaryCache APIs, though that might be difficult if `AsyncLookupHandle` is kept a fixed struct. ## cache.h (public API) Moves `secondary_cache` option from LRUCacheOptions to ShardedCacheOptions so that it is applicable to HyperClockCache. ## advanced_cache.h (advanced public API) * Add `Cache::CreateStandalone()` so that the SecondaryCache support wrapper can use it. * Add `SetEvictionCallback()` / `eviction_callback_` so that the SecondaryCache support wrapper can use it. Only a single callback is supported for efficiency. If there is ever a need for more than one, hopefully that can be handled with a broadcast callback wrapper. These are essentially the two "extra" pieces of `Cache` for pulling out specific SecondaryCache support from the `Cache` implementation. I think it's a good trade-off as these are reasonable, limited, and reusable "cut points" into the `Cache` implementations. * Remove async capability from standard `Lookup()` (getting rid of awkward restrictions on pending Handles) and add `AsyncLookupHandle` and `StartAsyncLookup()`. As noted in the comments, the full struct of `AsyncLookupHandle` is exposed so that it can be stack allocated, for efficiency, though more data is being copied around than before, which could impact performance. (Lookup info -> AsyncLookupHandle -> Handle vs. Lookup info -> Handle) I could foresee a future in which a Cache internally saves a pointer to the AsyncLookupHandle, which means it's dangerous to allow it to be copyable or even movable. It also means it's not compatible with std::vector (which I don't like requiring as an API parameter anyway), so `WaitAll()` expects any contiguous array of AsyncLookupHandles. I believe this is best for common case efficiency, while behaving well in other cases also. For example, `WaitAll()` has no effect on default-constructed AsyncLookupHandles, which look like a completed cache miss. ## cacheable_entry.h A couple of functions are obsolete because Cache::Handle can no longer be pending. ## cache.cc Provides default implementations for new or revamped Cache functions, especially appropriate for non-blocking caches. ## secondary_cache_adapter.{h,cc} The full details of the Cache wrapper adding SecondaryCache support. Essentially replicates the SecondaryCache handling that was in LRUCache, but obviously refactored. There is a bit of logic duplication, where Lookup() is essentially a manually optimized version of StartAsyncLookup() and Wait(), but it's roughly a dozen lines of code. ## sharded_cache.h, typed_cache.h, charged_cache.{h,cc}, sim_cache.cc Simply updated for Cache API changes. ## lru_cache.{h,cc} Carefully remove SecondaryCache logic, implement `CreateStandalone` and eviction handler functionality. ## clock_cache.{h,cc} Expose existing `CreateStandalone` functionality, add eviction handler functionality. Light refactoring. ## block_based_table_reader* Mostly re-worked the only usage of async Lookup, which is in BlockBasedTable::MultiGet. Used arrays in place of autovector in some places for efficiency. Simplified some logic by not trying to process some cache results before they're all ready. Created new function `BlockBasedTable::GetCachePriority()` to reduce some pre-existing code duplication (and avoid making it worse). Fixed at least one small bug from the prior confusing mixture of async and sync Lookups. In MaybeReadBlockAndLoadToCache(), called by RetrieveBlock(), called by MultiGet() with wait=false, is_cache_hit for the block_cache_tracer entry would not be set to true if the handle was pending after Lookup and before Wait. ## Intended follow-up work * Figure out if there are any missing stats or block_cache_tracer work in refactored BlockBasedTable::MultiGet * Stacked secondary caches (see above discussion) * See if we can make up for the small MultiGet performance regression. * Study more performance with SecondaryCache * Items evicted from over-full LRUCache in Release were not being demoted to SecondaryCache, and still aren't to minimize unit test churn. Ideally they would be demoted, but it's an exceptional case so not a big deal. * Use CreateStandalone for cache reservations (save unnecessary hash table operations). Not a big deal, but worthy cleanup. * Somehow I got the contract for SecondaryCache::Insert wrong in #10945. (Doesn't take ownership!) That API comment needs to be fixed, but didn't want to mingle that in here. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11301 Test Plan: ## Unit tests Generally updated to include HCC in SecondaryCache tests, though HyperClockCache has some different, less strict behaviors that leads to some tests not really being set up to work with it. Some of the tests remain disabled with it, but I think we have good coverage without them. ## Crash/stress test Updated to use the new combination. ## Performance First, let's check for regression on caches without secondary cache configured. Adding support for the eviction callback is likely to have a tiny effect, but it shouldn't be worrisome. LRUCache could benefit slightly from less logic around SecondaryCache handling. We can test with cache_bench default settings, built with DEBUG_LEVEL=0 and PORTABLE=0. ``` (while :; do base/cache_bench --cache_type=hyper_clock_cache | grep Rough; done) | awk '{ sum += $9; count++; print $0; print "Average: " int(sum / count) }' ``` **Before** this and #11299 (which could also have a small effect), running for about an hour, before & after running concurrently for each cache type: HyperClockCache: 3168662 (average parallel ops/sec) LRUCache: 2940127 **After** this and #11299, running for about an hour: HyperClockCache: 3164862 (average parallel ops/sec) (0.12% slower) LRUCache: 2940928 (0.03% faster) This is an acceptable difference IMHO. Next, let's consider essentially the worst case of new CPU overhead affecting overall performance. MultiGet uses the async lookup interface regardless of whether SecondaryCache or folly are used. We can configure a benchmark where all block cache queries are for data blocks, and all are hits. Create DB and test (before and after tests running simultaneously): ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=30000000 -disable_wal=1 -bloom_bits=16 TEST_TMPDIR=/dev/shm base/db_bench -benchmarks=multireadrandom[-X30] -readonly -multiread_batched -batch_size=32 -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: multireadrandom [AVG 30 runs] : 3444202 (± 57049) ops/sec; 240.9 (± 4.0) MB/sec multireadrandom [MEDIAN 30 runs] : 3514443 ops/sec; 245.8 MB/sec **After**: multireadrandom [AVG 30 runs] : 3291022 (± 58851) ops/sec; 230.2 (± 4.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3366179 ops/sec; 235.4 MB/sec So that's roughly a 3% regression, on kind of a *worst case* test of MultiGet CPU. Similar story with HyperClockCache: **Before**: multireadrandom [AVG 30 runs] : 3933777 (± 41840) ops/sec; 275.1 (± 2.9) MB/sec multireadrandom [MEDIAN 30 runs] : 3970667 ops/sec; 277.7 MB/sec **After**: multireadrandom [AVG 30 runs] : 3755338 (± 30391) ops/sec; 262.6 (± 2.1) MB/sec multireadrandom [MEDIAN 30 runs] : 3785696 ops/sec; 264.8 MB/sec Roughly a 4-5% regression. Not ideal, but not the whole story, fortunately. Let's also look at Get() in db_bench: ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=readrandom[-X30] -readonly -num=30000000 -bloom_bits=16 -cache_size=6789000000 -duration 20 -threads=16 ``` **Before**: readrandom [AVG 30 runs] : 2198685 (± 13412) ops/sec; 153.8 (± 0.9) MB/sec readrandom [MEDIAN 30 runs] : 2209498 ops/sec; 154.5 MB/sec **After**: readrandom [AVG 30 runs] : 2292814 (± 43508) ops/sec; 160.3 (± 3.0) MB/sec readrandom [MEDIAN 30 runs] : 2365181 ops/sec; 165.4 MB/sec That's showing roughly a 4% improvement, perhaps because of the secondary cache code that is no longer part of LRUCache. But weirdly, HyperClockCache is also showing 2-3% improvement: **Before**: readrandom [AVG 30 runs] : 2272333 (± 9992) ops/sec; 158.9 (± 0.7) MB/sec readrandom [MEDIAN 30 runs] : 2273239 ops/sec; 159.0 MB/sec **After**: readrandom [AVG 30 runs] : 2332407 (± 11252) ops/sec; 163.1 (± 0.8) MB/sec readrandom [MEDIAN 30 runs] : 2335329 ops/sec; 163.3 MB/sec Reviewed By: ltamasi Differential Revision: D44177044 Pulled By: pdillinger fbshipit-source-id: e808e48ff3fe2f792a79841ba617be98e48689f5
2 years ago
prefetch_buffer, ro, handle, uncompression_dict, for_compaction,
out_parsed_block, get_context, lookup_context,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
/*contents=*/nullptr, async_read);
if (!s.ok()) {
return s;
}
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
if (out_parsed_block->GetValue() != nullptr ||
out_parsed_block->GetCacheHandle() != nullptr) {
assert(s.ok());
return s;
}
}
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
assert(out_parsed_block->IsEmpty());
const bool no_io = ro.read_tier == kBlockCacheTier;
if (no_io) {
return Status::Incomplete("no blocking io");
}
const bool maybe_compressed =
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
TBlocklike::kBlockType != BlockType::kFilter &&
TBlocklike::kBlockType != BlockType::kCompressionDictionary &&
rep_->blocks_maybe_compressed;
std::unique_ptr<TBlocklike> block;
{
Histograms histogram =
for_compaction ? READ_BLOCK_COMPACTION_MICROS : READ_BLOCK_GET_MICROS;
StopWatch sw(rep_->ioptions.clock, rep_->ioptions.stats, histogram);
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
s = ReadAndParseBlockFromFile(
rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, &block,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
rep_->ioptions, rep_->create_context, maybe_compressed,
uncompression_dict, rep_->persistent_cache_options,
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
GetMemoryAllocator(rep_->table_options), for_compaction, async_read);
if (get_context) {
Major Cache refactoring, CPU efficiency improvement (#10975) Summary: This is several refactorings bundled into one to avoid having to incrementally re-modify uses of Cache several times. Overall, there are breaking changes to Cache class, and it becomes more of low-level interface for implementing caches, especially block cache. New internal APIs make using Cache cleaner than before, and more insulated from block cache evolution. Hopefully, this is the last really big block cache refactoring, because of rather effectively decoupling the implementations from the uses. This change also removes the EXPERIMENTAL designation on the SecondaryCache support in Cache. It seems reasonably mature at this point but still subject to change/evolution (as I warn in the API docs for Cache). The high-level motivation for this refactoring is to minimize code duplication / compounding complexity in adding SecondaryCache support to HyperClockCache (in a later PR). Other benefits listed below. * static_cast lines of code +29 -35 (net removed 6) * reinterpret_cast lines of code +6 -32 (net removed 26) ## cache.h and secondary_cache.h * Always use CacheItemHelper with entries instead of just a Deleter. There are several motivations / justifications: * Simpler for implementations to deal with just one Insert and one Lookup. * Simpler and more efficient implementation because we don't have to track which entries are using helpers and which are using deleters * Gets rid of hack to classify cache entries by their deleter. Instead, the CacheItemHelper includes a CacheEntryRole. This simplifies a lot of code (cache_entry_roles.h almost eliminated). Fixes https://github.com/facebook/rocksdb/issues/9428. * Makes it trivial to adjust SecondaryCache behavior based on kind of block (e.g. don't re-compress filter blocks). * It is arguably less convenient for many direct users of Cache, but direct users of Cache are now rare with introduction of typed_cache.h (below). * I considered and rejected an alternative approach in which we reduce customizability by assuming each secondary cache compatible value starts with a Slice referencing the uncompressed block contents (already true or mostly true), but we apparently intend to stack secondary caches. Saving an entry from a compressed secondary to a lower tier requires custom handling offered by SaveToCallback, etc. * Make CreateCallback part of the helper and introduce CreateContext to work with it (alternative to https://github.com/facebook/rocksdb/issues/10562). This cleans up the interface while still allowing context to be provided for loading/parsing values into primary cache. This model works for async lookup in BlockBasedTable reader (reader owns a CreateContext) under the assumption that it always waits on secondary cache operations to finish. (Otherwise, the CreateContext could be destroyed while async operation depending on it continues.) This likely contributes most to the observed performance improvement because it saves an std::function backed by a heap allocation. * Use char* for serialized data, e.g. in SaveToCallback, where void* was confusingly used. (We use `char*` for serialized byte data all over RocksDB, with many advantages over `void*`. `memcpy` etc. are legacy APIs that should not be mimicked.) * Add a type alias Cache::ObjectPtr = void*, so that we can better indicate the intent of the void* when it is to be the object associated with a Cache entry. Related: started (but did not complete) a refactoring to move away from "value" of a cache entry toward "object" or "obj". (It is confusing to call Cache a key-value store (like DB) when it is really storing arbitrary in-memory objects, not byte strings.) * Remove unnecessary key param from DeleterFn. This is good for efficiency in HyperClockCache, which does not directly store the cache key in memory. (Alternative to https://github.com/facebook/rocksdb/issues/10774) * Add allocator to Cache DeleterFn. This is a kind of future-proofing change in case we get more serious about using the Cache allocator for memory tracked by the Cache. Right now, only the uncompressed block contents are allocated using the allocator, and a pointer to that allocator is saved as part of the cached object so that the deleter can use it. (See CacheAllocationPtr.) If in the future we are able to "flatten out" our Cache objects some more, it would be good not to have to track the allocator as part of each object. * Removes legacy `ApplyToAllCacheEntries` and changes `ApplyToAllEntries` signature for Deleter->CacheItemHelper change. ## typed_cache.h Adds various "typed" interfaces to the Cache as internal APIs, so that most uses of Cache can use simple type safe code without casting and without explicit deleters, etc. Almost all of the non-test, non-glue code uses of Cache have been migrated. (Follow-up work: CompressedSecondaryCache deserves deeper attention to migrate.) This change expands RocksDB's internal usage of metaprogramming and SFINAE (https://en.cppreference.com/w/cpp/language/sfinae). The existing usages of Cache are divided up at a high level into these new interfaces. See updated existing uses of Cache for examples of how these are used. * PlaceholderCacheInterface - Used for making cache reservations, with entries that have a charge but no value. * BasicTypedCacheInterface<TValue> - Used for primary cache storage of objects of type TValue, which can be cleaned up with std::default_delete<TValue>. The role is provided by TValue::kCacheEntryRole or given in an optional template parameter. * FullTypedCacheInterface<TValue, TCreateContext> - Used for secondary cache compatible storage of objects of type TValue. In addition to BasicTypedCacheInterface constraints, we require TValue::ContentSlice() to return persistable data. This simplifies usage for the normal case of simple secondary cache compatibility (can give you a Slice to the data already in memory). In addition to TCreateContext performing the role of Cache::CreateContext, it is also expected to provide a factory function for creating TValue. * For each of these, there's a "Shared" version (e.g. FullTypedSharedCacheInterface) that holds a shared_ptr to the Cache, rather than assuming external ownership by holding only a raw `Cache*`. These interfaces introduce specific handle types for each interface instantiation, so that it's easy to see what kind of object is controlled by a handle. (Ultimately, this might not be worth the extra complexity, but it seems OK so far.) Note: I attempted to make the cache 'charge' automatically inferred from the cache object type, such as by expecting an ApproximateMemoryUsage() function, but this is not so clean because there are cases where we need to compute the charge ahead of time and don't want to re-compute it. ## block_cache.h This header is essentially the replacement for the old block_like_traits.h. It includes various things to support block cache access with typed_cache.h for block-based table. ## block_based_table_reader.cc Before this change, accessing the block cache here was an awkward mix of static polymorphism (template TBlocklike) and switch-case on a dynamic BlockType value. This change mostly unifies on static polymorphism, relying on minor hacks in block_cache.h to distinguish variants of Block. We still check BlockType in some places (especially for stats, which could be improved in follow-up work) but at least the BlockType is a static constant from the template parameter. (No more awkward partial redundancy between static and dynamic info.) This likely contributes to the overall performance improvement, but hasn't been tested in isolation. The other key source of simplification here is a more unified system of creating block cache objects: for directly populating from primary cache and for promotion from secondary cache. Both use BlockCreateContext, for context and for factory functions. ## block_based_table_builder.cc, cache_dump_load_impl.cc Before this change, warming caches was super ugly code. Both of these source files had switch statements to basically transition from the dynamic BlockType world to the static TBlocklike world. None of that mess is needed anymore as there's a new, untyped WarmInCache function that handles all the details just as promotion from SecondaryCache would. (Fixes `TODO akanksha: Dedup below code` in block_based_table_builder.cc.) ## Everything else Mostly just updating Cache users to use new typed APIs when reasonably possible, or changed Cache APIs when not. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10975 Test Plan: tests updated Performance test setup similar to https://github.com/facebook/rocksdb/issues/10626 (by cache size, LRUCache when not "hyper" for HyperClockCache): 34MB 1thread base.hyper -> kops/s: 0.745 io_bytes/op: 2.52504e+06 miss_ratio: 0.140906 max_rss_mb: 76.4844 34MB 1thread new.hyper -> kops/s: 0.751 io_bytes/op: 2.5123e+06 miss_ratio: 0.140161 max_rss_mb: 79.3594 34MB 1thread base -> kops/s: 0.254 io_bytes/op: 1.36073e+07 miss_ratio: 0.918818 max_rss_mb: 45.9297 34MB 1thread new -> kops/s: 0.252 io_bytes/op: 1.36157e+07 miss_ratio: 0.918999 max_rss_mb: 44.1523 34MB 32thread base.hyper -> kops/s: 7.272 io_bytes/op: 2.88323e+06 miss_ratio: 0.162532 max_rss_mb: 516.602 34MB 32thread new.hyper -> kops/s: 7.214 io_bytes/op: 2.99046e+06 miss_ratio: 0.168818 max_rss_mb: 518.293 34MB 32thread base -> kops/s: 3.528 io_bytes/op: 1.35722e+07 miss_ratio: 0.914691 max_rss_mb: 264.926 34MB 32thread new -> kops/s: 3.604 io_bytes/op: 1.35744e+07 miss_ratio: 0.915054 max_rss_mb: 264.488 233MB 1thread base.hyper -> kops/s: 53.909 io_bytes/op: 2552.35 miss_ratio: 0.0440566 max_rss_mb: 241.984 233MB 1thread new.hyper -> kops/s: 62.792 io_bytes/op: 2549.79 miss_ratio: 0.044043 max_rss_mb: 241.922 233MB 1thread base -> kops/s: 1.197 io_bytes/op: 2.75173e+06 miss_ratio: 0.103093 max_rss_mb: 241.559 233MB 1thread new -> kops/s: 1.199 io_bytes/op: 2.73723e+06 miss_ratio: 0.10305 max_rss_mb: 240.93 233MB 32thread base.hyper -> kops/s: 1298.69 io_bytes/op: 2539.12 miss_ratio: 0.0440307 max_rss_mb: 371.418 233MB 32thread new.hyper -> kops/s: 1421.35 io_bytes/op: 2538.75 miss_ratio: 0.0440307 max_rss_mb: 347.273 233MB 32thread base -> kops/s: 9.693 io_bytes/op: 2.77304e+06 miss_ratio: 0.103745 max_rss_mb: 569.691 233MB 32thread new -> kops/s: 9.75 io_bytes/op: 2.77559e+06 miss_ratio: 0.103798 max_rss_mb: 552.82 1597MB 1thread base.hyper -> kops/s: 58.607 io_bytes/op: 1449.14 miss_ratio: 0.0249324 max_rss_mb: 1583.55 1597MB 1thread new.hyper -> kops/s: 69.6 io_bytes/op: 1434.89 miss_ratio: 0.0247167 max_rss_mb: 1584.02 1597MB 1thread base -> kops/s: 60.478 io_bytes/op: 1421.28 miss_ratio: 0.024452 max_rss_mb: 1589.45 1597MB 1thread new -> kops/s: 63.973 io_bytes/op: 1416.07 miss_ratio: 0.0243766 max_rss_mb: 1589.24 1597MB 32thread base.hyper -> kops/s: 1436.2 io_bytes/op: 1357.93 miss_ratio: 0.0235353 max_rss_mb: 1692.92 1597MB 32thread new.hyper -> kops/s: 1605.03 io_bytes/op: 1358.04 miss_ratio: 0.023538 max_rss_mb: 1702.78 1597MB 32thread base -> kops/s: 280.059 io_bytes/op: 1350.34 miss_ratio: 0.023289 max_rss_mb: 1675.36 1597MB 32thread new -> kops/s: 283.125 io_bytes/op: 1351.05 miss_ratio: 0.0232797 max_rss_mb: 1703.83 Almost uniformly improving over base revision, especially for hot paths with HyperClockCache, up to 12% higher throughput seen (1597MB, 32thread, hyper). The improvement for that is likely coming from much simplified code for providing context for secondary cache promotion (CreateCallback/CreateContext), and possibly from less branching in block_based_table_reader. And likely a small improvement from not reconstituting key for DeleterFn. Reviewed By: anand1976 Differential Revision: D42417818 Pulled By: pdillinger fbshipit-source-id: f86bfdd584dce27c028b151ba56818ad14f7a432
2 years ago
switch (TBlocklike::kBlockType) {
case BlockType::kIndex:
++(get_context->get_context_stats_.num_index_read);
break;
case BlockType::kFilter:
case BlockType::kFilterPartitionIndex:
++(get_context->get_context_stats_.num_filter_read);
break;
default:
break;
}
}
}
if (!s.ok()) {
return s;
}
Refactor to avoid confusing "raw block" (#10408) Summary: We have a lot of confusing code because of mixed, sometimes completely opposite uses of of the term "raw block" or "raw contents", sometimes within the same source file. For example, in `BlockBasedTableBuilder`, `raw_block_contents` and `raw_size` generally referred to uncompressed block contents and size, while `WriteRawBlock` referred to writing a block that is already compressed if it is going to be. Meanwhile, in `BlockBasedTable`, `raw_block_contents` either referred to a (maybe compressed) block with trailer, or a maybe compressed block maybe without trailer. (Note: left as follow-up work to use C++ typing to better sort out the various kinds of BlockContents.) This change primarily tries to apply some consistent terminology around the kinds of block representations, avoiding the unclear "raw". (Any meaning of "raw" assumes some bias toward the storage layer or toward the logical data layer.) Preferred terminology: * **Serialized block** - bytes that go into storage. For block-based table (usually the case) this includes the block trailer. WART: block `size` may or may not include the trailer; need to be clear about whether it does or not. * **Maybe compressed block** - like a serialized block, but without the trailer (or no promise of including a trailer). Must be accompanied by a CompressionType. * **Uncompressed block** - "payload" bytes that are either stored with no compression, used as input to compression function, or result of decompression function. * **Parsed block** - an in-memory form of a block in block cache, as it is used by the table reader. Different C++ types are used depending on the block type (see block_like_traits.h). Other refactorings: * Misc corrections/improvements of internal API comments * Remove a few misleading / unhelpful / redundant comments. * Use move semantics in some places to simplify contracts * Use better parameter names to indicate which parameters are used for outputs * Remove some extraneous `extern` * Various clean-ups to `CacheDumperImpl` (mostly unnecessary code) Pull Request resolved: https://github.com/facebook/rocksdb/pull/10408 Test Plan: existing tests Reviewed By: akankshamahajan15 Differential Revision: D38172617 Pulled By: pdillinger fbshipit-source-id: ccb99299f324ac5ca46996d34c5089621a4f260c
2 years ago
out_parsed_block->SetOwnedValue(std::move(block));
assert(s.ok());
return s;
}
BlockBasedTable::PartitionedIndexIteratorState::PartitionedIndexIteratorState(
const BlockBasedTable* table,
Meta-internal folly integration with F14FastMap (#9546) Summary: Especially after updating to C++17, I don't see a compelling case for *requiring* any folly components in RocksDB. I was able to purge the existing hard dependencies, and it can be quite difficult to strip out non-trivial components from folly for use in RocksDB. (The prospect of doing that on F14 has changed my mind on the best approach here.) But this change creates an optional integration where we can plug in components from folly at compile time, starting here with F14FastMap to replace std::unordered_map when possible (probably no public APIs for example). I have replaced the biggest CPU users of std::unordered_map with compile-time pluggable UnorderedMap which will use F14FastMap when USE_FOLLY is set. USE_FOLLY is always set in the Meta-internal buck build, and a simulation of that is in the Makefile for public CI testing. A full folly build is not needed, but checking out the full folly repo is much simpler for getting the dependency, and anything else we might want to optionally integrate in the future. Some picky details: * I don't think the distributed mutex stuff is actually used, so it was easy to remove. * I implemented an alternative to `folly::constexpr_log2` (which is much easier in C++17 than C++11) so that I could pull out the hard dependencies on `ConstexprMath.h` * I had to add noexcept move constructors/operators to some types to make F14's complainUnlessNothrowMoveAndDestroy check happy, and I added a macro to make that easier in some common cases. * Updated Meta-internal buck build to use folly F14Map (always) No updates to HISTORY.md nor INSTALL.md as this is not (yet?) considered a production integration for open source users. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9546 Test Plan: CircleCI tests updated so that a couple of them use folly. Most internal unit & stress/crash tests updated to use Meta-internal latest folly. (Note: they should probably use buck but they currently use Makefile.) Example performance improvement: when filter partitions are pinned in cache, they are tracked by PartitionedFilterBlockReader::filter_map_ and we can build a test that exercises that heavily. Build DB with ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters ``` and test with (simultaneous runs with & without folly, ~20 times each to see convergence) ``` TEST_TMPDIR=/dev/shm/rocksdb ./db_bench_folly -readonly -use_existing_db -benchmarks=readrandom -num=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -partition_index_and_filters -duration=40 -pin_l0_filter_and_index_blocks_in_cache ``` Average ops/s no folly: 26229.2 Average ops/s with folly: 26853.3 (+2.4%) Reviewed By: ajkr Differential Revision: D34181736 Pulled By: pdillinger fbshipit-source-id: ffa6ad5104c2880321d8a1aa7187e00ab0d02e94
3 years ago
UnorderedMap<uint64_t, CachableEntry<Block>>* block_map)
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
: table_(table), block_map_(block_map) {}
InternalIteratorBase<IndexValue>*
BlockBasedTable::PartitionedIndexIteratorState::NewSecondaryIterator(
const BlockHandle& handle) {
// Return a block iterator on the index partition
auto block = block_map_->find(handle.offset());
// block_map_ must be exhaustive
if (block == block_map_->end()) {
assert(false);
// Signal problem to caller
return nullptr;
}
const Rep* rep = table_->get_rep();
assert(rep);
Statistics* kNullStats = nullptr;
// We don't return pinned data from index blocks, so no need
// to set `block_contents_pinned`.
return block->second.GetValue()->NewIndexIterator(
rep->internal_comparator.user_comparator(),
rep->get_global_seqno(BlockType::kIndex), nullptr, kNullStats, true,
rep->index_has_first_key, rep->index_key_includes_seq,
rep->index_value_is_full);
}
// This will be broken if the user specifies an unusual implementation
// of Options.comparator, or if the user specifies an unusual
// definition of prefixes in BlockBasedTableOptions.filter_policy.
// In particular, we require the following three properties:
//
// 1) key.starts_with(prefix(key))
// 2) Compare(prefix(key), key) <= 0.
// 3) If Compare(key1, key2) <= 0, then Compare(prefix(key1), prefix(key2)) <= 0
//
Fix iterator reading filter block despite read_tier == kBlockCacheTier (#6562) Summary: We're seeing iterators with `ReadOptions::read_tier == kBlockCacheTier` sometimes doing file reads. Stack trace: ``` rocksdb::RandomAccessFileReader::Read(unsigned long, unsigned long, rocksdb::Slice*, char*, bool) const rocksdb::BlockFetcher::ReadBlockContents() rocksdb::Status rocksdb::BlockBasedTable::MaybeReadBlockAndLoadToCache<rocksdb::ParsedFullFilterBlock>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::BlockContents*) const rocksdb::Status rocksdb::BlockBasedTable::RetrieveBlock<rocksdb::ParsedFullFilterBlock>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, bool, bool) const rocksdb::FilterBlockReaderCommon<rocksdb::ParsedFullFilterBlock>::ReadFilterBlock(rocksdb::BlockBasedTable const*, rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, bool, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*) rocksdb::FilterBlockReaderCommon<rocksdb::ParsedFullFilterBlock>::GetOrReadFilterBlock(bool, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*) const rocksdb::FullFilterBlockReader::MayMatch(rocksdb::Slice const&, bool, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*) const rocksdb::FullFilterBlockReader::RangeMayExist(rocksdb::Slice const*, rocksdb::Slice const&, rocksdb::SliceTransform const*, rocksdb::Comparator const*, rocksdb::Slice const*, bool*, bool, rocksdb::BlockCacheLookupContext*) rocksdb::BlockBasedTable::PrefixMayMatch(rocksdb::Slice const&, rocksdb::ReadOptions const&, rocksdb::SliceTransform const*, bool, rocksdb::BlockCacheLookupContext*) const rocksdb::BlockBasedTableIterator<rocksdb::DataBlockIter, rocksdb::Slice>::SeekImpl(rocksdb::Slice const*) rocksdb::ForwardIterator::SeekInternal(rocksdb::Slice const&, bool) rocksdb::DBIter::Seek(rocksdb::Slice const&) ``` `BlockBasedTableIterator::CheckPrefixMayMatch` was missing a check for `kBlockCacheTier`. This PR adds it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6562 Test Plan: deployed it to a logdevice test cluster and looked at logdevice's IO tracing. Reviewed By: siying Differential Revision: D20529368 Pulled By: al13n321 fbshipit-source-id: 65bf33964b1951464415c900336635fb20919611
5 years ago
// If read_options.read_tier == kBlockCacheTier, this method will do no I/O and
// will return true if the filter block is not in memory and not found in block
// cache.
//
// REQUIRES: this method shouldn't be called while the DB lock is held.
bool BlockBasedTable::PrefixRangeMayMatch(
const Slice& internal_key, const ReadOptions& read_options,
const SliceTransform* options_prefix_extractor,
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
const bool need_upper_bound_check, BlockCacheLookupContext* lookup_context,
bool* filter_checked) const {
if (!rep_->filter_policy) {
return true;
}
const SliceTransform* prefix_extractor;
if (rep_->table_prefix_extractor == nullptr) {
if (need_upper_bound_check) {
return true;
}
prefix_extractor = options_prefix_extractor;
} else {
prefix_extractor = rep_->table_prefix_extractor.get();
}
auto ts_sz = rep_->internal_comparator.user_comparator()->timestamp_size();
auto user_key_without_ts =
ExtractUserKeyAndStripTimestamp(internal_key, ts_sz);
if (!prefix_extractor->InDomain(user_key_without_ts)) {
return true;
}
bool may_match = true;
FilterBlockReader* const filter = rep_->filter.get();
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
*filter_checked = false;
if (filter != nullptr) {
Fix iterator reading filter block despite read_tier == kBlockCacheTier (#6562) Summary: We're seeing iterators with `ReadOptions::read_tier == kBlockCacheTier` sometimes doing file reads. Stack trace: ``` rocksdb::RandomAccessFileReader::Read(unsigned long, unsigned long, rocksdb::Slice*, char*, bool) const rocksdb::BlockFetcher::ReadBlockContents() rocksdb::Status rocksdb::BlockBasedTable::MaybeReadBlockAndLoadToCache<rocksdb::ParsedFullFilterBlock>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::BlockContents*) const rocksdb::Status rocksdb::BlockBasedTable::RetrieveBlock<rocksdb::ParsedFullFilterBlock>(rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, rocksdb::BlockHandle const&, rocksdb::UncompressionDict const&, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*, rocksdb::BlockType, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, bool, bool) const rocksdb::FilterBlockReaderCommon<rocksdb::ParsedFullFilterBlock>::ReadFilterBlock(rocksdb::BlockBasedTable const*, rocksdb::FilePrefetchBuffer*, rocksdb::ReadOptions const&, bool, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*) rocksdb::FilterBlockReaderCommon<rocksdb::ParsedFullFilterBlock>::GetOrReadFilterBlock(bool, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*, rocksdb::CachableEntry<rocksdb::ParsedFullFilterBlock>*) const rocksdb::FullFilterBlockReader::MayMatch(rocksdb::Slice const&, bool, rocksdb::GetContext*, rocksdb::BlockCacheLookupContext*) const rocksdb::FullFilterBlockReader::RangeMayExist(rocksdb::Slice const*, rocksdb::Slice const&, rocksdb::SliceTransform const*, rocksdb::Comparator const*, rocksdb::Slice const*, bool*, bool, rocksdb::BlockCacheLookupContext*) rocksdb::BlockBasedTable::PrefixMayMatch(rocksdb::Slice const&, rocksdb::ReadOptions const&, rocksdb::SliceTransform const*, bool, rocksdb::BlockCacheLookupContext*) const rocksdb::BlockBasedTableIterator<rocksdb::DataBlockIter, rocksdb::Slice>::SeekImpl(rocksdb::Slice const*) rocksdb::ForwardIterator::SeekInternal(rocksdb::Slice const&, bool) rocksdb::DBIter::Seek(rocksdb::Slice const&) ``` `BlockBasedTableIterator::CheckPrefixMayMatch` was missing a check for `kBlockCacheTier`. This PR adds it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6562 Test Plan: deployed it to a logdevice test cluster and looked at logdevice's IO tracing. Reviewed By: siying Differential Revision: D20529368 Pulled By: al13n321 fbshipit-source-id: 65bf33964b1951464415c900336635fb20919611
5 years ago
const bool no_io = read_options.read_tier == kBlockCacheTier;
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
const Slice* const const_ikey_ptr = &internal_key;
may_match = filter->RangeMayExist(
read_options.iterate_upper_bound, user_key_without_ts, prefix_extractor,
rep_->internal_comparator.user_comparator(), const_ikey_ptr,
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
filter_checked, need_upper_bound_check, no_io, lookup_context,
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
read_options);
}
return may_match;
}
Fast path for detecting unchanged prefix_extractor (#9407) Summary: Fixes a major performance regression in 6.26, where extra CPU is spent in SliceTransform::AsString when reads involve a prefix_extractor (Get, MultiGet, Seek). Common case performance is now better than 6.25. This change creates a "fast path" for verifying that the current prefix extractor is unchanged and compatible with what was used to generate a table file. This fast path detects the common case by pointer comparison on the current prefix_extractor and a "known good" prefix extractor (if applicable) that is saved at the time the table reader is opened. The "known good" prefix extractor is saved as another shared_ptr copy (in an existing field, however) to ensure the pointer is not recycled. When the prefix_extractor has changed to a different instance but same compatible configuration (rare, odd), performance is still a regression compared to 6.25, but this is likely acceptable because of the oddity of such a case. The performance of incompatible prefix_extractor is essentially unchanged. Also fixed a minor case (ForwardIterator) where a prefix_extractor could be used via a raw pointer after being freed as a shared_ptr, if replaced via SetOptions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9407 Test Plan: ## Performance Populate DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Running head-to-head comparisons simultaneously with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=seekrandom -num=10000000 -duration=20 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Below each is compared by ops/sec vs. baseline which is version 6.25 (multiple baseline runs because of variable machine load) v6.26: 4833 vs. 6698 (<- major regression!) v6.27: 4737 vs. 6397 (still) New: 6704 vs. 6461 (better than baseline in common case) Disabled fastpath: 4843 vs. 6389 (e.g. if prefix extractor instance changes but is still compatible) Changed prefix size (no usable filter) in new: 787 vs. 5927 Changed prefix size (no usable filter) in new & baseline: 773 vs. 784 Reviewed By: mrambacher Differential Revision: D33677812 Pulled By: pdillinger fbshipit-source-id: 571d9711c461fb97f957378a061b7e7dbc4d6a76
3 years ago
bool BlockBasedTable::PrefixExtractorChanged(
const SliceTransform* prefix_extractor) const {
if (prefix_extractor == nullptr) {
return true;
} else if (prefix_extractor == rep_->table_prefix_extractor.get()) {
return false;
} else {
return PrefixExtractorChangedHelper(rep_->table_properties.get(),
prefix_extractor);
}
}
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
Statistics* BlockBasedTable::GetStatistics() const {
return rep_->ioptions.stats;
}
bool BlockBasedTable::IsLastLevel() const {
return rep_->level == rep_->ioptions.num_levels - 1;
}
InternalIterator* BlockBasedTable::NewIterator(
const ReadOptions& read_options, const SliceTransform* prefix_extractor,
Arena* arena, bool skip_filters, TableReaderCaller 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
size_t compaction_readahead_size, bool allow_unprepared_value) {
BlockCacheLookupContext lookup_context{caller};
bool need_upper_bound_check =
Fast path for detecting unchanged prefix_extractor (#9407) Summary: Fixes a major performance regression in 6.26, where extra CPU is spent in SliceTransform::AsString when reads involve a prefix_extractor (Get, MultiGet, Seek). Common case performance is now better than 6.25. This change creates a "fast path" for verifying that the current prefix extractor is unchanged and compatible with what was used to generate a table file. This fast path detects the common case by pointer comparison on the current prefix_extractor and a "known good" prefix extractor (if applicable) that is saved at the time the table reader is opened. The "known good" prefix extractor is saved as another shared_ptr copy (in an existing field, however) to ensure the pointer is not recycled. When the prefix_extractor has changed to a different instance but same compatible configuration (rare, odd), performance is still a regression compared to 6.25, but this is likely acceptable because of the oddity of such a case. The performance of incompatible prefix_extractor is essentially unchanged. Also fixed a minor case (ForwardIterator) where a prefix_extractor could be used via a raw pointer after being freed as a shared_ptr, if replaced via SetOptions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9407 Test Plan: ## Performance Populate DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Running head-to-head comparisons simultaneously with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=seekrandom -num=10000000 -duration=20 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Below each is compared by ops/sec vs. baseline which is version 6.25 (multiple baseline runs because of variable machine load) v6.26: 4833 vs. 6698 (<- major regression!) v6.27: 4737 vs. 6397 (still) New: 6704 vs. 6461 (better than baseline in common case) Disabled fastpath: 4843 vs. 6389 (e.g. if prefix extractor instance changes but is still compatible) Changed prefix size (no usable filter) in new: 787 vs. 5927 Changed prefix size (no usable filter) in new & baseline: 773 vs. 784 Reviewed By: mrambacher Differential Revision: D33677812 Pulled By: pdillinger fbshipit-source-id: 571d9711c461fb97f957378a061b7e7dbc4d6a76
3 years ago
read_options.auto_prefix_mode || PrefixExtractorChanged(prefix_extractor);
De-template block based table iterator (#6531) Summary: Right now block based table iterator is used as both of iterating data for block based table, and for the index iterator for partitioend index. This was initially convenient for introducing a new iterator and block type for new index format, while reducing code change. However, these two usage doesn't go with each other very well. For example, Prev() is never called for partitioned index iterator, and some other complexity is maintained in block based iterators, which is not needed for index iterator but maintainers will always need to reason about it. Furthermore, the template usage is not following Google C++ Style which we are following, and makes a large chunk of code tangled together. This commit separate the two iterators. Right now, here is what it is done: 1. Copy the block based iterator code into partitioned index iterator, and de-template them. 2. Remove some code not needed for partitioned index. The upper bound check and tricks are removed. We never tested performance for those tricks when partitioned index is enabled in the first place. It's unlikelyl to generate performance regression, as creating new partitioned index block is much rarer than data blocks. 3. Separate out the prefetch logic to a helper class and both classes call them. This commit will enable future follow-ups. One direction is that we might separate index iterator interface for data blocks and index blocks, as they are quite different. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6531 Test Plan: build using make and cmake. And build release Differential Revision: D20473108 fbshipit-source-id: e48011783b339a4257c204cc07507b171b834b0f
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter(NewIndexIterator(
read_options,
/*disable_prefix_seek=*/need_upper_bound_check &&
De-template block based table iterator (#6531) Summary: Right now block based table iterator is used as both of iterating data for block based table, and for the index iterator for partitioend index. This was initially convenient for introducing a new iterator and block type for new index format, while reducing code change. However, these two usage doesn't go with each other very well. For example, Prev() is never called for partitioned index iterator, and some other complexity is maintained in block based iterators, which is not needed for index iterator but maintainers will always need to reason about it. Furthermore, the template usage is not following Google C++ Style which we are following, and makes a large chunk of code tangled together. This commit separate the two iterators. Right now, here is what it is done: 1. Copy the block based iterator code into partitioned index iterator, and de-template them. 2. Remove some code not needed for partitioned index. The upper bound check and tricks are removed. We never tested performance for those tricks when partitioned index is enabled in the first place. It's unlikelyl to generate performance regression, as creating new partitioned index block is much rarer than data blocks. 3. Separate out the prefetch logic to a helper class and both classes call them. This commit will enable future follow-ups. One direction is that we might separate index iterator interface for data blocks and index blocks, as they are quite different. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6531 Test Plan: build using make and cmake. And build release Differential Revision: D20473108 fbshipit-source-id: e48011783b339a4257c204cc07507b171b834b0f
5 years ago
rep_->index_type == BlockBasedTableOptions::kHashSearch,
/*input_iter=*/nullptr, /*get_context=*/nullptr, &lookup_context));
if (arena == nullptr) {
De-template block based table iterator (#6531) Summary: Right now block based table iterator is used as both of iterating data for block based table, and for the index iterator for partitioend index. This was initially convenient for introducing a new iterator and block type for new index format, while reducing code change. However, these two usage doesn't go with each other very well. For example, Prev() is never called for partitioned index iterator, and some other complexity is maintained in block based iterators, which is not needed for index iterator but maintainers will always need to reason about it. Furthermore, the template usage is not following Google C++ Style which we are following, and makes a large chunk of code tangled together. This commit separate the two iterators. Right now, here is what it is done: 1. Copy the block based iterator code into partitioned index iterator, and de-template them. 2. Remove some code not needed for partitioned index. The upper bound check and tricks are removed. We never tested performance for those tricks when partitioned index is enabled in the first place. It's unlikelyl to generate performance regression, as creating new partitioned index block is much rarer than data blocks. 3. Separate out the prefetch logic to a helper class and both classes call them. This commit will enable future follow-ups. One direction is that we might separate index iterator interface for data blocks and index blocks, as they are quite different. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6531 Test Plan: build using make and cmake. And build release Differential Revision: D20473108 fbshipit-source-id: e48011783b339a4257c204cc07507b171b834b0f
5 years ago
return new BlockBasedTableIterator(
this, read_options, rep_->internal_comparator, std::move(index_iter),
!skip_filters && !read_options.total_order_seek &&
prefix_extractor != nullptr,
De-template block based table iterator (#6531) Summary: Right now block based table iterator is used as both of iterating data for block based table, and for the index iterator for partitioend index. This was initially convenient for introducing a new iterator and block type for new index format, while reducing code change. However, these two usage doesn't go with each other very well. For example, Prev() is never called for partitioned index iterator, and some other complexity is maintained in block based iterators, which is not needed for index iterator but maintainers will always need to reason about it. Furthermore, the template usage is not following Google C++ Style which we are following, and makes a large chunk of code tangled together. This commit separate the two iterators. Right now, here is what it is done: 1. Copy the block based iterator code into partitioned index iterator, and de-template them. 2. Remove some code not needed for partitioned index. The upper bound check and tricks are removed. We never tested performance for those tricks when partitioned index is enabled in the first place. It's unlikelyl to generate performance regression, as creating new partitioned index block is much rarer than data blocks. 3. Separate out the prefetch logic to a helper class and both classes call them. This commit will enable future follow-ups. One direction is that we might separate index iterator interface for data blocks and index blocks, as they are quite different. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6531 Test Plan: build using make and cmake. And build release Differential Revision: D20473108 fbshipit-source-id: e48011783b339a4257c204cc07507b171b834b0f
5 years ago
need_upper_bound_check, prefix_extractor, 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
compaction_readahead_size, allow_unprepared_value);
} else {
De-template block based table iterator (#6531) Summary: Right now block based table iterator is used as both of iterating data for block based table, and for the index iterator for partitioend index. This was initially convenient for introducing a new iterator and block type for new index format, while reducing code change. However, these two usage doesn't go with each other very well. For example, Prev() is never called for partitioned index iterator, and some other complexity is maintained in block based iterators, which is not needed for index iterator but maintainers will always need to reason about it. Furthermore, the template usage is not following Google C++ Style which we are following, and makes a large chunk of code tangled together. This commit separate the two iterators. Right now, here is what it is done: 1. Copy the block based iterator code into partitioned index iterator, and de-template them. 2. Remove some code not needed for partitioned index. The upper bound check and tricks are removed. We never tested performance for those tricks when partitioned index is enabled in the first place. It's unlikelyl to generate performance regression, as creating new partitioned index block is much rarer than data blocks. 3. Separate out the prefetch logic to a helper class and both classes call them. This commit will enable future follow-ups. One direction is that we might separate index iterator interface for data blocks and index blocks, as they are quite different. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6531 Test Plan: build using make and cmake. And build release Differential Revision: D20473108 fbshipit-source-id: e48011783b339a4257c204cc07507b171b834b0f
5 years ago
auto* mem = arena->AllocateAligned(sizeof(BlockBasedTableIterator));
return new (mem) BlockBasedTableIterator(
this, read_options, rep_->internal_comparator, std::move(index_iter),
!skip_filters && !read_options.total_order_seek &&
prefix_extractor != nullptr,
De-template block based table iterator (#6531) Summary: Right now block based table iterator is used as both of iterating data for block based table, and for the index iterator for partitioend index. This was initially convenient for introducing a new iterator and block type for new index format, while reducing code change. However, these two usage doesn't go with each other very well. For example, Prev() is never called for partitioned index iterator, and some other complexity is maintained in block based iterators, which is not needed for index iterator but maintainers will always need to reason about it. Furthermore, the template usage is not following Google C++ Style which we are following, and makes a large chunk of code tangled together. This commit separate the two iterators. Right now, here is what it is done: 1. Copy the block based iterator code into partitioned index iterator, and de-template them. 2. Remove some code not needed for partitioned index. The upper bound check and tricks are removed. We never tested performance for those tricks when partitioned index is enabled in the first place. It's unlikelyl to generate performance regression, as creating new partitioned index block is much rarer than data blocks. 3. Separate out the prefetch logic to a helper class and both classes call them. This commit will enable future follow-ups. One direction is that we might separate index iterator interface for data blocks and index blocks, as they are quite different. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6531 Test Plan: build using make and cmake. And build release Differential Revision: D20473108 fbshipit-source-id: e48011783b339a4257c204cc07507b171b834b0f
5 years ago
need_upper_bound_check, prefix_extractor, 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
compaction_readahead_size, allow_unprepared_value);
}
}
FragmentedRangeTombstoneIterator* BlockBasedTable::NewRangeTombstoneIterator(
const ReadOptions& read_options) {
Cache fragmented range tombstones in BlockBasedTableReader (#4493) Summary: This allows tombstone fragmenting to only be performed when the table is opened, and cached for subsequent accesses. On the same DB used in #4449, running `readrandom` results in the following: ``` readrandom : 0.983 micros/op 1017076 ops/sec; 78.3 MB/s (63103 of 100000 found) ``` Now that Get performance in the presence of range tombstones is reasonable, I also compared the performance between a DB with range tombstones, "expanded" range tombstones (several point tombstones that cover the same keys the equivalent range tombstone would cover, a common workaround for DeleteRange), and no range tombstones. The created DBs had 5 million keys each, and DeleteRange was called at regular intervals (depending on the total number of range tombstones being written) after 4.5 million Puts. The table below summarizes the results of a `readwhilewriting` benchmark (in order to provide somewhat more realistic results): ``` Tombstones? | avg micros/op | stddev micros/op | avg ops/s | stddev ops/s ----------------- | ------------- | ---------------- | ------------ | ------------ None | 0.6186 | 0.04637 | 1,625,252.90 | 124,679.41 500 Expanded | 0.6019 | 0.03628 | 1,666,670.40 | 101,142.65 500 Unexpanded | 0.6435 | 0.03994 | 1,559,979.40 | 104,090.52 1k Expanded | 0.6034 | 0.04349 | 1,665,128.10 | 125,144.57 1k Unexpanded | 0.6261 | 0.03093 | 1,600,457.50 | 79,024.94 5k Expanded | 0.6163 | 0.05926 | 1,636,668.80 | 154,888.85 5k Unexpanded | 0.6402 | 0.04002 | 1,567,804.70 | 100,965.55 10k Expanded | 0.6036 | 0.05105 | 1,667,237.70 | 142,830.36 10k Unexpanded | 0.6128 | 0.02598 | 1,634,633.40 | 72,161.82 25k Expanded | 0.6198 | 0.04542 | 1,620,980.50 | 116,662.93 25k Unexpanded | 0.5478 | 0.0362 | 1,833,059.10 | 121,233.81 50k Expanded | 0.5104 | 0.04347 | 1,973,107.90 | 184,073.49 50k Unexpanded | 0.4528 | 0.03387 | 2,219,034.50 | 170,984.32 ``` After a large enough quantity of range tombstones are written, range tombstone Gets can become faster than reading from an equivalent DB with several point tombstones. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4493 Differential Revision: D10842844 Pulled By: abhimadan fbshipit-source-id: a7d44534f8120e6aabb65779d26c6b9df954c509
6 years ago
if (rep_->fragmented_range_dels == nullptr) {
return nullptr;
}
SequenceNumber snapshot = kMaxSequenceNumber;
if (read_options.snapshot != nullptr) {
snapshot = read_options.snapshot->GetSequenceNumber();
}
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
return new FragmentedRangeTombstoneIterator(rep_->fragmented_range_dels,
rep_->internal_comparator,
snapshot, read_options.timestamp);
Cache fragmented range tombstones in BlockBasedTableReader (#4493) Summary: This allows tombstone fragmenting to only be performed when the table is opened, and cached for subsequent accesses. On the same DB used in #4449, running `readrandom` results in the following: ``` readrandom : 0.983 micros/op 1017076 ops/sec; 78.3 MB/s (63103 of 100000 found) ``` Now that Get performance in the presence of range tombstones is reasonable, I also compared the performance between a DB with range tombstones, "expanded" range tombstones (several point tombstones that cover the same keys the equivalent range tombstone would cover, a common workaround for DeleteRange), and no range tombstones. The created DBs had 5 million keys each, and DeleteRange was called at regular intervals (depending on the total number of range tombstones being written) after 4.5 million Puts. The table below summarizes the results of a `readwhilewriting` benchmark (in order to provide somewhat more realistic results): ``` Tombstones? | avg micros/op | stddev micros/op | avg ops/s | stddev ops/s ----------------- | ------------- | ---------------- | ------------ | ------------ None | 0.6186 | 0.04637 | 1,625,252.90 | 124,679.41 500 Expanded | 0.6019 | 0.03628 | 1,666,670.40 | 101,142.65 500 Unexpanded | 0.6435 | 0.03994 | 1,559,979.40 | 104,090.52 1k Expanded | 0.6034 | 0.04349 | 1,665,128.10 | 125,144.57 1k Unexpanded | 0.6261 | 0.03093 | 1,600,457.50 | 79,024.94 5k Expanded | 0.6163 | 0.05926 | 1,636,668.80 | 154,888.85 5k Unexpanded | 0.6402 | 0.04002 | 1,567,804.70 | 100,965.55 10k Expanded | 0.6036 | 0.05105 | 1,667,237.70 | 142,830.36 10k Unexpanded | 0.6128 | 0.02598 | 1,634,633.40 | 72,161.82 25k Expanded | 0.6198 | 0.04542 | 1,620,980.50 | 116,662.93 25k Unexpanded | 0.5478 | 0.0362 | 1,833,059.10 | 121,233.81 50k Expanded | 0.5104 | 0.04347 | 1,973,107.90 | 184,073.49 50k Unexpanded | 0.4528 | 0.03387 | 2,219,034.50 | 170,984.32 ``` After a large enough quantity of range tombstones are written, range tombstone Gets can become faster than reading from an equivalent DB with several point tombstones. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4493 Differential Revision: D10842844 Pulled By: abhimadan fbshipit-source-id: a7d44534f8120e6aabb65779d26c6b9df954c509
6 years ago
}
bool BlockBasedTable::FullFilterKeyMayMatch(
Ignore `total_order_seek` in DB::Get (#9427) Summary: Apparently setting total_order_seek=true for DB::Get was intended to allow accurate read semantics if the current prefix extractor doesn't match what was used to generate SST files on disk. But since prefix_extractor was made a mutable option in 5.14.0, we have been able to detect this case and provide the correct semantics regardless of the total_order_seek option. Since that time, the option has only made Get() slower in a reasonably common case: prefix_extractor unchanged and whole_key_filtering=false. So this change primarily removes unnecessary effect of total_order_seek on Get. Also cleans up some related comments. Also adds a -total_order_seek option to db_bench and canonicalizes handling of ReadOptions in db_bench so that command line options have the expected association with library features. (There is potential for change in regression test behavior, but the old behavior is likely indefensible, or some other inconsistency would need to be fixed.) TODO in follow-up work: there should be no reason for Get() to depend on current prefix extractor at all. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9427 Test Plan: Unit tests updated. Performance (using db_bench update) Create DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12 -whole_key_filtering=0` Test with and without `-total_order_seek` on `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=readrandom -num=10000000 -duration=40 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Before this change, total_order_seek=false: 25188 ops/sec Before this change, total_order_seek=true: 1222 ops/sec (~20x slower) After this change, total_order_seek=false: 24570 ops/sec After this change, total_order_seek=true: 25012 ops/sec (indistinguishable) Reviewed By: siying Differential Revision: D33753458 Pulled By: pdillinger fbshipit-source-id: bf892f34907a5e407d9c40bd4d42f0adbcbe0014
3 years ago
FilterBlockReader* filter, const Slice& internal_key, const bool no_io,
const SliceTransform* prefix_extractor, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
const ReadOptions& read_options) const {
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
if (filter == nullptr) {
return true;
}
Slice user_key = ExtractUserKey(internal_key);
const Slice* const const_ikey_ptr = &internal_key;
bool may_match = true;
size_t ts_sz = rep_->internal_comparator.user_comparator()->timestamp_size();
Slice user_key_without_ts = StripTimestampFromUserKey(user_key, ts_sz);
if (rep_->whole_key_filtering) {
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
may_match = filter->KeyMayMatch(user_key_without_ts, no_io, const_ikey_ptr,
get_context, lookup_context, read_options);
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
if (may_match) {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, 1, rep_->level);
} else {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL);
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level);
}
Ignore `total_order_seek` in DB::Get (#9427) Summary: Apparently setting total_order_seek=true for DB::Get was intended to allow accurate read semantics if the current prefix extractor doesn't match what was used to generate SST files on disk. But since prefix_extractor was made a mutable option in 5.14.0, we have been able to detect this case and provide the correct semantics regardless of the total_order_seek option. Since that time, the option has only made Get() slower in a reasonably common case: prefix_extractor unchanged and whole_key_filtering=false. So this change primarily removes unnecessary effect of total_order_seek on Get. Also cleans up some related comments. Also adds a -total_order_seek option to db_bench and canonicalizes handling of ReadOptions in db_bench so that command line options have the expected association with library features. (There is potential for change in regression test behavior, but the old behavior is likely indefensible, or some other inconsistency would need to be fixed.) TODO in follow-up work: there should be no reason for Get() to depend on current prefix extractor at all. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9427 Test Plan: Unit tests updated. Performance (using db_bench update) Create DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12 -whole_key_filtering=0` Test with and without `-total_order_seek` on `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=readrandom -num=10000000 -duration=40 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Before this change, total_order_seek=false: 25188 ops/sec Before this change, total_order_seek=true: 1222 ops/sec (~20x slower) After this change, total_order_seek=false: 24570 ops/sec After this change, total_order_seek=true: 25012 ops/sec (indistinguishable) Reviewed By: siying Differential Revision: D33753458 Pulled By: pdillinger fbshipit-source-id: bf892f34907a5e407d9c40bd4d42f0adbcbe0014
3 years ago
} else if (!PrefixExtractorChanged(prefix_extractor) &&
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
prefix_extractor->InDomain(user_key_without_ts)) {
Ignore `total_order_seek` in DB::Get (#9427) Summary: Apparently setting total_order_seek=true for DB::Get was intended to allow accurate read semantics if the current prefix extractor doesn't match what was used to generate SST files on disk. But since prefix_extractor was made a mutable option in 5.14.0, we have been able to detect this case and provide the correct semantics regardless of the total_order_seek option. Since that time, the option has only made Get() slower in a reasonably common case: prefix_extractor unchanged and whole_key_filtering=false. So this change primarily removes unnecessary effect of total_order_seek on Get. Also cleans up some related comments. Also adds a -total_order_seek option to db_bench and canonicalizes handling of ReadOptions in db_bench so that command line options have the expected association with library features. (There is potential for change in regression test behavior, but the old behavior is likely indefensible, or some other inconsistency would need to be fixed.) TODO in follow-up work: there should be no reason for Get() to depend on current prefix extractor at all. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9427 Test Plan: Unit tests updated. Performance (using db_bench update) Create DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12 -whole_key_filtering=0` Test with and without `-total_order_seek` on `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=readrandom -num=10000000 -duration=40 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Before this change, total_order_seek=false: 25188 ops/sec Before this change, total_order_seek=true: 1222 ops/sec (~20x slower) After this change, total_order_seek=false: 24570 ops/sec After this change, total_order_seek=true: 25012 ops/sec (indistinguishable) Reviewed By: siying Differential Revision: D33753458 Pulled By: pdillinger fbshipit-source-id: bf892f34907a5e407d9c40bd4d42f0adbcbe0014
3 years ago
// FIXME ^^^: there should be no reason for Get() to depend on current
// prefix_extractor at all. It should always use table_prefix_extractor.
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
may_match = filter->PrefixMayMatch(
prefix_extractor->Transform(user_key_without_ts), no_io, const_ikey_ptr,
get_context, lookup_context, read_options);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_CHECKED);
if (may_match) {
// Includes prefix stats
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, 1, rep_->level);
} else {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_USEFUL);
// Includes prefix stats
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level);
}
}
return may_match;
}
Introduce a new MultiGet batching implementation (#5011) Summary: This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching. Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to - 1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch() 2. Bloom filter cachelines can be prefetched, hiding the cache miss latency The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress. Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32). Batch Sizes 1 | 2 | 4 | 8 | 16 | 32 Random pattern (Stride length 0) 4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get 4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching) 4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching) Good locality (Stride length 16) 4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753 4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781 4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135 Good locality (Stride length 256) 4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232 4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268 4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62 Medium locality (Stride length 4096) 4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555 4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465 4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891 dbbench command used (on a DB with 4 levels, 12 million keys)- TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011 Differential Revision: D14348703 Pulled By: anand1976 fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
6 years ago
void BlockBasedTable::FullFilterKeysMayMatch(
Ignore `total_order_seek` in DB::Get (#9427) Summary: Apparently setting total_order_seek=true for DB::Get was intended to allow accurate read semantics if the current prefix extractor doesn't match what was used to generate SST files on disk. But since prefix_extractor was made a mutable option in 5.14.0, we have been able to detect this case and provide the correct semantics regardless of the total_order_seek option. Since that time, the option has only made Get() slower in a reasonably common case: prefix_extractor unchanged and whole_key_filtering=false. So this change primarily removes unnecessary effect of total_order_seek on Get. Also cleans up some related comments. Also adds a -total_order_seek option to db_bench and canonicalizes handling of ReadOptions in db_bench so that command line options have the expected association with library features. (There is potential for change in regression test behavior, but the old behavior is likely indefensible, or some other inconsistency would need to be fixed.) TODO in follow-up work: there should be no reason for Get() to depend on current prefix extractor at all. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9427 Test Plan: Unit tests updated. Performance (using db_bench update) Create DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12 -whole_key_filtering=0` Test with and without `-total_order_seek` on `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=readrandom -num=10000000 -duration=40 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Before this change, total_order_seek=false: 25188 ops/sec Before this change, total_order_seek=true: 1222 ops/sec (~20x slower) After this change, total_order_seek=false: 24570 ops/sec After this change, total_order_seek=true: 25012 ops/sec (indistinguishable) Reviewed By: siying Differential Revision: D33753458 Pulled By: pdillinger fbshipit-source-id: bf892f34907a5e407d9c40bd4d42f0adbcbe0014
3 years ago
FilterBlockReader* filter, MultiGetRange* range, const bool no_io,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
const SliceTransform* prefix_extractor,
BlockCacheLookupContext* lookup_context,
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
const ReadOptions& read_options) const {
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
if (filter == nullptr) {
Introduce a new MultiGet batching implementation (#5011) Summary: This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching. Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to - 1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch() 2. Bloom filter cachelines can be prefetched, hiding the cache miss latency The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress. Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32). Batch Sizes 1 | 2 | 4 | 8 | 16 | 32 Random pattern (Stride length 0) 4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get 4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching) 4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching) Good locality (Stride length 16) 4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753 4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781 4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135 Good locality (Stride length 256) 4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232 4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268 4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62 Medium locality (Stride length 4096) 4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555 4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465 4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891 dbbench command used (on a DB with 4 levels, 12 million keys)- TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4 Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011 Differential Revision: D14348703 Pulled By: anand1976 fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
6 years ago
return;
}
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
uint64_t before_keys = range->KeysLeft();
assert(before_keys > 0); // Caller should ensure
if (rep_->whole_key_filtering) {
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
filter->KeysMayMatch(range, no_io, lookup_context, read_options);
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
uint64_t after_keys = range->KeysLeft();
if (after_keys) {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE, after_keys);
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, after_keys,
rep_->level);
}
uint64_t filtered_keys = before_keys - after_keys;
if (filtered_keys) {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL, filtered_keys);
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, filtered_keys,
rep_->level);
}
Ignore `total_order_seek` in DB::Get (#9427) Summary: Apparently setting total_order_seek=true for DB::Get was intended to allow accurate read semantics if the current prefix extractor doesn't match what was used to generate SST files on disk. But since prefix_extractor was made a mutable option in 5.14.0, we have been able to detect this case and provide the correct semantics regardless of the total_order_seek option. Since that time, the option has only made Get() slower in a reasonably common case: prefix_extractor unchanged and whole_key_filtering=false. So this change primarily removes unnecessary effect of total_order_seek on Get. Also cleans up some related comments. Also adds a -total_order_seek option to db_bench and canonicalizes handling of ReadOptions in db_bench so that command line options have the expected association with library features. (There is potential for change in regression test behavior, but the old behavior is likely indefensible, or some other inconsistency would need to be fixed.) TODO in follow-up work: there should be no reason for Get() to depend on current prefix extractor at all. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9427 Test Plan: Unit tests updated. Performance (using db_bench update) Create DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12 -whole_key_filtering=0` Test with and without `-total_order_seek` on `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=readrandom -num=10000000 -duration=40 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Before this change, total_order_seek=false: 25188 ops/sec Before this change, total_order_seek=true: 1222 ops/sec (~20x slower) After this change, total_order_seek=false: 24570 ops/sec After this change, total_order_seek=true: 25012 ops/sec (indistinguishable) Reviewed By: siying Differential Revision: D33753458 Pulled By: pdillinger fbshipit-source-id: bf892f34907a5e407d9c40bd4d42f0adbcbe0014
3 years ago
} else if (!PrefixExtractorChanged(prefix_extractor)) {
// FIXME ^^^: there should be no reason for MultiGet() to depend on current
// prefix_extractor at all. It should always use table_prefix_extractor.
filter->PrefixesMayMatch(range, prefix_extractor, false, lookup_context,
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
read_options);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_CHECKED, before_keys);
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
uint64_t after_keys = range->KeysLeft();
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
if (after_keys) {
// Includes prefix stats
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, after_keys,
rep_->level);
}
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
uint64_t filtered_keys = before_keys - after_keys;
if (filtered_keys) {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_USEFUL,
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
filtered_keys);
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
// Includes prefix stats
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, filtered_keys,
rep_->level);
Basic MultiGet support for partitioned filters (#6757) Summary: In MultiGet, access each applicable filter partition only once per batch, rather than for each applicable key. Also, * Fix Bloom stats for MultiGet * Fix/refactor MultiGetContext::Range::KeysLeft, including * Add efficient BitsSetToOne implementation * Assert that MultiGetContext::Range does not go beyond shift range Performance test: Generate db: $ ./db_bench --benchmarks=fillrandom --num=15000000 --cache_index_and_filter_blocks -bloom_bits=10 -partition_index_and_filters=true ... Before (middle performing run of three; note some missing Bloom stats): $ ./db_bench --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 26.403 micros/op 597517 ops/sec; (548427 of 671968 found) rocksdb.block.cache.filter.hit COUNT : 83443275 rocksdb.bloom.filter.useful COUNT : 0 rocksdb.bloom.filter.full.positive COUNT : 0 rocksdb.bloom.filter.full.true.positive COUNT : 7931450 rocksdb.number.multiget.get COUNT : 385984 rocksdb.number.multiget.keys.read COUNT : 12351488 rocksdb.number.multiget.bytes.read COUNT : 793145000 rocksdb.number.multiget.keys.found COUNT : 7931450 After (middle performing run of three): $ ./db_bench_new --use-existing-db --benchmarks=multireadrandom --num=15000000 --cache_index_and_filter_blocks --bloom_bits=10 --threads=16 --cache_size=20000000 -partition_index_and_filters -batch_size=32 -multiread_batched -statistics --duration=20 2>&1 | egrep 'micros/op|block.cache.filter.hit|bloom.filter.(full|use)|number.multiget' multireadrandom : 21.024 micros/op 752963 ops/sec; (705188 of 863968 found) rocksdb.block.cache.filter.hit COUNT : 49856682 rocksdb.bloom.filter.useful COUNT : 45684579 rocksdb.bloom.filter.full.positive COUNT : 10395458 rocksdb.bloom.filter.full.true.positive COUNT : 9908456 rocksdb.number.multiget.get COUNT : 481984 rocksdb.number.multiget.keys.read COUNT : 15423488 rocksdb.number.multiget.bytes.read COUNT : 990845600 rocksdb.number.multiget.keys.found COUNT : 9908456 So that's about 25% higher throughput even for random keys Pull Request resolved: https://github.com/facebook/rocksdb/pull/6757 Test Plan: unit test included Reviewed By: anand1976 Differential Revision: D21243256 Pulled By: pdillinger fbshipit-source-id: 5644a1468d9e8c8575be02f4e04bc5d62dbbb57f
5 years ago
}
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 BlockBasedTable::ApproximateKeyAnchors(const ReadOptions& read_options,
std::vector<Anchor>& anchors) {
// We iterator the whole index block here. More efficient implementation
// is possible if we push this operation into IndexReader. For example, we
// can directly sample from restart block entries in the index block and
// only read keys needed. Here we take a simple solution. Performance is
// likely not to be a problem. We are compacting the whole file, so all
// keys will be read out anyway. An extra read to index block might be
// a small share of the overhead. We can try to optimize if needed.
//
// `CacheDependencies()` brings all the blocks into cache using one I/O. That
// way the full index scan usually finds the index data it is looking for in
// cache rather than doing an I/O for each "dependency" (partition).
Record and use the tail size to prefetch table tail (#11406) Summary: **Context:** We prefetch the tail part of a SST file (i.e, the blocks after data blocks till the end of the file) during each SST file open in hope to prefetch all the stuff at once ahead of time for later read e.g, footer, meta index, filter/index etc. The existing approach to estimate the tail size to prefetch is through `TailPrefetchStats` heuristics introduced in https://github.com/facebook/rocksdb/pull/4156, which has caused small reads in unlucky case (e.g, small read into the tail buffer during table open in thread 1 under the same BlockBasedTableFactory object can make thread 2's tail prefetching use a small size that it shouldn't) and is hard to debug. Therefore we decide to record the exact tail size and use it directly to prefetch tail of the SST instead of relying heuristics. **Summary:** - Obtain and record in manifest the tail size in `BlockBasedTableBuilder::Finish()` - For backward compatibility, we fall back to TailPrefetchStats and last to simple heuristics that the tail size is a linear portion of the file size - see PR conversation for more. - Make`tail_start_offset` part of the table properties and deduct tail size to record in manifest for external files (e.g, file ingestion, import CF) and db repair (with no access to manifest). Pull Request resolved: https://github.com/facebook/rocksdb/pull/11406 Test Plan: 1. New UT 2. db bench Note: db bench on /tmp/ where direct read is supported is too slow to finish and the default pinning setting in db bench is not helpful to profile # sst read of Get. Therefore I hacked the following to obtain the following comparison. ``` diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index bd5669f0f..791484c1f 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -838,7 +838,7 @@ Status BlockBasedTable::PrefetchTail( &tail_prefetch_size); // Try file system prefetch - if (!file->use_direct_io() && !force_direct_prefetch) { + if (false && !file->use_direct_io() && !force_direct_prefetch) { if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) .IsNotSupported()) { prefetch_buffer->reset(new FilePrefetchBuffer( diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index ea40f5fa0..39a0ac385 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -4191,6 +4191,8 @@ class Benchmark { std::shared_ptr<TableFactory>(NewCuckooTableFactory(table_options)); } else { BlockBasedTableOptions block_based_options; + block_based_options.metadata_cache_options.partition_pinning = + PinningTier::kAll; block_based_options.checksum = static_cast<ChecksumType>(FLAGS_checksum_type); if (FLAGS_use_hash_search) { ``` Create DB ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` ReadRandom ``` ./db_bench --bloom_bits=3 --use_existing_db=1 --seed=1682546046158958 --partition_index_and_filters=1 --statistics=1 -db=/dev/shm/testdb/ -benchmarks=readrandom -key_size=3200 -value_size=512 -num=1000000 -write_buffer_size=6550000 -disable_auto_compactions=false -target_file_size_base=6550000 -compression_type=none ``` (a) Existing (Use TailPrefetchStats for tail size + use seperate prefetch buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 3395 rocksdb.sst.read.micros P50 : 5.655570 P95 : 9.931396 P99 : 14.845454 P100 : 585.000000 COUNT : 999905 SUM : 6590614 ``` (b) This PR (Record tail size + use the same tail buffer in PartitionedFilter/IndexReader::CacheDependencies()) ``` rocksdb.table.open.prefetch.tail.hit COUNT : 14257 rocksdb.sst.read.micros P50 : 5.173347 P95 : 9.015017 P99 : 12.912610 P100 : 228.000000 COUNT : 998547 SUM : 5976540 ``` As we can see, we increase the prefetch tail hit count and decrease SST read count with this PR 3. Test backward compatibility by stepping through reading with post-PR code on a db generated pre-PR. Reviewed By: pdillinger Differential Revision: D45413346 Pulled By: hx235 fbshipit-source-id: 7d5e36a60a72477218f79905168d688452a4c064
2 years ago
Status s = rep_->index_reader->CacheDependencies(
read_options, false /* pin */, nullptr /* prefetch_buffer */);
if (!s.ok()) {
return s;
}
IndexBlockIter iiter_on_stack;
auto iiter = NewIndexIterator(
read_options, /*disable_prefix_seek=*/false, &iiter_on_stack,
/*get_context=*/nullptr, /*lookup_context=*/nullptr);
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
if (iiter != &iiter_on_stack) {
iiter_unique_ptr.reset(iiter);
}
// If needed the threshold could be more adaptive. For example, it can be
// based on size, so that a larger will be sampled to more partitions than a
// smaller file. The size might also need to be passed in by the caller based
// on total compaction size.
const uint64_t kMaxNumAnchors = uint64_t{128};
uint64_t num_blocks = this->GetTableProperties()->num_data_blocks;
uint64_t num_blocks_per_anchor = num_blocks / kMaxNumAnchors;
if (num_blocks_per_anchor == 0) {
num_blocks_per_anchor = 1;
}
uint64_t count = 0;
std::string last_key;
uint64_t range_size = 0;
uint64_t prev_offset = 0;
for (iiter->SeekToFirst(); iiter->Valid(); iiter->Next()) {
const BlockHandle& bh = iiter->value().handle;
range_size += bh.offset() + bh.size() - prev_offset;
prev_offset = bh.offset() + bh.size();
if (++count % num_blocks_per_anchor == 0) {
count = 0;
anchors.emplace_back(iiter->user_key(), range_size);
range_size = 0;
} else {
last_key = iiter->user_key().ToString();
}
}
if (count != 0) {
anchors.emplace_back(last_key, range_size);
}
return Status::OK();
}
bool BlockBasedTable::TimestampMayMatch(const ReadOptions& read_options) const {
if (read_options.timestamp != nullptr && !rep_->min_timestamp.empty()) {
RecordTick(rep_->ioptions.stats, TIMESTAMP_FILTER_TABLE_CHECKED);
auto read_ts = read_options.timestamp;
auto comparator = rep_->internal_comparator.user_comparator();
if (comparator->CompareTimestamp(*read_ts, rep_->min_timestamp) < 0) {
RecordTick(rep_->ioptions.stats, TIMESTAMP_FILTER_TABLE_FILTERED);
return false;
}
}
return true;
}
Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
GetContext* get_context,
const SliceTransform* prefix_extractor,
bool skip_filters) {
// Similar to Bloom filter !may_match
// If timestamp is beyond the range of the table, skip
if (!TimestampMayMatch(read_options)) {
return Status::OK();
}
assert(key.size() >= 8); // key must be internal key
assert(get_context != nullptr);
Status s;
const bool no_io = read_options.read_tier == kBlockCacheTier;
FilterBlockReader* const filter =
!skip_filters ? rep_->filter.get() : nullptr;
// First check the full filter
// If full filter not useful, Then go into each block
uint64_t tracing_get_id = get_context->get_tracing_get_id();
BlockCacheLookupContext lookup_context{
TableReaderCaller::kUserGet, tracing_get_id,
/*get_from_user_specified_snapshot=*/read_options.snapshot != nullptr};
if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) {
// Trace the key since it contains both user key and sequence number.
lookup_context.referenced_key = key.ToString();
lookup_context.get_from_user_specified_snapshot =
read_options.snapshot != nullptr;
}
TEST_SYNC_POINT("BlockBasedTable::Get:BeforeFilterMatch");
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
const bool may_match =
FullFilterKeyMayMatch(filter, key, no_io, prefix_extractor, get_context,
&lookup_context, read_options);
TEST_SYNC_POINT("BlockBasedTable::Get:AfterFilterMatch");
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
if (may_match) {
IndexBlockIter iiter_on_stack;
// if prefix_extractor found in block differs from options, disable
// BlockPrefixIndex. Only do this check when index_type is kHashSearch.
bool need_upper_bound_check = false;
if (rep_->index_type == BlockBasedTableOptions::kHashSearch) {
Fast path for detecting unchanged prefix_extractor (#9407) Summary: Fixes a major performance regression in 6.26, where extra CPU is spent in SliceTransform::AsString when reads involve a prefix_extractor (Get, MultiGet, Seek). Common case performance is now better than 6.25. This change creates a "fast path" for verifying that the current prefix extractor is unchanged and compatible with what was used to generate a table file. This fast path detects the common case by pointer comparison on the current prefix_extractor and a "known good" prefix extractor (if applicable) that is saved at the time the table reader is opened. The "known good" prefix extractor is saved as another shared_ptr copy (in an existing field, however) to ensure the pointer is not recycled. When the prefix_extractor has changed to a different instance but same compatible configuration (rare, odd), performance is still a regression compared to 6.25, but this is likely acceptable because of the oddity of such a case. The performance of incompatible prefix_extractor is essentially unchanged. Also fixed a minor case (ForwardIterator) where a prefix_extractor could be used via a raw pointer after being freed as a shared_ptr, if replaced via SetOptions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9407 Test Plan: ## Performance Populate DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Running head-to-head comparisons simultaneously with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=seekrandom -num=10000000 -duration=20 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Below each is compared by ops/sec vs. baseline which is version 6.25 (multiple baseline runs because of variable machine load) v6.26: 4833 vs. 6698 (<- major regression!) v6.27: 4737 vs. 6397 (still) New: 6704 vs. 6461 (better than baseline in common case) Disabled fastpath: 4843 vs. 6389 (e.g. if prefix extractor instance changes but is still compatible) Changed prefix size (no usable filter) in new: 787 vs. 5927 Changed prefix size (no usable filter) in new & baseline: 773 vs. 784 Reviewed By: mrambacher Differential Revision: D33677812 Pulled By: pdillinger fbshipit-source-id: 571d9711c461fb97f957378a061b7e7dbc4d6a76
3 years ago
need_upper_bound_check = PrefixExtractorChanged(prefix_extractor);
}
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
auto iiter =
NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack,
get_context, &lookup_context);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
if (iiter != &iiter_on_stack) {
iiter_unique_ptr.reset(iiter);
}
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
Add support for timestamp in Get/Put (#5079) Summary: It's useful to be able to (optionally) associate key-value pairs with user-provided timestamps. This PR is an early effort towards this goal and continues the work of facebook#4942. A suite of new unit tests exist in DBBasicTestWithTimestampWithParam. Support for timestamp requires the user to provide timestamp as a slice in `ReadOptions` and `WriteOptions`. All timestamps of the same database must share the same length, format, etc. The format of the timestamp is the same throughout the same database, and the user is responsible for providing a comparator function (Comparator) to order the <key, timestamp> tuples. Once created, the format and length of the timestamp cannot change (at least for now). Test plan (on devserver): ``` $COMPILE_WITH_ASAN=1 make -j32 all $./db_basic_test --gtest_filter=Timestamp/DBBasicTestWithTimestampWithParam.PutAndGet/* $make check ``` All tests must pass. We also run the following db_bench tests to verify whether there is regression on Get/Put while timestamp is not enabled. ``` $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillseq,readrandom -num=1000000 $TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=1000000 ``` Repeat for 6 times for both versions. Results are as follows: ``` | | readrandom | fillrandom | | master | 16.77 MB/s | 47.05 MB/s | | PR5079 | 16.44 MB/s | 47.03 MB/s | ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/5079 Differential Revision: D15132946 Pulled By: riversand963 fbshipit-source-id: 833a0d657eac21182f0f206c910a6438154c742c
6 years ago
size_t ts_sz =
rep_->internal_comparator.user_comparator()->timestamp_size();
bool matched = false; // if such user key matched a key in SST
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
bool done = false;
for (iiter->Seek(key); iiter->Valid() && !done; iiter->Next()) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
IndexValue v = iiter->value();
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
if (!v.first_internal_key.empty() && !skip_filters &&
UserComparatorWrapper(rep_->internal_comparator.user_comparator())
.CompareWithoutTimestamp(
ExtractUserKey(key),
ExtractUserKey(v.first_internal_key)) < 0) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
// The requested key falls between highest key in previous block and
// lowest key in current block.
break;
}
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
BlockCacheLookupContext lookup_data_block_context{
TableReaderCaller::kUserGet, tracing_get_id,
/*get_from_user_specified_snapshot=*/read_options.snapshot !=
nullptr};
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
bool does_referenced_key_exist = false;
DataBlockIter biter;
uint64_t referenced_data_size = 0;
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
Status tmp_status;
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
NewDataBlockIterator<DataBlockIter>(
read_options, v.handle, &biter, BlockType::kData, get_context,
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
&lookup_data_block_context, /*prefetch_buffer=*/nullptr,
/*for_compaction=*/false, /*async_read=*/false, tmp_status);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
if (no_io && biter.status().IsIncomplete()) {
// couldn't get block from block_cache
// Update Saver.state to Found because we are only looking for
// whether we can guarantee the key is not there when "no_io" is set
get_context->MarkKeyMayExist();
s = biter.status();
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
break;
}
if (!biter.status().ok()) {
s = biter.status();
break;
}
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
bool may_exist = biter.SeekForGet(key);
// If user-specified timestamp is supported, we cannot end the search
// just because hash index lookup indicates the key+ts does not exist.
if (!may_exist && ts_sz == 0) {
// HashSeek cannot find the key this block and the the iter is not
// the end of the block, i.e. cannot be in the following blocks
// either. In this case, the seek_key cannot be found, so we break
// from the top level for-loop.
done = true;
} else {
// Call the *saver function on each entry/block until it returns false
for (; biter.Valid(); biter.Next()) {
ParsedInternalKey parsed_key;
Status pik_status = ParseInternalKey(
biter.key(), &parsed_key, false /* log_err_key */); // TODO
if (!pik_status.ok()) {
s = pik_status;
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
}
if (!get_context->SaveValue(
parsed_key, biter.value(), &matched,
biter.IsValuePinned() ? &biter : nullptr)) {
if (get_context->State() == GetContext::GetState::kFound) {
does_referenced_key_exist = true;
referenced_data_size = biter.key().size() + biter.value().size();
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
done = true;
break;
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
}
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
s = biter.status();
Block per key-value checksum (#11287) Summary: add option `block_protection_bytes_per_key` and implementation for block per key-value checksum. The main changes are 1. checksum construction and verification in block.cc/h 2. pass the option `block_protection_bytes_per_key` around (mainly for methods defined in table_cache.h) 3. unit tests/crash test updates Tests: * Added unit tests * Crash test: `python3 tools/db_crashtest.py blackbox --simple --block_protection_bytes_per_key=1 --write_buffer_size=1048576` Follow up (maybe as a separate PR): make sure corruption status returned from BlockIters are correctly handled. Performance: Turning on block per KV protection has a non-trivial negative impact on read performance and costs additional memory. For memory, each block includes additional 24 bytes for checksum-related states beside checksum itself. For CPU, I set up a DB of size ~1.2GB with 5M keys (32 bytes key and 200 bytes value) which compacts to ~5 SST files (target file size 256 MB) in L6 without compression. I tested readrandom performance with various block cache size (to mimic various cache hit rates): ``` SETUP make OPTIMIZE_LEVEL="-O3" USE_LTO=1 DEBUG_LEVEL=0 -j32 db_bench ./db_bench -benchmarks=fillseq,compact0,waitforcompaction,compact,waitforcompaction -write_buffer_size=33554432 -level_compaction_dynamic_level_bytes=true -max_background_jobs=8 -target_file_size_base=268435456 --num=5000000 --key_size=32 --value_size=200 --compression_type=none BENCHMARK ./db_bench --use_existing_db -benchmarks=readtocache,readrandom[-X10] --num=5000000 --key_size=32 --disable_auto_compactions --reads=1000000 --block_protection_bytes_per_key=[0|1] --cache_size=$CACHESIZE The readrandom ops/sec looks like the following: Block cache size: 2GB 1.2GB * 0.9 1.2GB * 0.8 1.2GB * 0.5 8MB Main 240805 223604 198176 161653 139040 PR prot_bytes=0 238691 226693 200127 161082 141153 PR prot_bytes=1 214983 193199 178532 137013 108211 prot_bytes=1 vs -10% -15% -10.8% -15% -23% prot_bytes=0 ``` The benchmark has a lot of variance, but there was a 5% to 25% regression in this benchmark with different cache hit rates. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11287 Reviewed By: ajkr Differential Revision: D43970708 Pulled By: cbi42 fbshipit-source-id: ef98d898b71779846fa74212b9ec9e08b7183940
2 years ago
if (!s.ok()) {
break;
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
}
// Write the block cache access record.
if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
// Avoid making copy of block_key, cf_name, and referenced_key when
// constructing the access record.
Slice referenced_key;
if (does_referenced_key_exist) {
referenced_key = biter.key();
} else {
referenced_key = key;
}
Refactor block cache tracing w/improved MultiGet (#11339) Summary: After https://github.com/facebook/rocksdb/issues/11301, I wasn't sure whether I had regressed block cache tracing with MultiGet. Demo PR https://github.com/facebook/rocksdb/issues/11330 shows the flawed state of tracing MultiGet before my change, and based on the unit test, there was essentially no change in tracing behavior with https://github.com/facebook/rocksdb/issues/11301. This change is to leave that code and behavior better than I found it. This change is not intended to change any production behaviors except when block cache tracing is active, though might improve general read path efficiency by disabling some related tracking when such tracing is disabled. More detail on production code: * Refactoring to consolidate the construction of BlockCacheTraceRecord, and other related functionality, in block-based table reader, though it's somewhat awkward to preserve an optimization to avoid copying Slices into temporary strings in BlockCacheLookupContext. * Accurately track cache hits and misses (etc.) for each data block accessed by a MultiGet(). (Previously reported hits as misses.) * Reduced repeated checking of `block_cache_tracer_` state (by creating lookup_context only when active) for efficiency and to reduce the risk of corner case bugs where tracing is enabled or disabled for different parts of a read op. (See a TODO below) * Improved estimate calculation for num_keys_in_block (see code comment) Possible follow-up: * `XXX:` use_cache=true means double cache query? (possible double-query of block cache when allow_mmap_reads=true) * `TODO:` need more than one lookup_context here to track individual filter and index partition hits and misses * `TODO:` optimize more state checks of `block_cache_tracer_` down to `lookup_context != nullptr` * Pre-existing `XXX:` There appear to be 'break' statements above that bypass this writing of the block cache trace record * Expand test coverage (see below) Pull Request resolved: https://github.com/facebook/rocksdb/pull/11339 Test Plan: * Added a basic unit test for block cache tracing MultiGet, for now just covering one data block with two keys. * Added HitMissCountingCache to independently verify that the actual block cache trace and expected block cache trace also agree with the actual number of cache hits / misses (nothing missing or mislabeled). For now only used with MultiGet test. * Better testing of num_keys_in_block, for now just with MultiGet * Misc improvements to table_test to improve clarity, such as making it clear that certain keys are auto-inserted at the start of every test. Performance test: Testing multireadrandom as in https://github.com/facebook/rocksdb/issues/11301, except averaging over distinct runs rather than [-X30] which doesn't seem to sufficiently reset after each run to work as an independent test run. Base with revert of 11301: 3148926 ops/sec Base: 3019146 ops/sec New: 2999529 ops/sec Possibly a tiny MultiGet CPU regression with this change. We are now always allocating an additional vector for the LookupContexts. I'm still contemplating options to try to correct the regression in https://github.com/facebook/rocksdb/issues/11301. Testing readrandom: Base with revert of 11301: 2311988 Base: 2281726 New: 2299722 Possibly a tiny Get CPU improvement with this change. We are now avoiding some unnecessary LookupContext population. Reviewed By: akankshamahajan15 Differential Revision: D44557845 Pulled By: pdillinger fbshipit-source-id: b841691799d2a48fb59cc8880dc7cbb1e107ae3d
2 years ago
FinishTraceRecord(lookup_data_block_context,
lookup_data_block_context.block_key, referenced_key,
does_referenced_key_exist, referenced_data_size);
}
if (done) {
// Avoid the extra Next which is expensive in two-level indexes
break;
}
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
}
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
if (matched && filter != nullptr) {
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
if (rep_->whole_key_filtering) {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE);
} else {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_TRUE_POSITIVE);
}
// Includes prefix stats
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1,
rep_->level);
}
Much better stats for seeks and prefix filtering (#11460) Summary: We want to know more about opportunities for better range filters, and the effectiveness of our own range filters. Currently the stats are very limited, essentially logging just hits and misses against prefix filters for range scans in BLOOM_FILTER_PREFIX_* without tracking the false positive rate. Perhaps confusingly, when prefix filters are used for point queries, the stats are currently going into the non-PREFIX tickers. This change does several things: * Introduce new stat tickers for seeks and related filtering, \*LEVEL_SEEK\* * Most importantly, allows us to see opportunities for range filtering. Specifically, we can count how many times a seek in an SST file accesses at least one data block, and how many times at least one value() is then accessed. If a data block was accessed but no value(), we can generally assume that the key(s) seen was(were) not of interest so could have been filtered with the right kind of filter, avoiding the data block access. * We can get the same level of detail when a filter (for now, prefix Bloom/ribbon) is used, or not. Specifically, we can infer a false positive rate for prefix filters (not available before) from the seek "false positive" rate: when a data block is accessed but no value() is called. (There can be other explanations for a seek false positive, but in typical iterator usage it would indicate a filter false positive.) * For efficiency, I wanted to avoid making additional calls to the prefix extractor (or key comparisons, etc.), which would be required if we wanted to more precisely detect filter false positives. I believe that instrumenting value() is the best balance of efficiency vs. accurately measuring what we are often interested in. * The stats are divided between last level and non-last levels, to help understand potential tiered storage use cases. * The old BLOOM_FILTER_PREFIX_* stats have a different meaning: no longer referring to iterators but to point queries using prefix filters. BLOOM_FILTER_PREFIX_TRUE_POSITIVE is added for computing the prefix false positive rate on point queries, which can be due to filter false positives as well as different keys with the same prefix. * Similarly, the non-PREFIX BLOOM_FILTER stats are now for whole key filtering only. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11460 Test Plan: unit tests updated, including updating many to pop the stat value since last read to improve test readability and maintainability. Performance test shows a consistent small improvement with these changes, both with clang and with gcc. CPU profile indicates that RecordTick is using less CPU, and this makes sense at least for a high filter miss rate. Before, we were recording two ticks per filter miss in iterators (CHECKED & USEFUL) and now recording just one (FILTERED). Create DB with ``` TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=30000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 ``` And run simultaneous before&after with ``` TEST_TMPDIR=/dev/shm ./db_bench -readonly -benchmarks=seekrandom[-X1000] -num=10000000 -bloom_bits=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=8 -seek_nexts=1 -duration=20 -seed=43 -threads=8 -cache_size=1000000000 -statistics ``` Before: seekrandom [AVG 275 runs] : 189680 (± 222) ops/sec; 18.4 (± 0.0) MB/sec After: seekrandom [AVG 275 runs] : 197110 (± 208) ops/sec; 19.1 (± 0.0) MB/sec Reviewed By: ajkr Differential Revision: D46029177 Pulled By: pdillinger fbshipit-source-id: cdace79a2ea548d46c5900b068c5b7c3a02e5822
2 years ago
if (s.ok() && !iiter->status().IsNotFound()) {
s = iiter->status();
}
}
return s;
}
Status BlockBasedTable::MultiGetFilter(const ReadOptions& read_options,
const SliceTransform* prefix_extractor,
MultiGetRange* mget_range) {
if (mget_range->empty()) {
// Caller should ensure non-empty (performance bug)
assert(false);
return Status::OK(); // Nothing to do
}
FilterBlockReader* const filter = rep_->filter.get();
if (!filter) {
return Status::OK();
}
// First check the full filter
// If full filter not useful, Then go into each block
const bool no_io = read_options.read_tier == kBlockCacheTier;
uint64_t tracing_mget_id = BlockCacheTraceHelper::kReservedGetId;
if (mget_range->begin()->get_context) {
tracing_mget_id = mget_range->begin()->get_context->get_tracing_get_id();
}
BlockCacheLookupContext lookup_context{
TableReaderCaller::kUserMultiGet, tracing_mget_id,
/*_get_from_user_specified_snapshot=*/read_options.snapshot != nullptr};
FullFilterKeysMayMatch(filter, mget_range, no_io, prefix_extractor,
Use user-provided ReadOptions for metadata block reads more often (#11208) Summary: This is mostly taken from https://github.com/facebook/rocksdb/issues/10427 with my own comments addressed. This PR plumbs the user’s `ReadOptions` down to `GetOrReadIndexBlock()`, `GetOrReadFilterBlock()`, and `GetFilterPartitionBlock()`. Now those functions no longer have to make up a `ReadOptions` with incomplete information. I also let `PartitionIndexReader::NewIterator()` pass through its caller's `ReadOptions::verify_checksums`, which was inexplicably dropped previously. Fixes https://github.com/facebook/rocksdb/issues/10463 Pull Request resolved: https://github.com/facebook/rocksdb/pull/11208 Test Plan: Functional: - Measured `-verify_checksum=false` applies to metadata blocks read outside of table open - setup command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56` - run command: `TEST_TMPDIR=/tmp/100M-DB/ ./db_bench -benchmarks=readrandom -use_existing_db=true -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -compression_type=none -num=1638400 -key_size=8 -value_size=56 -duration=10 -threads=32 -cache_size=131072 -statistics=true -verify_checksum=false -open_files=20 -cache_index_and_filter_blocks=true` - before: `rocksdb.block.checksum.compute.count COUNT : 384353` - after: `rocksdb.block.checksum.compute.count COUNT : 22` Performance: - Setup command (tmpfs, 128MB logical data size, cache indexes/filters without pinning so index/filter lookups go through table reader): `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=filluniquerandom,waitforcompaction -write_buffer_size=131072 -target_file_size_base=131072 -max_bytes_for_level_base=524288 -compression_type=none -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1` - Measured point lookup performance. Database is fully cached to emphasize any new callstack overheads - Command: `TEST_TMPDIR=/dev/shm/128M-DB/ ./db_bench -benchmarks=readrandom[-W1][-X20] -use_existing_db=true -cache_index_and_filter_blocks=true -disable_auto_compactions=true -num=4194304 -key_size=8 -value_size=24 -bloom_bits=8 -whole_key_filtering=1 -duration=10 -cache_size=1048576000` - Before: `readrandom [AVG 20 runs] : 274848 (± 3717) ops/sec; 8.4 (± 0.1) MB/sec` - After: `readrandom [AVG 20 runs] : 277904 (± 4474) ops/sec; 8.5 (± 0.1) MB/sec` Reviewed By: hx235 Differential Revision: D43145366 Pulled By: ajkr fbshipit-source-id: 75ec062ece86a82cd788783de9de2c72df57f994
2 years ago
&lookup_context, read_options);
return Status::OK();
}
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
Status BlockBasedTable::Prefetch(const ReadOptions& read_options,
const Slice* const begin,
const Slice* const end) {
auto& comparator = rep_->internal_comparator;
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
UserComparatorWrapper user_comparator(comparator.user_comparator());
// pre-condition
if (begin && end && comparator.Compare(*begin, *end) > 0) {
return Status::InvalidArgument(*begin, *end);
}
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
IndexBlockIter iiter_on_stack;
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
auto iiter = NewIndexIterator(read_options, /*need_upper_bound_check=*/false,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
&iiter_on_stack, /*get_context=*/nullptr,
&lookup_context);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
if (iiter != &iiter_on_stack) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter);
}
if (!iiter->status().ok()) {
// error opening index iterator
return iiter->status();
}
// indicates if we are on the last page that need to be pre-fetched
bool prefetching_boundary_page = false;
for (begin ? iiter->Seek(*begin) : iiter->SeekToFirst(); iiter->Valid();
iiter->Next()) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
BlockHandle block_handle = iiter->value().handle;
const bool is_user_key = !rep_->index_key_includes_seq;
if (end &&
((!is_user_key && comparator.Compare(iiter->key(), *end) >= 0) ||
(is_user_key &&
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
user_comparator.Compare(iiter->key(), ExtractUserKey(*end)) >= 0))) {
if (prefetching_boundary_page) {
break;
}
// The index entry represents the last key in the data block.
// We should load this page into memory as well, but no more
prefetching_boundary_page = true;
}
// Load the block specified by the block_handle into the block cache
DataBlockIter biter;
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
Status tmp_status;
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
NewDataBlockIterator<DataBlockIter>(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
read_options, block_handle, &biter, /*type=*/BlockType::kData,
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
/*get_context=*/nullptr, &lookup_context,
/*prefetch_buffer=*/nullptr, /*for_compaction=*/false,
/*async_read=*/false, tmp_status);
if (!biter.status().ok()) {
// there was an unexpected error while pre-fetching
return biter.status();
}
}
return Status::OK();
}
Status BlockBasedTable::VerifyChecksum(const ReadOptions& read_options,
TableReaderCaller caller) {
Status s;
// Check Meta blocks
std::unique_ptr<Block> metaindex;
std::unique_ptr<InternalIterator> metaindex_iter;
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
s = ReadMetaIndexBlock(read_options, nullptr /* prefetch buffer */,
&metaindex, &metaindex_iter);
if (s.ok()) {
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
s = VerifyChecksumInMetaBlocks(read_options, metaindex_iter.get());
if (!s.ok()) {
return s;
}
} else {
return s;
}
// Check Data blocks
IndexBlockIter iiter_on_stack;
BlockCacheLookupContext context{caller};
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
InternalIteratorBase<IndexValue>* iiter = NewIndexIterator(
read_options, /*disable_prefix_seek=*/false, &iiter_on_stack,
/*get_context=*/nullptr, &context);
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
if (iiter != &iiter_on_stack) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter);
}
if (!iiter->status().ok()) {
// error opening index iterator
return iiter->status();
}
s = VerifyChecksumInBlocks(read_options, iiter);
return s;
}
Status BlockBasedTable::VerifyChecksumInBlocks(
const ReadOptions& read_options,
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
InternalIteratorBase<IndexValue>* index_iter) {
Status s;
// We are scanning the whole file, so no need to do exponential
// increasing of the buffer size.
size_t readahead_size = (read_options.readahead_size != 0)
? read_options.readahead_size
: rep_->table_options.max_auto_readahead_size;
// FilePrefetchBuffer doesn't work in mmap mode and readahead is not
// needed there.
FilePrefetchBuffer prefetch_buffer(
readahead_size /* readahead_size */,
readahead_size /* max_readahead_size */,
!rep_->ioptions.allow_mmap_reads /* enable */);
for (index_iter->SeekToFirst(); index_iter->Valid(); index_iter->Next()) {
s = index_iter->status();
if (!s.ok()) {
break;
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
BlockHandle handle = index_iter->value().handle;
BlockContents contents;
BlockFetcher block_fetcher(
Add rate limiter priority to ReadOptions (#9424) Summary: Users can set the priority for file reads associated with their operation by setting `ReadOptions::rate_limiter_priority` to something other than `Env::IO_TOTAL`. Rate limiting `VerifyChecksum()` and `VerifyFileChecksums()` is the motivation for this PR, so it also includes benchmarks and minor bug fixes to get that working. `RandomAccessFileReader::Read()` already had support for rate limiting compaction reads. I changed that rate limiting to be non-specific to compaction, but rather performed according to the passed in `Env::IOPriority`. Now the compaction read rate limiting is supported by setting `rate_limiter_priority = Env::IO_LOW` on its `ReadOptions`. There is no default value for the new `Env::IOPriority` parameter to `RandomAccessFileReader::Read()`. That means this PR goes through all callers (in some cases multiple layers up the call stack) to find a `ReadOptions` to provide the priority. There are TODOs for cases I believe it would be good to let user control the priority some day (e.g., file footer reads), and no TODO in cases I believe it doesn't matter (e.g., trace file reads). The API doc only lists the missing cases where a file read associated with a provided `ReadOptions` cannot be rate limited. For cases like file ingestion checksum calculation, there is no API to provide `ReadOptions` or `Env::IOPriority`, so I didn't count that as missing. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9424 Test Plan: - new unit tests - new benchmarks on ~50MB database with 1MB/s read rate limit and 100ms refill interval; verified with strace reads are chunked (at 0.1MB per chunk) and spaced roughly 100ms apart. - setup command: `./db_bench -benchmarks=fillrandom,compact -db=/tmp/testdb -target_file_size_base=1048576 -disable_auto_compactions=true -file_checksum=true` - benchmarks command: `strace -ttfe pread64 ./db_bench -benchmarks=verifychecksum,verifyfilechecksums -use_existing_db=true -db=/tmp/testdb -rate_limiter_bytes_per_sec=1048576 -rate_limit_bg_reads=1 -rate_limit_user_ops=true -file_checksum=true` - crash test using IO_USER priority on non-validation reads with https://github.com/facebook/rocksdb/issues/9567 reverted: `python3 tools/db_crashtest.py blackbox --max_key=1000000 --write_buffer_size=524288 --target_file_size_base=524288 --level_compaction_dynamic_level_bytes=true --duration=3600 --rate_limit_bg_reads=true --rate_limit_user_ops=true --rate_limiter_bytes_per_sec=10485760 --interval=10` Reviewed By: hx235 Differential Revision: D33747386 Pulled By: ajkr fbshipit-source-id: a2d985e97912fba8c54763798e04f006ccc56e0c
3 years ago
rep_->file.get(), &prefetch_buffer, rep_->footer, read_options, handle,
&contents, rep_->ioptions, false /* decompress */,
false /*maybe_compressed*/, BlockType::kData,
UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options);
s = block_fetcher.ReadBlockContents();
if (!s.ok()) {
break;
}
}
if (s.ok()) {
// In the case of two level indexes, we would have exited the above loop
// by checking index_iter->Valid(), but Valid() might have returned false
// due to an IO error. So check the index_iter status
s = index_iter->status();
}
return s;
}
BlockType BlockBasedTable::GetBlockTypeForMetaBlockByName(
const Slice& meta_block_name) {
if (meta_block_name.starts_with(kFullFilterBlockPrefix)) {
return BlockType::kFilter;
}
if (meta_block_name.starts_with(kPartitionedFilterBlockPrefix)) {
return BlockType::kFilterPartitionIndex;
}
if (meta_block_name == kPropertiesBlockName) {
return BlockType::kProperties;
}
if (meta_block_name == kCompressionDictBlockName) {
return BlockType::kCompressionDictionary;
}
if (meta_block_name == kRangeDelBlockName) {
return BlockType::kRangeDeletion;
}
if (meta_block_name == kHashIndexPrefixesBlock) {
return BlockType::kHashIndexPrefixes;
}
if (meta_block_name == kHashIndexPrefixesMetadataBlock) {
return BlockType::kHashIndexMetadata;
}
format_version=6 and context-aware block checksums (#9058) Summary: ## Context checksum All RocksDB checksums currently use 32 bits of checking power, which should be 1 in 4 billion false negative (FN) probability (failing to detect corruption). This is true for random corruptions, and in some cases small corruptions are guaranteed to be detected. But some possible corruptions, such as in storage metadata rather than storage payload data, would have a much higher FN rate. For example: * Data larger than one SST block is replaced by data from elsewhere in the same or another SST file. Especially with block_align=true, the probability of exact block size match is probably around 1 in 100, making the FN probability around that same. Without `block_align=true` the probability of same block start location is probably around 1 in 10,000, for FN probability around 1 in a million. To solve this problem in new format_version=6, we add "context awareness" to block checksum checks. The stored and expected checksum value is modified based on the block's position in the file and which file it is in. The modifications are cleverly chosen so that, for example * blocks within about 4GB of each other are guaranteed to use different context * blocks that are offset by exactly some multiple of 4GiB are guaranteed to use different context * files generated by the same process are guaranteed to use different context for the same offsets, until wrap-around after 2^32 - 1 files Thus, with format_version=6, if a valid SST block and checksum is misplaced, its checksum FN probability should be essentially ideal, 1 in 4B. ## Footer checksum This change also adds checksum protection to the SST footer (with format_version=6), for the first time without relying on whole file checksum. To prevent a corruption of the format_version in the footer (e.g. 6 -> 5) to defeat the footer checksum, we change much of the footer data format including an "extended magic number" in format_version 6 that would be interpreted as empty index and metaindex block handles in older footer versions. We also change the encoding of handles to free up space for other new data in footer. ## More detail: making space in footer In order to keep footer the same size in format_version=6 (avoid change to IO patterns), we have to free up some space for new data. We do this two ways: * Metaindex block handle is encoded down to 4 bytes (from 10) by assuming it immediately precedes the footer, and by assuming it is < 4GB. * Index block handle is moved into metaindex. (I don't know why it was in footer to begin with.) ## Performance In case of small performance penalty, I've made a "pay as you go" optimization to compensate: replace `MutableCFOptions` in BlockBasedTableBuilder::Rep with the only field used in that structure after construction: `prefix_extractor`. This makes the PR an overall performance improvement (results below). Nevertheless I'm seeing essentially no difference going from fv=5 to fv=6, even including that improvement for both. That's based on extreme case table write performance testing, many files with many blocks. This is relatively checksum intensive (small blocks) and salt generation intensive (small files). ``` (for I in `seq 1 100`; do TEST_TMPDIR=/dev/shm/dbbench2 ./db_bench -benchmarks=fillseq -memtablerep=vector -disable_wal=1 -allow_concurrent_memtable_write=false -num=3000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -write_buffer_size=100000 -compression_type=none -block_size=1000; done) 2>&1 | grep micros/op | tee out awk '{ tot += $5; n += 1; } END { print int(1.0 * tot / n) }' < out ``` Each value below is ops/s averaged over 100 runs, run simultaneously with competing configuration for load fairness Before -> after (both fv=5): 483530 -> 483673 (negligible) Re-run 1: 480733 -> 485427 (1.0% faster) Re-run 2: 483821 -> 484541 (0.1% faster) Before (fv=5) -> after (fv=6): 482006 -> 485100 (0.6% faster) Re-run 1: 482212 -> 485075 (0.6% faster) Re-run 2: 483590 -> 484073 (0.1% faster) After fv=5 -> after fv=6: 483878 -> 485542 (0.3% faster) Re-run 1: 485331 -> 483385 (0.4% slower) Re-run 2: 485283 -> 483435 (0.4% slower) Re-run 3: 483647 -> 486109 (0.5% faster) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9058 Test Plan: unit tests included (table_test, db_properties_test, salt in env_test). General DB tests and crash test updated to test new format_version. Also temporarily updated the default format version to 6 and saw some test failures. Almost all were due to an inadvertent additional read in VerifyChecksum to verify the index block checksum, though it's arguably a bug that VerifyChecksum does not appear to (re-)verify the index block checksum, just assuming it was verified in opening the index reader (probably *usually* true but probably not always true). Some other concerns about VerifyChecksum are left in FIXME comments. The only remaining test failure on change of default (in block_fetcher_test) now has a comment about how to upgrade the test. The format compatibility test does not need updating because we have not updated the default format_version. Reviewed By: ajkr, mrambacher Differential Revision: D33100915 Pulled By: pdillinger fbshipit-source-id: 8679e3e572fa580181a737fd6d113ed53c5422ee
1 year ago
if (meta_block_name == kIndexBlockName) {
return BlockType::kIndex;
}
Remove deprecated block-based filter (#10184) Summary: In https://github.com/facebook/rocksdb/issues/9535, release 7.0, we hid the old block-based filter from being created using the public API, because of its inefficiency. Although we normally maintain read compatibility on old DBs forever, filters are not required for reading a DB, only for optimizing read performance. Thus, it should be acceptable to remove this code and the substantial maintenance burden it carries as useful features are developed and validated (such as user timestamp). This change completely removes the code for reading and writing the old block-based filters, net removing about 1370 lines of code no longer needed. Options removed from testing / benchmarking tools. The prior existence is only evident in a couple of places: * `CacheEntryRole::kDeprecatedFilterBlock` - We can update this public API enum in a major release to minimize source code incompatibilities. * A warning is logged when an old table file is opened that used the old block-based filter. This is provided as a courtesy, and would be a pain to unit test, so manual testing should suffice. Unfortunately, sst_dump does not tell you whether a file uses block-based filter, and the structure of the code makes it very difficult to fix. * To detect that case, `kObsoleteFilterBlockPrefix` (renamed from `kFilterBlockPrefix`) for metaindex is maintained (for now). Other notes: * In some cases where numbers are associated with filter configurations, we have had to update the assigned numbers so that they all correspond to something that exists. * Fixed potential stat counting bug by assuming `filter_checked = false` for cases like `filter == nullptr` rather than assuming `filter_checked = true` * Removed obsolete `block_offset` and `prefix_extractor` parameters from several functions. * Removed some unnecessary checks `if (!table_prefix_extractor() && !prefix_extractor)` because the caller guarantees the prefix extractor exists and is compatible Pull Request resolved: https://github.com/facebook/rocksdb/pull/10184 Test Plan: tests updated, manually test new warning in LOG using base version to generate a DB Reviewed By: riversand963 Differential Revision: D37212647 Pulled By: pdillinger fbshipit-source-id: 06ee020d8de3b81260ffc36ad0c1202cbf463a80
2 years ago
if (meta_block_name.starts_with(kObsoleteFilterBlockPrefix)) {
// Obsolete but possible in old files
return BlockType::kInvalid;
}
assert(false);
return BlockType::kInvalid;
}
Status BlockBasedTable::VerifyChecksumInMetaBlocks(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
const ReadOptions& read_options, InternalIteratorBase<Slice>* index_iter) {
Status s;
for (index_iter->SeekToFirst(); index_iter->Valid(); index_iter->Next()) {
s = index_iter->status();
if (!s.ok()) {
break;
}
BlockHandle handle;
Slice input = index_iter->value();
s = handle.DecodeFrom(&input);
format_version=6 and context-aware block checksums (#9058) Summary: ## Context checksum All RocksDB checksums currently use 32 bits of checking power, which should be 1 in 4 billion false negative (FN) probability (failing to detect corruption). This is true for random corruptions, and in some cases small corruptions are guaranteed to be detected. But some possible corruptions, such as in storage metadata rather than storage payload data, would have a much higher FN rate. For example: * Data larger than one SST block is replaced by data from elsewhere in the same or another SST file. Especially with block_align=true, the probability of exact block size match is probably around 1 in 100, making the FN probability around that same. Without `block_align=true` the probability of same block start location is probably around 1 in 10,000, for FN probability around 1 in a million. To solve this problem in new format_version=6, we add "context awareness" to block checksum checks. The stored and expected checksum value is modified based on the block's position in the file and which file it is in. The modifications are cleverly chosen so that, for example * blocks within about 4GB of each other are guaranteed to use different context * blocks that are offset by exactly some multiple of 4GiB are guaranteed to use different context * files generated by the same process are guaranteed to use different context for the same offsets, until wrap-around after 2^32 - 1 files Thus, with format_version=6, if a valid SST block and checksum is misplaced, its checksum FN probability should be essentially ideal, 1 in 4B. ## Footer checksum This change also adds checksum protection to the SST footer (with format_version=6), for the first time without relying on whole file checksum. To prevent a corruption of the format_version in the footer (e.g. 6 -> 5) to defeat the footer checksum, we change much of the footer data format including an "extended magic number" in format_version 6 that would be interpreted as empty index and metaindex block handles in older footer versions. We also change the encoding of handles to free up space for other new data in footer. ## More detail: making space in footer In order to keep footer the same size in format_version=6 (avoid change to IO patterns), we have to free up some space for new data. We do this two ways: * Metaindex block handle is encoded down to 4 bytes (from 10) by assuming it immediately precedes the footer, and by assuming it is < 4GB. * Index block handle is moved into metaindex. (I don't know why it was in footer to begin with.) ## Performance In case of small performance penalty, I've made a "pay as you go" optimization to compensate: replace `MutableCFOptions` in BlockBasedTableBuilder::Rep with the only field used in that structure after construction: `prefix_extractor`. This makes the PR an overall performance improvement (results below). Nevertheless I'm seeing essentially no difference going from fv=5 to fv=6, even including that improvement for both. That's based on extreme case table write performance testing, many files with many blocks. This is relatively checksum intensive (small blocks) and salt generation intensive (small files). ``` (for I in `seq 1 100`; do TEST_TMPDIR=/dev/shm/dbbench2 ./db_bench -benchmarks=fillseq -memtablerep=vector -disable_wal=1 -allow_concurrent_memtable_write=false -num=3000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -write_buffer_size=100000 -compression_type=none -block_size=1000; done) 2>&1 | grep micros/op | tee out awk '{ tot += $5; n += 1; } END { print int(1.0 * tot / n) }' < out ``` Each value below is ops/s averaged over 100 runs, run simultaneously with competing configuration for load fairness Before -> after (both fv=5): 483530 -> 483673 (negligible) Re-run 1: 480733 -> 485427 (1.0% faster) Re-run 2: 483821 -> 484541 (0.1% faster) Before (fv=5) -> after (fv=6): 482006 -> 485100 (0.6% faster) Re-run 1: 482212 -> 485075 (0.6% faster) Re-run 2: 483590 -> 484073 (0.1% faster) After fv=5 -> after fv=6: 483878 -> 485542 (0.3% faster) Re-run 1: 485331 -> 483385 (0.4% slower) Re-run 2: 485283 -> 483435 (0.4% slower) Re-run 3: 483647 -> 486109 (0.5% faster) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9058 Test Plan: unit tests included (table_test, db_properties_test, salt in env_test). General DB tests and crash test updated to test new format_version. Also temporarily updated the default format version to 6 and saw some test failures. Almost all were due to an inadvertent additional read in VerifyChecksum to verify the index block checksum, though it's arguably a bug that VerifyChecksum does not appear to (re-)verify the index block checksum, just assuming it was verified in opening the index reader (probably *usually* true but probably not always true). Some other concerns about VerifyChecksum are left in FIXME comments. The only remaining test failure on change of default (in block_fetcher_test) now has a comment about how to upgrade the test. The format compatibility test does not need updating because we have not updated the default format_version. Reviewed By: ajkr, mrambacher Differential Revision: D33100915 Pulled By: pdillinger fbshipit-source-id: 8679e3e572fa580181a737fd6d113ed53c5422ee
1 year ago
if (!s.ok()) {
break;
}
BlockContents contents;
const Slice meta_block_name = index_iter->key();
if (meta_block_name == kPropertiesBlockName) {
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
// Unfortunate special handling for properties block checksum w/
// global seqno
std::unique_ptr<TableProperties> table_properties;
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
s = ReadTablePropertiesHelper(read_options, handle, rep_->file.get(),
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
nullptr /* prefetch_buffer */, rep_->footer,
rep_->ioptions, &table_properties,
nullptr /* memory_allocator */);
format_version=6 and context-aware block checksums (#9058) Summary: ## Context checksum All RocksDB checksums currently use 32 bits of checking power, which should be 1 in 4 billion false negative (FN) probability (failing to detect corruption). This is true for random corruptions, and in some cases small corruptions are guaranteed to be detected. But some possible corruptions, such as in storage metadata rather than storage payload data, would have a much higher FN rate. For example: * Data larger than one SST block is replaced by data from elsewhere in the same or another SST file. Especially with block_align=true, the probability of exact block size match is probably around 1 in 100, making the FN probability around that same. Without `block_align=true` the probability of same block start location is probably around 1 in 10,000, for FN probability around 1 in a million. To solve this problem in new format_version=6, we add "context awareness" to block checksum checks. The stored and expected checksum value is modified based on the block's position in the file and which file it is in. The modifications are cleverly chosen so that, for example * blocks within about 4GB of each other are guaranteed to use different context * blocks that are offset by exactly some multiple of 4GiB are guaranteed to use different context * files generated by the same process are guaranteed to use different context for the same offsets, until wrap-around after 2^32 - 1 files Thus, with format_version=6, if a valid SST block and checksum is misplaced, its checksum FN probability should be essentially ideal, 1 in 4B. ## Footer checksum This change also adds checksum protection to the SST footer (with format_version=6), for the first time without relying on whole file checksum. To prevent a corruption of the format_version in the footer (e.g. 6 -> 5) to defeat the footer checksum, we change much of the footer data format including an "extended magic number" in format_version 6 that would be interpreted as empty index and metaindex block handles in older footer versions. We also change the encoding of handles to free up space for other new data in footer. ## More detail: making space in footer In order to keep footer the same size in format_version=6 (avoid change to IO patterns), we have to free up some space for new data. We do this two ways: * Metaindex block handle is encoded down to 4 bytes (from 10) by assuming it immediately precedes the footer, and by assuming it is < 4GB. * Index block handle is moved into metaindex. (I don't know why it was in footer to begin with.) ## Performance In case of small performance penalty, I've made a "pay as you go" optimization to compensate: replace `MutableCFOptions` in BlockBasedTableBuilder::Rep with the only field used in that structure after construction: `prefix_extractor`. This makes the PR an overall performance improvement (results below). Nevertheless I'm seeing essentially no difference going from fv=5 to fv=6, even including that improvement for both. That's based on extreme case table write performance testing, many files with many blocks. This is relatively checksum intensive (small blocks) and salt generation intensive (small files). ``` (for I in `seq 1 100`; do TEST_TMPDIR=/dev/shm/dbbench2 ./db_bench -benchmarks=fillseq -memtablerep=vector -disable_wal=1 -allow_concurrent_memtable_write=false -num=3000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -write_buffer_size=100000 -compression_type=none -block_size=1000; done) 2>&1 | grep micros/op | tee out awk '{ tot += $5; n += 1; } END { print int(1.0 * tot / n) }' < out ``` Each value below is ops/s averaged over 100 runs, run simultaneously with competing configuration for load fairness Before -> after (both fv=5): 483530 -> 483673 (negligible) Re-run 1: 480733 -> 485427 (1.0% faster) Re-run 2: 483821 -> 484541 (0.1% faster) Before (fv=5) -> after (fv=6): 482006 -> 485100 (0.6% faster) Re-run 1: 482212 -> 485075 (0.6% faster) Re-run 2: 483590 -> 484073 (0.1% faster) After fv=5 -> after fv=6: 483878 -> 485542 (0.3% faster) Re-run 1: 485331 -> 483385 (0.4% slower) Re-run 2: 485283 -> 483435 (0.4% slower) Re-run 3: 483647 -> 486109 (0.5% faster) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9058 Test Plan: unit tests included (table_test, db_properties_test, salt in env_test). General DB tests and crash test updated to test new format_version. Also temporarily updated the default format version to 6 and saw some test failures. Almost all were due to an inadvertent additional read in VerifyChecksum to verify the index block checksum, though it's arguably a bug that VerifyChecksum does not appear to (re-)verify the index block checksum, just assuming it was verified in opening the index reader (probably *usually* true but probably not always true). Some other concerns about VerifyChecksum are left in FIXME comments. The only remaining test failure on change of default (in block_fetcher_test) now has a comment about how to upgrade the test. The format compatibility test does not need updating because we have not updated the default format_version. Reviewed By: ajkr, mrambacher Differential Revision: D33100915 Pulled By: pdillinger fbshipit-source-id: 8679e3e572fa580181a737fd6d113ed53c5422ee
1 year ago
} else if (rep_->verify_checksum_set_on_open &&
meta_block_name == kIndexBlockName) {
// WART: For now, to maintain similar I/O behavior as before
// format_version=6, we skip verifying index block checksum--but only
// if it was checked on open.
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
} else {
format_version=6 and context-aware block checksums (#9058) Summary: ## Context checksum All RocksDB checksums currently use 32 bits of checking power, which should be 1 in 4 billion false negative (FN) probability (failing to detect corruption). This is true for random corruptions, and in some cases small corruptions are guaranteed to be detected. But some possible corruptions, such as in storage metadata rather than storage payload data, would have a much higher FN rate. For example: * Data larger than one SST block is replaced by data from elsewhere in the same or another SST file. Especially with block_align=true, the probability of exact block size match is probably around 1 in 100, making the FN probability around that same. Without `block_align=true` the probability of same block start location is probably around 1 in 10,000, for FN probability around 1 in a million. To solve this problem in new format_version=6, we add "context awareness" to block checksum checks. The stored and expected checksum value is modified based on the block's position in the file and which file it is in. The modifications are cleverly chosen so that, for example * blocks within about 4GB of each other are guaranteed to use different context * blocks that are offset by exactly some multiple of 4GiB are guaranteed to use different context * files generated by the same process are guaranteed to use different context for the same offsets, until wrap-around after 2^32 - 1 files Thus, with format_version=6, if a valid SST block and checksum is misplaced, its checksum FN probability should be essentially ideal, 1 in 4B. ## Footer checksum This change also adds checksum protection to the SST footer (with format_version=6), for the first time without relying on whole file checksum. To prevent a corruption of the format_version in the footer (e.g. 6 -> 5) to defeat the footer checksum, we change much of the footer data format including an "extended magic number" in format_version 6 that would be interpreted as empty index and metaindex block handles in older footer versions. We also change the encoding of handles to free up space for other new data in footer. ## More detail: making space in footer In order to keep footer the same size in format_version=6 (avoid change to IO patterns), we have to free up some space for new data. We do this two ways: * Metaindex block handle is encoded down to 4 bytes (from 10) by assuming it immediately precedes the footer, and by assuming it is < 4GB. * Index block handle is moved into metaindex. (I don't know why it was in footer to begin with.) ## Performance In case of small performance penalty, I've made a "pay as you go" optimization to compensate: replace `MutableCFOptions` in BlockBasedTableBuilder::Rep with the only field used in that structure after construction: `prefix_extractor`. This makes the PR an overall performance improvement (results below). Nevertheless I'm seeing essentially no difference going from fv=5 to fv=6, even including that improvement for both. That's based on extreme case table write performance testing, many files with many blocks. This is relatively checksum intensive (small blocks) and salt generation intensive (small files). ``` (for I in `seq 1 100`; do TEST_TMPDIR=/dev/shm/dbbench2 ./db_bench -benchmarks=fillseq -memtablerep=vector -disable_wal=1 -allow_concurrent_memtable_write=false -num=3000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -write_buffer_size=100000 -compression_type=none -block_size=1000; done) 2>&1 | grep micros/op | tee out awk '{ tot += $5; n += 1; } END { print int(1.0 * tot / n) }' < out ``` Each value below is ops/s averaged over 100 runs, run simultaneously with competing configuration for load fairness Before -> after (both fv=5): 483530 -> 483673 (negligible) Re-run 1: 480733 -> 485427 (1.0% faster) Re-run 2: 483821 -> 484541 (0.1% faster) Before (fv=5) -> after (fv=6): 482006 -> 485100 (0.6% faster) Re-run 1: 482212 -> 485075 (0.6% faster) Re-run 2: 483590 -> 484073 (0.1% faster) After fv=5 -> after fv=6: 483878 -> 485542 (0.3% faster) Re-run 1: 485331 -> 483385 (0.4% slower) Re-run 2: 485283 -> 483435 (0.4% slower) Re-run 3: 483647 -> 486109 (0.5% faster) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9058 Test Plan: unit tests included (table_test, db_properties_test, salt in env_test). General DB tests and crash test updated to test new format_version. Also temporarily updated the default format version to 6 and saw some test failures. Almost all were due to an inadvertent additional read in VerifyChecksum to verify the index block checksum, though it's arguably a bug that VerifyChecksum does not appear to (re-)verify the index block checksum, just assuming it was verified in opening the index reader (probably *usually* true but probably not always true). Some other concerns about VerifyChecksum are left in FIXME comments. The only remaining test failure on change of default (in block_fetcher_test) now has a comment about how to upgrade the test. The format compatibility test does not need updating because we have not updated the default format_version. Reviewed By: ajkr, mrambacher Differential Revision: D33100915 Pulled By: pdillinger fbshipit-source-id: 8679e3e572fa580181a737fd6d113ed53c5422ee
1 year ago
// FIXME? Need to verify checksums of index and filter partitions?
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
s = BlockFetcher(
rep_->file.get(), nullptr /* prefetch buffer */, rep_->footer,
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
read_options, handle, &contents, rep_->ioptions,
Improve / clean up meta block code & integrity (#9163) Summary: * Checksums are now checked on meta blocks unless specifically suppressed or not applicable (e.g. plain table). (Was other way around.) This means a number of cases that were not checking checksums now are, including direct read TableProperties in Version::GetTableProperties (fixed in meta_blocks ReadTableProperties), reading any block from PersistentCache (fixed in BlockFetcher), read TableProperties in SstFileDumper (ldb/sst_dump/BackupEngine) before table reader open, maybe more. * For that to work, I moved the global_seqno+TableProperties checksum logic to the shared table/ code, because that is used by many utilies such as SstFileDumper. * Also for that to work, we have to know when we're dealing with a block that has a checksum (trailer), so added that capability to Footer based on magic number, and from there BlockFetcher. * Knowledge of trailer presence has also fixed a problem where other table formats were reading blocks including bytes for a non-existant trailer--and awkwardly kind-of not using them, e.g. no shared code checking checksums. (BlockFetcher compression type was populated incorrectly.) Now we only read what is needed. * Minimized code duplication and differing/incompatible/awkward abstractions in meta_blocks.{cc,h} (e.g. SeekTo in metaindex block without parsing block handle) * Moved some meta block handling code from table_properties*.* * Moved some code specific to block-based table from shared table/ code to BlockBasedTable class. The checksum stuff means we can't completely separate it, but things that don't need to be in shared table/ code should not be. * Use unique_ptr rather than raw ptr in more places. (Note: you can std::move from unique_ptr to shared_ptr.) Without enhancements to GetPropertiesOfAllTablesTest (see below), net reduction of roughly 100 lines of code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9163 Test Plan: existing tests and * Enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to verify that checksums are now checked on direct read of table properties by TableCache (new test would fail before this change) * Also enhanced DBTablePropertiesTest.GetPropertiesOfAllTablesTest to test putting table properties under old meta name * Also generally enhanced that same test to actually test what it was supposed to be testing already, by kicking things out of table cache when we don't want them there. Reviewed By: ajkr, mrambacher Differential Revision: D32514757 Pulled By: pdillinger fbshipit-source-id: 507964b9311d186ae8d1131182290cbd97a99fa9
3 years ago
false /* decompress */, false /*maybe_compressed*/,
GetBlockTypeForMetaBlockByName(meta_block_name),
UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options)
.ReadBlockContents();
}
if (!s.ok()) {
break;
}
}
return s;
}
bool BlockBasedTable::TEST_BlockInCache(const BlockHandle& handle) const {
assert(rep_ != nullptr);
Cache* const cache = rep_->table_options.block_cache.get();
if (cache == nullptr) {
return false;
}
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
CacheKey key = GetCacheKey(rep_->base_cache_key, handle);
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
Cache::Handle* const cache_handle = cache->Lookup(key.AsSlice());
if (cache_handle == nullptr) {
return false;
}
cache->Release(cache_handle);
return true;
}
bool BlockBasedTable::TEST_KeyInCache(const ReadOptions& options,
const Slice& key) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter(NewIndexIterator(
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
options, /*need_upper_bound_check=*/false, /*input_iter=*/nullptr,
/*get_context=*/nullptr, /*lookup_context=*/nullptr));
iiter->Seek(key);
assert(iiter->Valid());
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
return TEST_BlockInCache(iiter->value().handle);
}
// REQUIRES: The following fields of rep_ should have already been populated:
// 1. file
// 2. index_handle,
// 3. options
// 4. internal_comparator
// 5. index_type
Status BlockBasedTable::CreateIndexReader(
const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
InternalIterator* meta_iter, bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context,
std::unique_ptr<IndexReader>* index_reader) {
format_version=6 and context-aware block checksums (#9058) Summary: ## Context checksum All RocksDB checksums currently use 32 bits of checking power, which should be 1 in 4 billion false negative (FN) probability (failing to detect corruption). This is true for random corruptions, and in some cases small corruptions are guaranteed to be detected. But some possible corruptions, such as in storage metadata rather than storage payload data, would have a much higher FN rate. For example: * Data larger than one SST block is replaced by data from elsewhere in the same or another SST file. Especially with block_align=true, the probability of exact block size match is probably around 1 in 100, making the FN probability around that same. Without `block_align=true` the probability of same block start location is probably around 1 in 10,000, for FN probability around 1 in a million. To solve this problem in new format_version=6, we add "context awareness" to block checksum checks. The stored and expected checksum value is modified based on the block's position in the file and which file it is in. The modifications are cleverly chosen so that, for example * blocks within about 4GB of each other are guaranteed to use different context * blocks that are offset by exactly some multiple of 4GiB are guaranteed to use different context * files generated by the same process are guaranteed to use different context for the same offsets, until wrap-around after 2^32 - 1 files Thus, with format_version=6, if a valid SST block and checksum is misplaced, its checksum FN probability should be essentially ideal, 1 in 4B. ## Footer checksum This change also adds checksum protection to the SST footer (with format_version=6), for the first time without relying on whole file checksum. To prevent a corruption of the format_version in the footer (e.g. 6 -> 5) to defeat the footer checksum, we change much of the footer data format including an "extended magic number" in format_version 6 that would be interpreted as empty index and metaindex block handles in older footer versions. We also change the encoding of handles to free up space for other new data in footer. ## More detail: making space in footer In order to keep footer the same size in format_version=6 (avoid change to IO patterns), we have to free up some space for new data. We do this two ways: * Metaindex block handle is encoded down to 4 bytes (from 10) by assuming it immediately precedes the footer, and by assuming it is < 4GB. * Index block handle is moved into metaindex. (I don't know why it was in footer to begin with.) ## Performance In case of small performance penalty, I've made a "pay as you go" optimization to compensate: replace `MutableCFOptions` in BlockBasedTableBuilder::Rep with the only field used in that structure after construction: `prefix_extractor`. This makes the PR an overall performance improvement (results below). Nevertheless I'm seeing essentially no difference going from fv=5 to fv=6, even including that improvement for both. That's based on extreme case table write performance testing, many files with many blocks. This is relatively checksum intensive (small blocks) and salt generation intensive (small files). ``` (for I in `seq 1 100`; do TEST_TMPDIR=/dev/shm/dbbench2 ./db_bench -benchmarks=fillseq -memtablerep=vector -disable_wal=1 -allow_concurrent_memtable_write=false -num=3000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -write_buffer_size=100000 -compression_type=none -block_size=1000; done) 2>&1 | grep micros/op | tee out awk '{ tot += $5; n += 1; } END { print int(1.0 * tot / n) }' < out ``` Each value below is ops/s averaged over 100 runs, run simultaneously with competing configuration for load fairness Before -> after (both fv=5): 483530 -> 483673 (negligible) Re-run 1: 480733 -> 485427 (1.0% faster) Re-run 2: 483821 -> 484541 (0.1% faster) Before (fv=5) -> after (fv=6): 482006 -> 485100 (0.6% faster) Re-run 1: 482212 -> 485075 (0.6% faster) Re-run 2: 483590 -> 484073 (0.1% faster) After fv=5 -> after fv=6: 483878 -> 485542 (0.3% faster) Re-run 1: 485331 -> 483385 (0.4% slower) Re-run 2: 485283 -> 483435 (0.4% slower) Re-run 3: 483647 -> 486109 (0.5% faster) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9058 Test Plan: unit tests included (table_test, db_properties_test, salt in env_test). General DB tests and crash test updated to test new format_version. Also temporarily updated the default format version to 6 and saw some test failures. Almost all were due to an inadvertent additional read in VerifyChecksum to verify the index block checksum, though it's arguably a bug that VerifyChecksum does not appear to (re-)verify the index block checksum, just assuming it was verified in opening the index reader (probably *usually* true but probably not always true). Some other concerns about VerifyChecksum are left in FIXME comments. The only remaining test failure on change of default (in block_fetcher_test) now has a comment about how to upgrade the test. The format compatibility test does not need updating because we have not updated the default format_version. Reviewed By: ajkr, mrambacher Differential Revision: D33100915 Pulled By: pdillinger fbshipit-source-id: 8679e3e572fa580181a737fd6d113ed53c5422ee
1 year ago
if (FormatVersionUsesIndexHandleInFooter(rep_->footer.format_version())) {
rep_->index_handle = rep_->footer.index_handle();
} else {
Status s = FindMetaBlock(meta_iter, kIndexBlockName, &rep_->index_handle);
if (!s.ok()) {
return s;
}
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
switch (rep_->index_type) {
case BlockBasedTableOptions::kTwoLevelIndexSearch: {
return PartitionIndexReader::Create(this, ro, prefetch_buffer, use_cache,
prefetch, pin, lookup_context,
index_reader);
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
case BlockBasedTableOptions::kBinarySearch:
FALLTHROUGH_INTENDED;
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
case BlockBasedTableOptions::kBinarySearchWithFirstKey: {
return BinarySearchIndexReader::Create(this, ro, prefetch_buffer,
use_cache, prefetch, pin,
lookup_context, index_reader);
}
case BlockBasedTableOptions::kHashSearch: {
if (!rep_->table_prefix_extractor) {
ROCKS_LOG_WARN(rep_->ioptions.logger,
"Missing prefix extractor for hash index. Fall back to"
" binary search index.");
return BinarySearchIndexReader::Create(this, ro, prefetch_buffer,
use_cache, prefetch, pin,
lookup_context, index_reader);
} else {
return HashIndexReader::Create(this, ro, prefetch_buffer, meta_iter,
use_cache, prefetch, pin, lookup_context,
index_reader);
}
}
default: {
std::string error_message =
"Unrecognized index type: " + std::to_string(rep_->index_type);
return Status::InvalidArgument(error_message.c_str());
}
}
}
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
uint64_t BlockBasedTable::ApproximateDataOffsetOf(
const InternalIteratorBase<IndexValue>& index_iter,
uint64_t data_size) const {
Handle failures in block-based table size/offset approximation (#9615) Summary: In crash test with fault injection, we were seeing stack traces like the following: ``` https://github.com/facebook/rocksdb/issues/3 0x00007f75f763c533 in __GI___assert_fail (assertion=assertion@entry=0x1c5b2a0 "end_offset >= start_offset", file=file@entry=0x1c580a0 "table/block_based/block_based_table_reader.cc", line=line@entry=3245, function=function@entry=0x1c60e60 "virtual uint64_t rocksdb::BlockBasedTable::ApproximateSize(const rocksdb::Slice&, const rocksdb::Slice&, rocksdb::TableReaderCaller)") at assert.c:101 https://github.com/facebook/rocksdb/issues/4 0x00000000010ea9b4 in rocksdb::BlockBasedTable::ApproximateSize (this=<optimized out>, start=..., end=..., caller=<optimized out>) at table/block_based/block_based_table_reader.cc:3224 https://github.com/facebook/rocksdb/issues/5 0x0000000000be61fb in rocksdb::TableCache::ApproximateSize (this=0x60f0000161b0, start=..., end=..., fd=..., caller=caller@entry=rocksdb::kCompaction, internal_comparator=..., prefix_extractor=...) at db/table_cache.cc:719 https://github.com/facebook/rocksdb/issues/6 0x0000000000c3eaec in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, v=<optimized out>, f=..., start=..., end=..., caller=<optimized out>) at ./db/version_set.h:850 https://github.com/facebook/rocksdb/issues/7 0x0000000000c6ebc3 in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, options=..., v=v@entry=0x621000047500, start=..., end=..., start_level=start_level@entry=0, end_level=<optimized out>, caller=<optimized out>) at db/version_set.cc:5657 https://github.com/facebook/rocksdb/issues/8 0x000000000166e894 in rocksdb::CompactionJob::GenSubcompactionBoundaries (this=<optimized out>) at ./include/rocksdb/options.h:1869 https://github.com/facebook/rocksdb/issues/9 0x000000000168c526 in rocksdb::CompactionJob::Prepare (this=this@entry=0x7f75f3ffcf00) at db/compaction/compaction_job.cc:546 ``` The problem occurred in `ApproximateSize()` when the index `Seek()` for the first `ApproximateDataOffsetOf()` encountered an I/O error, while the second `Seek()` did not. In the old code that scenario caused `start_offset == data_size` , thus it was easy to trip the assertion that `end_offset >= start_offset`. The fix is to set `start_offset == 0` when the first index `Seek()` fails, and `end_offset == data_size` when the second index `Seek()` fails. I doubt these give an "on average correct" answer for how this function is used, but I/O errors in index seeks are hopefully rare, it looked consistent with what was already there, and it was easier to calculate. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9615 Test Plan: run the repro command for a while and stopped seeing coredumps - ``` $ while ! ./db_stress --block_size=128 --cache_size=32768 --clear_column_family_one_in=0 --column_families=1 --continuous_verification_interval=0 --db=/dev/shm/rocksdb_crashtest --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --index_type=2 --iterpercent=10 --kill_random_test=18887 --max_key=1000000 --max_bytes_for_level_base=2048576 --nooverwritepercent=1 --open_files=-1 --open_read_fault_one_in=32 --ops_per_thread=1000000 --prefixpercent=5 --read_fault_one_in=0 --readpercent=45 --reopen=0 --skip_verifydb=1 --subcompactions=2 --target_file_size_base=524288 --test_batches_snapshots=0 --value_size_mult=32 --write_buffer_size=524288 --writepercent=35 ; do : ; done ``` Reviewed By: pdillinger Differential Revision: D34383069 Pulled By: ajkr fbshipit-source-id: fac26c3b20ea962e75387515ba5f2724dc48719f
3 years ago
assert(index_iter.status().ok());
if (index_iter.Valid()) {
BlockHandle handle = index_iter.value().handle;
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
return handle.offset();
} else {
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
// The iterator is past the last key in the file.
return data_size;
}
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
}
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
uint64_t BlockBasedTable::GetApproximateDataSize() {
// Should be in table properties unless super old version
if (rep_->table_properties) {
return rep_->table_properties->data_size;
}
// Fall back to rough estimate from footer
return rep_->footer.metaindex_handle().offset();
}
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
uint64_t BlockBasedTable::ApproximateOffsetOf(const ReadOptions& read_options,
const Slice& key,
TableReaderCaller caller) {
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
uint64_t data_size = GetApproximateDataSize();
if (UNLIKELY(data_size == 0)) {
// Hmm. Let's just split in half to avoid skewing one way or another,
// since we don't know whether we're operating on lower bound or
// upper bound.
return rep_->file_size / 2;
}
BlockCacheLookupContext context(caller);
IndexBlockIter iiter_on_stack;
ReadOptions ro;
ro.total_order_seek = true;
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
ro.io_activity = read_options.io_activity;
auto index_iter =
NewIndexIterator(ro, /*disable_prefix_seek=*/true,
/*input_iter=*/&iiter_on_stack, /*get_context=*/nullptr,
/*lookup_context=*/&context);
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
if (index_iter != &iiter_on_stack) {
iiter_unique_ptr.reset(index_iter);
}
index_iter->Seek(key);
Handle failures in block-based table size/offset approximation (#9615) Summary: In crash test with fault injection, we were seeing stack traces like the following: ``` https://github.com/facebook/rocksdb/issues/3 0x00007f75f763c533 in __GI___assert_fail (assertion=assertion@entry=0x1c5b2a0 "end_offset >= start_offset", file=file@entry=0x1c580a0 "table/block_based/block_based_table_reader.cc", line=line@entry=3245, function=function@entry=0x1c60e60 "virtual uint64_t rocksdb::BlockBasedTable::ApproximateSize(const rocksdb::Slice&, const rocksdb::Slice&, rocksdb::TableReaderCaller)") at assert.c:101 https://github.com/facebook/rocksdb/issues/4 0x00000000010ea9b4 in rocksdb::BlockBasedTable::ApproximateSize (this=<optimized out>, start=..., end=..., caller=<optimized out>) at table/block_based/block_based_table_reader.cc:3224 https://github.com/facebook/rocksdb/issues/5 0x0000000000be61fb in rocksdb::TableCache::ApproximateSize (this=0x60f0000161b0, start=..., end=..., fd=..., caller=caller@entry=rocksdb::kCompaction, internal_comparator=..., prefix_extractor=...) at db/table_cache.cc:719 https://github.com/facebook/rocksdb/issues/6 0x0000000000c3eaec in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, v=<optimized out>, f=..., start=..., end=..., caller=<optimized out>) at ./db/version_set.h:850 https://github.com/facebook/rocksdb/issues/7 0x0000000000c6ebc3 in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, options=..., v=v@entry=0x621000047500, start=..., end=..., start_level=start_level@entry=0, end_level=<optimized out>, caller=<optimized out>) at db/version_set.cc:5657 https://github.com/facebook/rocksdb/issues/8 0x000000000166e894 in rocksdb::CompactionJob::GenSubcompactionBoundaries (this=<optimized out>) at ./include/rocksdb/options.h:1869 https://github.com/facebook/rocksdb/issues/9 0x000000000168c526 in rocksdb::CompactionJob::Prepare (this=this@entry=0x7f75f3ffcf00) at db/compaction/compaction_job.cc:546 ``` The problem occurred in `ApproximateSize()` when the index `Seek()` for the first `ApproximateDataOffsetOf()` encountered an I/O error, while the second `Seek()` did not. In the old code that scenario caused `start_offset == data_size` , thus it was easy to trip the assertion that `end_offset >= start_offset`. The fix is to set `start_offset == 0` when the first index `Seek()` fails, and `end_offset == data_size` when the second index `Seek()` fails. I doubt these give an "on average correct" answer for how this function is used, but I/O errors in index seeks are hopefully rare, it looked consistent with what was already there, and it was easier to calculate. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9615 Test Plan: run the repro command for a while and stopped seeing coredumps - ``` $ while ! ./db_stress --block_size=128 --cache_size=32768 --clear_column_family_one_in=0 --column_families=1 --continuous_verification_interval=0 --db=/dev/shm/rocksdb_crashtest --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --index_type=2 --iterpercent=10 --kill_random_test=18887 --max_key=1000000 --max_bytes_for_level_base=2048576 --nooverwritepercent=1 --open_files=-1 --open_read_fault_one_in=32 --ops_per_thread=1000000 --prefixpercent=5 --read_fault_one_in=0 --readpercent=45 --reopen=0 --skip_verifydb=1 --subcompactions=2 --target_file_size_base=524288 --test_batches_snapshots=0 --value_size_mult=32 --write_buffer_size=524288 --writepercent=35 ; do : ; done ``` Reviewed By: pdillinger Differential Revision: D34383069 Pulled By: ajkr fbshipit-source-id: fac26c3b20ea962e75387515ba5f2724dc48719f
3 years ago
uint64_t offset;
if (index_iter->status().ok()) {
offset = ApproximateDataOffsetOf(*index_iter, data_size);
} else {
// Split in half to avoid skewing one way or another,
// since we don't know whether we're operating on lower bound or
// upper bound.
return rep_->file_size / 2;
}
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
// Pro-rate file metadata (incl filters) size-proportionally across data
// blocks.
double size_ratio =
static_cast<double>(offset) / static_cast<double>(data_size);
return static_cast<uint64_t>(size_ratio *
static_cast<double>(rep_->file_size));
}
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
uint64_t BlockBasedTable::ApproximateSize(const ReadOptions& read_options,
const Slice& start, const Slice& end,
TableReaderCaller caller) {
assert(rep_->internal_comparator.Compare(start, end) <= 0);
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
uint64_t data_size = GetApproximateDataSize();
if (UNLIKELY(data_size == 0)) {
// Hmm. Assume whole file is involved, since we have lower and upper
Handle failures in block-based table size/offset approximation (#9615) Summary: In crash test with fault injection, we were seeing stack traces like the following: ``` https://github.com/facebook/rocksdb/issues/3 0x00007f75f763c533 in __GI___assert_fail (assertion=assertion@entry=0x1c5b2a0 "end_offset >= start_offset", file=file@entry=0x1c580a0 "table/block_based/block_based_table_reader.cc", line=line@entry=3245, function=function@entry=0x1c60e60 "virtual uint64_t rocksdb::BlockBasedTable::ApproximateSize(const rocksdb::Slice&, const rocksdb::Slice&, rocksdb::TableReaderCaller)") at assert.c:101 https://github.com/facebook/rocksdb/issues/4 0x00000000010ea9b4 in rocksdb::BlockBasedTable::ApproximateSize (this=<optimized out>, start=..., end=..., caller=<optimized out>) at table/block_based/block_based_table_reader.cc:3224 https://github.com/facebook/rocksdb/issues/5 0x0000000000be61fb in rocksdb::TableCache::ApproximateSize (this=0x60f0000161b0, start=..., end=..., fd=..., caller=caller@entry=rocksdb::kCompaction, internal_comparator=..., prefix_extractor=...) at db/table_cache.cc:719 https://github.com/facebook/rocksdb/issues/6 0x0000000000c3eaec in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, v=<optimized out>, f=..., start=..., end=..., caller=<optimized out>) at ./db/version_set.h:850 https://github.com/facebook/rocksdb/issues/7 0x0000000000c6ebc3 in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, options=..., v=v@entry=0x621000047500, start=..., end=..., start_level=start_level@entry=0, end_level=<optimized out>, caller=<optimized out>) at db/version_set.cc:5657 https://github.com/facebook/rocksdb/issues/8 0x000000000166e894 in rocksdb::CompactionJob::GenSubcompactionBoundaries (this=<optimized out>) at ./include/rocksdb/options.h:1869 https://github.com/facebook/rocksdb/issues/9 0x000000000168c526 in rocksdb::CompactionJob::Prepare (this=this@entry=0x7f75f3ffcf00) at db/compaction/compaction_job.cc:546 ``` The problem occurred in `ApproximateSize()` when the index `Seek()` for the first `ApproximateDataOffsetOf()` encountered an I/O error, while the second `Seek()` did not. In the old code that scenario caused `start_offset == data_size` , thus it was easy to trip the assertion that `end_offset >= start_offset`. The fix is to set `start_offset == 0` when the first index `Seek()` fails, and `end_offset == data_size` when the second index `Seek()` fails. I doubt these give an "on average correct" answer for how this function is used, but I/O errors in index seeks are hopefully rare, it looked consistent with what was already there, and it was easier to calculate. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9615 Test Plan: run the repro command for a while and stopped seeing coredumps - ``` $ while ! ./db_stress --block_size=128 --cache_size=32768 --clear_column_family_one_in=0 --column_families=1 --continuous_verification_interval=0 --db=/dev/shm/rocksdb_crashtest --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --index_type=2 --iterpercent=10 --kill_random_test=18887 --max_key=1000000 --max_bytes_for_level_base=2048576 --nooverwritepercent=1 --open_files=-1 --open_read_fault_one_in=32 --ops_per_thread=1000000 --prefixpercent=5 --read_fault_one_in=0 --readpercent=45 --reopen=0 --skip_verifydb=1 --subcompactions=2 --target_file_size_base=524288 --test_batches_snapshots=0 --value_size_mult=32 --write_buffer_size=524288 --writepercent=35 ; do : ; done ``` Reviewed By: pdillinger Differential Revision: D34383069 Pulled By: ajkr fbshipit-source-id: fac26c3b20ea962e75387515ba5f2724dc48719f
3 years ago
// bound. This likely skews the estimate if we consider that this function
// is typically called with `[start, end]` fully contained in the file's
// key-range.
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
return rep_->file_size;
}
BlockCacheLookupContext context(caller);
IndexBlockIter iiter_on_stack;
ReadOptions ro;
ro.total_order_seek = true;
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
ro.io_activity = read_options.io_activity;
auto index_iter =
NewIndexIterator(ro, /*disable_prefix_seek=*/true,
/*input_iter=*/&iiter_on_stack, /*get_context=*/nullptr,
/*lookup_context=*/&context);
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
if (index_iter != &iiter_on_stack) {
iiter_unique_ptr.reset(index_iter);
}
index_iter->Seek(start);
Handle failures in block-based table size/offset approximation (#9615) Summary: In crash test with fault injection, we were seeing stack traces like the following: ``` https://github.com/facebook/rocksdb/issues/3 0x00007f75f763c533 in __GI___assert_fail (assertion=assertion@entry=0x1c5b2a0 "end_offset >= start_offset", file=file@entry=0x1c580a0 "table/block_based/block_based_table_reader.cc", line=line@entry=3245, function=function@entry=0x1c60e60 "virtual uint64_t rocksdb::BlockBasedTable::ApproximateSize(const rocksdb::Slice&, const rocksdb::Slice&, rocksdb::TableReaderCaller)") at assert.c:101 https://github.com/facebook/rocksdb/issues/4 0x00000000010ea9b4 in rocksdb::BlockBasedTable::ApproximateSize (this=<optimized out>, start=..., end=..., caller=<optimized out>) at table/block_based/block_based_table_reader.cc:3224 https://github.com/facebook/rocksdb/issues/5 0x0000000000be61fb in rocksdb::TableCache::ApproximateSize (this=0x60f0000161b0, start=..., end=..., fd=..., caller=caller@entry=rocksdb::kCompaction, internal_comparator=..., prefix_extractor=...) at db/table_cache.cc:719 https://github.com/facebook/rocksdb/issues/6 0x0000000000c3eaec in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, v=<optimized out>, f=..., start=..., end=..., caller=<optimized out>) at ./db/version_set.h:850 https://github.com/facebook/rocksdb/issues/7 0x0000000000c6ebc3 in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, options=..., v=v@entry=0x621000047500, start=..., end=..., start_level=start_level@entry=0, end_level=<optimized out>, caller=<optimized out>) at db/version_set.cc:5657 https://github.com/facebook/rocksdb/issues/8 0x000000000166e894 in rocksdb::CompactionJob::GenSubcompactionBoundaries (this=<optimized out>) at ./include/rocksdb/options.h:1869 https://github.com/facebook/rocksdb/issues/9 0x000000000168c526 in rocksdb::CompactionJob::Prepare (this=this@entry=0x7f75f3ffcf00) at db/compaction/compaction_job.cc:546 ``` The problem occurred in `ApproximateSize()` when the index `Seek()` for the first `ApproximateDataOffsetOf()` encountered an I/O error, while the second `Seek()` did not. In the old code that scenario caused `start_offset == data_size` , thus it was easy to trip the assertion that `end_offset >= start_offset`. The fix is to set `start_offset == 0` when the first index `Seek()` fails, and `end_offset == data_size` when the second index `Seek()` fails. I doubt these give an "on average correct" answer for how this function is used, but I/O errors in index seeks are hopefully rare, it looked consistent with what was already there, and it was easier to calculate. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9615 Test Plan: run the repro command for a while and stopped seeing coredumps - ``` $ while ! ./db_stress --block_size=128 --cache_size=32768 --clear_column_family_one_in=0 --column_families=1 --continuous_verification_interval=0 --db=/dev/shm/rocksdb_crashtest --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --index_type=2 --iterpercent=10 --kill_random_test=18887 --max_key=1000000 --max_bytes_for_level_base=2048576 --nooverwritepercent=1 --open_files=-1 --open_read_fault_one_in=32 --ops_per_thread=1000000 --prefixpercent=5 --read_fault_one_in=0 --readpercent=45 --reopen=0 --skip_verifydb=1 --subcompactions=2 --target_file_size_base=524288 --test_batches_snapshots=0 --value_size_mult=32 --write_buffer_size=524288 --writepercent=35 ; do : ; done ``` Reviewed By: pdillinger Differential Revision: D34383069 Pulled By: ajkr fbshipit-source-id: fac26c3b20ea962e75387515ba5f2724dc48719f
3 years ago
uint64_t start_offset;
if (index_iter->status().ok()) {
start_offset = ApproximateDataOffsetOf(*index_iter, data_size);
} else {
// Assume file is involved from the start. This likely skews the estimate
// but is consistent with the above error handling.
start_offset = 0;
}
index_iter->Seek(end);
Handle failures in block-based table size/offset approximation (#9615) Summary: In crash test with fault injection, we were seeing stack traces like the following: ``` https://github.com/facebook/rocksdb/issues/3 0x00007f75f763c533 in __GI___assert_fail (assertion=assertion@entry=0x1c5b2a0 "end_offset >= start_offset", file=file@entry=0x1c580a0 "table/block_based/block_based_table_reader.cc", line=line@entry=3245, function=function@entry=0x1c60e60 "virtual uint64_t rocksdb::BlockBasedTable::ApproximateSize(const rocksdb::Slice&, const rocksdb::Slice&, rocksdb::TableReaderCaller)") at assert.c:101 https://github.com/facebook/rocksdb/issues/4 0x00000000010ea9b4 in rocksdb::BlockBasedTable::ApproximateSize (this=<optimized out>, start=..., end=..., caller=<optimized out>) at table/block_based/block_based_table_reader.cc:3224 https://github.com/facebook/rocksdb/issues/5 0x0000000000be61fb in rocksdb::TableCache::ApproximateSize (this=0x60f0000161b0, start=..., end=..., fd=..., caller=caller@entry=rocksdb::kCompaction, internal_comparator=..., prefix_extractor=...) at db/table_cache.cc:719 https://github.com/facebook/rocksdb/issues/6 0x0000000000c3eaec in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, v=<optimized out>, f=..., start=..., end=..., caller=<optimized out>) at ./db/version_set.h:850 https://github.com/facebook/rocksdb/issues/7 0x0000000000c6ebc3 in rocksdb::VersionSet::ApproximateSize (this=<optimized out>, options=..., v=v@entry=0x621000047500, start=..., end=..., start_level=start_level@entry=0, end_level=<optimized out>, caller=<optimized out>) at db/version_set.cc:5657 https://github.com/facebook/rocksdb/issues/8 0x000000000166e894 in rocksdb::CompactionJob::GenSubcompactionBoundaries (this=<optimized out>) at ./include/rocksdb/options.h:1869 https://github.com/facebook/rocksdb/issues/9 0x000000000168c526 in rocksdb::CompactionJob::Prepare (this=this@entry=0x7f75f3ffcf00) at db/compaction/compaction_job.cc:546 ``` The problem occurred in `ApproximateSize()` when the index `Seek()` for the first `ApproximateDataOffsetOf()` encountered an I/O error, while the second `Seek()` did not. In the old code that scenario caused `start_offset == data_size` , thus it was easy to trip the assertion that `end_offset >= start_offset`. The fix is to set `start_offset == 0` when the first index `Seek()` fails, and `end_offset == data_size` when the second index `Seek()` fails. I doubt these give an "on average correct" answer for how this function is used, but I/O errors in index seeks are hopefully rare, it looked consistent with what was already there, and it was easier to calculate. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9615 Test Plan: run the repro command for a while and stopped seeing coredumps - ``` $ while ! ./db_stress --block_size=128 --cache_size=32768 --clear_column_family_one_in=0 --column_families=1 --continuous_verification_interval=0 --db=/dev/shm/rocksdb_crashtest --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --index_type=2 --iterpercent=10 --kill_random_test=18887 --max_key=1000000 --max_bytes_for_level_base=2048576 --nooverwritepercent=1 --open_files=-1 --open_read_fault_one_in=32 --ops_per_thread=1000000 --prefixpercent=5 --read_fault_one_in=0 --readpercent=45 --reopen=0 --skip_verifydb=1 --subcompactions=2 --target_file_size_base=524288 --test_batches_snapshots=0 --value_size_mult=32 --write_buffer_size=524288 --writepercent=35 ; do : ; done ``` Reviewed By: pdillinger Differential Revision: D34383069 Pulled By: ajkr fbshipit-source-id: fac26c3b20ea962e75387515ba5f2724dc48719f
3 years ago
uint64_t end_offset;
if (index_iter->status().ok()) {
end_offset = ApproximateDataOffsetOf(*index_iter, data_size);
} else {
// Assume file is involved until the end. This likely skews the estimate
// but is consistent with the above error handling.
end_offset = data_size;
}
assert(end_offset >= start_offset);
For ApproximateSizes, pro-rate table metadata size over data blocks (#6784) Summary: The implementation of GetApproximateSizes was inconsistent in its treatment of the size of non-data blocks of SST files, sometimes including and sometimes now. This was at its worst with large portion of table file used by filters and querying a small range that crossed a table boundary: the size estimate would include large filter size. It's conceivable that someone might want only to know the size in terms of data blocks, but I believe that's unlikely enough to ignore for now. Similarly, there's no evidence the internal function AppoximateOffsetOf is used for anything other than a one-sided ApproximateSize, so I intend to refactor to remove redundancy in a follow-up commit. So to fix this, GetApproximateSizes (and implementation details ApproximateSize and ApproximateOffsetOf) now consistently include in their returned sizes a portion of table file metadata (incl filters and indexes) based on the size portion of the data blocks in range. In other words, if a key range covers data blocks that are X% by size of all the table's data blocks, returned approximate size is X% of the total file size. It would technically be more accurate to attribute metadata based on number of keys, but that's not computationally efficient with data available and rarely a meaningful difference. Also includes miscellaneous comment improvements / clarifications. Also included is a new approximatesizerandom benchmark for db_bench. No significant performance difference seen with this change, whether ~700 ops/sec with cache_index_and_filter_blocks and small cache or ~150k ops/sec without cache_index_and_filter_blocks. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6784 Test Plan: Test added to DBTest.ApproximateSizesFilesWithErrorMargin. Old code running new test... [ RUN ] DBTest.ApproximateSizesFilesWithErrorMargin db/db_test.cc:1562: Failure Expected: (size) <= (11 * 100), actual: 9478 vs 1100 Other tests updated to reflect consistent accounting of metadata. Reviewed By: siying Differential Revision: D21334706 Pulled By: pdillinger fbshipit-source-id: 6f86870e45213334fedbe9c73b4ebb1d8d611185
5 years ago
// Pro-rate file metadata (incl filters) size-proportionally across data
// blocks.
double size_ratio = static_cast<double>(end_offset - start_offset) /
static_cast<double>(data_size);
return static_cast<uint64_t>(size_ratio *
static_cast<double>(rep_->file_size));
}
bool BlockBasedTable::TEST_FilterBlockInCache() const {
assert(rep_ != nullptr);
New stable, fixed-length cache keys (#9126) Summary: This change standardizes on a new 16-byte cache key format for block cache (incl compressed and secondary) and persistent cache (but not table cache and row cache). The goal is a really fast cache key with practically ideal stability and uniqueness properties without external dependencies (e.g. from FileSystem). A fixed key size of 16 bytes should enable future optimizations to the concurrent hash table for block cache, which is a heavy CPU user / bottleneck, but there appears to be measurable performance improvement even with no changes to LRUCache. This change replaces a lot of disjointed and ugly code handling cache keys with calls to a simple, clean new internal API (cache_key.h). (Preserving the old cache key logic under an option would be very ugly and likely negate the performance gain of the new approach. Complete replacement carries some inherent risk, but I think that's acceptable with sufficient analysis and testing.) The scheme for encoding new cache keys is complicated but explained in cache_key.cc. Also: EndianSwapValue is moved to math.h to be next to other bit operations. (Explains some new include "math.h".) ReverseBits operation added and unit tests added to hash_test for both. Fixes https://github.com/facebook/rocksdb/issues/7405 (presuming a root cause) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9126 Test Plan: ### Basic correctness Several tests needed updates to work with the new functionality, mostly because we are no longer relying on filesystem for stable cache keys so table builders & readers need more context info to agree on cache keys. This functionality is so core, a huge number of existing tests exercise the cache key functionality. ### Performance Create db with `TEST_TMPDIR=/dev/shm ./db_bench -bloom_bits=10 -benchmarks=fillrandom -num=3000000 -partition_index_and_filters` And test performance with `TEST_TMPDIR=/dev/shm ./db_bench -readonly -use_existing_db -bloom_bits=10 -benchmarks=readrandom -num=3000000 -duration=30 -cache_index_and_filter_blocks -cache_size=250000 -threads=4` using DEBUG_LEVEL=0 and simultaneous before & after runs. Before ops/sec, avg over 100 runs: 121924 After ops/sec, avg over 100 runs: 125385 (+2.8%) ### Collision probability I have built a tool, ./cache_bench -stress_cache_key to broadly simulate host-wide cache activity over many months, by making some pessimistic simplifying assumptions: * Every generated file has a cache entry for every byte offset in the file (contiguous range of cache keys) * All of every file is cached for its entire lifetime We use a simple table with skewed address assignment and replacement on address collision to simulate files coming & going, with quite a variance (super-Poisson) in ages. Some output with `./cache_bench -stress_cache_key -sck_keep_bits=40`: ``` Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day Multiply by 9.22337e+18 to correct for simulation losses (but still assume whole file cached) ``` These come from default settings of 2.5M files per day of 32 MB each, and `-sck_keep_bits=40` means that to represent a single file, we are only keeping 40 bits of the 128-bit cache key. With file size of 2\*\*25 contiguous keys (pessimistic), our simulation is about 2\*\*(128-40-25) or about 9 billion billion times more prone to collision than reality. More default assumptions, relatively pessimistic: * 100 DBs in same process (doesn't matter much) * Re-open DB in same process (new session ID related to old session ID) on average every 100 files generated * Restart process (all new session IDs unrelated to old) 24 times per day After enough data, we get a result at the end: ``` (keep 40 bits) 17 collisions after 2 x 90 days, est 10.5882 days between (9.76592e+19 corrected) ``` If we believe the (pessimistic) simulation and the mathematical generalization, we would need to run a billion machines all for 97 billion days to expect a cache key collision. To help verify that our generalization ("corrected") is robust, we can make our simulation more precise with `-sck_keep_bits=41` and `42`, which takes more running time to get enough data: ``` (keep 41 bits) 16 collisions after 4 x 90 days, est 22.5 days between (1.03763e+20 corrected) (keep 42 bits) 19 collisions after 10 x 90 days, est 47.3684 days between (1.09224e+20 corrected) ``` The generalized prediction still holds. With the `-sck_randomize` option, we can see that we are beating "random" cache keys (except offsets still non-randomized) by a modest amount (roughly 20x less collision prone than random), which should make us reasonably comfortable even in "degenerate" cases: ``` 197 collisions after 1 x 90 days, est 0.456853 days between (4.21372e+18 corrected) ``` I've run other tests to validate other conditions behave as expected, never behaving "worse than random" unless we start chopping off structured data. Reviewed By: zhichao-cao Differential Revision: D33171746 Pulled By: pdillinger fbshipit-source-id: f16a57e369ed37be5e7e33525ace848d0537c88f
3 years ago
return rep_->filter_type != Rep::FilterType::kNoFilter &&
TEST_BlockInCache(rep_->filter_handle);
}
bool BlockBasedTable::TEST_IndexBlockInCache() const {
assert(rep_ != nullptr);
format_version=6 and context-aware block checksums (#9058) Summary: ## Context checksum All RocksDB checksums currently use 32 bits of checking power, which should be 1 in 4 billion false negative (FN) probability (failing to detect corruption). This is true for random corruptions, and in some cases small corruptions are guaranteed to be detected. But some possible corruptions, such as in storage metadata rather than storage payload data, would have a much higher FN rate. For example: * Data larger than one SST block is replaced by data from elsewhere in the same or another SST file. Especially with block_align=true, the probability of exact block size match is probably around 1 in 100, making the FN probability around that same. Without `block_align=true` the probability of same block start location is probably around 1 in 10,000, for FN probability around 1 in a million. To solve this problem in new format_version=6, we add "context awareness" to block checksum checks. The stored and expected checksum value is modified based on the block's position in the file and which file it is in. The modifications are cleverly chosen so that, for example * blocks within about 4GB of each other are guaranteed to use different context * blocks that are offset by exactly some multiple of 4GiB are guaranteed to use different context * files generated by the same process are guaranteed to use different context for the same offsets, until wrap-around after 2^32 - 1 files Thus, with format_version=6, if a valid SST block and checksum is misplaced, its checksum FN probability should be essentially ideal, 1 in 4B. ## Footer checksum This change also adds checksum protection to the SST footer (with format_version=6), for the first time without relying on whole file checksum. To prevent a corruption of the format_version in the footer (e.g. 6 -> 5) to defeat the footer checksum, we change much of the footer data format including an "extended magic number" in format_version 6 that would be interpreted as empty index and metaindex block handles in older footer versions. We also change the encoding of handles to free up space for other new data in footer. ## More detail: making space in footer In order to keep footer the same size in format_version=6 (avoid change to IO patterns), we have to free up some space for new data. We do this two ways: * Metaindex block handle is encoded down to 4 bytes (from 10) by assuming it immediately precedes the footer, and by assuming it is < 4GB. * Index block handle is moved into metaindex. (I don't know why it was in footer to begin with.) ## Performance In case of small performance penalty, I've made a "pay as you go" optimization to compensate: replace `MutableCFOptions` in BlockBasedTableBuilder::Rep with the only field used in that structure after construction: `prefix_extractor`. This makes the PR an overall performance improvement (results below). Nevertheless I'm seeing essentially no difference going from fv=5 to fv=6, even including that improvement for both. That's based on extreme case table write performance testing, many files with many blocks. This is relatively checksum intensive (small blocks) and salt generation intensive (small files). ``` (for I in `seq 1 100`; do TEST_TMPDIR=/dev/shm/dbbench2 ./db_bench -benchmarks=fillseq -memtablerep=vector -disable_wal=1 -allow_concurrent_memtable_write=false -num=3000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -write_buffer_size=100000 -compression_type=none -block_size=1000; done) 2>&1 | grep micros/op | tee out awk '{ tot += $5; n += 1; } END { print int(1.0 * tot / n) }' < out ``` Each value below is ops/s averaged over 100 runs, run simultaneously with competing configuration for load fairness Before -> after (both fv=5): 483530 -> 483673 (negligible) Re-run 1: 480733 -> 485427 (1.0% faster) Re-run 2: 483821 -> 484541 (0.1% faster) Before (fv=5) -> after (fv=6): 482006 -> 485100 (0.6% faster) Re-run 1: 482212 -> 485075 (0.6% faster) Re-run 2: 483590 -> 484073 (0.1% faster) After fv=5 -> after fv=6: 483878 -> 485542 (0.3% faster) Re-run 1: 485331 -> 483385 (0.4% slower) Re-run 2: 485283 -> 483435 (0.4% slower) Re-run 3: 483647 -> 486109 (0.5% faster) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9058 Test Plan: unit tests included (table_test, db_properties_test, salt in env_test). General DB tests and crash test updated to test new format_version. Also temporarily updated the default format version to 6 and saw some test failures. Almost all were due to an inadvertent additional read in VerifyChecksum to verify the index block checksum, though it's arguably a bug that VerifyChecksum does not appear to (re-)verify the index block checksum, just assuming it was verified in opening the index reader (probably *usually* true but probably not always true). Some other concerns about VerifyChecksum are left in FIXME comments. The only remaining test failure on change of default (in block_fetcher_test) now has a comment about how to upgrade the test. The format compatibility test does not need updating because we have not updated the default format_version. Reviewed By: ajkr, mrambacher Differential Revision: D33100915 Pulled By: pdillinger fbshipit-source-id: 8679e3e572fa580181a737fd6d113ed53c5422ee
1 year ago
return TEST_BlockInCache(rep_->index_handle);
}
Status BlockBasedTable::GetKVPairsFromDataBlocks(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
const ReadOptions& read_options, std::vector<KVPairBlock>* kv_pair_blocks) {
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
NewIndexIterator(read_options, /*need_upper_bound_check=*/false,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
/*input_iter=*/nullptr, /*get_context=*/nullptr,
/*lookup_contex=*/nullptr));
Status s = blockhandles_iter->status();
if (!s.ok()) {
// Cannot read Index Block
return s;
}
for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
blockhandles_iter->Next()) {
s = blockhandles_iter->status();
if (!s.ok()) {
break;
}
std::unique_ptr<InternalIterator> datablock_iter;
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
Status tmp_status;
datablock_iter.reset(NewDataBlockIterator<DataBlockIter>(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
read_options, blockhandles_iter->value().handle,
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
/*input_iter=*/nullptr, /*type=*/BlockType::kData,
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
/*get_context=*/nullptr, /*lookup_context=*/nullptr,
/*prefetch_buffer=*/nullptr, /*for_compaction=*/false,
/*async_read=*/false, tmp_status));
s = datablock_iter->status();
if (!s.ok()) {
// Error reading the block - Skipped
continue;
}
KVPairBlock kv_pair_block;
for (datablock_iter->SeekToFirst(); datablock_iter->Valid();
datablock_iter->Next()) {
s = datablock_iter->status();
if (!s.ok()) {
// Error reading the block - Skipped
break;
}
const Slice& key = datablock_iter->key();
const Slice& value = datablock_iter->value();
std::string key_copy = std::string(key.data(), key.size());
std::string value_copy = std::string(value.data(), value.size());
kv_pair_block.push_back(
std::make_pair(std::move(key_copy), std::move(value_copy)));
}
kv_pair_blocks->push_back(std::move(kv_pair_block));
}
return Status::OK();
}
Status BlockBasedTable::DumpTable(WritableFile* out_file) {
WritableFileStringStreamAdapter out_file_wrapper(out_file);
std::ostream out_stream(&out_file_wrapper);
// Output Footer
out_stream << "Footer Details:\n"
"--------------------------------------\n";
out_stream << " " << rep_->footer.ToString() << "\n";
// Output MetaIndex
out_stream << "Metaindex Details:\n"
"--------------------------------------\n";
std::unique_ptr<Block> metaindex;
std::unique_ptr<InternalIterator> metaindex_iter;
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
// TODO: plumb Env::IOActivity
const ReadOptions ro;
Status s = ReadMetaIndexBlock(ro, nullptr /* prefetch_buffer */, &metaindex,
&metaindex_iter);
if (s.ok()) {
for (metaindex_iter->SeekToFirst(); metaindex_iter->Valid();
metaindex_iter->Next()) {
s = metaindex_iter->status();
if (!s.ok()) {
return s;
}
if (metaindex_iter->key() == kPropertiesBlockName) {
out_stream << " Properties block handle: "
<< metaindex_iter->value().ToString(true) << "\n";
} else if (metaindex_iter->key() == kCompressionDictBlockName) {
out_stream << " Compression dictionary block handle: "
<< metaindex_iter->value().ToString(true) << "\n";
} else if (strstr(metaindex_iter->key().ToString().c_str(),
"filter.rocksdb.") != nullptr) {
out_stream << " Filter block handle: "
<< metaindex_iter->value().ToString(true) << "\n";
} else if (metaindex_iter->key() == kRangeDelBlockName) {
out_stream << " Range deletion block handle: "
<< metaindex_iter->value().ToString(true) << "\n";
}
}
out_stream << "\n";
} else {
return s;
}
// Output TableProperties
const ROCKSDB_NAMESPACE::TableProperties* table_properties;
table_properties = rep_->table_properties.get();
if (table_properties != nullptr) {
out_stream << "Table Properties:\n"
"--------------------------------------\n";
out_stream << " " << table_properties->ToString("\n ", ": ") << "\n";
}
if (rep_->filter) {
out_stream << "Filter Details:\n"
"--------------------------------------\n";
out_stream << " " << rep_->filter->ToString() << "\n";
}
// Output Index block
s = DumpIndexBlock(out_stream);
if (!s.ok()) {
return s;
}
// Output compression dictionary
if (rep_->uncompression_dict_reader) {
CachableEntry<UncompressionDict> uncompression_dict;
s = rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
nullptr /* prefetch_buffer */, ro, false /* no_io */,
false, /* verify_checksums */
nullptr /* get_context */, nullptr /* lookup_context */,
&uncompression_dict);
if (!s.ok()) {
return s;
}
assert(uncompression_dict.GetValue());
const Slice& raw_dict = uncompression_dict.GetValue()->GetRawDict();
out_stream << "Compression Dictionary:\n"
"--------------------------------------\n";
out_stream << " size (bytes): " << raw_dict.size() << "\n\n";
out_stream << " HEX " << raw_dict.ToString(true) << "\n\n";
}
// Output range deletions block
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
auto* range_del_iter = NewRangeTombstoneIterator(ro);
if (range_del_iter != nullptr) {
range_del_iter->SeekToFirst();
if (range_del_iter->Valid()) {
out_stream << "Range deletions:\n"
"--------------------------------------\n";
for (; range_del_iter->Valid(); range_del_iter->Next()) {
DumpKeyValue(range_del_iter->key(), range_del_iter->value(),
out_stream);
}
out_stream << "\n";
}
delete range_del_iter;
}
// Output Data blocks
s = DumpDataBlocks(out_stream);
if (!s.ok()) {
return s;
}
if (!out_stream.good()) {
return Status::IOError("Failed to write to output file");
}
return Status::OK();
}
Status BlockBasedTable::DumpIndexBlock(std::ostream& out_stream) {
out_stream << "Index Details:\n"
"--------------------------------------\n";
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
// TODO: plumb Env::IOActivity
const ReadOptions read_options;
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
NewIndexIterator(read_options, /*need_upper_bound_check=*/false,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
/*input_iter=*/nullptr, /*get_context=*/nullptr,
/*lookup_contex=*/nullptr));
Status s = blockhandles_iter->status();
if (!s.ok()) {
out_stream << "Can not read Index Block \n\n";
return s;
}
out_stream << " Block key hex dump: Data block handle\n";
out_stream << " Block key ascii\n\n";
for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
blockhandles_iter->Next()) {
s = blockhandles_iter->status();
if (!s.ok()) {
break;
}
Slice key = blockhandles_iter->key();
Slice user_key;
InternalKey ikey;
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
if (!rep_->index_key_includes_seq) {
user_key = key;
} else {
ikey.DecodeFrom(key);
user_key = ikey.user_key();
}
out_stream << " HEX " << user_key.ToString(true) << ": "
<< blockhandles_iter->value().ToString(true,
rep_->index_has_first_key)
<< " offset " << blockhandles_iter->value().handle.offset()
<< " size " << blockhandles_iter->value().handle.size() << "\n";
std::string str_key = user_key.ToString();
std::string res_key("");
char cspace = ' ';
for (size_t i = 0; i < str_key.size(); i++) {
res_key.append(&str_key[i], 1);
res_key.append(1, cspace);
}
out_stream << " ASCII " << res_key << "\n";
out_stream << " ------\n";
}
out_stream << "\n";
return Status::OK();
}
Status BlockBasedTable::DumpDataBlocks(std::ostream& out_stream) {
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
// TODO: plumb Env::IOActivity
const ReadOptions read_options;
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
NewIndexIterator(read_options, /*need_upper_bound_check=*/false,
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
6 years ago
/*input_iter=*/nullptr, /*get_context=*/nullptr,
/*lookup_contex=*/nullptr));
Status s = blockhandles_iter->status();
if (!s.ok()) {
out_stream << "Can not read Index Block \n\n";
return s;
}
uint64_t datablock_size_min = std::numeric_limits<uint64_t>::max();
uint64_t datablock_size_max = 0;
uint64_t datablock_size_sum = 0;
size_t block_id = 1;
for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
block_id++, blockhandles_iter->Next()) {
s = blockhandles_iter->status();
if (!s.ok()) {
break;
}
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
BlockHandle bh = blockhandles_iter->value().handle;
uint64_t datablock_size = bh.size();
datablock_size_min = std::min(datablock_size_min, datablock_size);
datablock_size_max = std::max(datablock_size_max, datablock_size);
datablock_size_sum += datablock_size;
out_stream << "Data Block # " << block_id << " @ "
<< blockhandles_iter->value().handle.ToString(true) << "\n";
out_stream << "--------------------------------------\n";
std::unique_ptr<InternalIterator> datablock_iter;
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
Status tmp_status;
datablock_iter.reset(NewDataBlockIterator<DataBlockIter>(
Group rocksdb.sst.read.micros stat by IOActivity flush and compaction (#11288) Summary: **Context:** The existing stat rocksdb.sst.read.micros does not reflect each of compaction and flush cases but aggregate them, which is not so helpful for us to understand IO read behavior of each of them. **Summary** - Update `StopWatch` and `RandomAccessFileReader` to record `rocksdb.sst.read.micros` and `rocksdb.file.{flush/compaction}.read.micros` - Fixed the default histogram in `RandomAccessFileReader` - New field `ReadOptions/IOOptions::io_activity`; Pass `ReadOptions` through paths under db open, flush and compaction to where we can prepare `IOOptions` and pass it to `RandomAccessFileReader` - Use `thread_status_util` for assertion in `DbStressFSWrapper` for continuous testing on we are passing correct `io_activity` under db open, flush and compaction Pull Request resolved: https://github.com/facebook/rocksdb/pull/11288 Test Plan: - **Stress test** - **Db bench 1: rocksdb.sst.read.micros COUNT ≈ sum of rocksdb.file.read.flush.micros's and rocksdb.file.read.compaction.micros's.** (without blob) - May not be exactly the same due to `HistogramStat::Add` only guarantees atomic not accuracy across threads. ``` ./db_bench -db=/dev/shm/testdb/ -statistics=true -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -target_file_size_base=655 -disable_auto_compactions=false -compression_type=none -bloom_bits=3 (-use_plain_table=1 -prefix_size=10) ``` ``` // BlockBasedTable rocksdb.sst.read.micros P50 : 2.009374 P95 : 4.968548 P99 : 8.110362 P100 : 43.000000 COUNT : 40456 SUM : 114805 rocksdb.file.read.flush.micros P50 : 1.871841 P95 : 3.872407 P99 : 5.540541 P100 : 43.000000 COUNT : 2250 SUM : 6116 rocksdb.file.read.compaction.micros P50 : 2.023109 P95 : 5.029149 P99 : 8.196910 P100 : 26.000000 COUNT : 38206 SUM : 108689 // PlainTable Does not apply ``` - **Db bench 2: performance** **Read** SETUP: db with 900 files ``` ./db_bench -db=/dev/shm/testdb/ -benchmarks="fillseq" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=true -target_file_size_base=655 -compression_type=none ```run till convergence ``` ./db_bench -seed=1678564177044286 -use_existing_db=true -db=/dev/shm/testdb -benchmarks=readrandom[-X60] -statistics=true -num=1000000 -disable_auto_compactions=true -compression_type=none -bloom_bits=3 ``` Pre-change `readrandom [AVG 60 runs] : 21568 (± 248) ops/sec` Post-change (no regression, -0.3%) `readrandom [AVG 60 runs] : 21486 (± 236) ops/sec` **Compaction/Flush**run till convergence ``` ./db_bench -db=/dev/shm/testdb2/ -seed=1678564177044286 -benchmarks="fillseq[-X60]" -key_size=32 -value_size=512 -num=50000 -write_buffer_size=655 -disable_auto_compactions=false -target_file_size_base=655 -compression_type=none rocksdb.sst.read.micros COUNT : 33820 rocksdb.sst.read.flush.micros COUNT : 1800 rocksdb.sst.read.compaction.micros COUNT : 32020 ``` Pre-change `fillseq [AVG 46 runs] : 1391 (± 214) ops/sec; 0.7 (± 0.1) MB/sec` Post-change (no regression, ~-0.4%) `fillseq [AVG 46 runs] : 1385 (± 216) ops/sec; 0.7 (± 0.1) MB/sec` Reviewed By: ajkr Differential Revision: D44007011 Pulled By: hx235 fbshipit-source-id: a54c89e4846dfc9a135389edf3f3eedfea257132
2 years ago
read_options, blockhandles_iter->value().handle,
Add an option to put first key of each sst block in the index (#5289) Summary: The first key is used to defer reading the data block until this file gets to the top of merging iterator's heap. For short range scans, most files never make it to the top of the heap, so this change can reduce read amplification by a lot sometimes. Consider the following workload. There are a few data streams (we'll be calling them "logs"), each stream consisting of a sequence of blobs (we'll be calling them "records"). Each record is identified by log ID and a sequence number within the log. RocksDB key is concatenation of log ID and sequence number (big endian). Reads are mostly relatively short range scans, each within a single log. Writes are mostly sequential for each log, but writes to different logs are randomly interleaved. Compactions are disabled; instead, when we accumulate a few tens of sst files, we create a new column family and start writing to it. So, a typical sst file consists of a few ranges of blocks, each range corresponding to one log ID (we use FlushBlockPolicy to cut blocks at log boundaries). A typical read would go like this. First, iterator Seek() reads one block from each sst file. Then a series of Next()s move through one sst file (since writes to each log are mostly sequential) until the subiterator reaches the end of this log in this sst file; then Next() switches to the next sst file and reads sequentially from that, and so on. Often a range scan will only return records from a small number of blocks in small number of sst files; in this case, the cost of initial Seek() reading one block from each file may be bigger than the cost of reading the actually useful blocks. Neither iterate_upper_bound nor bloom filters can prevent reading one block from each file in Seek(). But this PR can: if the index contains first key from each block, we don't have to read the block until this block actually makes it to the top of merging iterator's heap, so for short range scans we won't read any blocks from most of the sst files. This PR does the deferred block loading inside value() call. This is not ideal: there's no good way to report an IO error from inside value(). As discussed with siying offline, it would probably be better to change InternalIterator's interface to explicitly fetch deferred value and get status. I'll do it in a separate PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5289 Differential Revision: D15256423 Pulled By: al13n321 fbshipit-source-id: 750e4c39ce88e8d41662f701cf6275d9388ba46a
5 years ago
/*input_iter=*/nullptr, /*type=*/BlockType::kData,
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
3 years ago
/*get_context=*/nullptr, /*lookup_context=*/nullptr,
/*prefetch_buffer=*/nullptr, /*for_compaction=*/false,
/*async_read=*/false, tmp_status));
s = datablock_iter->status();
if (!s.ok()) {
out_stream << "Error reading the block - Skipped \n\n";
continue;
}
for (datablock_iter->SeekToFirst(); datablock_iter->Valid();
datablock_iter->Next()) {
s = datablock_iter->status();
if (!s.ok()) {
out_stream << "Error reading the block - Skipped \n";
break;
}
DumpKeyValue(datablock_iter->key(), datablock_iter->value(), out_stream);
}
out_stream << "\n";
}
uint64_t num_datablocks = block_id - 1;
if (num_datablocks) {
double datablock_size_avg =
static_cast<double>(datablock_size_sum) / num_datablocks;
out_stream << "Data Block Summary:\n";
out_stream << "--------------------------------------\n";
out_stream << " # data blocks: " << num_datablocks << "\n";
out_stream << " min data block size: " << datablock_size_min << "\n";
out_stream << " max data block size: " << datablock_size_max << "\n";
out_stream << " avg data block size: "
<< std::to_string(datablock_size_avg) << "\n";
}
return Status::OK();
}
void BlockBasedTable::DumpKeyValue(const Slice& key, const Slice& value,
std::ostream& out_stream) {
InternalKey ikey;
ikey.DecodeFrom(key);
out_stream << " HEX " << ikey.user_key().ToString(true) << ": "
<< value.ToString(true) << "\n";
std::string str_key = ikey.user_key().ToString();
std::string str_value = value.ToString();
std::string res_key(""), res_value("");
char cspace = ' ';
for (size_t i = 0; i < str_key.size(); i++) {
if (str_key[i] == '\0') {
res_key.append("\\0", 2);
} else {
res_key.append(&str_key[i], 1);
}
res_key.append(1, cspace);
}
for (size_t i = 0; i < str_value.size(); i++) {
if (str_value[i] == '\0') {
res_value.append("\\0", 2);
} else {
res_value.append(&str_value[i], 1);
}
res_value.append(1, cspace);
}
out_stream << " ASCII " << res_key << ": " << res_value << "\n";
out_stream << " ------\n";
}
} // namespace ROCKSDB_NAMESPACE