rocksdb/file/sequence_file_reader.h
Changyu Bi 8515bd50c9 Support read rate-limiting in SequentialFileReader (#9973)
Summary:
Added rate limiter and read rate-limiting support to SequentialFileReader. I've updated call sites to SequentialFileReader::Read with appropriate IO priority (or left a TODO and specified IO_TOTAL for now).

The PR is separated into four commits: the first one added the rate-limiting support, but with some fixes in the unit test since the number of request bytes from rate limiter in SequentialFileReader are not accurate (there is overcharge at EOF). The second commit fixed this by allowing SequentialFileReader to check file size and determine how many bytes are left in the file to read. The third commit added benchmark related code. The fourth commit moved the logic of using file size to avoid overcharging the rate limiter into backup engine (the main user of SequentialFileReader).

Pull Request resolved: https://github.com/facebook/rocksdb/pull/9973

Test Plan:
- `make check`, backup_engine_test covers usage of SequentialFileReader with rate limiter.
- Run db_bench to check if rate limiting is throttling as expected: Verified that reads and writes are together throttled at 2MB/s, and at 0.2MB chunks that are 100ms apart.
  - Set up: `./db_bench --benchmarks=fillrandom -db=/dev/shm/test_rocksdb`
  - Benchmark:
```
strace -ttfe read,write ./db_bench --benchmarks=backup -db=/dev/shm/test_rocksdb --backup_rate_limit=2097152 --use_existing_db
strace -ttfe read,write ./db_bench --benchmarks=restore -db=/dev/shm/test_rocksdb --restore_rate_limit=2097152 --use_existing_db
```
- db bench on backup and restore to ensure no performance regression.
  - backup (avg over 50 runs): pre-change: 1.90443e+06 micros/op; post-change: 1.8993e+06 micros/op (improve by 0.2%)
  - restore (avg over 50 runs): pre-change: 1.79105e+06 micros/op; post-change: 1.78192e+06 micros/op (improve by 0.5%)

```
# Set up
./db_bench --benchmarks=fillrandom -db=/tmp/test_rocksdb -num=10000000

# benchmark
TEST_TMPDIR=/tmp/test_rocksdb
NUM_RUN=50
for ((j=0;j<$NUM_RUN;j++))
do
   ./db_bench -db=$TEST_TMPDIR -num=10000000 -benchmarks=backup -use_existing_db | egrep 'backup'
  # Restore
  #./db_bench -db=$TEST_TMPDIR -num=10000000 -benchmarks=restore -use_existing_db
done > rate_limit.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' rate_limit.txt >> rate_limit_2.txt
```

Reviewed By: hx235

Differential Revision: D36327418

Pulled By: cbi42

fbshipit-source-id: e75d4307cff815945482df5ba630c1e88d064691
2022-05-24 10:28:57 -07:00

130 lines
4.8 KiB
C++

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#pragma once
#include <atomic>
#include <string>
#include "env/file_system_tracer.h"
#include "port/port.h"
#include "rocksdb/env.h"
#include "rocksdb/file_system.h"
namespace ROCKSDB_NAMESPACE {
// SequentialFileReader is a wrapper on top of Env::SequentialFile. It handles
// Buffered (i.e when page cache is enabled) and Direct (with O_DIRECT / page
// cache disabled) reads appropriately, and also updates the IO stats.
class SequentialFileReader {
private:
#ifndef ROCKSDB_LITE
void NotifyOnFileReadFinish(
uint64_t offset, size_t length,
const FileOperationInfo::StartTimePoint& start_ts,
const FileOperationInfo::FinishTimePoint& finish_ts,
const Status& status) const {
FileOperationInfo info(FileOperationType::kRead, file_name_, start_ts,
finish_ts, status);
info.offset = offset;
info.length = length;
for (auto& listener : listeners_) {
listener->OnFileReadFinish(info);
}
info.status.PermitUncheckedError();
}
void AddFileIOListeners(
const std::vector<std::shared_ptr<EventListener>>& listeners) {
std::for_each(listeners.begin(), listeners.end(),
[this](const std::shared_ptr<EventListener>& e) {
if (e->ShouldBeNotifiedOnFileIO()) {
listeners_.emplace_back(e);
}
});
}
#endif // ROCKSDB_LITE
bool ShouldNotifyListeners() const { return !listeners_.empty(); }
std::string file_name_;
FSSequentialFilePtr file_;
std::atomic<size_t> offset_{0}; // read offset
std::vector<std::shared_ptr<EventListener>> listeners_{};
RateLimiter* rate_limiter_;
public:
explicit SequentialFileReader(
std::unique_ptr<FSSequentialFile>&& _file, const std::string& _file_name,
const std::shared_ptr<IOTracer>& io_tracer = nullptr,
const std::vector<std::shared_ptr<EventListener>>& listeners = {},
RateLimiter* rate_limiter =
nullptr) // TODO: migrate call sites to provide rate limiter
: file_name_(_file_name),
file_(std::move(_file), io_tracer, _file_name),
listeners_(),
rate_limiter_(rate_limiter) {
#ifndef ROCKSDB_LITE
AddFileIOListeners(listeners);
#else
(void)listeners;
#endif
}
explicit SequentialFileReader(
std::unique_ptr<FSSequentialFile>&& _file, const std::string& _file_name,
size_t _readahead_size,
const std::shared_ptr<IOTracer>& io_tracer = nullptr,
const std::vector<std::shared_ptr<EventListener>>& listeners = {},
RateLimiter* rate_limiter =
nullptr) // TODO: migrate call sites to provide rate limiter
: file_name_(_file_name),
file_(NewReadaheadSequentialFile(std::move(_file), _readahead_size),
io_tracer, _file_name),
listeners_(),
rate_limiter_(rate_limiter) {
#ifndef ROCKSDB_LITE
AddFileIOListeners(listeners);
#else
(void)listeners;
#endif
}
static IOStatus Create(const std::shared_ptr<FileSystem>& fs,
const std::string& fname, const FileOptions& file_opts,
std::unique_ptr<SequentialFileReader>* reader,
IODebugContext* dbg, RateLimiter* rate_limiter);
SequentialFileReader(const SequentialFileReader&) = delete;
SequentialFileReader& operator=(const SequentialFileReader&) = delete;
// `rate_limiter_priority` is used to charge the internal rate limiter when
// enabled. The special value `Env::IO_TOTAL` makes this operation bypass the
// rate limiter. The amount charged to the internal rate limiter is n, even
// when less than n bytes are actually read (e.g. at end of file). To avoid
// overcharging the rate limiter, the caller can use file size to cap n to
// read until end of file.
IOStatus Read(size_t n, Slice* result, char* scratch,
Env::IOPriority rate_limiter_priority);
IOStatus Skip(uint64_t n);
FSSequentialFile* file() { return file_.get(); }
std::string file_name() { return file_name_; }
bool use_direct_io() const { return file_->use_direct_io(); }
private:
// NewReadaheadSequentialFile provides a wrapper over SequentialFile to
// always prefetch additional data with every read.
static std::unique_ptr<FSSequentialFile> NewReadaheadSequentialFile(
std::unique_ptr<FSSequentialFile>&& file, size_t readahead_size);
};
} // namespace ROCKSDB_NAMESPACE