You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/file/file_prefetch_buffer.h

329 lines
13 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#pragma once
#include <algorithm>
#include <atomic>
#include <sstream>
#include <string>
#include "file/readahead_file_info.h"
#include "monitoring/statistics.h"
#include "port/port.h"
#include "rocksdb/env.h"
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
#include "rocksdb/file_system.h"
#include "rocksdb/options.h"
#include "util/aligned_buffer.h"
namespace ROCKSDB_NAMESPACE {
#define DEAFULT_DECREMENT 8 * 1024
struct IOOptions;
class RandomAccessFileReader;
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
struct BufferInfo {
AlignedBuffer buffer_;
uint64_t offset_ = 0;
};
// FilePrefetchBuffer is a smart buffer to store and read data from a file.
class FilePrefetchBuffer {
public:
static const int kMinNumFileReadsToStartAutoReadahead = 2;
Fix bug in rocksdb internal automatic prefetching (#9234) Summary: After introducing adaptive_readahead, the original flow got broken. Readahead size was set to 0 because of which rocksdb wasn't be able to do automatic prefetching which it enables after seeing sequential reads. This PR fixes it. ---------------------------------------------------------------------------------------------------- Before this patch: b_bench -use_existing_db=true -db=/tmp/prefix_scan -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 6.27 Date: Tue Nov 30 11:56:50 2021 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ DB path: [/tmp/prefix_scan] seekrandom : 5356367.174 micros/op 0 ops/sec; 29.4 MB/s (23 of 23 found) ---------------------------------------------------------------------------------------------------- After the patch: ./db_bench -use_existing_db=true -db=/tmp/prefix_scan -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 6.27 Date: Tue Nov 30 14:38:33 2021 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ DB path: [/tmp/prefix_scan] seekrandom : 456504.277 micros/op 2 ops/sec; 359.8 MB/s (264 of 264 found) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9234 Test Plan: Ran ./db_bench -db=/data/mysql/rocksdb/prefix_scan -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_d irect_io_for_flush_and_compaction=true -target_file_size_base=16777216 and then ./db_bench -use_existing_db=true -db=/data/mysql/rocksdb/prefix_scan -benchmarks="seekrandom" -key_size=32 -value_siz e=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 and compared the results. Reviewed By: anand1976 Differential Revision: D32743965 Pulled By: akankshamahajan15 fbshipit-source-id: b950fba68c91963b7deb5c20acdf471bc60251f5
3 years ago
// Constructor.
//
// All arguments are optional.
// readahead_size : the initial readahead size.
// max_readahead_size : the maximum readahead size.
// If max_readahead_size > readahead_size, the readahead size will be
// doubled on every IO until max_readahead_size is hit.
// Typically this is set as a multiple of readahead_size.
// max_readahead_size should be greater than equal to readahead_size.
// enable : controls whether reading from the buffer is enabled.
// If false, TryReadFromCache() always return false, and we only take stats
// for the minimum offset if track_min_offset = true.
// track_min_offset : Track the minimum offset ever read and collect stats on
// it. Used for adaptable readahead of the file footer/metadata.
// implicit_auto_readahead : Readahead is enabled implicitly by rocksdb after
// doing sequential scans for two times.
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// async_io : When async_io is enabled, if it's implicit_auto_readahead, it
// prefetches data asynchronously in second buffer while curr_ is being
// consumed.
//
// Automatic readhead is enabled for a file if readahead_size
// and max_readahead_size are passed in.
// A user can construct a FilePrefetchBuffer without any arguments, but use
// `Prefetch` to load data into the buffer.
FilePrefetchBuffer(size_t readahead_size = 0, size_t max_readahead_size = 0,
bool enable = true, bool track_min_offset = false,
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
bool implicit_auto_readahead = false,
uint64_t num_file_reads = 0, FileSystem* fs = nullptr,
SystemClock* clock = nullptr, Statistics* stats = nullptr)
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
: curr_(0),
readahead_size_(readahead_size),
Make initial auto readahead_size configurable (#9836) Summary: Make initial auto readahead_size configurable Pull Request resolved: https://github.com/facebook/rocksdb/pull/9836 Test Plan: Added new unit test Ran regression: Without change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` With this change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1649895440554504 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.2 Date: Wed Apr 13 17:17:20 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] ... finished 100 ops seekrandom : 476892.488 micros/op 2 ops/sec; 344.6 MB/s (252 of 252 found) ``` Reviewed By: anand1976 Differential Revision: D35632815 Pulled By: akankshamahajan15 fbshipit-source-id: c8057a88f9294c9d03b1d434b03affe02f74d796
2 years ago
initial_auto_readahead_size_(readahead_size),
max_readahead_size_(max_readahead_size),
min_offset_read_(std::numeric_limits<size_t>::max()),
enable_(enable),
track_min_offset_(track_min_offset),
implicit_auto_readahead_(implicit_auto_readahead),
prev_offset_(0),
prev_len_(0),
num_file_reads_(num_file_reads),
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
io_handle_(nullptr),
del_fn_(nullptr),
async_read_in_progress_(false),
async_request_submitted_(false),
fs_(fs),
clock_(clock),
stats_(stats) {
assert((num_file_reads_ >= kMinNumFileReadsToStartAutoReadahead + 1) ||
(num_file_reads_ == 0));
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// If async_io_ is enabled, data is asynchronously filled in second buffer
// while curr_ is being consumed. If data is overlapping in two buffers,
// data is copied to third buffer to return continuous buffer.
bufs_.resize(3);
}
~FilePrefetchBuffer() {
// Abort any pending async read request before destroying the class object.
if (async_read_in_progress_ && fs_ != nullptr) {
std::vector<void*> handles;
handles.emplace_back(io_handle_);
Status s = fs_->AbortIO(handles);
assert(s.ok());
}
// Prefetch buffer bytes discarded.
uint64_t bytes_discarded = 0;
if (bufs_[curr_].buffer_.CurrentSize() != 0) {
bytes_discarded = bufs_[curr_].buffer_.CurrentSize();
}
if (bufs_[curr_ ^ 1].buffer_.CurrentSize() != 0) {
bytes_discarded += bufs_[curr_ ^ 1].buffer_.CurrentSize();
}
RecordInHistogram(stats_, PREFETCHED_BYTES_DISCARDED, bytes_discarded);
// Release io_handle_.
if (io_handle_ != nullptr && del_fn_ != nullptr) {
del_fn_(io_handle_);
io_handle_ = nullptr;
del_fn_ = nullptr;
}
}
// Load data into the buffer from a file.
Add rate limiter priority to ReadOptions (#9424) Summary: Users can set the priority for file reads associated with their operation by setting `ReadOptions::rate_limiter_priority` to something other than `Env::IO_TOTAL`. Rate limiting `VerifyChecksum()` and `VerifyFileChecksums()` is the motivation for this PR, so it also includes benchmarks and minor bug fixes to get that working. `RandomAccessFileReader::Read()` already had support for rate limiting compaction reads. I changed that rate limiting to be non-specific to compaction, but rather performed according to the passed in `Env::IOPriority`. Now the compaction read rate limiting is supported by setting `rate_limiter_priority = Env::IO_LOW` on its `ReadOptions`. There is no default value for the new `Env::IOPriority` parameter to `RandomAccessFileReader::Read()`. That means this PR goes through all callers (in some cases multiple layers up the call stack) to find a `ReadOptions` to provide the priority. There are TODOs for cases I believe it would be good to let user control the priority some day (e.g., file footer reads), and no TODO in cases I believe it doesn't matter (e.g., trace file reads). The API doc only lists the missing cases where a file read associated with a provided `ReadOptions` cannot be rate limited. For cases like file ingestion checksum calculation, there is no API to provide `ReadOptions` or `Env::IOPriority`, so I didn't count that as missing. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9424 Test Plan: - new unit tests - new benchmarks on ~50MB database with 1MB/s read rate limit and 100ms refill interval; verified with strace reads are chunked (at 0.1MB per chunk) and spaced roughly 100ms apart. - setup command: `./db_bench -benchmarks=fillrandom,compact -db=/tmp/testdb -target_file_size_base=1048576 -disable_auto_compactions=true -file_checksum=true` - benchmarks command: `strace -ttfe pread64 ./db_bench -benchmarks=verifychecksum,verifyfilechecksums -use_existing_db=true -db=/tmp/testdb -rate_limiter_bytes_per_sec=1048576 -rate_limit_bg_reads=1 -rate_limit_user_ops=true -file_checksum=true` - crash test using IO_USER priority on non-validation reads with https://github.com/facebook/rocksdb/issues/9567 reverted: `python3 tools/db_crashtest.py blackbox --max_key=1000000 --write_buffer_size=524288 --target_file_size_base=524288 --level_compaction_dynamic_level_bytes=true --duration=3600 --rate_limit_bg_reads=true --rate_limit_user_ops=true --rate_limiter_bytes_per_sec=10485760 --interval=10` Reviewed By: hx235 Differential Revision: D33747386 Pulled By: ajkr fbshipit-source-id: a2d985e97912fba8c54763798e04f006ccc56e0c
2 years ago
// reader : the file reader.
// offset : the file offset to start reading from.
// n : the number of bytes to read.
// rate_limiter_priority : rate limiting priority, or `Env::IO_TOTAL` to
// bypass.
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// is_async_read : if the data should be prefetched by calling read
// asynchronously. It should be set true when called
// from TryReadFromCache.
Status Prefetch(const IOOptions& opts, RandomAccessFileReader* reader,
Add rate limiter priority to ReadOptions (#9424) Summary: Users can set the priority for file reads associated with their operation by setting `ReadOptions::rate_limiter_priority` to something other than `Env::IO_TOTAL`. Rate limiting `VerifyChecksum()` and `VerifyFileChecksums()` is the motivation for this PR, so it also includes benchmarks and minor bug fixes to get that working. `RandomAccessFileReader::Read()` already had support for rate limiting compaction reads. I changed that rate limiting to be non-specific to compaction, but rather performed according to the passed in `Env::IOPriority`. Now the compaction read rate limiting is supported by setting `rate_limiter_priority = Env::IO_LOW` on its `ReadOptions`. There is no default value for the new `Env::IOPriority` parameter to `RandomAccessFileReader::Read()`. That means this PR goes through all callers (in some cases multiple layers up the call stack) to find a `ReadOptions` to provide the priority. There are TODOs for cases I believe it would be good to let user control the priority some day (e.g., file footer reads), and no TODO in cases I believe it doesn't matter (e.g., trace file reads). The API doc only lists the missing cases where a file read associated with a provided `ReadOptions` cannot be rate limited. For cases like file ingestion checksum calculation, there is no API to provide `ReadOptions` or `Env::IOPriority`, so I didn't count that as missing. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9424 Test Plan: - new unit tests - new benchmarks on ~50MB database with 1MB/s read rate limit and 100ms refill interval; verified with strace reads are chunked (at 0.1MB per chunk) and spaced roughly 100ms apart. - setup command: `./db_bench -benchmarks=fillrandom,compact -db=/tmp/testdb -target_file_size_base=1048576 -disable_auto_compactions=true -file_checksum=true` - benchmarks command: `strace -ttfe pread64 ./db_bench -benchmarks=verifychecksum,verifyfilechecksums -use_existing_db=true -db=/tmp/testdb -rate_limiter_bytes_per_sec=1048576 -rate_limit_bg_reads=1 -rate_limit_user_ops=true -file_checksum=true` - crash test using IO_USER priority on non-validation reads with https://github.com/facebook/rocksdb/issues/9567 reverted: `python3 tools/db_crashtest.py blackbox --max_key=1000000 --write_buffer_size=524288 --target_file_size_base=524288 --level_compaction_dynamic_level_bytes=true --duration=3600 --rate_limit_bg_reads=true --rate_limit_user_ops=true --rate_limiter_bytes_per_sec=10485760 --interval=10` Reviewed By: hx235 Differential Revision: D33747386 Pulled By: ajkr fbshipit-source-id: a2d985e97912fba8c54763798e04f006ccc56e0c
2 years ago
uint64_t offset, size_t n,
Env::IOPriority rate_limiter_priority);
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
2 years ago
// Request for reading the data from a file asynchronously.
// If data already exists in the buffer, result will be updated.
// reader : the file reader.
// offset : the file offset to start reading from.
// n : the number of bytes to read.
// result : if data already exists in the buffer, result will
// be updated with the data.
//
// If data already exist in the buffer, it will return Status::OK, otherwise
// it will send asynchronous request and return Status::TryAgain.
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
Status PrefetchAsync(const IOOptions& opts, RandomAccessFileReader* reader,
uint64_t offset, size_t n, Slice* result);
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// Tries returning the data for a file read from this buffer if that data is
// in the buffer.
// It handles tracking the minimum read offset if track_min_offset = true.
// It also does the exponential readahead when readahead_size is set as part
// of the constructor.
//
Add rate limiter priority to ReadOptions (#9424) Summary: Users can set the priority for file reads associated with their operation by setting `ReadOptions::rate_limiter_priority` to something other than `Env::IO_TOTAL`. Rate limiting `VerifyChecksum()` and `VerifyFileChecksums()` is the motivation for this PR, so it also includes benchmarks and minor bug fixes to get that working. `RandomAccessFileReader::Read()` already had support for rate limiting compaction reads. I changed that rate limiting to be non-specific to compaction, but rather performed according to the passed in `Env::IOPriority`. Now the compaction read rate limiting is supported by setting `rate_limiter_priority = Env::IO_LOW` on its `ReadOptions`. There is no default value for the new `Env::IOPriority` parameter to `RandomAccessFileReader::Read()`. That means this PR goes through all callers (in some cases multiple layers up the call stack) to find a `ReadOptions` to provide the priority. There are TODOs for cases I believe it would be good to let user control the priority some day (e.g., file footer reads), and no TODO in cases I believe it doesn't matter (e.g., trace file reads). The API doc only lists the missing cases where a file read associated with a provided `ReadOptions` cannot be rate limited. For cases like file ingestion checksum calculation, there is no API to provide `ReadOptions` or `Env::IOPriority`, so I didn't count that as missing. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9424 Test Plan: - new unit tests - new benchmarks on ~50MB database with 1MB/s read rate limit and 100ms refill interval; verified with strace reads are chunked (at 0.1MB per chunk) and spaced roughly 100ms apart. - setup command: `./db_bench -benchmarks=fillrandom,compact -db=/tmp/testdb -target_file_size_base=1048576 -disable_auto_compactions=true -file_checksum=true` - benchmarks command: `strace -ttfe pread64 ./db_bench -benchmarks=verifychecksum,verifyfilechecksums -use_existing_db=true -db=/tmp/testdb -rate_limiter_bytes_per_sec=1048576 -rate_limit_bg_reads=1 -rate_limit_user_ops=true -file_checksum=true` - crash test using IO_USER priority on non-validation reads with https://github.com/facebook/rocksdb/issues/9567 reverted: `python3 tools/db_crashtest.py blackbox --max_key=1000000 --write_buffer_size=524288 --target_file_size_base=524288 --level_compaction_dynamic_level_bytes=true --duration=3600 --rate_limit_bg_reads=true --rate_limit_user_ops=true --rate_limiter_bytes_per_sec=10485760 --interval=10` Reviewed By: hx235 Differential Revision: D33747386 Pulled By: ajkr fbshipit-source-id: a2d985e97912fba8c54763798e04f006ccc56e0c
2 years ago
// opts : the IO options to use.
// reader : the file reader.
// offset : the file offset.
// n : the number of bytes.
// result : output buffer to put the data into.
// s : output status.
// rate_limiter_priority : rate limiting priority, or `Env::IO_TOTAL` to
// bypass.
// for_compaction : true if cache read is done for compaction read.
bool TryReadFromCache(const IOOptions& opts, RandomAccessFileReader* reader,
uint64_t offset, size_t n, Slice* result, Status* s,
Add rate limiter priority to ReadOptions (#9424) Summary: Users can set the priority for file reads associated with their operation by setting `ReadOptions::rate_limiter_priority` to something other than `Env::IO_TOTAL`. Rate limiting `VerifyChecksum()` and `VerifyFileChecksums()` is the motivation for this PR, so it also includes benchmarks and minor bug fixes to get that working. `RandomAccessFileReader::Read()` already had support for rate limiting compaction reads. I changed that rate limiting to be non-specific to compaction, but rather performed according to the passed in `Env::IOPriority`. Now the compaction read rate limiting is supported by setting `rate_limiter_priority = Env::IO_LOW` on its `ReadOptions`. There is no default value for the new `Env::IOPriority` parameter to `RandomAccessFileReader::Read()`. That means this PR goes through all callers (in some cases multiple layers up the call stack) to find a `ReadOptions` to provide the priority. There are TODOs for cases I believe it would be good to let user control the priority some day (e.g., file footer reads), and no TODO in cases I believe it doesn't matter (e.g., trace file reads). The API doc only lists the missing cases where a file read associated with a provided `ReadOptions` cannot be rate limited. For cases like file ingestion checksum calculation, there is no API to provide `ReadOptions` or `Env::IOPriority`, so I didn't count that as missing. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9424 Test Plan: - new unit tests - new benchmarks on ~50MB database with 1MB/s read rate limit and 100ms refill interval; verified with strace reads are chunked (at 0.1MB per chunk) and spaced roughly 100ms apart. - setup command: `./db_bench -benchmarks=fillrandom,compact -db=/tmp/testdb -target_file_size_base=1048576 -disable_auto_compactions=true -file_checksum=true` - benchmarks command: `strace -ttfe pread64 ./db_bench -benchmarks=verifychecksum,verifyfilechecksums -use_existing_db=true -db=/tmp/testdb -rate_limiter_bytes_per_sec=1048576 -rate_limit_bg_reads=1 -rate_limit_user_ops=true -file_checksum=true` - crash test using IO_USER priority on non-validation reads with https://github.com/facebook/rocksdb/issues/9567 reverted: `python3 tools/db_crashtest.py blackbox --max_key=1000000 --write_buffer_size=524288 --target_file_size_base=524288 --level_compaction_dynamic_level_bytes=true --duration=3600 --rate_limit_bg_reads=true --rate_limit_user_ops=true --rate_limiter_bytes_per_sec=10485760 --interval=10` Reviewed By: hx235 Differential Revision: D33747386 Pulled By: ajkr fbshipit-source-id: a2d985e97912fba8c54763798e04f006ccc56e0c
2 years ago
Env::IOPriority rate_limiter_priority,
bool for_compaction = false);
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
bool TryReadFromCacheAsync(const IOOptions& opts,
RandomAccessFileReader* reader, uint64_t offset,
size_t n, Slice* result, Status* status,
Fix stress test failure "Corruption: checksum mismatch" or "Iterator Diverged" with async_io enabled (#10032) Summary: In case of non sequential reads with `async_io`, `FilePRefetchBuffer::TryReadFromCacheAsync` can be called for previous blocks with `offset < bufs_[curr_].offset_` which wasn't handled correctly resulting wrong data being returned from buffer. Since `FilePRefetchBuffer::PrefetchAsync` can be called for any data block, it sets `prev_len_` to 0 indicating `FilePRefetchBuffer::TryReadFromCacheAsync` to go for the prefetching even though offset < bufs_[curr_].offset_ This is because async prefetching is always done in second buffer (to avoid mutex) even though curr_ is empty leading to offset < bufs_[curr_].offset_ in some cases. If prev_len_ is non zero then `TryReadFromCacheAsync` returns false if `offset < bufs_[curr_].offset_ && prev_len != 0` indicating reads are not sequential and previous call wasn't PrefetchAsync. - This PR also simplifies `FilePRefetchBuffer::TryReadFromCacheAsync` as it was getting complicated covering different scenarios based on `async_io` enabled/disabled. If `for_compaction` is set true, it now calls `FilePRefetchBufferTryReadFromCache` following synchronous flow as before. Its decided in BlockFetcher.cc Pull Request resolved: https://github.com/facebook/rocksdb/pull/10032 Test Plan: 1. export CRASH_TEST_EXT_ARGS=" --async_io=1" make crash_test -j completed successfully locally 2. make crash_test -j completed successfully locally 3. Reran CircleCi mini crashtest job 4 - 5 times. 4. Updated prefetch_test for more coverage. Reviewed By: anand1976 Differential Revision: D36579858 Pulled By: akankshamahajan15 fbshipit-source-id: 0c428d62b45e12e082a83acf533a5e37a584bedf
2 years ago
Env::IOPriority rate_limiter_priority);
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// The minimum `offset` ever passed to TryReadFromCache(). This will nly be
// tracked if track_min_offset = true.
size_t min_offset_read() const { return min_offset_read_; }
Fix bug in rocksdb internal automatic prefetching (#9234) Summary: After introducing adaptive_readahead, the original flow got broken. Readahead size was set to 0 because of which rocksdb wasn't be able to do automatic prefetching which it enables after seeing sequential reads. This PR fixes it. ---------------------------------------------------------------------------------------------------- Before this patch: b_bench -use_existing_db=true -db=/tmp/prefix_scan -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 6.27 Date: Tue Nov 30 11:56:50 2021 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ DB path: [/tmp/prefix_scan] seekrandom : 5356367.174 micros/op 0 ops/sec; 29.4 MB/s (23 of 23 found) ---------------------------------------------------------------------------------------------------- After the patch: ./db_bench -use_existing_db=true -db=/tmp/prefix_scan -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 6.27 Date: Tue Nov 30 14:38:33 2021 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 WARNING: Assertions are enabled; benchmarks unnecessarily slow ------------------------------------------------ DB path: [/tmp/prefix_scan] seekrandom : 456504.277 micros/op 2 ops/sec; 359.8 MB/s (264 of 264 found) Pull Request resolved: https://github.com/facebook/rocksdb/pull/9234 Test Plan: Ran ./db_bench -db=/data/mysql/rocksdb/prefix_scan -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_d irect_io_for_flush_and_compaction=true -target_file_size_base=16777216 and then ./db_bench -use_existing_db=true -db=/data/mysql/rocksdb/prefix_scan -benchmarks="seekrandom" -key_size=32 -value_siz e=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 and compared the results. Reviewed By: anand1976 Differential Revision: D32743965 Pulled By: akankshamahajan15 fbshipit-source-id: b950fba68c91963b7deb5c20acdf471bc60251f5
3 years ago
// Called in case of implicit auto prefetching.
void UpdateReadPattern(const uint64_t& offset, const size_t& len,
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
bool decrease_readaheadsize) {
if (decrease_readaheadsize) {
// Since this block was eligible for prefetch but it was found in
// cache, so check and decrease the readahead_size by 8KB (default)
// if eligible.
DecreaseReadAheadIfEligible(offset, len);
}
prev_offset_ = offset;
prev_len_ = len;
}
void GetReadaheadState(ReadaheadFileInfo::ReadaheadInfo* readahead_info) {
readahead_info->readahead_size = readahead_size_;
readahead_info->num_file_reads = num_file_reads_;
}
void DecreaseReadAheadIfEligible(uint64_t offset, size_t size,
size_t value = DEAFULT_DECREMENT) {
// Decrease the readahead_size if
// - its enabled internally by RocksDB (implicit_auto_readahead_) and,
// - readahead_size is greater than 0 and,
// - this block would have called prefetch API if not found in cache for
// which conditions are:
// - few/no bytes are in buffer and,
// - block is sequential with the previous read and,
// - num_file_reads_ + 1 (including this read) >
// kMinNumFileReadsToStartAutoReadahead
if (implicit_auto_readahead_ && readahead_size_ > 0) {
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
if ((offset + size >
bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize()) &&
IsBlockSequential(offset) &&
(num_file_reads_ + 1 > kMinNumFileReadsToStartAutoReadahead)) {
readahead_size_ =
Make initial auto readahead_size configurable (#9836) Summary: Make initial auto readahead_size configurable Pull Request resolved: https://github.com/facebook/rocksdb/pull/9836 Test Plan: Added new unit test Ran regression: Without change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` With this change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1649895440554504 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.2 Date: Wed Apr 13 17:17:20 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] ... finished 100 ops seekrandom : 476892.488 micros/op 2 ops/sec; 344.6 MB/s (252 of 252 found) ``` Reviewed By: anand1976 Differential Revision: D35632815 Pulled By: akankshamahajan15 fbshipit-source-id: c8057a88f9294c9d03b1d434b03affe02f74d796
2 years ago
std::max(initial_auto_readahead_size_,
(readahead_size_ >= value ? readahead_size_ - value : 0));
}
}
}
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// Callback function passed to underlying FS in case of asynchronous reads.
void PrefetchAsyncCallback(const FSReadRequest& req, void* cb_arg);
private:
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// Calculates roundoff offset and length to be prefetched based on alignment
// and data present in buffer_. It also allocates new buffer or refit tail if
// required.
void CalculateOffsetAndLen(size_t alignment, uint64_t offset,
size_t roundup_len, size_t index, bool refit_tail,
uint64_t& chunk_len);
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
2 years ago
// It calls Poll API if any there is any pending asynchronous request. It then
// checks if data is in any buffer. It clears the outdated data and swaps the
// buffers if required.
void PollAndUpdateBuffersIfNeeded(uint64_t offset);
Status PrefetchAsyncInternal(const IOOptions& opts,
RandomAccessFileReader* reader, uint64_t offset,
size_t length, size_t readahead_size,
Env::IOPriority rate_limiter_priority,
bool& copy_to_third_buffer);
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
Status Read(const IOOptions& opts, RandomAccessFileReader* reader,
Env::IOPriority rate_limiter_priority, uint64_t read_len,
uint64_t chunk_len, uint64_t rounddown_start, uint32_t index);
Status ReadAsync(const IOOptions& opts, RandomAccessFileReader* reader,
uint64_t read_len, uint64_t chunk_len,
uint64_t rounddown_start, uint32_t index);
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// Copy the data from src to third buffer.
void CopyDataToBuffer(uint32_t src, uint64_t& offset, size_t& length);
bool IsBlockSequential(const size_t& offset) {
return (prev_len_ == 0 || (prev_offset_ + prev_len_ == offset));
}
// Called in case of implicit auto prefetching.
void ResetValues() {
num_file_reads_ = 1;
Make initial auto readahead_size configurable (#9836) Summary: Make initial auto readahead_size configurable Pull Request resolved: https://github.com/facebook/rocksdb/pull/9836 Test Plan: Added new unit test Ran regression: Without change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` With this change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1649895440554504 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.2 Date: Wed Apr 13 17:17:20 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] ... finished 100 ops seekrandom : 476892.488 micros/op 2 ops/sec; 344.6 MB/s (252 of 252 found) ``` Reviewed By: anand1976 Differential Revision: D35632815 Pulled By: akankshamahajan15 fbshipit-source-id: c8057a88f9294c9d03b1d434b03affe02f74d796
2 years ago
readahead_size_ = initial_auto_readahead_size_;
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
}
// Called in case of implicit auto prefetching.
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
2 years ago
bool IsEligibleForPrefetch(uint64_t offset, size_t n) {
// Prefetch only if this read is sequential otherwise reset readahead_size_
// to initial value.
if (!IsBlockSequential(offset)) {
UpdateReadPattern(offset, n, false /*decrease_readaheadsize*/);
ResetValues();
return false;
}
num_file_reads_++;
// Since async request was submitted in last call directly by calling
// PrefetchAsync, it skips num_file_reads_ check as this call is to poll the
// data submitted in previous call.
if (async_request_submitted_) {
return true;
}
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
2 years ago
if (num_file_reads_ <= kMinNumFileReadsToStartAutoReadahead) {
UpdateReadPattern(offset, n, false /*decrease_readaheadsize*/);
return false;
}
return true;
}
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
std::vector<BufferInfo> bufs_;
// curr_ represents the index for bufs_ indicating which buffer is being
// consumed currently.
uint32_t curr_;
size_t readahead_size_;
Make initial auto readahead_size configurable (#9836) Summary: Make initial auto readahead_size configurable Pull Request resolved: https://github.com/facebook/rocksdb/pull/9836 Test Plan: Added new unit test Ran regression: Without change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` With this change: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1649895440554504 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.2 Date: Wed Apr 13 17:17:20 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] ... finished 100 ops seekrandom : 476892.488 micros/op 2 ops/sec; 344.6 MB/s (252 of 252 found) ``` Reviewed By: anand1976 Differential Revision: D35632815 Pulled By: akankshamahajan15 fbshipit-source-id: c8057a88f9294c9d03b1d434b03affe02f74d796
2 years ago
size_t initial_auto_readahead_size_;
// FilePrefetchBuffer object won't be created from Iterator flow if
// max_readahead_size_ = 0.
size_t max_readahead_size_;
// The minimum `offset` ever passed to TryReadFromCache().
size_t min_offset_read_;
// if false, TryReadFromCache() always return false, and we only take stats
// for track_min_offset_ if track_min_offset_ = true
bool enable_;
// If true, track minimum `offset` ever passed to TryReadFromCache(), which
// can be fetched from min_offset_read().
bool track_min_offset_;
// implicit_auto_readahead is enabled by rocksdb internally after 2
// sequential IOs.
bool implicit_auto_readahead_;
uint64_t prev_offset_;
size_t prev_len_;
// num_file_reads_ is only used when implicit_auto_readahead_ is set.
uint64_t num_file_reads_;
Provide implementation to prefetch data asynchronously in FilePrefetchBuffer (#9674) Summary: In FilePrefetchBuffer if reads are sequential, after prefetching call ReadAsync API to prefetch data asynchronously so that in next prefetching data will be available. Data prefetched asynchronously will be readahead_size/2. It uses two buffers, one for synchronous prefetching and one for asynchronous. In case, the data is overlapping, the data is copied from both buffers to third buffer to make it continuous. This feature is under ReadOptions::async_io and is under experimental. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9674 Test Plan: 1. Add new unit tests 2. Run **db_stress** to make sure nothing crashes. - Normal prefetch without `async_io` ran successfully: ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` 3. **Run Regressions**. i) Main branch without any change for normal prefetching with async_io disabled: ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 - use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_withchange -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 14:11:31 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_withchange] seekrandom : 471347.227 micros/op 2 ops/sec; 348.1 MB/s (255 of 255 found) ``` Reviewed By: anand1976 Differential Revision: D34731543 Pulled By: akankshamahajan15 fbshipit-source-id: 8e23aa93453d5fe3c672b9231ad582f60207937f
2 years ago
// io_handle_ is allocated and used by underlying file system in case of
// asynchronous reads.
void* io_handle_;
IOHandleDeleter del_fn_;
bool async_read_in_progress_;
// If async_request_submitted_ is set then it indicates RocksDB called
// PrefetchAsync to submit request. It needs to TryReadFromCacheAsync to poll
// the submitted request without checking if data is sequential and
// num_file_reads_.
bool async_request_submitted_;
FileSystem* fs_;
SystemClock* clock_;
Statistics* stats_;
};
} // namespace ROCKSDB_NAMESPACE