rocksdb/file/file_prefetch_buffer.cc

837 lines
30 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "file/file_prefetch_buffer.h"
#include <algorithm>
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
#include <cassert>
#include "file/random_access_file_reader.h"
#include "monitoring/histogram.h"
#include "monitoring/iostats_context_imp.h"
#include "port/port.h"
#include "test_util/sync_point.h"
#include "util/random.h"
#include "util/rate_limiter.h"
namespace ROCKSDB_NAMESPACE {
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
2022-03-21 14:12:43 +00:00
void FilePrefetchBuffer::CalculateOffsetAndLen(size_t alignment,
uint64_t offset,
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
size_t roundup_len,
uint32_t index, bool refit_tail,
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
2022-03-21 14:12:43 +00:00
uint64_t& chunk_len) {
uint64_t chunk_offset_in_buffer = 0;
bool copy_data_to_new_buffer = false;
// Check if requested bytes are in the existing buffer_.
// If only a few bytes exist -- reuse them & read only what is really needed.
// This is typically the case of incremental reading of data.
// If no bytes exist in buffer -- full pread.
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (DoesBufferContainData(index) && IsOffsetInBuffer(offset, 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
2022-03-21 14:12:43 +00:00
// Only a few requested bytes are in the buffer. memmove those chunk of
// bytes to the beginning, and memcpy them back into the new buffer if a
// new buffer is created.
chunk_offset_in_buffer = Rounddown(
static_cast<size_t>(offset - bufs_[index].offset_), alignment);
chunk_len = static_cast<uint64_t>(bufs_[index].buffer_.CurrentSize()) -
chunk_offset_in_buffer;
assert(chunk_offset_in_buffer % alignment == 0);
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
assert(chunk_len % alignment == 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
2022-03-21 14:12:43 +00:00
assert(chunk_offset_in_buffer + chunk_len <=
bufs_[index].offset_ + bufs_[index].buffer_.CurrentSize());
if (chunk_len > 0) {
copy_data_to_new_buffer = true;
} else {
// this reset is not necessary, but just to be safe.
chunk_offset_in_buffer = 0;
}
}
// Create a new buffer only if current capacity is not sufficient, and memcopy
// bytes from old buffer if needed (i.e., if chunk_len is greater than 0).
if (bufs_[index].buffer_.Capacity() < roundup_len) {
bufs_[index].buffer_.Alignment(alignment);
bufs_[index].buffer_.AllocateNewBuffer(
static_cast<size_t>(roundup_len), copy_data_to_new_buffer,
chunk_offset_in_buffer, static_cast<size_t>(chunk_len));
} else if (chunk_len > 0 && refit_tail) {
// New buffer not needed. But memmove bytes from tail to the beginning since
// chunk_len is greater than 0.
bufs_[index].buffer_.RefitTail(static_cast<size_t>(chunk_offset_in_buffer),
static_cast<size_t>(chunk_len));
Fix some errors in async prefetching in FilePrefetchBuffer (#9734) Summary: In ReadOption `async_io` which prefetches the data asynchronously, db_bench and db_stress runs were failing because wrong data was prefetched which resulted in Error: Checksum mismatched. Wrong data was copied because capacity was less than actual size needed. It has been fixed in this PR. Since there are two separate methods for async and sync prefetching, these changes are in async prefetching methods and any changes would not effect normal prefetching. I ran the regressions to make sure normal prefetching is fine. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9734 Test Plan: 1. CircleCI jobs 2. Ran db_bench ``` . /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 ``` 3. Ran db_stress test ``` export CRASH_TEST_EXT_ARGS=" --async_io=1 --adaptive_readahead=1" make crash_test -j ``` 4. Run regressions for async_io disabled. Old flow without any async changes: ``` ./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 async prefetching changes and async_io disabled to make sure in normal prefetching there is no regression. ``` ./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=0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.1 Date: Wed Mar 23 15:56:37 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 : 481819.816 micros/op 2 ops/sec; 340.2 MB/s (250 of 250 found) ``` Reviewed By: riversand963 Differential Revision: D35058471 Pulled By: akankshamahajan15 fbshipit-source-id: 9233a1e6d97cea0c7a8111bfb9e8ac3251c341ce
2022-03-26 01:26:22 +00:00
} else if (chunk_len > 0) {
// For async prefetching, it doesn't call RefitTail with chunk_len > 0.
// Allocate new buffer if needed because aligned buffer calculate remaining
// buffer as capacity_ - cursize_ which might not be the case in this as we
// are not refitting.
// TODO akanksha: Update the condition when asynchronous prefetching is
// stable.
bufs_[index].buffer_.Alignment(alignment);
bufs_[index].buffer_.AllocateNewBuffer(
static_cast<size_t>(roundup_len), copy_data_to_new_buffer,
chunk_offset_in_buffer, static_cast<size_t>(chunk_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
2022-03-21 14:12:43 +00:00
}
}
Status FilePrefetchBuffer::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) {
Slice result;
Status s = reader->Read(opts, rounddown_start + chunk_len, read_len, &result,
bufs_[index].buffer_.BufferStart() + chunk_len,
/*aligned_buf=*/nullptr, 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
2022-03-21 14:12:43 +00:00
#ifndef NDEBUG
if (result.size() < read_len) {
// Fake an IO error to force db_stress fault injection to ignore
// truncated read errors
IGNORE_STATUS_IF_ERROR(Status::IOError());
}
#endif
if (!s.ok()) {
return s;
}
// Update the buffer offset and size.
bufs_[index].offset_ = rounddown_start;
bufs_[index].buffer_.Size(static_cast<size_t>(chunk_len) + result.size());
return s;
}
Status FilePrefetchBuffer::ReadAsync(const IOOptions& opts,
RandomAccessFileReader* reader,
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
uint64_t read_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
2022-03-21 14:12:43 +00:00
uint64_t rounddown_start, uint32_t index) {
// callback for async read request.
auto fp = std::bind(&FilePrefetchBuffer::PrefetchAsyncCallback, this,
std::placeholders::_1, std::placeholders::_2);
FSReadRequest req;
Slice result;
req.len = read_len;
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
req.offset = rounddown_start;
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
2022-03-21 14:12:43 +00:00
req.result = result;
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
req.scratch = bufs_[index].buffer_.BufferStart();
bufs_[index].async_req_len_ = req.len;
Status s =
reader->ReadAsync(req, opts, fp, &(bufs_[index].pos_),
&(bufs_[index].io_handle_), &(bufs_[index].del_fn_),
/*aligned_buf=*/nullptr);
req.status.PermitUncheckedError();
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
2022-03-21 14:12:43 +00:00
if (s.ok()) {
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
bufs_[index].async_read_in_progress_ = 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
2022-03-21 14:12:43 +00:00
}
return s;
}
Status FilePrefetchBuffer::Prefetch(const IOOptions& opts,
RandomAccessFileReader* reader,
uint64_t offset, size_t n,
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
2022-02-17 07:17:03 +00:00
Env::IOPriority rate_limiter_priority) {
if (!enable_ || reader == nullptr) {
return Status::OK();
}
TEST_SYNC_POINT("FilePrefetchBuffer::Prefetch:Start");
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
2022-03-21 14:12:43 +00:00
if (offset + n <= bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize()) {
// All requested bytes are already in the curr_ buffer. So no need to Read
// again.
return Status::OK();
}
size_t alignment = reader->file()->GetRequiredBufferAlignment();
size_t offset_ = static_cast<size_t>(offset);
uint64_t rounddown_offset = Rounddown(offset_, alignment);
uint64_t roundup_end = Roundup(offset_ + n, alignment);
uint64_t roundup_len = roundup_end - rounddown_offset;
assert(roundup_len >= alignment);
assert(roundup_len % alignment == 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
2022-03-21 14:12:43 +00:00
uint64_t chunk_len = 0;
CalculateOffsetAndLen(alignment, offset, roundup_len, curr_,
true /*refit_tail*/, chunk_len);
size_t read_len = static_cast<size_t>(roundup_len - chunk_len);
Status s = Read(opts, reader, rate_limiter_priority, read_len, chunk_len,
rounddown_offset, curr_);
return s;
}
// Copy data from src to third buffer.
void FilePrefetchBuffer::CopyDataToBuffer(uint32_t src, uint64_t& offset,
size_t& length) {
if (length == 0) {
return;
}
uint64_t copy_offset = (offset - bufs_[src].offset_);
size_t copy_len = 0;
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (IsDataBlockInBuffer(offset, length, src)) {
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
2022-03-21 14:12:43 +00:00
// All the bytes are in src.
copy_len = length;
} else {
copy_len = bufs_[src].buffer_.CurrentSize() - copy_offset;
}
memcpy(bufs_[2].buffer_.BufferStart() + bufs_[2].buffer_.CurrentSize(),
bufs_[src].buffer_.BufferStart() + copy_offset, copy_len);
bufs_[2].buffer_.Size(bufs_[2].buffer_.CurrentSize() + copy_len);
// Update offset and length.
offset += copy_len;
length -= copy_len;
// length > 0 indicates it has consumed all data from the src buffer and it
// still needs to read more other buffer.
if (length > 0) {
bufs_[src].buffer_.Clear();
}
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// Clear the buffers if it contains outdated data. Outdated data can be
// because previous sequential reads were read from the cache instead of these
// buffer. In that case outdated IOs should be aborted.
void FilePrefetchBuffer::AbortIOIfNeeded(uint64_t offset) {
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
2022-03-21 14:12:43 +00:00
uint32_t second = curr_ ^ 1;
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
std::vector<void*> handles;
autovector<uint32_t> buf_pos;
if (IsBufferOutdatedWithAsyncProgress(offset, curr_)) {
handles.emplace_back(bufs_[curr_].io_handle_);
buf_pos.emplace_back(curr_);
}
if (IsBufferOutdatedWithAsyncProgress(offset, second)) {
handles.emplace_back(bufs_[second].io_handle_);
buf_pos.emplace_back(second);
}
if (!handles.empty()) {
StopWatch sw(clock_, stats_, ASYNC_PREFETCH_ABORT_MICROS);
Status s = fs_->AbortIO(handles);
assert(s.ok());
}
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
2022-03-21 14:12:43 +00:00
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
for (auto& pos : buf_pos) {
// Release io_handle.
DestroyAndClearIOHandle(pos);
}
if (bufs_[second].io_handle_ == nullptr) {
bufs_[second].async_read_in_progress_ = false;
}
if (bufs_[curr_].io_handle_ == nullptr &&
bufs_[curr_].async_read_in_progress_) {
bufs_[curr_].async_read_in_progress_ = false;
curr_ = curr_ ^ 1;
}
}
void FilePrefetchBuffer::AbortAllIOs() {
uint32_t second = curr_ ^ 1;
std::vector<void*> handles;
for (uint32_t i = 0; i < 2; i++) {
if (bufs_[i].async_read_in_progress_ && bufs_[i].io_handle_ != nullptr) {
handles.emplace_back(bufs_[i].io_handle_);
Fix some errors in async prefetching in FilePrefetchBuffer (#9734) Summary: In ReadOption `async_io` which prefetches the data asynchronously, db_bench and db_stress runs were failing because wrong data was prefetched which resulted in Error: Checksum mismatched. Wrong data was copied because capacity was less than actual size needed. It has been fixed in this PR. Since there are two separate methods for async and sync prefetching, these changes are in async prefetching methods and any changes would not effect normal prefetching. I ran the regressions to make sure normal prefetching is fine. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9734 Test Plan: 1. CircleCI jobs 2. Ran db_bench ``` . /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 ``` 3. Ran db_stress test ``` export CRASH_TEST_EXT_ARGS=" --async_io=1 --adaptive_readahead=1" make crash_test -j ``` 4. Run regressions for async_io disabled. Old flow without any async changes: ``` ./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 async prefetching changes and async_io disabled to make sure in normal prefetching there is no regression. ``` ./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=0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.1 Date: Wed Mar 23 15:56:37 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 : 481819.816 micros/op 2 ops/sec; 340.2 MB/s (250 of 250 found) ``` Reviewed By: riversand963 Differential Revision: D35058471 Pulled By: akankshamahajan15 fbshipit-source-id: 9233a1e6d97cea0c7a8111bfb9e8ac3251c341ce
2022-03-26 01:26:22 +00:00
}
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (!handles.empty()) {
StopWatch sw(clock_, stats_, ASYNC_PREFETCH_ABORT_MICROS);
Status s = fs_->AbortIO(handles);
assert(s.ok());
}
Fix some errors in async prefetching in FilePrefetchBuffer (#9734) Summary: In ReadOption `async_io` which prefetches the data asynchronously, db_bench and db_stress runs were failing because wrong data was prefetched which resulted in Error: Checksum mismatched. Wrong data was copied because capacity was less than actual size needed. It has been fixed in this PR. Since there are two separate methods for async and sync prefetching, these changes are in async prefetching methods and any changes would not effect normal prefetching. I ran the regressions to make sure normal prefetching is fine. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9734 Test Plan: 1. CircleCI jobs 2. Ran db_bench ``` . /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 ``` 3. Ran db_stress test ``` export CRASH_TEST_EXT_ARGS=" --async_io=1 --adaptive_readahead=1" make crash_test -j ``` 4. Run regressions for async_io disabled. Old flow without any async changes: ``` ./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 async prefetching changes and async_io disabled to make sure in normal prefetching there is no regression. ``` ./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=0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.1 Date: Wed Mar 23 15:56:37 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 : 481819.816 micros/op 2 ops/sec; 340.2 MB/s (250 of 250 found) ``` Reviewed By: riversand963 Differential Revision: D35058471 Pulled By: akankshamahajan15 fbshipit-source-id: 9233a1e6d97cea0c7a8111bfb9e8ac3251c341ce
2022-03-26 01:26:22 +00:00
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// Release io_handles.
if (bufs_[curr_].io_handle_ != nullptr && bufs_[curr_].del_fn_ != nullptr) {
DestroyAndClearIOHandle(curr_);
}
if (bufs_[second].io_handle_ != nullptr && bufs_[second].del_fn_ != nullptr) {
DestroyAndClearIOHandle(second);
}
}
// Clear the buffers if it contains outdated data. Outdated data can be
// because previous sequential reads were read from the cache instead of these
// buffer.
void FilePrefetchBuffer::UpdateBuffersIfNeeded(uint64_t offset) {
uint32_t second = curr_ ^ 1;
if (IsBufferOutdated(offset, curr_)) {
bufs_[curr_].buffer_.Clear();
}
if (IsBufferOutdated(offset, second)) {
bufs_[second].buffer_.Clear();
}
// If data starts from second buffer, make it curr_. Second buffer can be
// either partial filled or full.
if (!bufs_[second].async_read_in_progress_ && DoesBufferContainData(second) &&
IsOffsetInBuffer(offset, second)) {
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
2022-03-21 14:12:43 +00:00
// Clear the curr_ as buffers have been swapped and curr_ contains the
Fix some errors in async prefetching in FilePrefetchBuffer (#9734) Summary: In ReadOption `async_io` which prefetches the data asynchronously, db_bench and db_stress runs were failing because wrong data was prefetched which resulted in Error: Checksum mismatched. Wrong data was copied because capacity was less than actual size needed. It has been fixed in this PR. Since there are two separate methods for async and sync prefetching, these changes are in async prefetching methods and any changes would not effect normal prefetching. I ran the regressions to make sure normal prefetching is fine. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9734 Test Plan: 1. CircleCI jobs 2. Ran db_bench ``` . /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 ``` 3. Ran db_stress test ``` export CRASH_TEST_EXT_ARGS=" --async_io=1 --adaptive_readahead=1" make crash_test -j ``` 4. Run regressions for async_io disabled. Old flow without any async changes: ``` ./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 async prefetching changes and async_io disabled to make sure in normal prefetching there is no regression. ``` ./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=0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.1 Date: Wed Mar 23 15:56:37 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 : 481819.816 micros/op 2 ops/sec; 340.2 MB/s (250 of 250 found) ``` Reviewed By: riversand963 Differential Revision: D35058471 Pulled By: akankshamahajan15 fbshipit-source-id: 9233a1e6d97cea0c7a8111bfb9e8ac3251c341ce
2022-03-26 01:26:22 +00:00
// outdated data and switch the buffers.
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (!bufs_[curr_].async_read_in_progress_) {
bufs_[curr_].buffer_.Clear();
}
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
2022-03-21 14:12:43 +00:00
curr_ = curr_ ^ 1;
}
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
2022-05-20 23:09:33 +00:00
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
void FilePrefetchBuffer::PollAndUpdateBuffersIfNeeded(uint64_t offset) {
if (bufs_[curr_].async_read_in_progress_ && fs_ != nullptr) {
if (bufs_[curr_].io_handle_ != nullptr) {
// Wait for prefetch data to complete.
// No mutex is needed as async_read_in_progress behaves as mutex and is
// updated by main thread only.
std::vector<void*> handles;
handles.emplace_back(bufs_[curr_].io_handle_);
StopWatch sw(clock_, stats_, POLL_WAIT_MICROS);
fs_->Poll(handles, 1).PermitUncheckedError();
}
// Reset and Release io_handle after the Poll API as request has been
// completed.
DestroyAndClearIOHandle(curr_);
}
UpdateBuffersIfNeeded(offset);
}
// If async_io is enabled in case of sequential reads, PrefetchAsyncInternal is
// called. When buffers are switched, we clear the curr_ buffer as we assume the
// data has been consumed because of sequential reads.
// Data in buffers will always be sequential with curr_ following second and
// not vice versa.
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
2022-05-20 23:09:33 +00:00
//
// Scenarios for prefetching asynchronously:
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// Case1: If both buffers are empty, prefetch n + readahead_size_/2 bytes
// synchronously in curr_ and prefetch readahead_size_/2 async in second
// buffer.
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
2022-05-20 23:09:33 +00:00
// Case2: If second buffer has partial or full data, make it current and
// prefetch readahead_size_/2 async in second buffer. In case of
// partial data, prefetch remaining bytes from size n synchronously to
// fulfill the requested bytes request.
// Case3: If curr_ has partial data, prefetch remaining bytes from size n
// synchronously in curr_ to fulfill the requested bytes request and
// prefetch readahead_size_/2 bytes async in second buffer.
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// Case4: (Special case) If data is in both buffers, copy requested data from
// curr_, send async request on curr_, wait for poll to fill second
// buffer (if any), and copy remaining data from second buffer to third
// buffer.
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
2022-05-20 23:09:33 +00:00
Status FilePrefetchBuffer::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) {
if (!enable_) {
return Status::OK();
}
TEST_SYNC_POINT("FilePrefetchBuffer::PrefetchAsyncInternal:Start");
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
size_t alignment = reader->file()->GetRequiredBufferAlignment();
Status s;
uint64_t tmp_offset = offset;
size_t tmp_length = length;
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// 1. Abort IO and swap buffers if needed to point curr_ to first buffer with
// data.
{
if (!explicit_prefetch_submitted_) {
AbortIOIfNeeded(offset);
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
UpdateBuffersIfNeeded(offset);
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
2022-03-21 14:12:43 +00:00
}
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
2022-05-20 23:09:33 +00:00
uint32_t second = curr_ ^ 1;
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// 2. If data is overlapping over two buffers, copy the data from curr_ and
// call ReadAsync on curr_.
if (!bufs_[curr_].async_read_in_progress_ && DoesBufferContainData(curr_) &&
IsOffsetInBuffer(offset, curr_) &&
(/*Data extends over curr_ buffer and second buffer either has data or in
process of population=*/
(offset + length > bufs_[second].offset_) &&
(bufs_[second].async_read_in_progress_ ||
DoesBufferContainData(second)))) {
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
2022-03-21 14:12:43 +00:00
// Allocate new buffer to third buffer;
bufs_[2].buffer_.Clear();
bufs_[2].buffer_.Alignment(alignment);
bufs_[2].buffer_.AllocateNewBuffer(length);
bufs_[2].offset_ = offset;
copy_to_third_buffer = true;
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
CopyDataToBuffer(curr_, tmp_offset, tmp_length);
// Call async prefetching on curr_ since data has been consumed in curr_
// only if data lies within second buffer.
size_t second_size = bufs_[second].async_read_in_progress_
? bufs_[second].async_req_len_
: bufs_[second].buffer_.CurrentSize();
if (tmp_offset + tmp_length <= bufs_[second].offset_ + second_size) {
uint64_t rounddown_start = bufs_[second].offset_ + second_size;
uint64_t roundup_end =
Roundup(rounddown_start + readahead_size, alignment);
uint64_t roundup_len = roundup_end - rounddown_start;
uint64_t chunk_len = 0;
CalculateOffsetAndLen(alignment, rounddown_start, roundup_len, curr_,
false, chunk_len);
assert(chunk_len == 0);
assert(roundup_len >= chunk_len);
bufs_[curr_].offset_ = rounddown_start;
uint64_t read_len = static_cast<size_t>(roundup_len - chunk_len);
s = ReadAsync(opts, reader, read_len, rounddown_start, curr_);
if (!s.ok()) {
DestroyAndClearIOHandle(curr_);
bufs_[curr_].buffer_.Clear();
return s;
}
}
curr_ = curr_ ^ 1;
}
// 3. Call Poll only if data is needed for the second buffer.
// - Return if whole data is in curr_ and second buffer in progress.
// - If second buffer is empty, it will go for ReadAsync for second buffer.
if (!bufs_[curr_].async_read_in_progress_ && DoesBufferContainData(curr_) &&
IsDataBlockInBuffer(offset, length, curr_)) {
// Whole data is in curr_.
UpdateBuffersIfNeeded(offset);
second = curr_ ^ 1;
if (bufs_[second].async_read_in_progress_) {
return s;
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
} else {
PollAndUpdateBuffersIfNeeded(offset);
second = curr_ ^ 1;
}
if (copy_to_third_buffer) {
offset = tmp_offset;
length = tmp_length;
}
// 4. After polling and swapping buffers, if all the requested bytes are in
// curr_, it will only go for async prefetching.
// copy_to_third_buffer is a special case so it will be handled separately.
if (!copy_to_third_buffer && DoesBufferContainData(curr_) &&
IsDataBlockInBuffer(offset, length, curr_)) {
offset += length;
length = 0;
// Since async request was submitted directly by calling PrefetchAsync in
// last call, we don't need to prefetch further as this call is to poll
// the data submitted in previous call.
if (explicit_prefetch_submitted_) {
return s;
}
}
// 5. Data is overlapping i.e. some of the data has been copied to third
// buffer
// and remaining will be updated below.
if (copy_to_third_buffer) {
CopyDataToBuffer(curr_, offset, length);
// Length == 0: All the requested data has been copied to third buffer and
// it has already gone for async prefetching. It can return without doing
// anything further.
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
2022-03-21 14:12:43 +00:00
// Length > 0: More data needs to be consumed so it will continue async and
// sync prefetching and copy the remaining data to third buffer in the end.
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (length == 0) {
return s;
}
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// 6. Go for ReadAsync and Read (if needed).
size_t prefetch_size = length + 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
2022-03-21 14:12:43 +00:00
size_t _offset = static_cast<size_t>(offset);
// offset and size alignment for curr_ buffer with synchronous prefetching
uint64_t rounddown_start1 = Rounddown(_offset, alignment);
uint64_t roundup_end1 = Roundup(_offset + prefetch_size, alignment);
uint64_t roundup_len1 = roundup_end1 - rounddown_start1;
assert(roundup_len1 >= alignment);
assert(roundup_len1 % alignment == 0);
uint64_t chunk_len1 = 0;
uint64_t read_len1 = 0;
// For length == 0, skip the synchronous prefetching. read_len1 will be 0.
if (length > 0) {
CalculateOffsetAndLen(alignment, offset, roundup_len1, curr_,
false /*refit_tail*/, chunk_len1);
Fix some errors in async prefetching in FilePrefetchBuffer (#9734) Summary: In ReadOption `async_io` which prefetches the data asynchronously, db_bench and db_stress runs were failing because wrong data was prefetched which resulted in Error: Checksum mismatched. Wrong data was copied because capacity was less than actual size needed. It has been fixed in this PR. Since there are two separate methods for async and sync prefetching, these changes are in async prefetching methods and any changes would not effect normal prefetching. I ran the regressions to make sure normal prefetching is fine. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9734 Test Plan: 1. CircleCI jobs 2. Ran db_bench ``` . /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 ``` 3. Ran db_stress test ``` export CRASH_TEST_EXT_ARGS=" --async_io=1 --adaptive_readahead=1" make crash_test -j ``` 4. Run regressions for async_io disabled. Old flow without any async changes: ``` ./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 async prefetching changes and async_io disabled to make sure in normal prefetching there is no regression. ``` ./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=0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.1 Date: Wed Mar 23 15:56:37 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 : 481819.816 micros/op 2 ops/sec; 340.2 MB/s (250 of 250 found) ``` Reviewed By: riversand963 Differential Revision: D35058471 Pulled By: akankshamahajan15 fbshipit-source-id: 9233a1e6d97cea0c7a8111bfb9e8ac3251c341ce
2022-03-26 01:26:22 +00:00
assert(roundup_len1 >= chunk_len1);
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
2022-03-21 14:12:43 +00:00
read_len1 = static_cast<size_t>(roundup_len1 - chunk_len1);
}
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
2022-03-21 14:12:43 +00:00
{
// offset and size alignment for second buffer for asynchronous
// prefetching
uint64_t rounddown_start2 = roundup_end1;
uint64_t roundup_end2 =
Roundup(rounddown_start2 + readahead_size, alignment);
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
2022-03-21 14:12:43 +00:00
// For length == 0, do the asynchronous prefetching in second instead of
Fix some errors in async prefetching in FilePrefetchBuffer (#9734) Summary: In ReadOption `async_io` which prefetches the data asynchronously, db_bench and db_stress runs were failing because wrong data was prefetched which resulted in Error: Checksum mismatched. Wrong data was copied because capacity was less than actual size needed. It has been fixed in this PR. Since there are two separate methods for async and sync prefetching, these changes are in async prefetching methods and any changes would not effect normal prefetching. I ran the regressions to make sure normal prefetching is fine. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9734 Test Plan: 1. CircleCI jobs 2. Ran db_bench ``` . /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 ``` 3. Ran db_stress test ``` export CRASH_TEST_EXT_ARGS=" --async_io=1 --adaptive_readahead=1" make crash_test -j ``` 4. Run regressions for async_io disabled. Old flow without any async changes: ``` ./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 async prefetching changes and async_io disabled to make sure in normal prefetching there is no regression. ``` ./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=0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.1 Date: Wed Mar 23 15:56:37 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 : 481819.816 micros/op 2 ops/sec; 340.2 MB/s (250 of 250 found) ``` Reviewed By: riversand963 Differential Revision: D35058471 Pulled By: akankshamahajan15 fbshipit-source-id: 9233a1e6d97cea0c7a8111bfb9e8ac3251c341ce
2022-03-26 01:26:22 +00:00
// synchronous prefetching in curr_.
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
2022-03-21 14:12:43 +00:00
if (length == 0) {
rounddown_start2 =
bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize();
roundup_end2 = Roundup(rounddown_start2 + prefetch_size, alignment);
}
uint64_t roundup_len2 = roundup_end2 - rounddown_start2;
uint64_t chunk_len2 = 0;
CalculateOffsetAndLen(alignment, rounddown_start2, roundup_len2, second,
false /*refit_tail*/, chunk_len2);
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
assert(chunk_len2 == 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
2022-03-21 14:12:43 +00:00
// Update the buffer offset.
bufs_[second].offset_ = rounddown_start2;
Fix some errors in async prefetching in FilePrefetchBuffer (#9734) Summary: In ReadOption `async_io` which prefetches the data asynchronously, db_bench and db_stress runs were failing because wrong data was prefetched which resulted in Error: Checksum mismatched. Wrong data was copied because capacity was less than actual size needed. It has been fixed in this PR. Since there are two separate methods for async and sync prefetching, these changes are in async prefetching methods and any changes would not effect normal prefetching. I ran the regressions to make sure normal prefetching is fine. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9734 Test Plan: 1. CircleCI jobs 2. Ran db_bench ``` . /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 ``` 3. Ran db_stress test ``` export CRASH_TEST_EXT_ARGS=" --async_io=1 --adaptive_readahead=1" make crash_test -j ``` 4. Run regressions for async_io disabled. Old flow without any async changes: ``` ./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 async prefetching changes and async_io disabled to make sure in normal prefetching there is no regression. ``` ./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=0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.1 Date: Wed Mar 23 15:56:37 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 : 481819.816 micros/op 2 ops/sec; 340.2 MB/s (250 of 250 found) ``` Reviewed By: riversand963 Differential Revision: D35058471 Pulled By: akankshamahajan15 fbshipit-source-id: 9233a1e6d97cea0c7a8111bfb9e8ac3251c341ce
2022-03-26 01:26:22 +00:00
assert(roundup_len2 >= chunk_len2);
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
2022-03-21 14:12:43 +00:00
uint64_t read_len2 = static_cast<size_t>(roundup_len2 - chunk_len2);
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
Status tmp_s = ReadAsync(opts, reader, read_len2, rounddown_start2, second);
if (!tmp_s.ok()) {
DestroyAndClearIOHandle(second);
bufs_[second].buffer_.Clear();
}
}
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
2022-03-21 14:12:43 +00:00
if (read_len1 > 0) {
s = Read(opts, reader, rate_limiter_priority, read_len1, chunk_len1,
rounddown_start1, curr_);
if (!s.ok()) {
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (bufs_[second].io_handle_ != nullptr) {
std::vector<void*> handles;
handles.emplace_back(bufs_[second].io_handle_);
{
StopWatch sw(clock_, stats_, ASYNC_PREFETCH_ABORT_MICROS);
Status status = fs_->AbortIO(handles);
assert(status.ok());
}
}
DestroyAndClearIOHandle(second);
bufs_[second].buffer_.Clear();
bufs_[curr_].buffer_.Clear();
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
2022-03-21 14:12:43 +00:00
return s;
}
}
// Copy remaining requested bytes to third_buffer.
if (copy_to_third_buffer && length > 0) {
CopyDataToBuffer(curr_, offset, length);
}
return s;
}
bool FilePrefetchBuffer::TryReadFromCache(const IOOptions& opts,
RandomAccessFileReader* reader,
uint64_t offset, size_t n,
Slice* result, Status* status,
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
2022-02-17 07:17:03 +00:00
Env::IOPriority rate_limiter_priority,
bool for_compaction /* = false */) {
if (track_min_offset_ && offset < min_offset_read_) {
min_offset_read_ = static_cast<size_t>(offset);
}
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
2022-03-21 14:12:43 +00:00
if (!enable_ || (offset < bufs_[curr_].offset_)) {
return false;
}
// If the buffer contains only a few of the requested bytes:
// If readahead is enabled: prefetch the remaining bytes + readahead bytes
// and satisfy the request.
// If readahead is not enabled: return false.
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
2021-12-01 06:52:14 +00:00
TEST_SYNC_POINT_CALLBACK("FilePrefetchBuffer::TryReadFromCache",
&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
2022-03-21 14:12:43 +00:00
if (offset + n > bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize()) {
if (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
2022-03-21 14:12:43 +00:00
Status s;
assert(reader != nullptr);
assert(max_readahead_size_ >= readahead_size_);
if (for_compaction) {
s = Prefetch(opts, reader, offset, std::max(n, readahead_size_),
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
2022-02-17 07:17:03 +00:00
rate_limiter_priority);
} else {
if (implicit_auto_readahead_) {
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
2022-03-21 14:12:43 +00:00
if (!IsEligibleForPrefetch(offset, n)) {
// Ignore status as Prefetch is not called.
s.PermitUncheckedError();
return 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
2022-03-21 14:12:43 +00:00
}
s = Prefetch(opts, reader, offset, n + readahead_size_,
rate_limiter_priority);
}
if (!s.ok()) {
if (status) {
*status = s;
}
#ifndef NDEBUG
IGNORE_STATUS_IF_ERROR(s);
#endif
return false;
}
readahead_size_ = std::min(max_readahead_size_, readahead_size_ * 2);
} else {
return false;
}
}
UpdateReadPattern(offset, n, false /*decrease_readaheadsize*/);
uint64_t offset_in_buffer = offset - bufs_[curr_].offset_;
*result = Slice(bufs_[curr_].buffer_.BufferStart() + offset_in_buffer, n);
return true;
}
bool FilePrefetchBuffer::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
2022-05-23 19:15:26 +00:00
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
2022-03-21 14:12:43 +00:00
if (track_min_offset_ && offset < min_offset_read_) {
min_offset_read_ = static_cast<size_t>(offset);
}
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
2022-05-20 23:09:33 +00:00
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
2022-05-23 19:15:26 +00:00
if (!enable_) {
return false;
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (explicit_prefetch_submitted_) {
if (prev_offset_ != offset) {
// Random offset called. So abort the IOs.
AbortAllIOs();
bufs_[curr_].buffer_.Clear();
bufs_[curr_ ^ 1].buffer_.Clear();
explicit_prefetch_submitted_ = false;
return false;
}
}
if (!explicit_prefetch_submitted_ && offset < bufs_[curr_].offset_) {
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
2022-03-21 14:12:43 +00:00
return false;
}
bool prefetched = false;
bool copy_to_third_buffer = false;
// If the buffer contains only a few of the requested bytes:
// If readahead is enabled: prefetch the remaining bytes + readahead bytes
// and satisfy the request.
// If readahead is not enabled: return false.
TEST_SYNC_POINT_CALLBACK("FilePrefetchBuffer::TryReadFromCache",
&readahead_size_);
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (explicit_prefetch_submitted_ ||
(bufs_[curr_].async_read_in_progress_ ||
offset + n >
bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize())) {
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
2022-03-21 14:12:43 +00:00
if (readahead_size_ > 0) {
Status s;
assert(reader != nullptr);
assert(max_readahead_size_ >= readahead_size_);
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
2022-05-23 19:15:26 +00:00
if (implicit_auto_readahead_) {
if (!IsEligibleForPrefetch(offset, n)) {
// Ignore status as Prefetch is not called.
s.PermitUncheckedError();
return 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
2022-03-21 14:12:43 +00:00
}
}
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
2022-05-23 19:15:26 +00:00
// Prefetch n + readahead_size_/2 synchronously as remaining
// readahead_size_/2 will be prefetched asynchronously.
s = PrefetchAsyncInternal(opts, reader, offset, n, readahead_size_ / 2,
rate_limiter_priority, copy_to_third_buffer);
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
explicit_prefetch_submitted_ = false;
if (!s.ok()) {
if (status) {
*status = s;
}
#ifndef NDEBUG
IGNORE_STATUS_IF_ERROR(s);
#endif
return false;
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
prefetched = explicit_prefetch_submitted_ ? false : true;
} else {
return false;
}
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
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
2022-03-21 14:12:43 +00:00
UpdateReadPattern(offset, n, false /*decrease_readaheadsize*/);
uint32_t index = curr_;
if (copy_to_third_buffer) {
index = 2;
}
uint64_t offset_in_buffer = offset - bufs_[index].offset_;
*result = Slice(bufs_[index].buffer_.BufferStart() + offset_in_buffer, n);
if (prefetched) {
readahead_size_ = std::min(max_readahead_size_, readahead_size_ * 2);
}
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
2022-03-21 14:12:43 +00:00
void FilePrefetchBuffer::PrefetchAsyncCallback(const FSReadRequest& req,
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
void* cb_arg) {
uint32_t index = *(static_cast<uint32_t*>(cb_arg));
Fix stress test failure in ReadAsync. (#9824) Summary: Fix stress test failure in ReadAsync by ignoring errors injected during async read by FaultInjectionFS. Failure: ``` WARNING: prefix_size is non-zero but memtablerep != prefix_hash Didn't get expected error from MultiGet. num_keys 14 Expected 1 errors, seen 0 Callstack that injected the fault Injected error type = 32538 Message: error; #0 ./db_stress() [0x6f7dd4] rocksdb::port::SaveStack(int*, int) /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/port/stack_trace.cc:152 https://github.com/facebook/rocksdb/issues/1 ./db_stress() [0x7f2bda] rocksdb::FaultInjectionTestFS::InjectThreadSpecificReadError(rocksdb::FaultInjectionTestFS::ErrorOperation, rocksdb::Slice*, bool, char*, bool, bool*) /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:891 https://github.com/facebook/rocksdb/issues/2 ./db_stress() [0x7f2e78] rocksdb::TestFSRandomAccessFile::Read(unsigned long, unsigned long, rocksdb::IOOptions const&, rocksdb::Slice*, char*, rocksdb::IODebugContext*) const /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:367 https://github.com/facebook/rocksdb/issues/3 ./db_stress() [0x6483d7] rocksdb::(anonymous namespace)::CompositeRandomAccessFileWrapper::Read(unsigned long, unsigned long, rocksdb::Slice*, char*) const /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/env/composite_env.cc:61 https://github.com/facebook/rocksdb/issues/4 ./db_stress() [0x654564] rocksdb::(anonymous namespace)::LegacyRandomAccessFileWrapper::Read(unsigned long, unsigned long, rocksdb::IOOptions const&, rocksdb::Slice*, char*, rocksdb::IODebugContext*) const /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/env/env.cc:152 https://github.com/facebook/rocksdb/issues/5 ./db_stress() [0x659b3b] rocksdb::FSRandomAccessFile::ReadAsync(rocksdb::FSReadRequest&, rocksdb::IOOptions const&, std::function<void (rocksdb::FSReadRequest const&, void*)>, void*, void**, std::function<void (void*)>*, rocksdb::IODebugContext*) /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/./include/rocksdb/file_system.h:896 https://github.com/facebook/rocksdb/issues/6 ./db_stress() [0x8b8bab] rocksdb::RandomAccessFileReader::ReadAsync(rocksdb::FSReadRequest&, rocksdb::IOOptions const&, std::function<void (rocksdb::FSReadRequest const&, void*)>, void*, void**, std::function<void (void*)>*, rocksdb::Env::IOPriority) /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/file/random_access_file_reader.cc:459 https://github.com/facebook/rocksdb/issues/7 ./db_stress() [0x8b501f] rocksdb::FilePrefetchBuffer::ReadAsync(rocksdb::IOOptions const&, rocksdb::RandomAccessFileReader*, rocksdb::Env::IOPriority, unsigned long, unsigned long, unsigned long, unsigned int) /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/file/file_prefetch_buffer.cc:124 https://github.com/facebook/rocksdb/issues/8 ./db_stress() [0x8b55fc] rocksdb::FilePrefetchBuffer::PrefetchAsync(rocksdb::IOOptions const&, rocksdb::RandomAccessFileReader*, unsigned long, unsigned long, unsigned long, rocksdb::Env::IOPriority, bool&) /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/file/file_prefetch_buffer.cc:363 https://github.com/facebook/rocksdb/issues/9 ./db_stress() [0x8b61f8] rocksdb::FilePrefetchBuffer::TryReadFromCacheAsync(rocksdb::IOOptions const&, rocksdb::RandomAccessFileReader*, unsigned long, unsigned long, rocksdb::Slice*, rocksdb::Status*, rocksdb::Env::IOPriority, bool) /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/file/file_prefetch_buffer.cc:482 https://github.com/facebook/rocksdb/issues/10 ./db_stress() [0x745e04] rocksdb::BlockFetcher::TryGetFromPrefetchBuffer() /data/sandcastle/boxes/trunk-hg-fbcode-fbsource/fbcode/internal_repo_rocksdb/repo/table/block_fetcher.cc:76 ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9824 Test Plan: ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_concurrent_memtable_write=0 --async_io=1 --atomic_flush=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=0 -- backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=0 --block_size=16384 --bloom_bits=5.037629726741734 --bottommost_compression_type=lz4hc --cache_index_and_filter_blocks=0 --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kxxHash --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_ttl=100 --compression_max_dict_buffer_bytes=1073741823 --compression_max_dict_bytes=16384 --compression_parallel_threads=1 --compression_type=zstd --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --db=/home/akankshamahajan/dev/shm/rocksdb/rocksdb_crashtest_blackbox --db_write_buffer_size=8388608 --delpercent=0 --delrangepercent=0 --destroy_db_initially=0 - detect_filter_construct_corruption=1 --disable_wal=1 --enable_compaction_filter=0 --enable_pipelined_write=0 --expected_values_dir=/home/akankshamahajan/dev/shm/rocksdb/rocksdb_crashtest_expected --experimental_mempurge_threshold=8.772789063014715 --fail_if_options_file_error=0 --file_checksum_impl=crc32c --flush_one_in=1000000 --format_version=3 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=15 --index_type=3 --iterpercent=0 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=False --long_running_snapshots=0 --mark_for_compaction_one_file_in=0 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=25000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=16777216 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=2097152 --memtable_prefix_bloom_size_ratio=0.001 --memtable_whole_key_filtering=1 --memtablerep=skip_list --mmap_read=0 --mock_direct_io=True --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=1000 --prefix_size=-1 --prefixpercent=0 --prepopulate_block_cache=0 --progress_reports=0 --read_fault_one_in=32 --readpercent=100 --recycle_log_file_num=1 --reopen=0 --reserve_table_reader_memory=1 --ribbon_starting_level=999 --secondary_cache_fault_one_in=0 --set_options_one_in=0 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=2 --sync=0 --sync_fault_injection=False --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=3 --unpartitioned_pinning=2 --use_block_based_filter=0 --use_clock_cache=0 --use_direct_io_for_flush_and_compaction=1 --use_direct_reads=0 --use_full_merge_v1=0 --use_merge=1 --use_multiget=1 --user_timestamp_size=0 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --wal_compression=none --write_buffer_size=33554432 --write_dbid_to_manifest=1 --write_fault_one_in=0 --writepercent=0 ``` Reviewed By: anand1976 Differential Revision: D35514566 Pulled By: akankshamahajan15 fbshipit-source-id: e2a868fdd7422604774c1419738f9926a21e92a4
2022-04-11 17:56:11 +00:00
#ifndef NDEBUG
if (req.result.size() < req.len) {
// Fake an IO error to force db_stress fault injection to ignore
// truncated read errors
IGNORE_STATUS_IF_ERROR(Status::IOError());
}
IGNORE_STATUS_IF_ERROR(req.status);
#endif
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
2022-03-21 14:12:43 +00:00
if (req.status.ok()) {
if (req.offset + req.result.size() <=
bufs_[index].offset_ + bufs_[index].buffer_.CurrentSize()) {
// All requested bytes are already in the buffer or no data is read
// because of EOF. So no need to update.
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
2022-03-21 14:12:43 +00:00
return;
}
if (req.offset < bufs_[index].offset_) {
// Next block to be read has changed (Recent read was not a sequential
// read). So ignore this read.
return;
}
size_t current_size = bufs_[index].buffer_.CurrentSize();
bufs_[index].buffer_.Size(current_size + req.result.size());
}
}
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
2022-05-20 23:09:33 +00:00
Status FilePrefetchBuffer::PrefetchAsync(const IOOptions& opts,
RandomAccessFileReader* reader,
uint64_t offset, size_t n,
Slice* result) {
assert(reader != nullptr);
if (!enable_) {
return Status::NotSupported();
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
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
2022-05-20 23:09:33 +00:00
TEST_SYNC_POINT("FilePrefetchBuffer::PrefetchAsync:Start");
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
num_file_reads_ = 0;
explicit_prefetch_submitted_ = false;
bool is_eligible_for_prefetching = false;
if (readahead_size_ > 0 &&
(!implicit_auto_readahead_ ||
num_file_reads_ + 1 >= num_file_reads_for_auto_readahead_)) {
is_eligible_for_prefetching = 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
2022-05-20 23:09:33 +00:00
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// 1. Cancel any pending async read to make code simpler as buffers can be out
// of sync.
AbortAllIOs();
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
2022-05-20 23:09:33 +00:00
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// 2. Clear outdated data.
UpdateBuffersIfNeeded(offset);
uint32_t second = curr_ ^ 1;
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
2022-05-23 19:15:26 +00:00
// Since PrefetchAsync can be called on non sequential reads. So offset can
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
// be less than curr_ buffers' offset. In that case also it clears both
// buffers.
if (DoesBufferContainData(curr_) && !IsOffsetInBuffer(offset, curr_)) {
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
2022-05-20 23:09:33 +00:00
bufs_[curr_].buffer_.Clear();
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
bufs_[second].buffer_.Clear();
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
2022-05-20 23:09:33 +00:00
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
UpdateReadPattern(offset, n, /*decrease_readaheadsize=*/false);
bool data_found = false;
// 3. If curr_ has full data.
if (DoesBufferContainData(curr_) && IsDataBlockInBuffer(offset, n, curr_)) {
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
2022-05-20 23:09:33 +00:00
uint64_t offset_in_buffer = offset - bufs_[curr_].offset_;
*result = Slice(bufs_[curr_].buffer_.BufferStart() + offset_in_buffer, n);
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
data_found = true;
// Update num_file_reads_ as TryReadFromCacheAsync won't be called for
// poll and update num_file_reads_ if data is found.
num_file_reads_++;
// 3.1 If second also has some data or is not eligible for prefetching,
// return.
if (!is_eligible_for_prefetching || DoesBufferContainData(second)) {
return Status::OK();
}
} else {
// Partial data in curr_.
bufs_[curr_].buffer_.Clear();
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
2022-05-20 23:09:33 +00:00
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
bufs_[second].buffer_.Clear();
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
2022-05-20 23:09:33 +00:00
Status s;
size_t alignment = reader->file()->GetRequiredBufferAlignment();
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
size_t prefetch_size = is_eligible_for_prefetching ? readahead_size_ / 2 : 0;
Seek parallelization (#9994) Summary: The RocksDB iterator is a hierarchy of iterators. MergingIterator maintains a heap of LevelIterators, one for each L0 file and for each non-zero level. The Seek() operation naturally lends itself to parallelization, as it involves positioning every LevelIterator on the correct data block in the correct SST file. It lookups a level for a target key, to find the first key that's >= the target key. This typically involves reading one data block that is likely to contain the target key, and scan forward to find the first valid key. The forward scan may read more data blocks. In order to find the right data block, the iterator may read some metadata blocks (required for opening a file and searching the index). This flow can be parallelized. Design: Seek will be called two times under async_io option. First seek will send asynchronous request to prefetch the data blocks at each level and second seek will follow the normal flow and in FilePrefetchBuffer::TryReadFromCacheAsync it will wait for the Poll() to get the results and add the iterator to min_heap. - Status::TryAgain is passed down from FilePrefetchBuffer::PrefetchAsync to block_iter_.Status indicating asynchronous request has been submitted. - If for some reason asynchronous request returns error in submitting the request, it will fallback to sequential reading of blocks in one pass. - If the data already exists in prefetch_buffer, it will return the data without prefetching further and it will be treated as single pass of seek. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9994 Test Plan: - **Run Regressions.** ``` ./db_bench -db=/tmp/prefix_scan_prefetch_main -benchmarks="fillseq" -key_size=32 -value_size=512 -num=5000000 -use_direct_io_for_flush_and_compaction=true -target_file_size_base=16777216 ``` i) Previous release 7.0 run for normal prefetching with async_io disabled: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.0 Date: Thu Mar 17 13:11:34 2022 CPU: 24 * Intel Core Processor (Broadwell) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483618.390 micros/op 2 ops/sec; 338.9 MB/s (249 of 249 found) ``` ii) normal prefetching after changes with async_io disable: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 Set seed to 1652922591315307 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:09:51 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 483080.466 micros/op 2 ops/sec 120.287 seconds 249 operations; 340.8 MB/s (249 of 249 found) ``` iii) db_bench with async_io enabled completed succesfully ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=5000000 -use_direct_reads=true -seek_nexts=327680 -duration=120 -ops_between_duration_checks=1 -async_io=1 -adaptive_readahead=1 Set seed to 1652924062021732 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags RocksDB: version 7.3 Date: Wed May 18 18:34:22 2022 CPU: 32 * Intel Xeon Processor (Skylake) CPUCache: 16384 KB Keys: 32 bytes each (+ 0 bytes user-defined timestamp) Values: 512 bytes each (256 bytes after compression) Entries: 5000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 2594.0 MB (estimated) FileSize: 1373.3 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: Snappy Compression sampling rate: 0 Memtablerep: SkipListFactory Perf Level: 1 ------------------------------------------------ DB path: [/tmp/prefix_scan_prefetch_main] seekrandom : 553913.576 micros/op 1 ops/sec 120.199 seconds 217 operations; 293.6 MB/s (217 of 217 found) ``` - db_stress with async_io disabled completed succesfully ``` export CRASH_TEST_EXT_ARGS=" --async_io=0" make crash_test -j ``` I**n Progress**: db_stress with async_io is failing and working on debugging/fixing it. Reviewed By: anand1976 Differential Revision: D36459323 Pulled By: akankshamahajan15 fbshipit-source-id: abb1cd944abe712bae3986ae5b16704b3338917c
2022-05-20 23:09:33 +00:00
size_t offset_to_read = static_cast<size_t>(offset);
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
uint64_t rounddown_start1 = 0;
uint64_t roundup_end1 = 0;
uint64_t rounddown_start2 = 0;
uint64_t roundup_end2 = 0;
uint64_t chunk_len1 = 0;
uint64_t chunk_len2 = 0;
size_t read_len1 = 0;
size_t read_len2 = 0;
// - If curr_ is empty.
// - Call async read for full data + prefetch_size on curr_.
// - Call async read for prefetch_size on second if eligible.
// - If curr_ is filled.
// - prefetch_size on second.
// Calculate length and offsets for reading.
if (!DoesBufferContainData(curr_)) {
// Prefetch full data + prefetch_size in curr_.
rounddown_start1 = Rounddown(offset_to_read, alignment);
roundup_end1 = Roundup(offset_to_read + n + prefetch_size, alignment);
uint64_t roundup_len1 = roundup_end1 - rounddown_start1;
assert(roundup_len1 >= alignment);
assert(roundup_len1 % alignment == 0);
CalculateOffsetAndLen(alignment, rounddown_start1, roundup_len1, curr_,
false, chunk_len1);
assert(chunk_len1 == 0);
assert(roundup_len1 >= chunk_len1);
read_len1 = static_cast<size_t>(roundup_len1 - chunk_len1);
bufs_[curr_].offset_ = rounddown_start1;
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
2022-05-20 23:09:33 +00:00
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (is_eligible_for_prefetching) {
if (DoesBufferContainData(curr_)) {
rounddown_start2 =
bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize();
} else {
rounddown_start2 = roundup_end1;
}
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
2022-05-20 23:09:33 +00:00
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
roundup_end2 = Roundup(rounddown_start2 + prefetch_size, alignment);
uint64_t roundup_len2 = roundup_end2 - rounddown_start2;
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
2022-05-20 23:09:33 +00:00
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
assert(roundup_len2 >= alignment);
CalculateOffsetAndLen(alignment, rounddown_start2, roundup_len2, second,
false, chunk_len2);
assert(chunk_len2 == 0);
assert(roundup_len2 >= chunk_len2);
read_len2 = static_cast<size_t>(roundup_len2 - chunk_len2);
// Update the buffer offset.
bufs_[second].offset_ = rounddown_start2;
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
2022-05-20 23:09:33 +00:00
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
if (read_len1) {
s = ReadAsync(opts, reader, read_len1, rounddown_start1, curr_);
if (!s.ok()) {
DestroyAndClearIOHandle(curr_);
bufs_[curr_].buffer_.Clear();
return s;
}
explicit_prefetch_submitted_ = true;
prev_len_ = 0;
}
if (read_len2) {
s = ReadAsync(opts, reader, read_len2, rounddown_start2, second);
if (!s.ok()) {
DestroyAndClearIOHandle(second);
bufs_[second].buffer_.Clear();
return s;
}
readahead_size_ = std::min(max_readahead_size_, readahead_size_ * 2);
}
return (data_found ? Status::OK() : Status::TryAgain());
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
2022-05-20 23:09:33 +00:00
}
Async optimization in scan path (#10602) Summary: Optimizations 1. In FilePrefetchBuffer, when data is overlapping between two buffers, it copies the data from first to third buffer, then from second to third buffer to return continuous buffer. This optimization will call ReadAsync on first buffer as soon as buffer is empty instead of getting blocked by second buffer to copy the data. 2. For fixed size readahead_size, FilePrefetchBuffer will issues two async read calls. One with length + readahead_size_/2 on first buffer(if buffer is empty) and readahead_size_/2 on second buffer during seek. - Add readahead_size to db_stress for stress testing these changes in https://github.com/facebook/rocksdb/pull/10632 Pull Request resolved: https://github.com/facebook/rocksdb/pull/10602 Test Plan: - CircleCI tests - stress_test completed successfully export CRASH_TEST_EXT_ARGS="--async_io=1" make crash_test -j32 - db_bench showed no regression With this PR: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876074584472 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:14:34 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 270878.018 micros/op 3 ops/sec 30.068 seconds 111 operations; 618.7 MB/s (111 of 111 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=true -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661875332862922 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:02:12 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_prefetch_main1] seekrandom : 358352.488 micros/op 2 ops/sec 30.102 seconds 84 operations; 474.4 MB/s (84 of 84 found) ``` Without PR in main: ``` ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=1 Set seed to 1661876425983045 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:20:26 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 280881.953 micros/op 3 ops/sec 30.054 seconds 107 operations; 605.2 MB/s (107 of 107 found) ./db_bench -use_existing_db=true -db=/tmp/prefix_scan_prefetch_main1 -benchmarks="seekrandom" -key_size=32 -value_size=512 -num=50000000 -use_direct_reads=false -seek_nexts=327680 -duration=30 -ops_between_duration_checks=1 -async_io=0 Set seed to 1661876475267771 because --seed was 0 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags Integrated BlobDB: blob cache disabled RocksDB: version 7.7.0 Date: Tue Aug 30 09:21:15 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: 50000000 Prefix: 0 bytes Keys per prefix: 0 RawSize: 25939.9 MB (estimated) FileSize: 13732.9 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_main1] seekrandom : 363155.084 micros/op 2 ops/sec 30.142 seconds 83 operations; 468.1 MB/s (83 of 83 found) ``` Reviewed By: anand1976 Differential Revision: D39141328 Pulled By: akankshamahajan15 fbshipit-source-id: 560655922c1a437a8569c228abb31b8c0b413120
2022-09-13 00:42:01 +00:00
} // namespace ROCKSDB_NAMESPACE