2019-04-18 19:51:19 +02:00
|
|
|
// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
|
2013-10-29 01:54:09 +01:00
|
|
|
// 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.
|
2014-01-28 06:58:46 +01:00
|
|
|
//
|
|
|
|
// Currently we support two types of tables: plain table and block-based table.
|
|
|
|
// 1. Block-based table: this is the default table type that we inherited from
|
|
|
|
// LevelDB, which was designed for storing data in hard disk or flash
|
|
|
|
// device.
|
|
|
|
// 2. Plain table: it is one of RocksDB's SST file format optimized
|
|
|
|
// for low query latency on pure-memory or really low-latency media.
|
|
|
|
//
|
|
|
|
// A tutorial of rocksdb table formats is available here:
|
|
|
|
// https://github.com/facebook/rocksdb/wiki/A-Tutorial-of-RocksDB-SST-formats
|
|
|
|
//
|
|
|
|
// Example code is also available
|
|
|
|
// https://github.com/facebook/rocksdb/wiki/A-Tutorial-of-RocksDB-SST-formats#wiki-examples
|
2013-10-29 01:54:09 +01:00
|
|
|
|
|
|
|
#pragma once
|
2018-09-06 03:07:53 +02:00
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
#include <memory>
|
2014-01-28 06:58:46 +01:00
|
|
|
#include <string>
|
|
|
|
#include <unordered_map>
|
|
|
|
|
2020-11-12 00:09:14 +01:00
|
|
|
#include "rocksdb/customizable.h"
|
2013-10-29 01:54:09 +01:00
|
|
|
#include "rocksdb/env.h"
|
2020-05-08 00:52:58 +02:00
|
|
|
#include "rocksdb/options.h"
|
2014-01-28 06:58:46 +01:00
|
|
|
#include "rocksdb/status.h"
|
2013-10-29 01:54:09 +01:00
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2013-10-29 01:54:09 +01:00
|
|
|
|
2014-01-28 06:58:46 +01:00
|
|
|
// -- Block-based Table
|
2020-09-15 01:59:00 +02:00
|
|
|
class Cache;
|
2020-04-22 02:35:28 +02:00
|
|
|
class FilterPolicy;
|
2014-01-28 06:58:46 +01:00
|
|
|
class FlushBlockPolicyFactory;
|
2015-12-16 03:20:10 +01:00
|
|
|
class PersistentCache;
|
2014-02-04 04:48:45 +01:00
|
|
|
class RandomAccessFile;
|
2015-09-11 20:36:33 +02:00
|
|
|
struct TableReaderOptions;
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
2015-04-06 19:04:30 +02:00
|
|
|
struct TableBuilderOptions;
|
2014-02-04 04:48:45 +01:00
|
|
|
class TableBuilder;
|
2020-04-22 02:35:28 +02:00
|
|
|
class TableFactory;
|
2014-02-04 04:48:45 +01:00
|
|
|
class TableReader;
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
class WritableFileWriter;
|
2020-04-22 02:35:28 +02:00
|
|
|
struct ConfigOptions;
|
2014-02-04 04:48:45 +01:00
|
|
|
struct EnvOptions;
|
|
|
|
|
2014-05-01 20:09:32 +02:00
|
|
|
enum ChecksumType : char {
|
2017-08-24 04:31:40 +02:00
|
|
|
kNoChecksum = 0x0,
|
2014-05-01 20:09:32 +02:00
|
|
|
kCRC32c = 0x1,
|
|
|
|
kxxHash = 0x2,
|
2018-11-01 23:39:40 +01:00
|
|
|
kxxHash64 = 0x3,
|
2014-05-01 20:09:32 +02:00
|
|
|
};
|
|
|
|
|
2020-10-11 23:52:49 +02:00
|
|
|
// `PinningTier` is used to specify which tier of block-based tables should
|
|
|
|
// be affected by a block cache pinning setting (see
|
|
|
|
// `MetadataCacheOptions` below).
|
|
|
|
enum class PinningTier {
|
|
|
|
// For compatibility, this value specifies to fallback to the behavior
|
|
|
|
// indicated by the deprecated options,
|
|
|
|
// `pin_l0_filter_and_index_blocks_in_cache` and
|
|
|
|
// `pin_top_level_index_and_filter`.
|
|
|
|
kFallback,
|
|
|
|
|
|
|
|
// This tier contains no block-based tables.
|
|
|
|
kNone,
|
|
|
|
|
|
|
|
// This tier contains block-based tables that may have originated from a
|
|
|
|
// memtable flush. In particular, it includes tables from L0 that are smaller
|
|
|
|
// than 1.5 times the current `write_buffer_size`. Note these criteria imply
|
|
|
|
// it can include intra-L0 compaction outputs and ingested files, as long as
|
|
|
|
// they are not abnormally large compared to flushed files in L0.
|
|
|
|
kFlushedAndSimilar,
|
|
|
|
|
|
|
|
// This tier contains all block-based tables.
|
|
|
|
kAll,
|
|
|
|
};
|
|
|
|
|
|
|
|
// `MetadataCacheOptions` contains members indicating the desired caching
|
|
|
|
// behavior for the different categories of metadata blocks.
|
|
|
|
struct MetadataCacheOptions {
|
|
|
|
// The tier of block-based tables whose top-level index into metadata
|
|
|
|
// partitions will be pinned. Currently indexes and filters may be
|
|
|
|
// partitioned.
|
|
|
|
//
|
|
|
|
// Note `cache_index_and_filter_blocks` must be true for this option to have
|
|
|
|
// any effect. Otherwise any top-level index into metadata partitions would be
|
|
|
|
// held in table reader memory, outside the block cache.
|
|
|
|
PinningTier top_level_index_pinning = PinningTier::kFallback;
|
|
|
|
|
|
|
|
// The tier of block-based tables whose metadata partitions will be pinned.
|
|
|
|
// Currently indexes and filters may be partitioned.
|
|
|
|
PinningTier partition_pinning = PinningTier::kFallback;
|
|
|
|
|
|
|
|
// The tier of block-based tables whose unpartitioned metadata blocks will be
|
|
|
|
// pinned.
|
|
|
|
//
|
|
|
|
// Note `cache_index_and_filter_blocks` must be true for this option to have
|
|
|
|
// any effect. Otherwise the unpartitioned meta-blocks would be held in table
|
|
|
|
// reader memory, outside the block cache.
|
|
|
|
PinningTier unpartitioned_pinning = PinningTier::kFallback;
|
|
|
|
};
|
|
|
|
|
2014-01-28 06:58:46 +01:00
|
|
|
// For advanced user only
|
|
|
|
struct BlockBasedTableOptions {
|
2020-09-15 01:59:00 +02:00
|
|
|
static const char* kName() { return "BlockTableOptions"; };
|
2014-01-28 06:58:46 +01:00
|
|
|
// @flush_block_policy_factory creates the instances of flush block policy.
|
|
|
|
// which provides a configurable way to determine when to flush a block in
|
|
|
|
// the block based tables. If not set, table builder will use the default
|
|
|
|
// block flush policy, which cut blocks by block size (please refer to
|
|
|
|
// `FlushBlockBySizePolicy`).
|
|
|
|
std::shared_ptr<FlushBlockPolicyFactory> flush_block_policy_factory;
|
2013-10-29 01:54:09 +01:00
|
|
|
|
2014-01-28 06:58:46 +01:00
|
|
|
// TODO(kailiu) Temporarily disable this feature by making the default value
|
|
|
|
// to be false.
|
2013-10-30 18:52:33 +01:00
|
|
|
//
|
2019-01-24 03:11:08 +01:00
|
|
|
// TODO(ajkr) we need to update names of variables controlling meta-block
|
|
|
|
// caching as they should now apply to range tombstone and compression
|
|
|
|
// dictionary meta-blocks, in addition to index and filter meta-blocks.
|
|
|
|
//
|
2014-01-28 06:58:46 +01:00
|
|
|
// Indicating if we'd put index/filter blocks to the block cache.
|
|
|
|
// If not specified, each "table reader" object will pre-load index/filter
|
|
|
|
// block during table initialization.
|
|
|
|
bool cache_index_and_filter_blocks = false;
|
2014-03-01 03:19:07 +01:00
|
|
|
|
2016-08-23 22:44:13 +02:00
|
|
|
// If cache_index_and_filter_blocks is enabled, cache index and filter
|
|
|
|
// blocks with high priority. If set to true, depending on implementation of
|
2017-05-18 08:03:54 +02:00
|
|
|
// block cache, index and filter blocks may be less likely to be evicted
|
2016-08-23 22:44:13 +02:00
|
|
|
// than data blocks.
|
2019-06-27 19:16:21 +02:00
|
|
|
bool cache_index_and_filter_blocks_with_high_priority = true;
|
2016-08-23 22:44:13 +02:00
|
|
|
|
2020-10-11 23:52:49 +02:00
|
|
|
// DEPRECATED: This option will be removed in a future version. For now, this
|
|
|
|
// option still takes effect by updating each of the following variables that
|
|
|
|
// has the default value, `PinningTier::kFallback`:
|
|
|
|
//
|
|
|
|
// - `MetadataCacheOptions::partition_pinning`
|
|
|
|
// - `MetadataCacheOptions::unpartitioned_pinning`
|
|
|
|
//
|
|
|
|
// The updated value is chosen as follows:
|
|
|
|
//
|
|
|
|
// - `pin_l0_filter_and_index_blocks_in_cache == false` ->
|
|
|
|
// `PinningTier::kNone`
|
|
|
|
// - `pin_l0_filter_and_index_blocks_in_cache == true` ->
|
|
|
|
// `PinningTier::kFlushedAndSimilar`
|
|
|
|
//
|
|
|
|
// To migrate away from this flag, explicitly configure
|
|
|
|
// `MetadataCacheOptions` as described above.
|
|
|
|
//
|
Adding pin_l0_filter_and_index_blocks_in_cache feature and related fixes.
Summary:
When a block based table file is opened, if prefetch_index_and_filter is true, it will prefetch the index and filter blocks, putting them into the block cache.
What this feature adds: when a L0 block based table file is opened, if pin_l0_filter_and_index_blocks_in_cache is true in the options (and prefetch_index_and_filter is true), then the filter and index blocks aren't released back to the block cache at the end of BlockBasedTableReader::Open(). Instead the table reader takes ownership of them, hence pinning them, ie. the LRU cache will never push them out. Meanwhile in the table reader, further accesses will not hit the block cache, thus avoiding lock contention.
Test Plan:
'export TEST_TMPDIR=/dev/shm/ && DISABLE_JEMALLOC=1 OPT=-g make all valgrind_check -j32' is OK.
I didn't run the Java tests, I don't have Java set up on my devserver.
Reviewers: sdong
Reviewed By: sdong
Subscribers: andrewkr, dhruba
Differential Revision: https://reviews.facebook.net/D56133
2016-04-01 19:42:39 +02:00
|
|
|
// if cache_index_and_filter_blocks is true and the below is true, then
|
|
|
|
// filter and index blocks are stored in the cache, but a reference is
|
|
|
|
// held in the "table reader" object so the blocks are pinned and only
|
|
|
|
// evicted from cache when the table reader is freed.
|
|
|
|
bool pin_l0_filter_and_index_blocks_in_cache = false;
|
|
|
|
|
2020-10-11 23:52:49 +02:00
|
|
|
// DEPRECATED: This option will be removed in a future version. For now, this
|
|
|
|
// option still takes effect by updating
|
|
|
|
// `MetadataCacheOptions::top_level_index_pinning` when it has the
|
|
|
|
// default value, `PinningTier::kFallback`.
|
|
|
|
//
|
|
|
|
// The updated value is chosen as follows:
|
|
|
|
//
|
|
|
|
// - `pin_top_level_index_and_filter == false` ->
|
|
|
|
// `PinningTier::kNone`
|
|
|
|
// - `pin_top_level_index_and_filter == true` ->
|
|
|
|
// `PinningTier::kAll`
|
|
|
|
//
|
|
|
|
// To migrate away from this flag, explicitly configure
|
|
|
|
// `MetadataCacheOptions` as described above.
|
|
|
|
//
|
2018-06-23 00:14:05 +02:00
|
|
|
// If cache_index_and_filter_blocks is true and the below is true, then
|
|
|
|
// the top-level index of partitioned filter and index blocks are stored in
|
|
|
|
// the cache, but a reference is held in the "table reader" object so the
|
|
|
|
// blocks are pinned and only evicted from cache when the table reader is
|
|
|
|
// freed. This is not limited to l0 in LSM tree.
|
|
|
|
bool pin_top_level_index_and_filter = true;
|
|
|
|
|
2020-10-11 23:52:49 +02:00
|
|
|
// The desired block cache pinning behavior for the different categories of
|
|
|
|
// metadata blocks. While pinning can reduce block cache contention, users
|
|
|
|
// must take care not to pin excessive amounts of data, which risks
|
|
|
|
// overflowing block cache.
|
|
|
|
MetadataCacheOptions metadata_cache_options;
|
|
|
|
|
2014-03-01 03:19:07 +01:00
|
|
|
// The index type that will be used for this table.
|
|
|
|
enum IndexType : char {
|
|
|
|
// A space efficient index block that is optimized for
|
|
|
|
// binary-search-based index.
|
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
2019-06-25 05:50:35 +02:00
|
|
|
kBinarySearch = 0x00,
|
2014-04-10 23:19:43 +02:00
|
|
|
|
|
|
|
// The hash index, if enabled, will do the hash lookup when
|
2014-04-25 21:21:34 +02:00
|
|
|
// `Options.prefix_extractor` is provided.
|
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
2019-06-25 05:50:35 +02:00
|
|
|
kHashSearch = 0x01,
|
2017-02-07 01:29:29 +01:00
|
|
|
|
|
|
|
// A two-level index implementation. Both levels are binary search indexes.
|
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
2019-06-25 05:50:35 +02:00
|
|
|
kTwoLevelIndexSearch = 0x02,
|
|
|
|
|
|
|
|
// Like kBinarySearch, but index also contains first key of each block.
|
|
|
|
// This allows iterators to defer reading the block until it's actually
|
|
|
|
// needed. May significantly reduce read amplification of short range scans.
|
|
|
|
// Without it, iterator seek usually reads one block from each level-0 file
|
|
|
|
// and from each level, which may be expensive.
|
|
|
|
// Works best in combination with:
|
|
|
|
// - IndexShorteningMode::kNoShortening,
|
|
|
|
// - custom FlushBlockPolicy to cut blocks at some meaningful boundaries,
|
|
|
|
// e.g. when prefix changes.
|
|
|
|
// Makes the index significantly bigger (2x or more), especially when keys
|
|
|
|
// are long.
|
|
|
|
kBinarySearchWithFirstKey = 0x03,
|
2014-03-01 03:19:07 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
IndexType index_type = kBinarySearch;
|
2014-05-01 20:09:32 +02:00
|
|
|
|
2018-07-28 00:35:41 +02:00
|
|
|
// The index type that will be used for the data block.
|
|
|
|
enum DataBlockIndexType : char {
|
2018-08-17 03:29:13 +02:00
|
|
|
kDataBlockBinarySearch = 0, // traditional block type
|
|
|
|
kDataBlockBinaryAndHash = 1, // additional hash index
|
2018-07-28 00:35:41 +02:00
|
|
|
};
|
|
|
|
|
|
|
|
DataBlockIndexType data_block_index_type = kDataBlockBinarySearch;
|
|
|
|
|
2018-08-15 23:27:47 +02:00
|
|
|
// #entries/#buckets. It is valid only when data_block_hash_index_type is
|
|
|
|
// kDataBlockBinaryAndHash.
|
|
|
|
double data_block_hash_table_util_ratio = 0.75;
|
|
|
|
|
2016-05-21 02:14:38 +02:00
|
|
|
// This option is now deprecated. No matter what value it is set to,
|
|
|
|
// it will behave as if hash_index_allow_collision=true.
|
2014-06-13 04:03:22 +02:00
|
|
|
bool hash_index_allow_collision = true;
|
|
|
|
|
2014-05-01 20:09:32 +02:00
|
|
|
// Use the specified checksum type. Newly created table files will be
|
|
|
|
// protected with this checksum type. Old table files will still be readable,
|
|
|
|
// even though they have different checksum type.
|
|
|
|
ChecksumType checksum = kCRC32c;
|
2014-08-25 23:22:05 +02:00
|
|
|
|
|
|
|
// Disable block cache. If this is set to true,
|
|
|
|
// then no block cache should be used, and the block_cache should
|
|
|
|
// point to a nullptr object.
|
|
|
|
bool no_block_cache = false;
|
|
|
|
|
|
|
|
// If non-NULL use the specified cache for blocks.
|
|
|
|
// If NULL, rocksdb will automatically create and use an 8MB internal cache.
|
|
|
|
std::shared_ptr<Cache> block_cache = nullptr;
|
|
|
|
|
2015-12-16 03:20:10 +01:00
|
|
|
// If non-NULL use the specified cache for pages read from device
|
|
|
|
// IF NULL, no page cache is used
|
|
|
|
std::shared_ptr<PersistentCache> persistent_cache = nullptr;
|
|
|
|
|
2014-08-25 23:22:05 +02:00
|
|
|
// If non-NULL use the specified cache for compressed blocks.
|
|
|
|
// If NULL, rocksdb will not use a compressed block cache.
|
2018-11-14 02:00:49 +01:00
|
|
|
// Note: though it looks similar to `block_cache`, RocksDB doesn't put the
|
|
|
|
// same type of object there.
|
2014-08-25 23:22:05 +02:00
|
|
|
std::shared_ptr<Cache> block_cache_compressed = nullptr;
|
|
|
|
|
|
|
|
// Approximate size of user data packed per block. Note that the
|
|
|
|
// block size specified here corresponds to uncompressed data. The
|
|
|
|
// actual size of the unit read from disk may be smaller if
|
|
|
|
// compression is enabled. This parameter can be changed dynamically.
|
|
|
|
size_t block_size = 4 * 1024;
|
|
|
|
|
|
|
|
// This is used to close a block before it reaches the configured
|
|
|
|
// 'block_size'. If the percentage of free space in the current block is less
|
|
|
|
// than this specified number and adding a new record to the block will
|
|
|
|
// exceed the configured block size, then this block will be closed and the
|
|
|
|
// new record will be written to the next block.
|
|
|
|
int block_size_deviation = 10;
|
|
|
|
|
|
|
|
// Number of keys between restart points for delta encoding of keys.
|
|
|
|
// This parameter can be changed dynamically. Most clients should
|
2016-01-04 19:51:00 +01:00
|
|
|
// leave this parameter alone. The minimum value allowed is 1. Any smaller
|
|
|
|
// value will be silently overwritten with 1.
|
2014-08-25 23:22:05 +02:00
|
|
|
int block_restart_interval = 16;
|
|
|
|
|
2016-02-05 19:22:37 +01:00
|
|
|
// Same as block_restart_interval but used for the index block.
|
|
|
|
int index_block_restart_interval = 1;
|
|
|
|
|
2017-03-28 20:56:56 +02:00
|
|
|
// Block size for partitioned metadata. Currently applied to indexes when
|
|
|
|
// kTwoLevelIndexSearch is used and to filters when partition_filters is used.
|
|
|
|
// Note: Since in the current implementation the filters and index partitions
|
2017-05-18 08:03:54 +02:00
|
|
|
// are aligned, an index/filter block is created when either index or filter
|
2017-03-28 20:56:56 +02:00
|
|
|
// block size reaches the specified limit.
|
|
|
|
// Note: this limit is currently applied to only index blocks; a filter
|
|
|
|
// partition is cut right after an index block is cut
|
|
|
|
// TODO(myabandeh): remove the note above when filter partitions are cut
|
|
|
|
// separately
|
|
|
|
uint64_t metadata_block_size = 4096;
|
2017-02-07 01:29:29 +01:00
|
|
|
|
2017-03-07 22:48:02 +01:00
|
|
|
// Note: currently this option requires kTwoLevelIndexSearch to be set as
|
|
|
|
// well.
|
|
|
|
// TODO(myabandeh): remove the note above once the limitation is lifted
|
2017-11-02 19:06:10 +01:00
|
|
|
// Use partitioned full filters for each SST file. This option is
|
2018-03-08 19:18:34 +01:00
|
|
|
// incompatible with block-based filters.
|
2017-03-07 22:48:02 +01:00
|
|
|
bool partition_filters = false;
|
|
|
|
|
2021-06-17 21:28:00 +02:00
|
|
|
// Option to generate Bloom/Ribbon filters that minimize memory
|
Minimize memory internal fragmentation for Bloom filters (#6427)
Summary:
New experimental option BBTO::optimize_filters_for_memory builds
filters that maximize their use of "usable size" from malloc_usable_size,
which is also used to compute block cache charges.
Rather than always "rounding up," we track state in the
BloomFilterPolicy object to mix essentially "rounding down" and
"rounding up" so that the average FP rate of all generated filters is
the same as without the option. (YMMV as heavily accessed filters might
be unluckily lower accuracy.)
Thus, the option near-minimizes what the block cache considers as
"memory used" for a given target Bloom filter false positive rate and
Bloom filter implementation. There are no forward or backward
compatibility issues with this change, though it only works on the
format_version=5 Bloom filter.
With Jemalloc, we see about 10% reduction in memory footprint (and block
cache charge) for Bloom filters, but 1-2% increase in storage footprint,
due to encoding efficiency losses (FP rate is non-linear with bits/key).
Why not weighted random round up/down rather than state tracking? By
only requiring malloc_usable_size, we don't actually know what the next
larger and next smaller usable sizes for the allocator are. We pick a
requested size, accept and use whatever usable size it has, and use the
difference to inform our next choice. This allows us to narrow in on the
right balance without tracking/predicting usable sizes.
Why not weight history of generated filter false positive rates by
number of keys? This could lead to excess skew in small filters after
generating a large filter.
Results from filter_bench with jemalloc (irrelevant details omitted):
(normal keys/filter, but high variance)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=30000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.6278
Number of filters: 5516
Total size (MB): 200.046
Reported total allocated memory (MB): 220.597
Reported internal fragmentation: 10.2732%
Bits/key stored: 10.0097
Average FP rate %: 0.965228
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=30000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 30.5104
Number of filters: 5464
Total size (MB): 200.015
Reported total allocated memory (MB): 200.322
Reported internal fragmentation: 0.153709%
Bits/key stored: 10.1011
Average FP rate %: 0.966313
(very few keys / filter, optimization not as effective due to ~59 byte
internal fragmentation in blocked Bloom filter representation)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.5649
Number of filters: 162950
Total size (MB): 200.001
Reported total allocated memory (MB): 224.624
Reported internal fragmentation: 12.3117%
Bits/key stored: 10.2951
Average FP rate %: 0.821534
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 31.8057
Number of filters: 159849
Total size (MB): 200
Reported total allocated memory (MB): 208.846
Reported internal fragmentation: 4.42297%
Bits/key stored: 10.4948
Average FP rate %: 0.811006
(high keys/filter)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.7017
Number of filters: 164
Total size (MB): 200.352
Reported total allocated memory (MB): 221.5
Reported internal fragmentation: 10.5552%
Bits/key stored: 10.0003
Average FP rate %: 0.969358
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 30.7131
Number of filters: 160
Total size (MB): 200.928
Reported total allocated memory (MB): 200.938
Reported internal fragmentation: 0.00448054%
Bits/key stored: 10.1852
Average FP rate %: 0.963387
And from db_bench (block cache) with jemalloc:
$ ./db_bench -db=/dev/shm/dbbench.no_optimize -benchmarks=fillrandom -format_version=5 -value_size=90 -bloom_bits=10 -num=2000000 -threads=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false
$ ./db_bench -db=/dev/shm/dbbench -benchmarks=fillrandom -format_version=5 -value_size=90 -bloom_bits=10 -num=2000000 -threads=8 -optimize_filters_for_memory -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false
$ (for FILE in /dev/shm/dbbench.no_optimize/*.sst; do ./sst_dump --file=$FILE --show_properties | grep 'filter block' ; done) | awk '{ t += $4; } END { print t; }'
17063835
$ (for FILE in /dev/shm/dbbench/*.sst; do ./sst_dump --file=$FILE --show_properties | grep 'filter block' ; done) | awk '{ t += $4; } END { print t; }'
17430747
$ #^ 2.1% additional filter storage
$ ./db_bench -db=/dev/shm/dbbench.no_optimize -use_existing_db -benchmarks=readrandom,stats -statistics -bloom_bits=10 -num=2000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false -duration=10 -cache_index_and_filter_blocks -cache_size=1000000000
rocksdb.block.cache.index.add COUNT : 33
rocksdb.block.cache.index.bytes.insert COUNT : 8440400
rocksdb.block.cache.filter.add COUNT : 33
rocksdb.block.cache.filter.bytes.insert COUNT : 21087528
rocksdb.bloom.filter.useful COUNT : 4963889
rocksdb.bloom.filter.full.positive COUNT : 1214081
rocksdb.bloom.filter.full.true.positive COUNT : 1161999
$ #^ 1.04 % observed FP rate
$ ./db_bench -db=/dev/shm/dbbench -use_existing_db -benchmarks=readrandom,stats -statistics -bloom_bits=10 -num=2000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false -optimize_filters_for_memory -duration=10 -cache_index_and_filter_blocks -cache_size=1000000000
rocksdb.block.cache.index.add COUNT : 33
rocksdb.block.cache.index.bytes.insert COUNT : 8448592
rocksdb.block.cache.filter.add COUNT : 33
rocksdb.block.cache.filter.bytes.insert COUNT : 18220328
rocksdb.bloom.filter.useful COUNT : 5360933
rocksdb.bloom.filter.full.positive COUNT : 1321315
rocksdb.bloom.filter.full.true.positive COUNT : 1262999
$ #^ 1.08 % observed FP rate, 13.6% less memory usage for filters
(Due to specific key density, this example tends to generate filters that are "worse than average" for internal fragmentation. "Better than average" cases can show little or no improvement.)
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6427
Test Plan: unit test added, 'make check' with gcc, clang and valgrind
Reviewed By: siying
Differential Revision: D22124374
Pulled By: pdillinger
fbshipit-source-id: f3e3aa152f9043ddf4fae25799e76341d0d8714e
2020-06-22 22:30:57 +02:00
|
|
|
// internal fragmentation.
|
|
|
|
//
|
|
|
|
// When false, malloc_usable_size is not available, or format_version < 5,
|
|
|
|
// filters are generated without regard to internal fragmentation when
|
|
|
|
// loaded into memory (historical behavior). When true (and
|
2021-06-17 21:28:00 +02:00
|
|
|
// malloc_usable_size is available and format_version >= 5), then
|
Minimize memory internal fragmentation for Bloom filters (#6427)
Summary:
New experimental option BBTO::optimize_filters_for_memory builds
filters that maximize their use of "usable size" from malloc_usable_size,
which is also used to compute block cache charges.
Rather than always "rounding up," we track state in the
BloomFilterPolicy object to mix essentially "rounding down" and
"rounding up" so that the average FP rate of all generated filters is
the same as without the option. (YMMV as heavily accessed filters might
be unluckily lower accuracy.)
Thus, the option near-minimizes what the block cache considers as
"memory used" for a given target Bloom filter false positive rate and
Bloom filter implementation. There are no forward or backward
compatibility issues with this change, though it only works on the
format_version=5 Bloom filter.
With Jemalloc, we see about 10% reduction in memory footprint (and block
cache charge) for Bloom filters, but 1-2% increase in storage footprint,
due to encoding efficiency losses (FP rate is non-linear with bits/key).
Why not weighted random round up/down rather than state tracking? By
only requiring malloc_usable_size, we don't actually know what the next
larger and next smaller usable sizes for the allocator are. We pick a
requested size, accept and use whatever usable size it has, and use the
difference to inform our next choice. This allows us to narrow in on the
right balance without tracking/predicting usable sizes.
Why not weight history of generated filter false positive rates by
number of keys? This could lead to excess skew in small filters after
generating a large filter.
Results from filter_bench with jemalloc (irrelevant details omitted):
(normal keys/filter, but high variance)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=30000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.6278
Number of filters: 5516
Total size (MB): 200.046
Reported total allocated memory (MB): 220.597
Reported internal fragmentation: 10.2732%
Bits/key stored: 10.0097
Average FP rate %: 0.965228
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=30000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 30.5104
Number of filters: 5464
Total size (MB): 200.015
Reported total allocated memory (MB): 200.322
Reported internal fragmentation: 0.153709%
Bits/key stored: 10.1011
Average FP rate %: 0.966313
(very few keys / filter, optimization not as effective due to ~59 byte
internal fragmentation in blocked Bloom filter representation)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.5649
Number of filters: 162950
Total size (MB): 200.001
Reported total allocated memory (MB): 224.624
Reported internal fragmentation: 12.3117%
Bits/key stored: 10.2951
Average FP rate %: 0.821534
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 31.8057
Number of filters: 159849
Total size (MB): 200
Reported total allocated memory (MB): 208.846
Reported internal fragmentation: 4.42297%
Bits/key stored: 10.4948
Average FP rate %: 0.811006
(high keys/filter)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.7017
Number of filters: 164
Total size (MB): 200.352
Reported total allocated memory (MB): 221.5
Reported internal fragmentation: 10.5552%
Bits/key stored: 10.0003
Average FP rate %: 0.969358
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 30.7131
Number of filters: 160
Total size (MB): 200.928
Reported total allocated memory (MB): 200.938
Reported internal fragmentation: 0.00448054%
Bits/key stored: 10.1852
Average FP rate %: 0.963387
And from db_bench (block cache) with jemalloc:
$ ./db_bench -db=/dev/shm/dbbench.no_optimize -benchmarks=fillrandom -format_version=5 -value_size=90 -bloom_bits=10 -num=2000000 -threads=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false
$ ./db_bench -db=/dev/shm/dbbench -benchmarks=fillrandom -format_version=5 -value_size=90 -bloom_bits=10 -num=2000000 -threads=8 -optimize_filters_for_memory -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false
$ (for FILE in /dev/shm/dbbench.no_optimize/*.sst; do ./sst_dump --file=$FILE --show_properties | grep 'filter block' ; done) | awk '{ t += $4; } END { print t; }'
17063835
$ (for FILE in /dev/shm/dbbench/*.sst; do ./sst_dump --file=$FILE --show_properties | grep 'filter block' ; done) | awk '{ t += $4; } END { print t; }'
17430747
$ #^ 2.1% additional filter storage
$ ./db_bench -db=/dev/shm/dbbench.no_optimize -use_existing_db -benchmarks=readrandom,stats -statistics -bloom_bits=10 -num=2000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false -duration=10 -cache_index_and_filter_blocks -cache_size=1000000000
rocksdb.block.cache.index.add COUNT : 33
rocksdb.block.cache.index.bytes.insert COUNT : 8440400
rocksdb.block.cache.filter.add COUNT : 33
rocksdb.block.cache.filter.bytes.insert COUNT : 21087528
rocksdb.bloom.filter.useful COUNT : 4963889
rocksdb.bloom.filter.full.positive COUNT : 1214081
rocksdb.bloom.filter.full.true.positive COUNT : 1161999
$ #^ 1.04 % observed FP rate
$ ./db_bench -db=/dev/shm/dbbench -use_existing_db -benchmarks=readrandom,stats -statistics -bloom_bits=10 -num=2000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false -optimize_filters_for_memory -duration=10 -cache_index_and_filter_blocks -cache_size=1000000000
rocksdb.block.cache.index.add COUNT : 33
rocksdb.block.cache.index.bytes.insert COUNT : 8448592
rocksdb.block.cache.filter.add COUNT : 33
rocksdb.block.cache.filter.bytes.insert COUNT : 18220328
rocksdb.bloom.filter.useful COUNT : 5360933
rocksdb.bloom.filter.full.positive COUNT : 1321315
rocksdb.bloom.filter.full.true.positive COUNT : 1262999
$ #^ 1.08 % observed FP rate, 13.6% less memory usage for filters
(Due to specific key density, this example tends to generate filters that are "worse than average" for internal fragmentation. "Better than average" cases can show little or no improvement.)
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6427
Test Plan: unit test added, 'make check' with gcc, clang and valgrind
Reviewed By: siying
Differential Revision: D22124374
Pulled By: pdillinger
fbshipit-source-id: f3e3aa152f9043ddf4fae25799e76341d0d8714e
2020-06-22 22:30:57 +02:00
|
|
|
// filters are generated to "round up" and "round down" their sizes to
|
|
|
|
// minimize internal fragmentation when loaded into memory, assuming the
|
|
|
|
// reading DB has the same memory allocation characteristics as the
|
|
|
|
// generating DB. This option does not break forward or backward
|
|
|
|
// compatibility.
|
|
|
|
//
|
|
|
|
// While individual filters will vary in bits/key and false positive rate
|
|
|
|
// when setting is true, the implementation attempts to maintain a weighted
|
|
|
|
// average FP rate for filters consistent with this option set to false.
|
|
|
|
//
|
|
|
|
// With Jemalloc for example, this setting is expected to save about 10% of
|
|
|
|
// the memory footprint and block cache charge of filters, while increasing
|
|
|
|
// disk usage of filters by about 1-2% due to encoding efficiency losses
|
|
|
|
// with variance in bits/key.
|
|
|
|
//
|
|
|
|
// NOTE: Because some memory counted by block cache might be unmapped pages
|
|
|
|
// within internal fragmentation, this option can increase observed RSS
|
|
|
|
// memory usage. With cache_index_and_filter_blocks=true, this option makes
|
2021-06-17 21:28:00 +02:00
|
|
|
// the block cache better at using space it is allowed. (These issues
|
|
|
|
// should not arise with partitioned filters.)
|
Minimize memory internal fragmentation for Bloom filters (#6427)
Summary:
New experimental option BBTO::optimize_filters_for_memory builds
filters that maximize their use of "usable size" from malloc_usable_size,
which is also used to compute block cache charges.
Rather than always "rounding up," we track state in the
BloomFilterPolicy object to mix essentially "rounding down" and
"rounding up" so that the average FP rate of all generated filters is
the same as without the option. (YMMV as heavily accessed filters might
be unluckily lower accuracy.)
Thus, the option near-minimizes what the block cache considers as
"memory used" for a given target Bloom filter false positive rate and
Bloom filter implementation. There are no forward or backward
compatibility issues with this change, though it only works on the
format_version=5 Bloom filter.
With Jemalloc, we see about 10% reduction in memory footprint (and block
cache charge) for Bloom filters, but 1-2% increase in storage footprint,
due to encoding efficiency losses (FP rate is non-linear with bits/key).
Why not weighted random round up/down rather than state tracking? By
only requiring malloc_usable_size, we don't actually know what the next
larger and next smaller usable sizes for the allocator are. We pick a
requested size, accept and use whatever usable size it has, and use the
difference to inform our next choice. This allows us to narrow in on the
right balance without tracking/predicting usable sizes.
Why not weight history of generated filter false positive rates by
number of keys? This could lead to excess skew in small filters after
generating a large filter.
Results from filter_bench with jemalloc (irrelevant details omitted):
(normal keys/filter, but high variance)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=30000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.6278
Number of filters: 5516
Total size (MB): 200.046
Reported total allocated memory (MB): 220.597
Reported internal fragmentation: 10.2732%
Bits/key stored: 10.0097
Average FP rate %: 0.965228
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=30000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 30.5104
Number of filters: 5464
Total size (MB): 200.015
Reported total allocated memory (MB): 200.322
Reported internal fragmentation: 0.153709%
Bits/key stored: 10.1011
Average FP rate %: 0.966313
(very few keys / filter, optimization not as effective due to ~59 byte
internal fragmentation in blocked Bloom filter representation)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.5649
Number of filters: 162950
Total size (MB): 200.001
Reported total allocated memory (MB): 224.624
Reported internal fragmentation: 12.3117%
Bits/key stored: 10.2951
Average FP rate %: 0.821534
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 31.8057
Number of filters: 159849
Total size (MB): 200
Reported total allocated memory (MB): 208.846
Reported internal fragmentation: 4.42297%
Bits/key stored: 10.4948
Average FP rate %: 0.811006
(high keys/filter)
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000000 -vary_key_count_ratio=0.9
Build avg ns/key: 29.7017
Number of filters: 164
Total size (MB): 200.352
Reported total allocated memory (MB): 221.5
Reported internal fragmentation: 10.5552%
Bits/key stored: 10.0003
Average FP rate %: 0.969358
$ ./filter_bench -quick -impl=2 -average_keys_per_filter=1000000 -vary_key_count_ratio=0.9 -optimize_filters_for_memory
Build avg ns/key: 30.7131
Number of filters: 160
Total size (MB): 200.928
Reported total allocated memory (MB): 200.938
Reported internal fragmentation: 0.00448054%
Bits/key stored: 10.1852
Average FP rate %: 0.963387
And from db_bench (block cache) with jemalloc:
$ ./db_bench -db=/dev/shm/dbbench.no_optimize -benchmarks=fillrandom -format_version=5 -value_size=90 -bloom_bits=10 -num=2000000 -threads=8 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false
$ ./db_bench -db=/dev/shm/dbbench -benchmarks=fillrandom -format_version=5 -value_size=90 -bloom_bits=10 -num=2000000 -threads=8 -optimize_filters_for_memory -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false
$ (for FILE in /dev/shm/dbbench.no_optimize/*.sst; do ./sst_dump --file=$FILE --show_properties | grep 'filter block' ; done) | awk '{ t += $4; } END { print t; }'
17063835
$ (for FILE in /dev/shm/dbbench/*.sst; do ./sst_dump --file=$FILE --show_properties | grep 'filter block' ; done) | awk '{ t += $4; } END { print t; }'
17430747
$ #^ 2.1% additional filter storage
$ ./db_bench -db=/dev/shm/dbbench.no_optimize -use_existing_db -benchmarks=readrandom,stats -statistics -bloom_bits=10 -num=2000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false -duration=10 -cache_index_and_filter_blocks -cache_size=1000000000
rocksdb.block.cache.index.add COUNT : 33
rocksdb.block.cache.index.bytes.insert COUNT : 8440400
rocksdb.block.cache.filter.add COUNT : 33
rocksdb.block.cache.filter.bytes.insert COUNT : 21087528
rocksdb.bloom.filter.useful COUNT : 4963889
rocksdb.bloom.filter.full.positive COUNT : 1214081
rocksdb.bloom.filter.full.true.positive COUNT : 1161999
$ #^ 1.04 % observed FP rate
$ ./db_bench -db=/dev/shm/dbbench -use_existing_db -benchmarks=readrandom,stats -statistics -bloom_bits=10 -num=2000000 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=false -optimize_filters_for_memory -duration=10 -cache_index_and_filter_blocks -cache_size=1000000000
rocksdb.block.cache.index.add COUNT : 33
rocksdb.block.cache.index.bytes.insert COUNT : 8448592
rocksdb.block.cache.filter.add COUNT : 33
rocksdb.block.cache.filter.bytes.insert COUNT : 18220328
rocksdb.bloom.filter.useful COUNT : 5360933
rocksdb.bloom.filter.full.positive COUNT : 1321315
rocksdb.bloom.filter.full.true.positive COUNT : 1262999
$ #^ 1.08 % observed FP rate, 13.6% less memory usage for filters
(Due to specific key density, this example tends to generate filters that are "worse than average" for internal fragmentation. "Better than average" cases can show little or no improvement.)
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6427
Test Plan: unit test added, 'make check' with gcc, clang and valgrind
Reviewed By: siying
Differential Revision: D22124374
Pulled By: pdillinger
fbshipit-source-id: f3e3aa152f9043ddf4fae25799e76341d0d8714e
2020-06-22 22:30:57 +02:00
|
|
|
//
|
|
|
|
// NOTE: Do not set to true if you do not trust malloc_usable_size. With
|
|
|
|
// this option, RocksDB might access an allocated memory object beyond its
|
|
|
|
// original size if malloc_usable_size says it is safe to do so. While this
|
|
|
|
// can be considered bad practice, it should not produce undefined behavior
|
|
|
|
// unless malloc_usable_size is buggy or broken.
|
|
|
|
bool optimize_filters_for_memory = false;
|
|
|
|
|
2015-12-16 21:08:30 +01:00
|
|
|
// Use delta encoding to compress keys in blocks.
|
2016-04-26 21:41:07 +02:00
|
|
|
// ReadOptions::pin_data requires this option to be disabled.
|
2015-12-16 21:08:30 +01:00
|
|
|
//
|
|
|
|
// Default: true
|
|
|
|
bool use_delta_encoding = true;
|
|
|
|
|
2014-08-25 23:22:05 +02:00
|
|
|
// If non-nullptr, use the specified filter policy to reduce disk reads.
|
|
|
|
// Many applications will benefit from passing the result of
|
|
|
|
// NewBloomFilterPolicy() here.
|
|
|
|
std::shared_ptr<const FilterPolicy> filter_policy = nullptr;
|
|
|
|
|
|
|
|
// If true, place whole keys in the filter (not just prefixes).
|
|
|
|
// This must generally be true for gets to be efficient.
|
|
|
|
bool whole_key_filtering = true;
|
2015-01-13 23:33:04 +01:00
|
|
|
|
2016-06-11 03:20:54 +02:00
|
|
|
// Verify that decompressing the compressed block gives back the input. This
|
|
|
|
// is a verification mode that we use to detect bugs in compression
|
|
|
|
// algorithms.
|
|
|
|
bool verify_compression = false;
|
|
|
|
|
2016-08-27 03:55:58 +02:00
|
|
|
// If used, For every data block we load into memory, we will create a bitmap
|
|
|
|
// of size ((block_size / `read_amp_bytes_per_bit`) / 8) bytes. This bitmap
|
|
|
|
// will be used to figure out the percentage we actually read of the blocks.
|
|
|
|
//
|
|
|
|
// When this feature is used Tickers::READ_AMP_ESTIMATE_USEFUL_BYTES and
|
|
|
|
// Tickers::READ_AMP_TOTAL_READ_BYTES can be used to calculate the
|
|
|
|
// read amplification using this formula
|
|
|
|
// (READ_AMP_TOTAL_READ_BYTES / READ_AMP_ESTIMATE_USEFUL_BYTES)
|
|
|
|
//
|
|
|
|
// value => memory usage (percentage of loaded blocks memory)
|
|
|
|
// 1 => 12.50 %
|
|
|
|
// 2 => 06.25 %
|
|
|
|
// 4 => 03.12 %
|
|
|
|
// 8 => 01.56 %
|
|
|
|
// 16 => 00.78 %
|
|
|
|
//
|
|
|
|
// Note: This number must be a power of 2, if not it will be sanitized
|
|
|
|
// to be the next lowest power of 2, for example a value of 7 will be
|
|
|
|
// treated as 4, a value of 19 will be treated as 16.
|
|
|
|
//
|
|
|
|
// Default: 0 (disabled)
|
|
|
|
uint32_t read_amp_bytes_per_bit = 0;
|
|
|
|
|
2019-02-22 23:36:38 +01:00
|
|
|
// We currently have five versions:
|
2015-01-13 23:33:04 +01:00
|
|
|
// 0 -- This version is currently written out by all RocksDB's versions by
|
|
|
|
// default. Can be read by really old RocksDB's. Doesn't support changing
|
|
|
|
// checksum (default is CRC32).
|
|
|
|
// 1 -- Can be read by RocksDB's versions since 3.0. Supports non-default
|
|
|
|
// checksum, like xxHash. It is written by RocksDB when
|
|
|
|
// BlockBasedTableOptions::checksum is something other than kCRC32c. (version
|
|
|
|
// 0 is silently upconverted)
|
2015-01-16 18:18:45 +01:00
|
|
|
// 2 -- Can be read by RocksDB's versions since 3.10. Changes the way we
|
|
|
|
// encode compressed blocks with LZ4, BZip2 and Zlib compression. If you
|
|
|
|
// don't plan to run RocksDB before version 3.10, you should probably use
|
|
|
|
// this.
|
2018-05-26 03:41:31 +02:00
|
|
|
// 3 -- Can be read by RocksDB's versions since 5.15. Changes the way we
|
|
|
|
// encode the keys in index blocks. If you don't plan to run RocksDB before
|
|
|
|
// version 5.15, you should probably use this.
|
|
|
|
// This option only affects newly written tables. When reading existing
|
|
|
|
// tables, the information about version is read from the footer.
|
2018-09-07 16:51:26 +02:00
|
|
|
// 4 -- Can be read by RocksDB's versions since 5.16. Changes the way we
|
|
|
|
// encode the values in index blocks. If you don't plan to run RocksDB before
|
|
|
|
// version 5.16 and you are using index_block_restart_interval > 1, you should
|
|
|
|
// probably use this as it would reduce the index size.
|
|
|
|
// This option only affects newly written tables. When reading existing
|
|
|
|
// tables, the information about version is read from the footer.
|
2019-11-27 19:22:45 +01:00
|
|
|
// 5 -- Can be read by RocksDB's versions since 6.6.0. Full and partitioned
|
|
|
|
// filters use a generally faster and more accurate Bloom filter
|
|
|
|
// implementation, with a different schema.
|
2021-03-09 21:41:15 +01:00
|
|
|
uint32_t format_version = 5;
|
2018-01-11 00:06:29 +01:00
|
|
|
|
|
|
|
// Store index blocks on disk in compressed format. Changing this option to
|
|
|
|
// false will avoid the overhead of decompression if index blocks are evicted
|
|
|
|
// and read back
|
|
|
|
bool enable_index_compression = true;
|
2018-03-27 05:14:24 +02:00
|
|
|
|
|
|
|
// Align data blocks on lesser of page size and block size
|
|
|
|
bool block_align = false;
|
2019-04-22 17:17:45 +02:00
|
|
|
|
|
|
|
// This enum allows trading off increased index size for improved iterator
|
|
|
|
// seek performance in some situations, particularly when block cache is
|
|
|
|
// disabled (ReadOptions::fill_cache = false) and direct IO is
|
|
|
|
// enabled (DBOptions::use_direct_reads = true).
|
|
|
|
// The default mode is the best tradeoff for most use cases.
|
|
|
|
// This option only affects newly written tables.
|
|
|
|
//
|
|
|
|
// The index contains a key separating each pair of consecutive blocks.
|
|
|
|
// Let A be the highest key in one block, B the lowest key in the next block,
|
|
|
|
// and I the index entry separating these two blocks:
|
|
|
|
// [ ... A] I [B ...]
|
|
|
|
// I is allowed to be anywhere in [A, B).
|
|
|
|
// If an iterator is seeked to a key in (A, I], we'll unnecessarily read the
|
|
|
|
// first block, then immediately fall through to the second block.
|
|
|
|
// However, if I=A, this can't happen, and we'll read only the second block.
|
|
|
|
// In kNoShortening mode, we use I=A. In other modes, we use the shortest
|
|
|
|
// key in [A, B), which usually significantly reduces index size.
|
|
|
|
//
|
|
|
|
// There's a similar story for the last index entry, which is an upper bound
|
|
|
|
// of the highest key in the file. If it's shortened and therefore
|
|
|
|
// overestimated, iterator is likely to unnecessarily read the last data block
|
|
|
|
// from each file on each seek.
|
|
|
|
enum class IndexShorteningMode : char {
|
|
|
|
// Use full keys.
|
|
|
|
kNoShortening,
|
|
|
|
// Shorten index keys between blocks, but use full key for the last index
|
|
|
|
// key, which is the upper bound of the whole file.
|
|
|
|
kShortenSeparators,
|
|
|
|
// Shorten both keys between blocks and key after last block.
|
|
|
|
kShortenSeparatorsAndSuccessor,
|
|
|
|
};
|
|
|
|
|
|
|
|
IndexShorteningMode index_shortening =
|
|
|
|
IndexShorteningMode::kShortenSeparators;
|
2021-02-24 01:52:35 +01:00
|
|
|
|
|
|
|
// RocksDB does auto-readahead for iterators on noticing more than two reads
|
|
|
|
// for a table file if user doesn't provide readahead_size. The readahead
|
|
|
|
// starts at 8KB and doubles on every additional read upto
|
|
|
|
// max_auto_readahead_size and max_auto_readahead_size can be configured.
|
|
|
|
//
|
|
|
|
// Special Value: 0 - If max_auto_readahead_size is set 0 then no implicit
|
|
|
|
// auto prefetching will be done. If max_auto_readahead_size provided is less
|
|
|
|
// than 8KB (which is initial readahead size used by rocksdb in case of
|
|
|
|
// auto-readahead), readahead size will remain same as
|
|
|
|
// max_auto_readahead_size.
|
|
|
|
//
|
|
|
|
// Value should be provided along with KB i.e. 256 * 1024 as it will prefetch
|
|
|
|
// the blocks.
|
|
|
|
//
|
|
|
|
// Found that 256 KB readahead size provides the best performance, based on
|
|
|
|
// experiments, for auto readahead. Experiment data is in PR #3282.
|
|
|
|
//
|
|
|
|
// This parameter can be changed dynamically by
|
|
|
|
// DB::SetOptions({{"block_based_table_factory",
|
|
|
|
// "{max_auto_readahead_size=0;}"}}));
|
|
|
|
//
|
|
|
|
// Changing the value dynamically will only affect files opened after the
|
|
|
|
// change.
|
|
|
|
//
|
|
|
|
// Default: 256 KB (256 * 1024).
|
|
|
|
size_t max_auto_readahead_size = 256 * 1024;
|
2021-06-18 06:55:42 +02:00
|
|
|
|
|
|
|
// If enabled, prepopulate warm/hot data blocks which are already in memory
|
|
|
|
// into block cache at the time of flush. On a flush, the data block that is
|
|
|
|
// in memory (in memtables) get flushed to the device. If using Direct IO,
|
|
|
|
// additional IO is incurred to read this data back into memory again, which
|
|
|
|
// is avoided by enabling this option. This further helps if the workload
|
|
|
|
// exhibits high temporal locality, where most of the reads go to recently
|
|
|
|
// written data. This also helps in case of Distributed FileSystem.
|
|
|
|
//
|
|
|
|
// Right now, this is enabled only for flush for data blocks. We plan to
|
|
|
|
// expand this option to cover compactions in the future and for other types
|
|
|
|
// of blocks.
|
|
|
|
enum class PrepopulateBlockCache : char {
|
|
|
|
// Disable prepopulate block cache.
|
|
|
|
kDisable,
|
|
|
|
// Prepopulate data blocks during flush only. Plan to extend it to all block
|
|
|
|
// types.
|
|
|
|
kFlushOnly,
|
|
|
|
};
|
|
|
|
|
|
|
|
PrepopulateBlockCache prepopulate_block_cache =
|
|
|
|
PrepopulateBlockCache::kDisable;
|
2014-03-01 03:19:07 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
// Table Properties that are specific to block-based table properties.
|
|
|
|
struct BlockBasedTablePropertyNames {
|
2018-03-08 19:18:34 +01:00
|
|
|
// value of this properties is a fixed int32 number.
|
2014-03-01 03:19:07 +01:00
|
|
|
static const std::string kIndexType;
|
2015-02-05 02:03:57 +01:00
|
|
|
// value is "1" for true and "0" for false.
|
|
|
|
static const std::string kWholeKeyFiltering;
|
|
|
|
// value is "1" for true and "0" for false.
|
|
|
|
static const std::string kPrefixFiltering;
|
2013-10-29 01:54:09 +01:00
|
|
|
};
|
|
|
|
|
2014-01-28 06:58:46 +01:00
|
|
|
// Create default block based table factory.
|
|
|
|
extern TableFactory* NewBlockBasedTableFactory(
|
|
|
|
const BlockBasedTableOptions& table_options = BlockBasedTableOptions());
|
|
|
|
|
2014-04-15 22:39:26 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-06-19 01:36:48 +02:00
|
|
|
|
|
|
|
enum EncodingType : char {
|
|
|
|
// Always write full keys without any special encoding.
|
|
|
|
kPlain,
|
|
|
|
// Find opportunity to write the same prefix once for multiple rows.
|
|
|
|
// In some cases, when a key follows a previous key with the same prefix,
|
|
|
|
// instead of writing out the full key, it just writes out the size of the
|
|
|
|
// shared prefix, as well as other bytes, to save some bytes.
|
|
|
|
//
|
|
|
|
// When using this option, the user is required to use the same prefix
|
|
|
|
// extractor to make sure the same prefix will be extracted from the same key.
|
|
|
|
// The Name() value of the prefix extractor will be stored in the file. When
|
|
|
|
// reopening the file, the name of the options.prefix_extractor given will be
|
|
|
|
// bitwise compared to the prefix extractors stored in the file. An error
|
|
|
|
// will be returned if the two don't match.
|
|
|
|
kPrefix,
|
|
|
|
};
|
|
|
|
|
|
|
|
// Table Properties that are specific to plain table properties.
|
|
|
|
struct PlainTablePropertyNames {
|
|
|
|
static const std::string kEncodingType;
|
2014-07-19 01:58:13 +02:00
|
|
|
static const std::string kBloomVersion;
|
|
|
|
static const std::string kNumBloomBlocks;
|
2014-06-19 01:36:48 +02:00
|
|
|
};
|
|
|
|
|
2014-07-18 09:08:38 +02:00
|
|
|
const uint32_t kPlainTableVariableLength = 0;
|
|
|
|
|
|
|
|
struct PlainTableOptions {
|
2020-09-15 01:59:00 +02:00
|
|
|
static const char* kName() { return "PlainTableOptions"; };
|
2014-08-25 23:24:09 +02:00
|
|
|
// @user_key_len: plain table has optimization for fix-sized keys, which can
|
|
|
|
// be specified via user_key_len. Alternatively, you can pass
|
|
|
|
// `kPlainTableVariableLength` if your keys have variable
|
|
|
|
// lengths.
|
|
|
|
uint32_t user_key_len = kPlainTableVariableLength;
|
|
|
|
|
|
|
|
// @bloom_bits_per_key: the number of bits used for bloom filer per prefix.
|
|
|
|
// You may disable it by passing a zero.
|
|
|
|
int bloom_bits_per_key = 10;
|
|
|
|
|
|
|
|
// @hash_table_ratio: the desired utilization of the hash table used for
|
|
|
|
// prefix hashing.
|
|
|
|
// hash_table_ratio = number of prefixes / #buckets in the
|
|
|
|
// hash table
|
|
|
|
double hash_table_ratio = 0.75;
|
|
|
|
|
|
|
|
// @index_sparseness: inside each prefix, need to build one index record for
|
|
|
|
// how many keys for binary search inside each hash bucket.
|
|
|
|
// For encoding type kPrefix, the value will be used when
|
|
|
|
// writing to determine an interval to rewrite the full
|
|
|
|
// key. It will also be used as a suggestion and satisfied
|
|
|
|
// when possible.
|
|
|
|
size_t index_sparseness = 16;
|
|
|
|
|
|
|
|
// @huge_page_tlb_size: if <=0, allocate hash indexes and blooms from malloc.
|
|
|
|
// Otherwise from huge page TLB. The user needs to
|
|
|
|
// reserve huge pages for it to be allocated, like:
|
|
|
|
// sysctl -w vm.nr_hugepages=20
|
|
|
|
// See linux doc Documentation/vm/hugetlbpage.txt
|
|
|
|
size_t huge_page_tlb_size = 0;
|
|
|
|
|
|
|
|
// @encoding_type: how to encode the keys. See enum EncodingType above for
|
|
|
|
// the choices. The value will determine how to encode keys
|
|
|
|
// when writing to a new SST file. This value will be stored
|
|
|
|
// inside the SST file which will be used when reading from
|
|
|
|
// the file, which makes it possible for users to choose
|
|
|
|
// different encoding type when reopening a DB. Files with
|
|
|
|
// different encoding types can co-exist in the same DB and
|
|
|
|
// can be read.
|
|
|
|
EncodingType encoding_type = kPlain;
|
|
|
|
|
|
|
|
// @full_scan_mode: mode for reading the whole file one record by one without
|
|
|
|
// using the index.
|
2014-07-18 09:08:38 +02:00
|
|
|
bool full_scan_mode = false;
|
2014-07-19 01:58:13 +02:00
|
|
|
|
|
|
|
// @store_index_in_file: compute plain table index and bloom filter during
|
|
|
|
// file building and store it in file. When reading
|
2021-03-29 14:04:06 +02:00
|
|
|
// file, index will be mapped instead of recomputation.
|
2019-03-02 00:41:55 +01:00
|
|
|
bool store_index_in_file = false;
|
2014-07-18 09:08:38 +02:00
|
|
|
};
|
|
|
|
|
|
|
|
// -- Plain Table with prefix-only seek
|
2019-03-27 21:21:27 +01:00
|
|
|
// For this factory, you need to set Options.prefix_extractor properly to make
|
|
|
|
// it work. Look-up will starts with prefix hash lookup for key prefix. Inside
|
|
|
|
// the hash bucket found, a binary search is executed for hash conflicts.
|
|
|
|
// Finally, a linear search is used.
|
2014-07-18 09:08:38 +02:00
|
|
|
|
2019-03-27 21:21:27 +01:00
|
|
|
extern TableFactory* NewPlainTableFactory(
|
|
|
|
const PlainTableOptions& options = PlainTableOptions());
|
2013-10-29 01:54:09 +01:00
|
|
|
|
2014-07-21 22:26:09 +02:00
|
|
|
struct CuckooTablePropertyNames {
|
2014-08-28 19:42:23 +02:00
|
|
|
// The key that is used to fill empty buckets.
|
2014-07-24 19:07:41 +02:00
|
|
|
static const std::string kEmptyKey;
|
2014-08-28 19:42:23 +02:00
|
|
|
// Fixed length of value.
|
2014-07-24 19:07:41 +02:00
|
|
|
static const std::string kValueLength;
|
2014-08-28 19:42:23 +02:00
|
|
|
// Number of hash functions used in Cuckoo Hash.
|
|
|
|
static const std::string kNumHashFunc;
|
|
|
|
// It denotes the number of buckets in a Cuckoo Block. Given a key and a
|
|
|
|
// particular hash function, a Cuckoo Block is a set of consecutive buckets,
|
|
|
|
// where starting bucket id is given by the hash function on the key. In case
|
|
|
|
// of a collision during inserting the key, the builder tries to insert the
|
|
|
|
// key in other locations of the cuckoo block before using the next hash
|
|
|
|
// function. This reduces cache miss during read operation in case of
|
|
|
|
// collision.
|
|
|
|
static const std::string kCuckooBlockSize;
|
|
|
|
// Size of the hash table. Use this number to compute the modulo of hash
|
|
|
|
// function. The actual number of buckets will be kMaxHashTableSize +
|
|
|
|
// kCuckooBlockSize - 1. The last kCuckooBlockSize-1 buckets are used to
|
|
|
|
// accommodate the Cuckoo Block from end of hash table, due to cache friendly
|
|
|
|
// implementation.
|
|
|
|
static const std::string kHashTableSize;
|
|
|
|
// Denotes if the key sorted in the file is Internal Key (if false)
|
|
|
|
// or User Key only (if true).
|
2014-07-26 01:37:32 +02:00
|
|
|
static const std::string kIsLastLevel;
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
// Indicate if using identity function for the first hash function.
|
|
|
|
static const std::string kIdentityAsFirstHash;
|
2014-09-25 22:53:27 +02:00
|
|
|
// Indicate if using module or bit and to calculate hash value
|
|
|
|
static const std::string kUseModuleHash;
|
2014-09-26 01:15:23 +02:00
|
|
|
// Fixed user key length
|
|
|
|
static const std::string kUserKeyLength;
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
};
|
|
|
|
|
|
|
|
struct CuckooTableOptions {
|
2020-09-15 01:59:00 +02:00
|
|
|
static const char* kName() { return "CuckooTableOptions"; };
|
|
|
|
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
// Determines the utilization of hash tables. Smaller values
|
|
|
|
// result in larger hash tables with fewer collisions.
|
|
|
|
double hash_table_ratio = 0.9;
|
|
|
|
// A property used by builder to determine the depth to go to
|
|
|
|
// to search for a path to displace elements in case of
|
|
|
|
// collision. See Builder.MakeSpaceForKey method. Higher
|
|
|
|
// values result in more efficient hash tables with fewer
|
|
|
|
// lookups but take more time to build.
|
|
|
|
uint32_t max_search_depth = 100;
|
|
|
|
// In case of collision while inserting, the builder
|
|
|
|
// attempts to insert in the next cuckoo_block_size
|
|
|
|
// locations before skipping over to the next Cuckoo hash
|
|
|
|
// function. This makes lookups more cache friendly in case
|
|
|
|
// of collisions.
|
|
|
|
uint32_t cuckoo_block_size = 5;
|
2014-09-25 22:53:27 +02:00
|
|
|
// If this option is enabled, user key is treated as uint64_t and its value
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
// is used as hash value directly. This option changes builder's behavior.
|
|
|
|
// Reader ignore this option and behave according to what specified in table
|
|
|
|
// property.
|
|
|
|
bool identity_as_first_hash = false;
|
2014-09-25 22:53:27 +02:00
|
|
|
// If this option is set to true, module is used during hash calculation.
|
|
|
|
// This often yields better space efficiency at the cost of performance.
|
2018-03-08 19:18:34 +01:00
|
|
|
// If this option is set to false, # of entries in table is constrained to be
|
2014-09-25 22:53:27 +02:00
|
|
|
// power of two, and bit and is used to calculate hash, which is faster in
|
|
|
|
// general.
|
|
|
|
bool use_module_hash = true;
|
2014-07-21 22:26:09 +02:00
|
|
|
};
|
|
|
|
|
2014-08-28 19:42:23 +02:00
|
|
|
// Cuckoo Table Factory for SST table format using Cache Friendly Cuckoo Hashing
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
extern TableFactory* NewCuckooTableFactory(
|
|
|
|
const CuckooTableOptions& table_options = CuckooTableOptions());
|
2014-08-12 05:21:07 +02:00
|
|
|
|
2014-04-15 22:39:26 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
class RandomAccessFileReader;
|
|
|
|
|
2014-02-04 04:48:45 +01:00
|
|
|
// A base class for table factories.
|
2020-11-12 00:09:14 +01:00
|
|
|
class TableFactory : public Customizable {
|
2014-02-04 04:48:45 +01:00
|
|
|
public:
|
2020-09-15 01:59:00 +02:00
|
|
|
virtual ~TableFactory() override {}
|
|
|
|
|
|
|
|
static const char* kBlockCacheOpts() { return "BlockCache"; };
|
|
|
|
static const char* kBlockBasedTableName() { return "BlockBasedTable"; };
|
|
|
|
static const char* kPlainTableName() { return "PlainTable"; }
|
|
|
|
static const char* kCuckooTableName() { return "CuckooTable"; };
|
|
|
|
|
|
|
|
// Creates and configures a new TableFactory from the input options and id.
|
|
|
|
static Status CreateFromString(const ConfigOptions& config_options,
|
|
|
|
const std::string& id,
|
|
|
|
std::shared_ptr<TableFactory>* factory);
|
2014-02-04 04:48:45 +01:00
|
|
|
|
2020-11-12 00:09:14 +01:00
|
|
|
static const char* Type() { return "TableFactory"; }
|
2020-09-15 01:59:00 +02:00
|
|
|
|
2014-02-04 04:48:45 +01:00
|
|
|
// Returns a Table object table that can fetch data from file specified
|
|
|
|
// in parameter file. It's the caller's responsibility to make sure
|
|
|
|
// file is in the correct format.
|
|
|
|
//
|
2015-09-23 21:42:43 +02:00
|
|
|
// NewTableReader() is called in three places:
|
2014-02-04 04:48:45 +01:00
|
|
|
// (1) TableCache::FindTable() calls the function when table cache miss
|
|
|
|
// and cache the table object returned.
|
2018-11-27 21:59:27 +01:00
|
|
|
// (2) SstFileDumper (for SST Dump) opens the table and dump the table
|
2017-05-18 08:03:54 +02:00
|
|
|
// contents using the iterator of the table.
|
2018-12-13 23:12:02 +01:00
|
|
|
// (3) DBImpl::IngestExternalFile() calls this function to read the contents
|
|
|
|
// of the sst file it's attempting to add
|
2015-09-11 20:36:33 +02:00
|
|
|
//
|
|
|
|
// table_reader_options is a TableReaderOptions which contain all the
|
|
|
|
// needed parameters and configuration to open the table.
|
|
|
|
// file is a file handler to handle the file for the table.
|
|
|
|
// file_size is the physical file size of the file.
|
|
|
|
// table_reader is the output table reader.
|
2014-02-04 04:48:45 +01:00
|
|
|
virtual Status NewTableReader(
|
2015-09-11 20:36:33 +02:00
|
|
|
const TableReaderOptions& table_reader_options,
|
2018-11-09 20:17:34 +01:00
|
|
|
std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
|
|
|
|
std::unique_ptr<TableReader>* table_reader,
|
2020-06-29 23:51:57 +02:00
|
|
|
bool prefetch_index_and_filter_in_cache = true) const {
|
|
|
|
ReadOptions ro;
|
|
|
|
return NewTableReader(ro, table_reader_options, std::move(file), file_size,
|
|
|
|
table_reader, prefetch_index_and_filter_in_cache);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Overload of the above function that allows the caller to pass in a
|
|
|
|
// ReadOptions
|
|
|
|
virtual Status NewTableReader(
|
|
|
|
const ReadOptions& ro, const TableReaderOptions& table_reader_options,
|
|
|
|
std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
|
|
|
|
std::unique_ptr<TableReader>* table_reader,
|
|
|
|
bool prefetch_index_and_filter_in_cache) const = 0;
|
2014-02-04 04:48:45 +01:00
|
|
|
|
|
|
|
// Return a table builder to write to a file for this table type.
|
|
|
|
//
|
|
|
|
// It is called in several places:
|
|
|
|
// (1) When flushing memtable to a level-0 output file, it creates a table
|
|
|
|
// builder (In DBImpl::WriteLevel0Table(), by calling BuildTable())
|
|
|
|
// (2) During compaction, it gets the builder for writing compaction output
|
|
|
|
// files in DBImpl::OpenCompactionOutputFile().
|
|
|
|
// (3) When recovering from transaction logs, it creates a table builder to
|
|
|
|
// write to a level-0 output file (In DBImpl::WriteLevel0TableForRecovery,
|
|
|
|
// by calling BuildTable())
|
|
|
|
// (4) When running Repairer, it creates a table builder to convert logs to
|
|
|
|
// SST files (In Repairer::ConvertLogToTable() by calling BuildTable())
|
|
|
|
//
|
2017-05-18 08:03:54 +02:00
|
|
|
// Multiple configured can be accessed from there, including and not limited
|
2014-09-05 01:18:36 +02:00
|
|
|
// to compression options. file is a handle of a writable file.
|
|
|
|
// It is the caller's responsibility to keep the file open and close the file
|
|
|
|
// after closing the table builder. compression_type is the compression type
|
|
|
|
// to use in this table.
|
2014-02-04 04:48:45 +01:00
|
|
|
virtual TableBuilder* NewTableBuilder(
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
2015-04-06 19:04:30 +02:00
|
|
|
const TableBuilderOptions& table_builder_options,
|
2021-04-29 15:59:53 +02:00
|
|
|
WritableFileWriter* file) const = 0;
|
2014-08-21 00:53:39 +02:00
|
|
|
|
2017-07-13 01:49:56 +02:00
|
|
|
// Return is delete range supported
|
|
|
|
virtual bool IsDeleteRangeSupported() const { return false; }
|
2014-02-04 04:48:45 +01:00
|
|
|
};
|
|
|
|
|
2014-06-17 05:06:18 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-11-14 20:34:32 +01:00
|
|
|
// Create a special table factory that can open either of the supported
|
|
|
|
// table formats, based on setting inside the SST files. It should be used to
|
2014-06-17 05:06:18 +02:00
|
|
|
// convert a DB from one table format to another.
|
2014-06-18 07:04:37 +02:00
|
|
|
// @table_factory_to_write: the table factory used when writing to new files.
|
2014-06-17 05:06:18 +02:00
|
|
|
// @block_based_table_factory: block based table factory to use. If NULL, use
|
|
|
|
// a default one.
|
|
|
|
// @plain_table_factory: plain table factory to use. If NULL, use a default one.
|
2019-03-27 21:21:27 +01:00
|
|
|
// @cuckoo_table_factory: cuckoo table factory to use. If NULL, use a default
|
|
|
|
// one.
|
2014-06-17 05:06:18 +02:00
|
|
|
extern TableFactory* NewAdaptiveTableFactory(
|
2014-06-18 07:04:37 +02:00
|
|
|
std::shared_ptr<TableFactory> table_factory_to_write = nullptr,
|
2014-06-17 05:06:18 +02:00
|
|
|
std::shared_ptr<TableFactory> block_based_table_factory = nullptr,
|
2014-08-12 05:21:07 +02:00
|
|
|
std::shared_ptr<TableFactory> plain_table_factory = nullptr,
|
|
|
|
std::shared_ptr<TableFactory> cuckoo_table_factory = nullptr);
|
2014-06-17 05:06:18 +02:00
|
|
|
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|