summaryrefslogtreecommitdiffstats
path: root/src/rocksdb/file
diff options
context:
space:
mode:
Diffstat (limited to 'src/rocksdb/file')
-rw-r--r--src/rocksdb/file/delete_scheduler.cc411
-rw-r--r--src/rocksdb/file/delete_scheduler.h149
-rw-r--r--src/rocksdb/file/delete_scheduler_test.cc724
-rw-r--r--src/rocksdb/file/file_prefetch_buffer.cc918
-rw-r--r--src/rocksdb/file/file_prefetch_buffer.h446
-rw-r--r--src/rocksdb/file/file_util.cc282
-rw-r--r--src/rocksdb/file/file_util.h89
-rw-r--r--src/rocksdb/file/filename.cc523
-rw-r--r--src/rocksdb/file/filename.h188
-rw-r--r--src/rocksdb/file/line_file_reader.cc73
-rw-r--r--src/rocksdb/file/line_file_reader.h60
-rw-r--r--src/rocksdb/file/prefetch_test.cc2109
-rw-r--r--src/rocksdb/file/random_access_file_reader.cc602
-rw-r--r--src/rocksdb/file/random_access_file_reader.h217
-rw-r--r--src/rocksdb/file/random_access_file_reader_test.cc481
-rw-r--r--src/rocksdb/file/read_write_util.cc33
-rw-r--r--src/rocksdb/file/read_write_util.h31
-rw-r--r--src/rocksdb/file/readahead_file_info.h33
-rw-r--r--src/rocksdb/file/readahead_raf.cc169
-rw-r--r--src/rocksdb/file/readahead_raf.h29
-rw-r--r--src/rocksdb/file/sequence_file_reader.cc328
-rw-r--r--src/rocksdb/file/sequence_file_reader.h129
-rw-r--r--src/rocksdb/file/sst_file_manager_impl.cc525
-rw-r--r--src/rocksdb/file/sst_file_manager_impl.h195
-rw-r--r--src/rocksdb/file/writable_file_writer.cc1025
-rw-r--r--src/rocksdb/file/writable_file_writer.h336
26 files changed, 10105 insertions, 0 deletions
diff --git a/src/rocksdb/file/delete_scheduler.cc b/src/rocksdb/file/delete_scheduler.cc
new file mode 100644
index 000000000..b97a0f224
--- /dev/null
+++ b/src/rocksdb/file/delete_scheduler.cc
@@ -0,0 +1,411 @@
+// 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).
+
+#ifndef ROCKSDB_LITE
+
+#include "file/delete_scheduler.h"
+
+#include <cinttypes>
+#include <thread>
+#include <vector>
+
+#include "file/sst_file_manager_impl.h"
+#include "logging/logging.h"
+#include "port/port.h"
+#include "rocksdb/env.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/system_clock.h"
+#include "test_util/sync_point.h"
+#include "util/mutexlock.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+DeleteScheduler::DeleteScheduler(SystemClock* clock, FileSystem* fs,
+ int64_t rate_bytes_per_sec, Logger* info_log,
+ SstFileManagerImpl* sst_file_manager,
+ double max_trash_db_ratio,
+ uint64_t bytes_max_delete_chunk)
+ : clock_(clock),
+ fs_(fs),
+ total_trash_size_(0),
+ rate_bytes_per_sec_(rate_bytes_per_sec),
+ pending_files_(0),
+ bytes_max_delete_chunk_(bytes_max_delete_chunk),
+ closing_(false),
+ cv_(&mu_),
+ bg_thread_(nullptr),
+ info_log_(info_log),
+ sst_file_manager_(sst_file_manager),
+ max_trash_db_ratio_(max_trash_db_ratio) {
+ assert(sst_file_manager != nullptr);
+ assert(max_trash_db_ratio >= 0);
+ MaybeCreateBackgroundThread();
+}
+
+DeleteScheduler::~DeleteScheduler() {
+ {
+ InstrumentedMutexLock l(&mu_);
+ closing_ = true;
+ cv_.SignalAll();
+ }
+ if (bg_thread_) {
+ bg_thread_->join();
+ }
+ for (const auto& it : bg_errors_) {
+ it.second.PermitUncheckedError();
+ }
+}
+
+Status DeleteScheduler::DeleteFile(const std::string& file_path,
+ const std::string& dir_to_sync,
+ const bool force_bg) {
+ if (rate_bytes_per_sec_.load() <= 0 ||
+ (!force_bg &&
+ total_trash_size_.load() >
+ sst_file_manager_->GetTotalSize() * max_trash_db_ratio_.load())) {
+ // Rate limiting is disabled or trash size makes up more than
+ // max_trash_db_ratio_ (default 25%) of the total DB size
+ TEST_SYNC_POINT("DeleteScheduler::DeleteFile");
+ Status s = fs_->DeleteFile(file_path, IOOptions(), nullptr);
+ if (s.ok()) {
+ s = sst_file_manager_->OnDeleteFile(file_path);
+ ROCKS_LOG_INFO(info_log_,
+ "Deleted file %s immediately, rate_bytes_per_sec %" PRIi64
+ ", total_trash_size %" PRIu64 " max_trash_db_ratio %lf",
+ file_path.c_str(), rate_bytes_per_sec_.load(),
+ total_trash_size_.load(), max_trash_db_ratio_.load());
+ InstrumentedMutexLock l(&mu_);
+ RecordTick(stats_.get(), FILES_DELETED_IMMEDIATELY);
+ }
+ return s;
+ }
+
+ // Move file to trash
+ std::string trash_file;
+ Status s = MarkAsTrash(file_path, &trash_file);
+ ROCKS_LOG_INFO(info_log_, "Mark file: %s as trash -- %s", trash_file.c_str(),
+ s.ToString().c_str());
+
+ if (!s.ok()) {
+ ROCKS_LOG_ERROR(info_log_, "Failed to mark %s as trash -- %s",
+ file_path.c_str(), s.ToString().c_str());
+ s = fs_->DeleteFile(file_path, IOOptions(), nullptr);
+ if (s.ok()) {
+ s = sst_file_manager_->OnDeleteFile(file_path);
+ ROCKS_LOG_INFO(info_log_, "Deleted file %s immediately",
+ trash_file.c_str());
+ InstrumentedMutexLock l(&mu_);
+ RecordTick(stats_.get(), FILES_DELETED_IMMEDIATELY);
+ }
+ return s;
+ }
+
+ // Update the total trash size
+ uint64_t trash_file_size = 0;
+ IOStatus io_s =
+ fs_->GetFileSize(trash_file, IOOptions(), &trash_file_size, nullptr);
+ if (io_s.ok()) {
+ total_trash_size_.fetch_add(trash_file_size);
+ }
+ //**TODO: What should we do if we failed to
+ // get the file size?
+
+ // Add file to delete queue
+ {
+ InstrumentedMutexLock l(&mu_);
+ RecordTick(stats_.get(), FILES_MARKED_TRASH);
+ queue_.emplace(trash_file, dir_to_sync);
+ pending_files_++;
+ if (pending_files_ == 1) {
+ cv_.SignalAll();
+ }
+ }
+ return s;
+}
+
+std::map<std::string, Status> DeleteScheduler::GetBackgroundErrors() {
+ InstrumentedMutexLock l(&mu_);
+ return bg_errors_;
+}
+
+const std::string DeleteScheduler::kTrashExtension = ".trash";
+bool DeleteScheduler::IsTrashFile(const std::string& file_path) {
+ return (file_path.size() >= kTrashExtension.size() &&
+ file_path.rfind(kTrashExtension) ==
+ file_path.size() - kTrashExtension.size());
+}
+
+Status DeleteScheduler::CleanupDirectory(Env* env, SstFileManagerImpl* sfm,
+ const std::string& path) {
+ Status s;
+ // Check if there are any files marked as trash in this path
+ std::vector<std::string> files_in_path;
+ const auto& fs = env->GetFileSystem();
+ IOOptions io_opts;
+ io_opts.do_not_recurse = true;
+ s = fs->GetChildren(path, io_opts, &files_in_path,
+ /*IODebugContext*=*/nullptr);
+ if (!s.ok()) {
+ return s;
+ }
+ for (const std::string& current_file : files_in_path) {
+ if (!DeleteScheduler::IsTrashFile(current_file)) {
+ // not a trash file, skip
+ continue;
+ }
+
+ Status file_delete;
+ std::string trash_file = path + "/" + current_file;
+ if (sfm) {
+ // We have an SstFileManager that will schedule the file delete
+ s = sfm->OnAddFile(trash_file);
+ file_delete = sfm->ScheduleFileDeletion(trash_file, path);
+ } else {
+ // Delete the file immediately
+ file_delete = env->DeleteFile(trash_file);
+ }
+
+ if (s.ok() && !file_delete.ok()) {
+ s = file_delete;
+ }
+ }
+
+ return s;
+}
+
+Status DeleteScheduler::MarkAsTrash(const std::string& file_path,
+ std::string* trash_file) {
+ // Sanity check of the path
+ size_t idx = file_path.rfind("/");
+ if (idx == std::string::npos || idx == file_path.size() - 1) {
+ return Status::InvalidArgument("file_path is corrupted");
+ }
+
+ if (DeleteScheduler::IsTrashFile(file_path)) {
+ // This is already a trash file
+ *trash_file = file_path;
+ return Status::OK();
+ }
+
+ *trash_file = file_path + kTrashExtension;
+ // TODO(tec) : Implement Env::RenameFileIfNotExist and remove
+ // file_move_mu mutex.
+ int cnt = 0;
+ Status s;
+ InstrumentedMutexLock l(&file_move_mu_);
+ while (true) {
+ s = fs_->FileExists(*trash_file, IOOptions(), nullptr);
+ if (s.IsNotFound()) {
+ // We found a path for our file in trash
+ s = fs_->RenameFile(file_path, *trash_file, IOOptions(), nullptr);
+ break;
+ } else if (s.ok()) {
+ // Name conflict, generate new random suffix
+ *trash_file = file_path + std::to_string(cnt) + kTrashExtension;
+ } else {
+ // Error during FileExists call, we cannot continue
+ break;
+ }
+ cnt++;
+ }
+ if (s.ok()) {
+ s = sst_file_manager_->OnMoveFile(file_path, *trash_file);
+ }
+ return s;
+}
+
+void DeleteScheduler::BackgroundEmptyTrash() {
+ TEST_SYNC_POINT("DeleteScheduler::BackgroundEmptyTrash");
+
+ while (true) {
+ InstrumentedMutexLock l(&mu_);
+ while (queue_.empty() && !closing_) {
+ cv_.Wait();
+ }
+
+ if (closing_) {
+ return;
+ }
+
+ // Delete all files in queue_
+ uint64_t start_time = clock_->NowMicros();
+ uint64_t total_deleted_bytes = 0;
+ int64_t current_delete_rate = rate_bytes_per_sec_.load();
+ while (!queue_.empty() && !closing_) {
+ if (current_delete_rate != rate_bytes_per_sec_.load()) {
+ // User changed the delete rate
+ current_delete_rate = rate_bytes_per_sec_.load();
+ start_time = clock_->NowMicros();
+ total_deleted_bytes = 0;
+ ROCKS_LOG_INFO(info_log_, "rate_bytes_per_sec is changed to %" PRIi64,
+ current_delete_rate);
+ }
+
+ // Get new file to delete
+ const FileAndDir& fad = queue_.front();
+ std::string path_in_trash = fad.fname;
+
+ // We don't need to hold the lock while deleting the file
+ mu_.Unlock();
+ uint64_t deleted_bytes = 0;
+ bool is_complete = true;
+ // Delete file from trash and update total_penlty value
+ Status s =
+ DeleteTrashFile(path_in_trash, fad.dir, &deleted_bytes, &is_complete);
+ total_deleted_bytes += deleted_bytes;
+ mu_.Lock();
+ if (is_complete) {
+ queue_.pop();
+ }
+
+ if (!s.ok()) {
+ bg_errors_[path_in_trash] = s;
+ }
+
+ // Apply penalty if necessary
+ uint64_t total_penalty;
+ if (current_delete_rate > 0) {
+ // rate limiting is enabled
+ total_penalty =
+ ((total_deleted_bytes * kMicrosInSecond) / current_delete_rate);
+ ROCKS_LOG_INFO(info_log_,
+ "Rate limiting is enabled with penalty %" PRIu64
+ " after deleting file %s",
+ total_penalty, path_in_trash.c_str());
+ while (!closing_ && !cv_.TimedWait(start_time + total_penalty)) {
+ }
+ } else {
+ // rate limiting is disabled
+ total_penalty = 0;
+ ROCKS_LOG_INFO(info_log_,
+ "Rate limiting is disabled after deleting file %s",
+ path_in_trash.c_str());
+ }
+ TEST_SYNC_POINT_CALLBACK("DeleteScheduler::BackgroundEmptyTrash:Wait",
+ &total_penalty);
+
+ if (is_complete) {
+ pending_files_--;
+ }
+ if (pending_files_ == 0) {
+ // Unblock WaitForEmptyTrash since there are no more files waiting
+ // to be deleted
+ cv_.SignalAll();
+ }
+ }
+ }
+}
+
+Status DeleteScheduler::DeleteTrashFile(const std::string& path_in_trash,
+ const std::string& dir_to_sync,
+ uint64_t* deleted_bytes,
+ bool* is_complete) {
+ uint64_t file_size;
+ Status s = fs_->GetFileSize(path_in_trash, IOOptions(), &file_size, nullptr);
+ *is_complete = true;
+ TEST_SYNC_POINT("DeleteScheduler::DeleteTrashFile:DeleteFile");
+ if (s.ok()) {
+ bool need_full_delete = true;
+ if (bytes_max_delete_chunk_ != 0 && file_size > bytes_max_delete_chunk_) {
+ uint64_t num_hard_links = 2;
+ // We don't have to worry aobut data race between linking a new
+ // file after the number of file link check and ftruncte because
+ // the file is now in trash and no hardlink is supposed to create
+ // to trash files by RocksDB.
+ Status my_status = fs_->NumFileLinks(path_in_trash, IOOptions(),
+ &num_hard_links, nullptr);
+ if (my_status.ok()) {
+ if (num_hard_links == 1) {
+ std::unique_ptr<FSWritableFile> wf;
+ my_status = fs_->ReopenWritableFile(path_in_trash, FileOptions(), &wf,
+ nullptr);
+ if (my_status.ok()) {
+ my_status = wf->Truncate(file_size - bytes_max_delete_chunk_,
+ IOOptions(), nullptr);
+ if (my_status.ok()) {
+ TEST_SYNC_POINT("DeleteScheduler::DeleteTrashFile:Fsync");
+ my_status = wf->Fsync(IOOptions(), nullptr);
+ }
+ }
+ if (my_status.ok()) {
+ *deleted_bytes = bytes_max_delete_chunk_;
+ need_full_delete = false;
+ *is_complete = false;
+ } else {
+ ROCKS_LOG_WARN(info_log_,
+ "Failed to partially delete %s from trash -- %s",
+ path_in_trash.c_str(), my_status.ToString().c_str());
+ }
+ } else {
+ ROCKS_LOG_INFO(info_log_,
+ "Cannot delete %s slowly through ftruncate from trash "
+ "as it has other links",
+ path_in_trash.c_str());
+ }
+ } else if (!num_link_error_printed_) {
+ ROCKS_LOG_INFO(
+ info_log_,
+ "Cannot delete files slowly through ftruncate from trash "
+ "as Env::NumFileLinks() returns error: %s",
+ my_status.ToString().c_str());
+ num_link_error_printed_ = true;
+ }
+ }
+
+ if (need_full_delete) {
+ s = fs_->DeleteFile(path_in_trash, IOOptions(), nullptr);
+ if (!dir_to_sync.empty()) {
+ std::unique_ptr<FSDirectory> dir_obj;
+ if (s.ok()) {
+ s = fs_->NewDirectory(dir_to_sync, IOOptions(), &dir_obj, nullptr);
+ }
+ if (s.ok()) {
+ s = dir_obj->FsyncWithDirOptions(
+ IOOptions(), nullptr,
+ DirFsyncOptions(DirFsyncOptions::FsyncReason::kFileDeleted));
+ TEST_SYNC_POINT_CALLBACK(
+ "DeleteScheduler::DeleteTrashFile::AfterSyncDir",
+ reinterpret_cast<void*>(const_cast<std::string*>(&dir_to_sync)));
+ }
+ }
+ if (s.ok()) {
+ *deleted_bytes = file_size;
+ s = sst_file_manager_->OnDeleteFile(path_in_trash);
+ }
+ }
+ }
+ if (!s.ok()) {
+ // Error while getting file size or while deleting
+ ROCKS_LOG_ERROR(info_log_, "Failed to delete %s from trash -- %s",
+ path_in_trash.c_str(), s.ToString().c_str());
+ *deleted_bytes = 0;
+ } else {
+ total_trash_size_.fetch_sub(*deleted_bytes);
+ }
+
+ return s;
+}
+
+void DeleteScheduler::WaitForEmptyTrash() {
+ InstrumentedMutexLock l(&mu_);
+ while (pending_files_ > 0 && !closing_) {
+ cv_.Wait();
+ }
+}
+
+void DeleteScheduler::MaybeCreateBackgroundThread() {
+ if (bg_thread_ == nullptr && rate_bytes_per_sec_.load() > 0) {
+ bg_thread_.reset(
+ new port::Thread(&DeleteScheduler::BackgroundEmptyTrash, this));
+ ROCKS_LOG_INFO(info_log_,
+ "Created background thread for deletion scheduler with "
+ "rate_bytes_per_sec: %" PRIi64,
+ rate_bytes_per_sec_.load());
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/file/delete_scheduler.h b/src/rocksdb/file/delete_scheduler.h
new file mode 100644
index 000000000..2904ec621
--- /dev/null
+++ b/src/rocksdb/file/delete_scheduler.h
@@ -0,0 +1,149 @@
+// 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).
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+
+#include <map>
+#include <queue>
+#include <string>
+#include <thread>
+
+#include "monitoring/instrumented_mutex.h"
+#include "port/port.h"
+#include "rocksdb/status.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Env;
+class FileSystem;
+class Logger;
+class SstFileManagerImpl;
+class SystemClock;
+
+// DeleteScheduler allows the DB to enforce a rate limit on file deletion,
+// Instead of deleteing files immediately, files are marked as trash
+// and deleted in a background thread that apply sleep penalty between deletes
+// if they are happening in a rate faster than rate_bytes_per_sec,
+//
+// Rate limiting can be turned off by setting rate_bytes_per_sec = 0, In this
+// case DeleteScheduler will delete files immediately.
+class DeleteScheduler {
+ public:
+ DeleteScheduler(SystemClock* clock, FileSystem* fs,
+ int64_t rate_bytes_per_sec, Logger* info_log,
+ SstFileManagerImpl* sst_file_manager,
+ double max_trash_db_ratio, uint64_t bytes_max_delete_chunk);
+
+ ~DeleteScheduler();
+
+ // Return delete rate limit in bytes per second
+ int64_t GetRateBytesPerSecond() { return rate_bytes_per_sec_.load(); }
+
+ // Set delete rate limit in bytes per second
+ void SetRateBytesPerSecond(int64_t bytes_per_sec) {
+ rate_bytes_per_sec_.store(bytes_per_sec);
+ MaybeCreateBackgroundThread();
+ }
+
+ // Mark file as trash directory and schedule its deletion. If force_bg is
+ // set, it forces the file to always be deleted in the background thread,
+ // except when rate limiting is disabled
+ Status DeleteFile(const std::string& fname, const std::string& dir_to_sync,
+ const bool force_bg = false);
+
+ // Wait for all files being deleteing in the background to finish or for
+ // destructor to be called.
+ void WaitForEmptyTrash();
+
+ // Return a map containing errors that happened in BackgroundEmptyTrash
+ // file_path => error status
+ std::map<std::string, Status> GetBackgroundErrors();
+
+ uint64_t GetTotalTrashSize() { return total_trash_size_.load(); }
+
+ // Return trash/DB size ratio where new files will be deleted immediately
+ double GetMaxTrashDBRatio() { return max_trash_db_ratio_.load(); }
+
+ // Update trash/DB size ratio where new files will be deleted immediately
+ void SetMaxTrashDBRatio(double r) {
+ assert(r >= 0);
+ max_trash_db_ratio_.store(r);
+ }
+
+ static const std::string kTrashExtension;
+ static bool IsTrashFile(const std::string& file_path);
+
+ // Check if there are any .trash files in path, and schedule their deletion
+ // Or delete immediately if sst_file_manager is nullptr
+ static Status CleanupDirectory(Env* env, SstFileManagerImpl* sfm,
+ const std::string& path);
+
+ void SetStatisticsPtr(const std::shared_ptr<Statistics>& stats) {
+ InstrumentedMutexLock l(&mu_);
+ stats_ = stats;
+ }
+
+ private:
+ Status MarkAsTrash(const std::string& file_path, std::string* path_in_trash);
+
+ Status DeleteTrashFile(const std::string& path_in_trash,
+ const std::string& dir_to_sync,
+ uint64_t* deleted_bytes, bool* is_complete);
+
+ void BackgroundEmptyTrash();
+
+ void MaybeCreateBackgroundThread();
+
+ SystemClock* clock_;
+ FileSystem* fs_;
+
+ // total size of trash files
+ std::atomic<uint64_t> total_trash_size_;
+ // Maximum number of bytes that should be deleted per second
+ std::atomic<int64_t> rate_bytes_per_sec_;
+ // Mutex to protect queue_, pending_files_, bg_errors_, closing_, stats_
+ InstrumentedMutex mu_;
+
+ struct FileAndDir {
+ FileAndDir(const std::string& f, const std::string& d) : fname(f), dir(d) {}
+ std::string fname;
+ std::string dir; // empty will be skipped.
+ };
+
+ // Queue of trash files that need to be deleted
+ std::queue<FileAndDir> queue_;
+ // Number of trash files that are waiting to be deleted
+ int32_t pending_files_;
+ uint64_t bytes_max_delete_chunk_;
+ // Errors that happened in BackgroundEmptyTrash (file_path => error)
+ std::map<std::string, Status> bg_errors_;
+
+ bool num_link_error_printed_ = false;
+ // Set to true in ~DeleteScheduler() to force BackgroundEmptyTrash to stop
+ bool closing_;
+ // Condition variable signaled in these conditions
+ // - pending_files_ value change from 0 => 1
+ // - pending_files_ value change from 1 => 0
+ // - closing_ value is set to true
+ InstrumentedCondVar cv_;
+ // Background thread running BackgroundEmptyTrash
+ std::unique_ptr<port::Thread> bg_thread_;
+ // Mutex to protect threads from file name conflicts
+ InstrumentedMutex file_move_mu_;
+ Logger* info_log_;
+ SstFileManagerImpl* sst_file_manager_;
+ // If the trash size constitutes for more than this fraction of the total DB
+ // size we will start deleting new files passed to DeleteScheduler
+ // immediately
+ std::atomic<double> max_trash_db_ratio_;
+ static const uint64_t kMicrosInSecond = 1000 * 1000LL;
+ std::shared_ptr<Statistics> stats_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/file/delete_scheduler_test.cc b/src/rocksdb/file/delete_scheduler_test.cc
new file mode 100644
index 000000000..d825da32a
--- /dev/null
+++ b/src/rocksdb/file/delete_scheduler_test.cc
@@ -0,0 +1,724 @@
+// 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).
+
+#include "file/delete_scheduler.h"
+
+#include <atomic>
+#include <cinttypes>
+#include <thread>
+#include <vector>
+
+#include "file/file_util.h"
+#include "file/sst_file_manager_impl.h"
+#include "rocksdb/env.h"
+#include "rocksdb/options.h"
+#include "test_util/sync_point.h"
+#include "test_util/testharness.h"
+#include "util/string_util.h"
+
+#ifndef ROCKSDB_LITE
+
+namespace ROCKSDB_NAMESPACE {
+
+class DeleteSchedulerTest : public testing::Test {
+ public:
+ DeleteSchedulerTest() : env_(Env::Default()) {
+ const int kNumDataDirs = 3;
+ dummy_files_dirs_.reserve(kNumDataDirs);
+ for (size_t i = 0; i < kNumDataDirs; ++i) {
+ dummy_files_dirs_.emplace_back(
+ test::PerThreadDBPath(env_, "delete_scheduler_dummy_data_dir") +
+ std::to_string(i));
+ DestroyAndCreateDir(dummy_files_dirs_.back());
+ }
+ stats_ = ROCKSDB_NAMESPACE::CreateDBStatistics();
+ }
+
+ ~DeleteSchedulerTest() override {
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({});
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+ for (const auto& dummy_files_dir : dummy_files_dirs_) {
+ DestroyDir(env_, dummy_files_dir);
+ }
+ }
+
+ void DestroyAndCreateDir(const std::string& dir) {
+ ASSERT_OK(DestroyDir(env_, dir));
+ EXPECT_OK(env_->CreateDir(dir));
+ }
+
+ int CountNormalFiles(size_t dummy_files_dirs_idx = 0) {
+ std::vector<std::string> files_in_dir;
+ EXPECT_OK(env_->GetChildren(dummy_files_dirs_[dummy_files_dirs_idx],
+ &files_in_dir));
+
+ int normal_cnt = 0;
+ for (auto& f : files_in_dir) {
+ if (!DeleteScheduler::IsTrashFile(f)) {
+ normal_cnt++;
+ }
+ }
+ return normal_cnt;
+ }
+
+ int CountTrashFiles(size_t dummy_files_dirs_idx = 0) {
+ std::vector<std::string> files_in_dir;
+ EXPECT_OK(env_->GetChildren(dummy_files_dirs_[dummy_files_dirs_idx],
+ &files_in_dir));
+
+ int trash_cnt = 0;
+ for (auto& f : files_in_dir) {
+ if (DeleteScheduler::IsTrashFile(f)) {
+ trash_cnt++;
+ }
+ }
+ return trash_cnt;
+ }
+
+ std::string NewDummyFile(const std::string& file_name, uint64_t size = 1024,
+ size_t dummy_files_dirs_idx = 0) {
+ std::string file_path =
+ dummy_files_dirs_[dummy_files_dirs_idx] + "/" + file_name;
+ std::unique_ptr<WritableFile> f;
+ env_->NewWritableFile(file_path, &f, EnvOptions());
+ std::string data(size, 'A');
+ EXPECT_OK(f->Append(data));
+ EXPECT_OK(f->Close());
+ sst_file_mgr_->OnAddFile(file_path);
+ return file_path;
+ }
+
+ void NewDeleteScheduler() {
+ // Tests in this file are for DeleteScheduler component and don't create any
+ // DBs, so we need to set max_trash_db_ratio to 100% (instead of default
+ // 25%)
+ sst_file_mgr_.reset(
+ new SstFileManagerImpl(env_->GetSystemClock(), env_->GetFileSystem(),
+ nullptr, rate_bytes_per_sec_,
+ /* max_trash_db_ratio= */ 1.1, 128 * 1024));
+ delete_scheduler_ = sst_file_mgr_->delete_scheduler();
+ sst_file_mgr_->SetStatisticsPtr(stats_);
+ }
+
+ Env* env_;
+ std::vector<std::string> dummy_files_dirs_;
+ int64_t rate_bytes_per_sec_;
+ DeleteScheduler* delete_scheduler_;
+ std::unique_ptr<SstFileManagerImpl> sst_file_mgr_;
+ std::shared_ptr<Statistics> stats_;
+};
+
+// Test the basic functionality of DeleteScheduler (Rate Limiting).
+// 1- Create 100 dummy files
+// 2- Delete the 100 dummy files using DeleteScheduler
+// --- Hold DeleteScheduler::BackgroundEmptyTrash ---
+// 3- Wait for DeleteScheduler to delete all files in trash
+// 4- Verify that BackgroundEmptyTrash used to correct penlties for the files
+// 5- Make sure that all created files were completely deleted
+TEST_F(DeleteSchedulerTest, BasicRateLimiting) {
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"DeleteSchedulerTest::BasicRateLimiting:1",
+ "DeleteScheduler::BackgroundEmptyTrash"},
+ });
+
+ std::vector<uint64_t> penalties;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::BackgroundEmptyTrash:Wait",
+ [&](void* arg) { penalties.push_back(*(static_cast<uint64_t*>(arg))); });
+ int dir_synced = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile::AfterSyncDir", [&](void* arg) {
+ dir_synced++;
+ std::string* dir = reinterpret_cast<std::string*>(arg);
+ EXPECT_EQ(dummy_files_dirs_[0], *dir);
+ });
+
+ int num_files = 100; // 100 files
+ uint64_t file_size = 1024; // every file is 1 kb
+ std::vector<uint64_t> delete_kbs_per_sec = {512, 200, 100, 50, 25};
+
+ for (size_t t = 0; t < delete_kbs_per_sec.size(); t++) {
+ penalties.clear();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearTrace();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ DestroyAndCreateDir(dummy_files_dirs_[0]);
+ rate_bytes_per_sec_ = delete_kbs_per_sec[t] * 1024;
+ NewDeleteScheduler();
+
+ dir_synced = 0;
+ // Create 100 dummy files, every file is 1 Kb
+ std::vector<std::string> generated_files;
+ for (int i = 0; i < num_files; i++) {
+ std::string file_name = "file" + std::to_string(i) + ".data";
+ generated_files.push_back(NewDummyFile(file_name, file_size));
+ }
+
+ // Delete dummy files and measure time spent to empty trash
+ for (int i = 0; i < num_files; i++) {
+ ASSERT_OK(delete_scheduler_->DeleteFile(generated_files[i],
+ dummy_files_dirs_[0]));
+ }
+ ASSERT_EQ(CountNormalFiles(), 0);
+
+ uint64_t delete_start_time = env_->NowMicros();
+ TEST_SYNC_POINT("DeleteSchedulerTest::BasicRateLimiting:1");
+ delete_scheduler_->WaitForEmptyTrash();
+ uint64_t time_spent_deleting = env_->NowMicros() - delete_start_time;
+
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 0);
+
+ uint64_t total_files_size = 0;
+ uint64_t expected_penlty = 0;
+ ASSERT_EQ(penalties.size(), num_files);
+ for (int i = 0; i < num_files; i++) {
+ total_files_size += file_size;
+ expected_penlty = ((total_files_size * 1000000) / rate_bytes_per_sec_);
+ ASSERT_EQ(expected_penlty, penalties[i]);
+ }
+ ASSERT_GT(time_spent_deleting, expected_penlty * 0.9);
+
+ ASSERT_EQ(num_files, dir_synced);
+
+ ASSERT_EQ(CountTrashFiles(), 0);
+ ASSERT_EQ(num_files, stats_->getAndResetTickerCount(FILES_MARKED_TRASH));
+ ASSERT_EQ(0, stats_->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ }
+}
+
+TEST_F(DeleteSchedulerTest, MultiDirectoryDeletionsScheduled) {
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"DeleteSchedulerTest::MultiDbPathDeletionsScheduled:1",
+ "DeleteScheduler::BackgroundEmptyTrash"},
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ rate_bytes_per_sec_ = 1 << 20; // 1MB
+ NewDeleteScheduler();
+
+ // Generate dummy files in multiple directories
+ const size_t kNumFiles = dummy_files_dirs_.size();
+ const size_t kFileSize = 1 << 10; // 1KB
+ std::vector<std::string> generated_files;
+ for (size_t i = 0; i < kNumFiles; i++) {
+ generated_files.push_back(NewDummyFile("file", kFileSize, i));
+ ASSERT_EQ(1, CountNormalFiles(i));
+ }
+
+ // Mark dummy files as trash
+ for (size_t i = 0; i < kNumFiles; i++) {
+ ASSERT_OK(delete_scheduler_->DeleteFile(generated_files[i], ""));
+ ASSERT_EQ(0, CountNormalFiles(i));
+ ASSERT_EQ(1, CountTrashFiles(i));
+ }
+ TEST_SYNC_POINT("DeleteSchedulerTest::MultiDbPathDeletionsScheduled:1");
+ delete_scheduler_->WaitForEmptyTrash();
+
+ // Verify dummy files eventually got deleted
+ for (size_t i = 0; i < kNumFiles; i++) {
+ ASSERT_EQ(0, CountNormalFiles(i));
+ ASSERT_EQ(0, CountTrashFiles(i));
+ }
+
+ ASSERT_EQ(kNumFiles, stats_->getAndResetTickerCount(FILES_MARKED_TRASH));
+ ASSERT_EQ(0, stats_->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+}
+
+// Same as the BasicRateLimiting test but delete files in multiple threads.
+// 1- Create 100 dummy files
+// 2- Delete the 100 dummy files using DeleteScheduler using 10 threads
+// --- Hold DeleteScheduler::BackgroundEmptyTrash ---
+// 3- Wait for DeleteScheduler to delete all files in queue
+// 4- Verify that BackgroundEmptyTrash used to correct penlties for the files
+// 5- Make sure that all created files were completely deleted
+TEST_F(DeleteSchedulerTest, RateLimitingMultiThreaded) {
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"DeleteSchedulerTest::RateLimitingMultiThreaded:1",
+ "DeleteScheduler::BackgroundEmptyTrash"},
+ });
+
+ std::vector<uint64_t> penalties;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::BackgroundEmptyTrash:Wait",
+ [&](void* arg) { penalties.push_back(*(static_cast<uint64_t*>(arg))); });
+
+ int thread_cnt = 10;
+ int num_files = 10; // 10 files per thread
+ uint64_t file_size = 1024; // every file is 1 kb
+
+ std::vector<uint64_t> delete_kbs_per_sec = {512, 200, 100, 50, 25};
+ for (size_t t = 0; t < delete_kbs_per_sec.size(); t++) {
+ penalties.clear();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearTrace();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ DestroyAndCreateDir(dummy_files_dirs_[0]);
+ rate_bytes_per_sec_ = delete_kbs_per_sec[t] * 1024;
+ NewDeleteScheduler();
+
+ // Create 100 dummy files, every file is 1 Kb
+ std::vector<std::string> generated_files;
+ for (int i = 0; i < num_files * thread_cnt; i++) {
+ std::string file_name = "file" + std::to_string(i) + ".data";
+ generated_files.push_back(NewDummyFile(file_name, file_size));
+ }
+
+ // Delete dummy files using 10 threads and measure time spent to empty trash
+ std::atomic<int> thread_num(0);
+ std::vector<port::Thread> threads;
+ std::function<void()> delete_thread = [&]() {
+ int idx = thread_num.fetch_add(1);
+ int range_start = idx * num_files;
+ int range_end = range_start + num_files;
+ for (int j = range_start; j < range_end; j++) {
+ ASSERT_OK(delete_scheduler_->DeleteFile(generated_files[j], ""));
+ }
+ };
+
+ for (int i = 0; i < thread_cnt; i++) {
+ threads.emplace_back(delete_thread);
+ }
+
+ for (size_t i = 0; i < threads.size(); i++) {
+ threads[i].join();
+ }
+
+ uint64_t delete_start_time = env_->NowMicros();
+ TEST_SYNC_POINT("DeleteSchedulerTest::RateLimitingMultiThreaded:1");
+ delete_scheduler_->WaitForEmptyTrash();
+ uint64_t time_spent_deleting = env_->NowMicros() - delete_start_time;
+
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 0);
+
+ uint64_t total_files_size = 0;
+ uint64_t expected_penlty = 0;
+ ASSERT_EQ(penalties.size(), num_files * thread_cnt);
+ for (int i = 0; i < num_files * thread_cnt; i++) {
+ total_files_size += file_size;
+ expected_penlty = ((total_files_size * 1000000) / rate_bytes_per_sec_);
+ ASSERT_EQ(expected_penlty, penalties[i]);
+ }
+ ASSERT_GT(time_spent_deleting, expected_penlty * 0.9);
+
+ ASSERT_EQ(CountNormalFiles(), 0);
+ ASSERT_EQ(CountTrashFiles(), 0);
+ ASSERT_EQ(num_files * thread_cnt,
+ stats_->getAndResetTickerCount(FILES_MARKED_TRASH));
+ ASSERT_EQ(0, stats_->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ }
+}
+
+// Disable rate limiting by setting rate_bytes_per_sec_ to 0 and make sure
+// that when DeleteScheduler delete a file it delete it immediately and don't
+// move it to trash
+TEST_F(DeleteSchedulerTest, DisableRateLimiting) {
+ int bg_delete_file = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:DeleteFile",
+ [&](void* /*arg*/) { bg_delete_file++; });
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 0;
+ NewDeleteScheduler();
+ constexpr int num_files = 10;
+
+ for (int i = 0; i < num_files; i++) {
+ // Every file we delete will be deleted immediately
+ std::string dummy_file = NewDummyFile("dummy.data");
+ ASSERT_OK(delete_scheduler_->DeleteFile(dummy_file, ""));
+ ASSERT_TRUE(env_->FileExists(dummy_file).IsNotFound());
+ ASSERT_EQ(CountNormalFiles(), 0);
+ ASSERT_EQ(CountTrashFiles(), 0);
+ }
+
+ ASSERT_EQ(bg_delete_file, 0);
+ ASSERT_EQ(0, stats_->getAndResetTickerCount(FILES_MARKED_TRASH));
+ ASSERT_EQ(num_files,
+ stats_->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+}
+
+// Testing that moving files to trash with the same name is not a problem
+// 1- Create 10 files with the same name "conflict.data"
+// 2- Delete the 10 files using DeleteScheduler
+// 3- Make sure that trash directory contain 10 files ("conflict.data" x 10)
+// --- Hold DeleteScheduler::BackgroundEmptyTrash ---
+// 4- Make sure that files are deleted from trash
+TEST_F(DeleteSchedulerTest, ConflictNames) {
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"DeleteSchedulerTest::ConflictNames:1",
+ "DeleteScheduler::BackgroundEmptyTrash"},
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 1024 * 1024; // 1 Mb/sec
+ NewDeleteScheduler();
+
+ // Create "conflict.data" and move it to trash 10 times
+ for (int i = 0; i < 10; i++) {
+ std::string dummy_file = NewDummyFile("conflict.data");
+ ASSERT_OK(delete_scheduler_->DeleteFile(dummy_file, ""));
+ }
+ ASSERT_EQ(CountNormalFiles(), 0);
+ // 10 files ("conflict.data" x 10) in trash
+ ASSERT_EQ(CountTrashFiles(), 10);
+
+ // Hold BackgroundEmptyTrash
+ TEST_SYNC_POINT("DeleteSchedulerTest::ConflictNames:1");
+ delete_scheduler_->WaitForEmptyTrash();
+ ASSERT_EQ(CountTrashFiles(), 0);
+
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 0);
+ ASSERT_EQ(10, stats_->getAndResetTickerCount(FILES_MARKED_TRASH));
+ ASSERT_EQ(0, stats_->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+}
+
+// 1- Create 10 dummy files
+// 2- Delete the 10 files using DeleteScheduler (move them to trsah)
+// 3- Delete the 10 files directly (using env_->DeleteFile)
+// --- Hold DeleteScheduler::BackgroundEmptyTrash ---
+// 4- Make sure that DeleteScheduler failed to delete the 10 files and
+// reported 10 background errors
+TEST_F(DeleteSchedulerTest, BackgroundError) {
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"DeleteSchedulerTest::BackgroundError:1",
+ "DeleteScheduler::BackgroundEmptyTrash"},
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 1024 * 1024; // 1 Mb/sec
+ NewDeleteScheduler();
+
+ // Generate 10 dummy files and move them to trash
+ for (int i = 0; i < 10; i++) {
+ std::string file_name = "data_" + std::to_string(i) + ".data";
+ ASSERT_OK(delete_scheduler_->DeleteFile(NewDummyFile(file_name), ""));
+ }
+ ASSERT_EQ(CountNormalFiles(), 0);
+ ASSERT_EQ(CountTrashFiles(), 10);
+
+ // Delete 10 files from trash, this will cause background errors in
+ // BackgroundEmptyTrash since we already deleted the files it was
+ // goind to delete
+ for (int i = 0; i < 10; i++) {
+ std::string file_name = "data_" + std::to_string(i) + ".data.trash";
+ ASSERT_OK(env_->DeleteFile(dummy_files_dirs_[0] + "/" + file_name));
+ }
+
+ // Hold BackgroundEmptyTrash
+ TEST_SYNC_POINT("DeleteSchedulerTest::BackgroundError:1");
+ delete_scheduler_->WaitForEmptyTrash();
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 10);
+ for (const auto& it : bg_errors) {
+ ASSERT_TRUE(it.second.IsPathNotFound());
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+}
+
+// 1- Create kTestFileNum dummy files
+// 2- Delete kTestFileNum dummy files using DeleteScheduler
+// 3- Wait for DeleteScheduler to delete all files in queue
+// 4- Make sure all files in trash directory were deleted
+// 5- Repeat previous steps 5 times
+TEST_F(DeleteSchedulerTest, StartBGEmptyTrashMultipleTimes) {
+ constexpr int kTestFileNum = 10;
+ std::atomic_int bg_delete_file = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:DeleteFile",
+ [&](void* /*arg*/) { bg_delete_file++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 1024 * 1024; // 1 MB / sec
+ NewDeleteScheduler();
+
+ // If trash file is generated faster than deleting, delete_scheduler will
+ // delete it directly instead of waiting for background trash empty thread to
+ // clean it. Set the ratio higher to avoid that.
+ sst_file_mgr_->SetMaxTrashDBRatio(kTestFileNum + 1);
+
+ // Move files to trash, wait for empty trash, start again
+ for (int run = 1; run <= 5; run++) {
+ // Generate kTestFileNum dummy files and move them to trash
+ for (int i = 0; i < kTestFileNum; i++) {
+ std::string file_name = "data_" + std::to_string(i) + ".data";
+ ASSERT_OK(delete_scheduler_->DeleteFile(NewDummyFile(file_name), ""));
+ }
+ ASSERT_EQ(CountNormalFiles(), 0);
+ delete_scheduler_->WaitForEmptyTrash();
+ ASSERT_EQ(bg_delete_file, kTestFileNum * run);
+ ASSERT_EQ(CountTrashFiles(), 0);
+
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 0);
+ ASSERT_EQ(kTestFileNum, stats_->getAndResetTickerCount(FILES_MARKED_TRASH));
+ ASSERT_EQ(0, stats_->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
+ }
+
+ ASSERT_EQ(bg_delete_file, 5 * kTestFileNum);
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+}
+
+TEST_F(DeleteSchedulerTest, DeletePartialFile) {
+ int bg_delete_file = 0;
+ int bg_fsync = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:DeleteFile",
+ [&](void*) { bg_delete_file++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:Fsync", [&](void*) { bg_fsync++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 1024 * 1024; // 1 MB / sec
+ NewDeleteScheduler();
+
+ // Should delete in 4 batch
+ ASSERT_OK(
+ delete_scheduler_->DeleteFile(NewDummyFile("data_1", 500 * 1024), ""));
+ ASSERT_OK(
+ delete_scheduler_->DeleteFile(NewDummyFile("data_2", 100 * 1024), ""));
+ // Should delete in 2 batch
+ ASSERT_OK(
+ delete_scheduler_->DeleteFile(NewDummyFile("data_2", 200 * 1024), ""));
+
+ delete_scheduler_->WaitForEmptyTrash();
+
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 0);
+ ASSERT_EQ(7, bg_delete_file);
+ ASSERT_EQ(4, bg_fsync);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+}
+
+#ifdef OS_LINUX
+TEST_F(DeleteSchedulerTest, NoPartialDeleteWithLink) {
+ int bg_delete_file = 0;
+ int bg_fsync = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:DeleteFile",
+ [&](void*) { bg_delete_file++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:Fsync", [&](void*) { bg_fsync++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 1024 * 1024; // 1 MB / sec
+ NewDeleteScheduler();
+
+ std::string file1 = NewDummyFile("data_1", 500 * 1024);
+ std::string file2 = NewDummyFile("data_2", 100 * 1024);
+
+ ASSERT_OK(env_->LinkFile(file1, dummy_files_dirs_[0] + "/data_1b"));
+ ASSERT_OK(env_->LinkFile(file2, dummy_files_dirs_[0] + "/data_2b"));
+
+ // Should delete in 4 batch if there is no hardlink
+ ASSERT_OK(delete_scheduler_->DeleteFile(file1, ""));
+ ASSERT_OK(delete_scheduler_->DeleteFile(file2, ""));
+
+ delete_scheduler_->WaitForEmptyTrash();
+
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 0);
+ ASSERT_EQ(2, bg_delete_file);
+ ASSERT_EQ(0, bg_fsync);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+}
+#endif
+
+// 1- Create a DeleteScheduler with very slow rate limit (1 Byte / sec)
+// 2- Delete 100 files using DeleteScheduler
+// 3- Delete the DeleteScheduler (call the destructor while queue is not empty)
+// 4- Make sure that not all files were deleted from trash and that
+// DeleteScheduler background thread did not delete all files
+TEST_F(DeleteSchedulerTest, DestructorWithNonEmptyQueue) {
+ int bg_delete_file = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:DeleteFile",
+ [&](void* /*arg*/) { bg_delete_file++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 1; // 1 Byte / sec
+ NewDeleteScheduler();
+
+ for (int i = 0; i < 100; i++) {
+ std::string file_name = "data_" + std::to_string(i) + ".data";
+ ASSERT_OK(delete_scheduler_->DeleteFile(NewDummyFile(file_name), ""));
+ }
+
+ // Deleting 100 files will need >28 hours to delete
+ // we will delete the DeleteScheduler while delete queue is not empty
+ sst_file_mgr_.reset();
+
+ ASSERT_LT(bg_delete_file, 100);
+ ASSERT_GT(CountTrashFiles(), 0);
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+}
+
+TEST_F(DeleteSchedulerTest, DISABLED_DynamicRateLimiting1) {
+ std::vector<uint64_t> penalties;
+ int bg_delete_file = 0;
+ int fg_delete_file = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:DeleteFile",
+ [&](void* /*arg*/) { bg_delete_file++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteFile", [&](void* /*arg*/) { fg_delete_file++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::BackgroundEmptyTrash:Wait",
+ [&](void* arg) { penalties.push_back(*(static_cast<int*>(arg))); });
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"DeleteSchedulerTest::DynamicRateLimiting1:1",
+ "DeleteScheduler::BackgroundEmptyTrash"},
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ rate_bytes_per_sec_ = 0; // Disable rate limiting initially
+ NewDeleteScheduler();
+
+ int num_files = 10; // 10 files
+ uint64_t file_size = 1024; // every file is 1 kb
+
+ std::vector<int64_t> delete_kbs_per_sec = {512, 200, 0, 100, 50, -2, 25};
+ for (size_t t = 0; t < delete_kbs_per_sec.size(); t++) {
+ penalties.clear();
+ bg_delete_file = 0;
+ fg_delete_file = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearTrace();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ DestroyAndCreateDir(dummy_files_dirs_[0]);
+ rate_bytes_per_sec_ = delete_kbs_per_sec[t] * 1024;
+ delete_scheduler_->SetRateBytesPerSecond(rate_bytes_per_sec_);
+
+ // Create 100 dummy files, every file is 1 Kb
+ std::vector<std::string> generated_files;
+ for (int i = 0; i < num_files; i++) {
+ std::string file_name = "file" + std::to_string(i) + ".data";
+ generated_files.push_back(NewDummyFile(file_name, file_size));
+ }
+
+ // Delete dummy files and measure time spent to empty trash
+ for (int i = 0; i < num_files; i++) {
+ ASSERT_OK(delete_scheduler_->DeleteFile(generated_files[i], ""));
+ }
+ ASSERT_EQ(CountNormalFiles(), 0);
+
+ if (rate_bytes_per_sec_ > 0) {
+ uint64_t delete_start_time = env_->NowMicros();
+ TEST_SYNC_POINT("DeleteSchedulerTest::DynamicRateLimiting1:1");
+ delete_scheduler_->WaitForEmptyTrash();
+ uint64_t time_spent_deleting = env_->NowMicros() - delete_start_time;
+
+ auto bg_errors = delete_scheduler_->GetBackgroundErrors();
+ ASSERT_EQ(bg_errors.size(), 0);
+
+ uint64_t total_files_size = 0;
+ uint64_t expected_penlty = 0;
+ ASSERT_EQ(penalties.size(), num_files);
+ for (int i = 0; i < num_files; i++) {
+ total_files_size += file_size;
+ expected_penlty = ((total_files_size * 1000000) / rate_bytes_per_sec_);
+ ASSERT_EQ(expected_penlty, penalties[i]);
+ }
+ ASSERT_GT(time_spent_deleting, expected_penlty * 0.9);
+ ASSERT_EQ(bg_delete_file, num_files);
+ ASSERT_EQ(fg_delete_file, 0);
+ } else {
+ ASSERT_EQ(penalties.size(), 0);
+ ASSERT_EQ(bg_delete_file, 0);
+ ASSERT_EQ(fg_delete_file, num_files);
+ }
+
+ ASSERT_EQ(CountTrashFiles(), 0);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ }
+}
+
+TEST_F(DeleteSchedulerTest, ImmediateDeleteOn25PercDBSize) {
+ int bg_delete_file = 0;
+ int fg_delete_file = 0;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteTrashFile:DeleteFile",
+ [&](void* /*arg*/) { bg_delete_file++; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "DeleteScheduler::DeleteFile", [&](void* /*arg*/) { fg_delete_file++; });
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ int num_files = 100; // 100 files
+ uint64_t file_size = 1024 * 10; // 100 KB as a file size
+ rate_bytes_per_sec_ = 1; // 1 byte per sec (very slow trash delete)
+
+ NewDeleteScheduler();
+ delete_scheduler_->SetMaxTrashDBRatio(0.25);
+
+ std::vector<std::string> generated_files;
+ for (int i = 0; i < num_files; i++) {
+ std::string file_name = "file" + std::to_string(i) + ".data";
+ generated_files.push_back(NewDummyFile(file_name, file_size));
+ }
+
+ for (std::string& file_name : generated_files) {
+ ASSERT_OK(delete_scheduler_->DeleteFile(file_name, ""));
+ }
+
+ // When we end up with 26 files in trash we will start
+ // deleting new files immediately
+ ASSERT_EQ(fg_delete_file, 74);
+ ASSERT_EQ(26, stats_->getAndResetTickerCount(FILES_MARKED_TRASH));
+ ASSERT_EQ(74, stats_->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+}
+
+TEST_F(DeleteSchedulerTest, IsTrashCheck) {
+ // Trash files
+ ASSERT_TRUE(DeleteScheduler::IsTrashFile("x.trash"));
+ ASSERT_TRUE(DeleteScheduler::IsTrashFile(".trash"));
+ ASSERT_TRUE(DeleteScheduler::IsTrashFile("abc.sst.trash"));
+ ASSERT_TRUE(DeleteScheduler::IsTrashFile("/a/b/c/abc..sst.trash"));
+ ASSERT_TRUE(DeleteScheduler::IsTrashFile("log.trash"));
+ ASSERT_TRUE(DeleteScheduler::IsTrashFile("^^^^^.log.trash"));
+ ASSERT_TRUE(DeleteScheduler::IsTrashFile("abc.t.trash"));
+
+ // Not trash files
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile("abc.sst"));
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile("abc.txt"));
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile("/a/b/c/abc.sst"));
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile("/a/b/c/abc.sstrash"));
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile("^^^^^.trashh"));
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile("abc.ttrash"));
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile(".ttrash"));
+ ASSERT_FALSE(DeleteScheduler::IsTrashFile("abc.trashx"));
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+int main(int /*argc*/, char** /*argv*/) {
+ printf("DeleteScheduler is not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/file/file_prefetch_buffer.cc b/src/rocksdb/file/file_prefetch_buffer.cc
new file mode 100644
index 000000000..4ac0d0504
--- /dev/null
+++ b/src/rocksdb/file/file_prefetch_buffer.cc
@@ -0,0 +1,918 @@
+// 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>
+#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 {
+
+void FilePrefetchBuffer::CalculateOffsetAndLen(size_t alignment,
+ uint64_t offset,
+ size_t roundup_len,
+ uint32_t index, bool refit_tail,
+ 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.
+ if (DoesBufferContainData(index) && IsOffsetInBuffer(offset, index)) {
+ // 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);
+ assert(chunk_len % alignment == 0);
+ 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));
+ } 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));
+ }
+}
+
+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);
+#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,
+ uint64_t read_len,
+ 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;
+ req.offset = rounddown_start;
+ req.result = result;
+ 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();
+ if (s.ok()) {
+ bufs_[index].async_read_in_progress_ = true;
+ }
+ return s;
+}
+
+Status FilePrefetchBuffer::Prefetch(const IOOptions& opts,
+ RandomAccessFileReader* reader,
+ uint64_t offset, size_t n,
+ Env::IOPriority rate_limiter_priority) {
+ if (!enable_ || reader == nullptr) {
+ return Status::OK();
+ }
+ TEST_SYNC_POINT("FilePrefetchBuffer::Prefetch:Start");
+
+ 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);
+
+ 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;
+ if (IsDataBlockInBuffer(offset, length, src)) {
+ // 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();
+ }
+}
+
+// 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) {
+ uint32_t second = curr_ ^ 1;
+ 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());
+ }
+
+ 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_ = false;
+ }
+}
+
+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_);
+ }
+ }
+ if (!handles.empty()) {
+ StopWatch sw(clock_, stats_, ASYNC_PREFETCH_ABORT_MICROS);
+ Status s = fs_->AbortIO(handles);
+ assert(s.ok());
+ }
+
+ // Release io_handles.
+ if (bufs_[curr_].io_handle_ != nullptr && bufs_[curr_].del_fn_ != nullptr) {
+ DestroyAndClearIOHandle(curr_);
+ } else {
+ bufs_[curr_].async_read_in_progress_ = false;
+ }
+
+ if (bufs_[second].io_handle_ != nullptr && bufs_[second].del_fn_ != nullptr) {
+ DestroyAndClearIOHandle(second);
+ } else {
+ bufs_[second].async_read_in_progress_ = false;
+ }
+}
+
+// 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();
+ }
+
+ {
+ // In case buffers do not align, reset second buffer. This can happen in
+ // case readahead_size is set.
+ if (!bufs_[second].async_read_in_progress_ &&
+ !bufs_[curr_].async_read_in_progress_) {
+ if (DoesBufferContainData(curr_)) {
+ if (bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize() !=
+ bufs_[second].offset_) {
+ bufs_[second].buffer_.Clear();
+ }
+ } else {
+ if (!IsOffsetInBuffer(offset, second)) {
+ bufs_[second].buffer_.Clear();
+ }
+ }
+ }
+ }
+
+ // If data starts from second buffer, make it curr_. Second buffer can be
+ // either partial filled, full or async read is in progress.
+ if (bufs_[second].async_read_in_progress_) {
+ if (IsOffsetInBufferWithAsyncProgress(offset, second)) {
+ curr_ = curr_ ^ 1;
+ }
+ } else {
+ if (DoesBufferContainData(second) && IsOffsetInBuffer(offset, second)) {
+ assert(bufs_[curr_].async_read_in_progress_ ||
+ bufs_[curr_].buffer_.CurrentSize() == 0);
+ curr_ = curr_ ^ 1;
+ }
+ }
+}
+
+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);
+}
+
+Status FilePrefetchBuffer::HandleOverlappingData(
+ 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,
+ uint64_t& tmp_offset, size_t& tmp_length) {
+ Status s;
+ size_t alignment = reader->file()->GetRequiredBufferAlignment();
+ uint32_t second;
+
+ // Check if the first buffer has the required offset and the async read is
+ // still in progress. This should only happen if a prefetch was initiated
+ // by Seek, but the next access is at another offset.
+ if (bufs_[curr_].async_read_in_progress_ &&
+ IsOffsetInBufferWithAsyncProgress(offset, curr_)) {
+ PollAndUpdateBuffersIfNeeded(offset);
+ }
+ second = curr_ ^ 1;
+
+ // 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)))) {
+ // 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;
+
+ 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;
+ }
+ return s;
+}
+// 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.
+//
+// Scenarios for prefetching asynchronously:
+// Case1: If both buffers are empty, prefetch n + readahead_size_/2 bytes
+// synchronously in curr_ and prefetch readahead_size_/2 async in second
+// buffer.
+// 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.
+// 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.
+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");
+
+ size_t alignment = reader->file()->GetRequiredBufferAlignment();
+ Status s;
+ uint64_t tmp_offset = offset;
+ size_t tmp_length = length;
+
+ // 1. Abort IO and swap buffers if needed to point curr_ to first buffer with
+ // data.
+ if (!explicit_prefetch_submitted_) {
+ AbortIOIfNeeded(offset);
+ }
+ UpdateBuffersIfNeeded(offset);
+
+ // 2. Handle overlapping data over two buffers. If data is overlapping then
+ // during this call:
+ // - data from curr_ is copied into third buffer,
+ // - curr_ is send for async prefetching of further data if second buffer
+ // contains remaining requested data or in progress for async prefetch,
+ // - switch buffers and curr_ now points to second buffer to copy remaining
+ // data.
+ s = HandleOverlappingData(opts, reader, offset, length, readahead_size,
+ rate_limiter_priority, copy_to_third_buffer,
+ tmp_offset, tmp_length);
+ if (!s.ok()) {
+ return s;
+ }
+
+ // 3. Call Poll only if data is needed for the second buffer.
+ // - Return if whole data is in curr_ and second buffer is in progress or
+ // already full.
+ // - 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);
+ if (!IsSecondBuffEligibleForPrefetching()) {
+ return s;
+ }
+ } else {
+ // After poll request, curr_ might be empty because of IOError in
+ // callback while reading or may contain required data.
+ PollAndUpdateBuffersIfNeeded(offset);
+ }
+
+ 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;
+ }
+ if (!IsSecondBuffEligibleForPrefetching()) {
+ return s;
+ }
+ }
+
+ uint32_t second = curr_ ^ 1;
+ assert(!bufs_[curr_].async_read_in_progress_);
+
+ // In case because of some IOError curr_ got empty, abort IO for second as
+ // well. Otherwise data might not align if more data needs to be read in curr_
+ // which might overlap with second buffer.
+ if (!DoesBufferContainData(curr_) && bufs_[second].async_read_in_progress_) {
+ 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();
+ }
+
+ // 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 && DoesBufferContainData(curr_)) {
+ 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.
+ // 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.
+ if (length == 0) {
+ return s;
+ }
+ }
+
+ // 6. Go for ReadAsync and Read (if needed).
+ size_t prefetch_size = length + readahead_size;
+ 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;
+
+ assert(!bufs_[second].async_read_in_progress_ &&
+ !DoesBufferContainData(second));
+
+ // 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);
+ assert(roundup_len1 >= chunk_len1);
+ read_len1 = static_cast<size_t>(roundup_len1 - chunk_len1);
+ }
+ {
+ // 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);
+
+ // For length == 0, do the asynchronous prefetching in second instead of
+ // synchronous prefetching in curr_.
+ 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);
+ assert(chunk_len2 == 0);
+ // Update the buffer offset.
+ bufs_[second].offset_ = rounddown_start2;
+ assert(roundup_len2 >= chunk_len2);
+ uint64_t read_len2 = static_cast<size_t>(roundup_len2 - chunk_len2);
+ Status tmp_s = ReadAsync(opts, reader, read_len2, rounddown_start2, second);
+ if (!tmp_s.ok()) {
+ DestroyAndClearIOHandle(second);
+ bufs_[second].buffer_.Clear();
+ }
+ }
+
+ if (read_len1 > 0) {
+ s = Read(opts, reader, rate_limiter_priority, read_len1, chunk_len1,
+ rounddown_start1, curr_);
+ if (!s.ok()) {
+ 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();
+ 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,
+ 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);
+ }
+ 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.
+ TEST_SYNC_POINT_CALLBACK("FilePrefetchBuffer::TryReadFromCache",
+ &readahead_size_);
+ if (offset + n > bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize()) {
+ if (readahead_size_ > 0) {
+ Status s;
+ assert(reader != nullptr);
+ assert(max_readahead_size_ >= readahead_size_);
+ if (for_compaction) {
+ s = Prefetch(opts, reader, offset, std::max(n, readahead_size_),
+ rate_limiter_priority);
+ } else {
+ if (implicit_auto_readahead_) {
+ if (!IsEligibleForPrefetch(offset, n)) {
+ // Ignore status as Prefetch is not called.
+ s.PermitUncheckedError();
+ return false;
+ }
+ }
+ 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,
+ Env::IOPriority rate_limiter_priority) {
+ if (track_min_offset_ && offset < min_offset_read_) {
+ min_offset_read_ = static_cast<size_t>(offset);
+ }
+
+ if (!enable_) {
+ return false;
+ }
+
+ if (explicit_prefetch_submitted_) {
+ // explicit_prefetch_submitted_ is special case where it expects request
+ // submitted in PrefetchAsync should match with this request. Otherwise
+ // buffers will be outdated.
+ // Random offset called. So abort the IOs.
+ if (prev_offset_ != offset) {
+ AbortAllIOs();
+ bufs_[curr_].buffer_.Clear();
+ bufs_[curr_ ^ 1].buffer_.Clear();
+ explicit_prefetch_submitted_ = false;
+ return false;
+ }
+ }
+
+ if (!explicit_prefetch_submitted_ && offset < bufs_[curr_].offset_) {
+ 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_);
+
+ if (explicit_prefetch_submitted_ ||
+ (bufs_[curr_].async_read_in_progress_ ||
+ offset + n >
+ bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize())) {
+ if (readahead_size_ > 0) {
+ Status s;
+ assert(reader != nullptr);
+ assert(max_readahead_size_ >= readahead_size_);
+
+ if (implicit_auto_readahead_) {
+ if (!IsEligibleForPrefetch(offset, n)) {
+ // Ignore status as Prefetch is not called.
+ s.PermitUncheckedError();
+ return false;
+ }
+ }
+ // 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);
+ explicit_prefetch_submitted_ = false;
+ if (!s.ok()) {
+ if (status) {
+ *status = s;
+ }
+#ifndef NDEBUG
+ IGNORE_STATUS_IF_ERROR(s);
+#endif
+ return false;
+ }
+ prefetched = explicit_prefetch_submitted_ ? false : true;
+ } else {
+ return false;
+ }
+ }
+
+ 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;
+}
+
+void FilePrefetchBuffer::PrefetchAsyncCallback(const FSReadRequest& req,
+ void* cb_arg) {
+ uint32_t index = *(static_cast<uint32_t*>(cb_arg));
+#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
+
+ 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.
+ 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());
+ }
+}
+
+Status FilePrefetchBuffer::PrefetchAsync(const IOOptions& opts,
+ RandomAccessFileReader* reader,
+ uint64_t offset, size_t n,
+ Slice* result) {
+ assert(reader != nullptr);
+ if (!enable_) {
+ return Status::NotSupported();
+ }
+
+ TEST_SYNC_POINT("FilePrefetchBuffer::PrefetchAsync:Start");
+
+ 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;
+ }
+
+ // 1. Cancel any pending async read to make code simpler as buffers can be out
+ // of sync.
+ AbortAllIOs();
+
+ // 2. Clear outdated data.
+ UpdateBuffersIfNeeded(offset);
+ uint32_t second = curr_ ^ 1;
+ // Since PrefetchAsync can be called on non sequential reads. So offset can
+ // be less than curr_ buffers' offset. In that case also it clears both
+ // buffers.
+ if (DoesBufferContainData(curr_) && !IsOffsetInBuffer(offset, curr_)) {
+ bufs_[curr_].buffer_.Clear();
+ bufs_[second].buffer_.Clear();
+ }
+
+ UpdateReadPattern(offset, n, /*decrease_readaheadsize=*/false);
+
+ bool data_found = false;
+
+ // 3. If curr_ has full data.
+ if (DoesBufferContainData(curr_) && IsDataBlockInBuffer(offset, n, curr_)) {
+ uint64_t offset_in_buffer = offset - bufs_[curr_].offset_;
+ *result = Slice(bufs_[curr_].buffer_.BufferStart() + offset_in_buffer, n);
+ 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();
+ }
+ bufs_[second].buffer_.Clear();
+
+ Status s;
+ size_t alignment = reader->file()->GetRequiredBufferAlignment();
+ size_t prefetch_size = is_eligible_for_prefetching ? readahead_size_ / 2 : 0;
+ size_t offset_to_read = static_cast<size_t>(offset);
+ 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;
+ }
+
+ if (is_eligible_for_prefetching) {
+ if (DoesBufferContainData(curr_)) {
+ rounddown_start2 =
+ bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize();
+ } else {
+ rounddown_start2 = roundup_end1;
+ }
+
+ roundup_end2 = Roundup(rounddown_start2 + prefetch_size, alignment);
+ uint64_t roundup_len2 = roundup_end2 - rounddown_start2;
+
+ 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;
+ }
+
+ 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());
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/file_prefetch_buffer.h b/src/rocksdb/file/file_prefetch_buffer.h
new file mode 100644
index 000000000..a4a75fe2b
--- /dev/null
+++ b/src/rocksdb/file/file_prefetch_buffer.h
@@ -0,0 +1,446 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <algorithm>
+#include <atomic>
+#include <sstream>
+#include <string>
+
+#include "file/readahead_file_info.h"
+#include "monitoring/statistics.h"
+#include "port/port.h"
+#include "rocksdb/env.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/options.h"
+#include "util/aligned_buffer.h"
+#include "util/autovector.h"
+#include "util/stop_watch.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+#define DEFAULT_DECREMENT 8 * 1024
+
+struct IOOptions;
+class RandomAccessFileReader;
+
+struct BufferInfo {
+ AlignedBuffer buffer_;
+
+ uint64_t offset_ = 0;
+
+ // Below parameters are used in case of async read flow.
+ // Length requested for in ReadAsync.
+ size_t async_req_len_ = 0;
+
+ // async_read_in_progress can be used as mutex. Callback can update the buffer
+ // and its size but async_read_in_progress is only set by main thread.
+ bool async_read_in_progress_ = false;
+
+ // io_handle is allocated and used by underlying file system in case of
+ // asynchronous reads.
+ void* io_handle_ = nullptr;
+
+ IOHandleDeleter del_fn_ = nullptr;
+
+ // pos represents the index of this buffer in vector of BufferInfo.
+ uint32_t pos_ = 0;
+};
+
+// FilePrefetchBuffer is a smart buffer to store and read data from a file.
+class FilePrefetchBuffer {
+ public:
+ // Constructor.
+ //
+ // All arguments are optional.
+ // readahead_size : the initial readahead size.
+ // max_readahead_size : the maximum readahead size.
+ // If max_readahead_size > readahead_size, the readahead size will be
+ // doubled on every IO until max_readahead_size is hit.
+ // Typically this is set as a multiple of readahead_size.
+ // max_readahead_size should be greater than equal to readahead_size.
+ // enable : controls whether reading from the buffer is enabled.
+ // If false, TryReadFromCache() always return false, and we only take stats
+ // for the minimum offset if track_min_offset = true.
+ // track_min_offset : Track the minimum offset ever read and collect stats on
+ // it. Used for adaptable readahead of the file footer/metadata.
+ // implicit_auto_readahead : Readahead is enabled implicitly by rocksdb after
+ // doing sequential scans for two times.
+ //
+ // Automatic readhead is enabled for a file if readahead_size
+ // and max_readahead_size are passed in.
+ // A user can construct a FilePrefetchBuffer without any arguments, but use
+ // `Prefetch` to load data into the buffer.
+ FilePrefetchBuffer(size_t readahead_size = 0, size_t max_readahead_size = 0,
+ bool enable = true, bool track_min_offset = false,
+ bool implicit_auto_readahead = false,
+ uint64_t num_file_reads = 0,
+ uint64_t num_file_reads_for_auto_readahead = 0,
+ FileSystem* fs = nullptr, SystemClock* clock = nullptr,
+ Statistics* stats = nullptr)
+ : curr_(0),
+ readahead_size_(readahead_size),
+ initial_auto_readahead_size_(readahead_size),
+ max_readahead_size_(max_readahead_size),
+ min_offset_read_(std::numeric_limits<size_t>::max()),
+ enable_(enable),
+ track_min_offset_(track_min_offset),
+ implicit_auto_readahead_(implicit_auto_readahead),
+ prev_offset_(0),
+ prev_len_(0),
+ num_file_reads_for_auto_readahead_(num_file_reads_for_auto_readahead),
+ num_file_reads_(num_file_reads),
+ explicit_prefetch_submitted_(false),
+ fs_(fs),
+ clock_(clock),
+ stats_(stats) {
+ assert((num_file_reads_ >= num_file_reads_for_auto_readahead_ + 1) ||
+ (num_file_reads_ == 0));
+ // If ReadOptions.async_io is enabled, data is asynchronously filled in
+ // second buffer while curr_ is being consumed. If data is overlapping in
+ // two buffers, data is copied to third buffer to return continuous buffer.
+ bufs_.resize(3);
+ for (uint32_t i = 0; i < 2; i++) {
+ bufs_[i].pos_ = i;
+ }
+ }
+
+ ~FilePrefetchBuffer() {
+ // Abort any pending async read request before destroying the class object.
+ if (fs_ != nullptr) {
+ 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_);
+ }
+ }
+ if (!handles.empty()) {
+ StopWatch sw(clock_, stats_, ASYNC_PREFETCH_ABORT_MICROS);
+ Status s = fs_->AbortIO(handles);
+ assert(s.ok());
+ }
+ }
+
+ // Prefetch buffer bytes discarded.
+ uint64_t bytes_discarded = 0;
+ // Iterated over 2 buffers.
+ for (int i = 0; i < 2; i++) {
+ int first = i;
+ int second = i ^ 1;
+
+ if (DoesBufferContainData(first)) {
+ // If last block was read completely from first and some bytes in
+ // first buffer are still unconsumed.
+ if (prev_offset_ >= bufs_[first].offset_ &&
+ prev_offset_ + prev_len_ <
+ bufs_[first].offset_ + bufs_[first].buffer_.CurrentSize()) {
+ bytes_discarded += bufs_[first].buffer_.CurrentSize() -
+ (prev_offset_ + prev_len_ - bufs_[first].offset_);
+ }
+ // If data was in second buffer and some/whole block bytes were read
+ // from second buffer.
+ else if (prev_offset_ < bufs_[first].offset_ &&
+ !DoesBufferContainData(second)) {
+ // If last block read was completely from different buffer, this
+ // buffer is unconsumed.
+ if (prev_offset_ + prev_len_ <= bufs_[first].offset_) {
+ bytes_discarded += bufs_[first].buffer_.CurrentSize();
+ }
+ // If last block read overlaps with this buffer and some data is
+ // still unconsumed and previous buffer (second) is not cleared.
+ else if (prev_offset_ + prev_len_ > bufs_[first].offset_ &&
+ bufs_[first].offset_ + bufs_[first].buffer_.CurrentSize() ==
+ bufs_[second].offset_) {
+ bytes_discarded += bufs_[first].buffer_.CurrentSize() -
+ (/*bytes read from this buffer=*/prev_len_ -
+ (bufs_[first].offset_ - prev_offset_));
+ }
+ }
+ }
+ }
+
+ for (uint32_t i = 0; i < 2; i++) {
+ // Release io_handle.
+ DestroyAndClearIOHandle(i);
+ }
+ RecordInHistogram(stats_, PREFETCHED_BYTES_DISCARDED, bytes_discarded);
+ }
+
+ // Load data into the buffer from a file.
+ // reader : the file reader.
+ // offset : the file offset to start reading from.
+ // n : the number of bytes to read.
+ // rate_limiter_priority : rate limiting priority, or `Env::IO_TOTAL` to
+ // bypass.
+ Status Prefetch(const IOOptions& opts, RandomAccessFileReader* reader,
+ uint64_t offset, size_t n,
+ Env::IOPriority rate_limiter_priority);
+
+ // Request for reading the data from a file asynchronously.
+ // If data already exists in the buffer, result will be updated.
+ // reader : the file reader.
+ // offset : the file offset to start reading from.
+ // n : the number of bytes to read.
+ // result : if data already exists in the buffer, result will
+ // be updated with the data.
+ //
+ // If data already exist in the buffer, it will return Status::OK, otherwise
+ // it will send asynchronous request and return Status::TryAgain.
+ Status PrefetchAsync(const IOOptions& opts, RandomAccessFileReader* reader,
+ uint64_t offset, size_t n, Slice* result);
+
+ // Tries returning the data for a file read from this buffer if that data is
+ // in the buffer.
+ // It handles tracking the minimum read offset if track_min_offset = true.
+ // It also does the exponential readahead when readahead_size is set as part
+ // of the constructor.
+ //
+ // opts : the IO options to use.
+ // reader : the file reader.
+ // offset : the file offset.
+ // n : the number of bytes.
+ // result : output buffer to put the data into.
+ // s : output status.
+ // rate_limiter_priority : rate limiting priority, or `Env::IO_TOTAL` to
+ // bypass.
+ // for_compaction : true if cache read is done for compaction read.
+ bool TryReadFromCache(const IOOptions& opts, RandomAccessFileReader* reader,
+ uint64_t offset, size_t n, Slice* result, Status* s,
+ Env::IOPriority rate_limiter_priority,
+ bool for_compaction = false);
+
+ bool TryReadFromCacheAsync(const IOOptions& opts,
+ RandomAccessFileReader* reader, uint64_t offset,
+ size_t n, Slice* result, Status* status,
+ Env::IOPriority rate_limiter_priority);
+
+ // The minimum `offset` ever passed to TryReadFromCache(). This will nly be
+ // tracked if track_min_offset = true.
+ size_t min_offset_read() const { return min_offset_read_; }
+
+ // Called in case of implicit auto prefetching.
+ void UpdateReadPattern(const uint64_t& offset, const size_t& len,
+ bool decrease_readaheadsize) {
+ if (decrease_readaheadsize) {
+ // Since this block was eligible for prefetch but it was found in
+ // cache, so check and decrease the readahead_size by 8KB (default)
+ // if eligible.
+ DecreaseReadAheadIfEligible(offset, len);
+ }
+ prev_offset_ = offset;
+ prev_len_ = len;
+ explicit_prefetch_submitted_ = false;
+ }
+
+ void GetReadaheadState(ReadaheadFileInfo::ReadaheadInfo* readahead_info) {
+ readahead_info->readahead_size = readahead_size_;
+ readahead_info->num_file_reads = num_file_reads_;
+ }
+
+ void DecreaseReadAheadIfEligible(uint64_t offset, size_t size,
+ size_t value = DEFAULT_DECREMENT) {
+ // Decrease the readahead_size if
+ // - its enabled internally by RocksDB (implicit_auto_readahead_) and,
+ // - readahead_size is greater than 0 and,
+ // - this block would have called prefetch API if not found in cache for
+ // which conditions are:
+ // - few/no bytes are in buffer and,
+ // - block is sequential with the previous read and,
+ // - num_file_reads_ + 1 (including this read) >
+ // num_file_reads_for_auto_readahead_
+ size_t curr_size = bufs_[curr_].async_read_in_progress_
+ ? bufs_[curr_].async_req_len_
+ : bufs_[curr_].buffer_.CurrentSize();
+ if (implicit_auto_readahead_ && readahead_size_ > 0) {
+ if ((offset + size > bufs_[curr_].offset_ + curr_size) &&
+ IsBlockSequential(offset) &&
+ (num_file_reads_ + 1 > num_file_reads_for_auto_readahead_)) {
+ readahead_size_ =
+ std::max(initial_auto_readahead_size_,
+ (readahead_size_ >= value ? readahead_size_ - value : 0));
+ }
+ }
+ }
+
+ // Callback function passed to underlying FS in case of asynchronous reads.
+ void PrefetchAsyncCallback(const FSReadRequest& req, void* cb_arg);
+
+ private:
+ // Calculates roundoff offset and length to be prefetched based on alignment
+ // and data present in buffer_. It also allocates new buffer or refit tail if
+ // required.
+ void CalculateOffsetAndLen(size_t alignment, uint64_t offset,
+ size_t roundup_len, uint32_t index,
+ bool refit_tail, uint64_t& chunk_len);
+
+ void AbortIOIfNeeded(uint64_t offset);
+
+ void AbortAllIOs();
+
+ void UpdateBuffersIfNeeded(uint64_t offset);
+
+ // It calls Poll API if any there is any pending asynchronous request. It then
+ // checks if data is in any buffer. It clears the outdated data and swaps the
+ // buffers if required.
+ void PollAndUpdateBuffersIfNeeded(uint64_t offset);
+
+ Status PrefetchAsyncInternal(const IOOptions& opts,
+ RandomAccessFileReader* reader, uint64_t offset,
+ size_t length, size_t readahead_size,
+ Env::IOPriority rate_limiter_priority,
+ bool& copy_to_third_buffer);
+
+ Status Read(const IOOptions& opts, RandomAccessFileReader* reader,
+ Env::IOPriority rate_limiter_priority, uint64_t read_len,
+ uint64_t chunk_len, uint64_t rounddown_start, uint32_t index);
+
+ Status ReadAsync(const IOOptions& opts, RandomAccessFileReader* reader,
+ uint64_t read_len, uint64_t rounddown_start, uint32_t index);
+
+ // Copy the data from src to third buffer.
+ void CopyDataToBuffer(uint32_t src, uint64_t& offset, size_t& length);
+
+ bool IsBlockSequential(const size_t& offset) {
+ return (prev_len_ == 0 || (prev_offset_ + prev_len_ == offset));
+ }
+
+ // Called in case of implicit auto prefetching.
+ void ResetValues() {
+ num_file_reads_ = 1;
+ readahead_size_ = initial_auto_readahead_size_;
+ }
+
+ // Called in case of implicit auto prefetching.
+ bool IsEligibleForPrefetch(uint64_t offset, size_t n) {
+ // Prefetch only if this read is sequential otherwise reset readahead_size_
+ // to initial value.
+ if (!IsBlockSequential(offset)) {
+ UpdateReadPattern(offset, n, false /*decrease_readaheadsize*/);
+ ResetValues();
+ return false;
+ }
+ num_file_reads_++;
+
+ // Since async request was submitted in last call directly by calling
+ // PrefetchAsync, it skips num_file_reads_ check as this call is to poll the
+ // data submitted in previous call.
+ if (explicit_prefetch_submitted_) {
+ return true;
+ }
+ if (num_file_reads_ <= num_file_reads_for_auto_readahead_) {
+ UpdateReadPattern(offset, n, false /*decrease_readaheadsize*/);
+ return false;
+ }
+ return true;
+ }
+
+ // Helper functions.
+ bool IsDataBlockInBuffer(uint64_t offset, size_t length, uint32_t index) {
+ return (offset >= bufs_[index].offset_ &&
+ offset + length <=
+ bufs_[index].offset_ + bufs_[index].buffer_.CurrentSize());
+ }
+ bool IsOffsetInBuffer(uint64_t offset, uint32_t index) {
+ return (offset >= bufs_[index].offset_ &&
+ offset < bufs_[index].offset_ + bufs_[index].buffer_.CurrentSize());
+ }
+ bool DoesBufferContainData(uint32_t index) {
+ return bufs_[index].buffer_.CurrentSize() > 0;
+ }
+ bool IsBufferOutdated(uint64_t offset, uint32_t index) {
+ return (
+ !bufs_[index].async_read_in_progress_ && DoesBufferContainData(index) &&
+ offset >= bufs_[index].offset_ + bufs_[index].buffer_.CurrentSize());
+ }
+ bool IsBufferOutdatedWithAsyncProgress(uint64_t offset, uint32_t index) {
+ return (bufs_[index].async_read_in_progress_ &&
+ bufs_[index].io_handle_ != nullptr &&
+ offset >= bufs_[index].offset_ + bufs_[index].async_req_len_);
+ }
+ bool IsOffsetInBufferWithAsyncProgress(uint64_t offset, uint32_t index) {
+ return (bufs_[index].async_read_in_progress_ &&
+ offset >= bufs_[index].offset_ &&
+ offset < bufs_[index].offset_ + bufs_[index].async_req_len_);
+ }
+
+ bool IsSecondBuffEligibleForPrefetching() {
+ uint32_t second = curr_ ^ 1;
+ if (bufs_[second].async_read_in_progress_) {
+ return false;
+ }
+ assert(!bufs_[curr_].async_read_in_progress_);
+
+ if (DoesBufferContainData(curr_) && DoesBufferContainData(second) &&
+ (bufs_[curr_].offset_ + bufs_[curr_].buffer_.CurrentSize() ==
+ bufs_[second].offset_)) {
+ return false;
+ }
+ bufs_[second].buffer_.Clear();
+ return true;
+ }
+
+ void DestroyAndClearIOHandle(uint32_t index) {
+ if (bufs_[index].io_handle_ != nullptr && bufs_[index].del_fn_ != nullptr) {
+ bufs_[index].del_fn_(bufs_[index].io_handle_);
+ bufs_[index].io_handle_ = nullptr;
+ bufs_[index].del_fn_ = nullptr;
+ }
+ bufs_[index].async_read_in_progress_ = false;
+ }
+
+ Status HandleOverlappingData(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, uint64_t& tmp_offset,
+ size_t& tmp_length);
+
+ std::vector<BufferInfo> bufs_;
+ // curr_ represents the index for bufs_ indicating which buffer is being
+ // consumed currently.
+ uint32_t curr_;
+
+ size_t readahead_size_;
+ size_t initial_auto_readahead_size_;
+ // FilePrefetchBuffer object won't be created from Iterator flow if
+ // max_readahead_size_ = 0.
+ size_t max_readahead_size_;
+
+ // The minimum `offset` ever passed to TryReadFromCache().
+ size_t min_offset_read_;
+ // if false, TryReadFromCache() always return false, and we only take stats
+ // for track_min_offset_ if track_min_offset_ = true
+ bool enable_;
+ // If true, track minimum `offset` ever passed to TryReadFromCache(), which
+ // can be fetched from min_offset_read().
+ bool track_min_offset_;
+
+ // implicit_auto_readahead is enabled by rocksdb internally after 2
+ // sequential IOs.
+ bool implicit_auto_readahead_;
+ uint64_t prev_offset_;
+ size_t prev_len_;
+ // num_file_reads_ and num_file_reads_for_auto_readahead_ is only used when
+ // implicit_auto_readahead_ is set.
+ uint64_t num_file_reads_for_auto_readahead_;
+ uint64_t num_file_reads_;
+
+ // If explicit_prefetch_submitted_ is set then it indicates RocksDB called
+ // PrefetchAsync to submit request. It needs to call TryReadFromCacheAsync to
+ // poll the submitted request without checking if data is sequential and
+ // num_file_reads_.
+ bool explicit_prefetch_submitted_;
+
+ FileSystem* fs_;
+ SystemClock* clock_;
+ Statistics* stats_;
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/file_util.cc b/src/rocksdb/file/file_util.cc
new file mode 100644
index 000000000..7997d6e11
--- /dev/null
+++ b/src/rocksdb/file/file_util.cc
@@ -0,0 +1,282 @@
+// 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).
+//
+#include "file/file_util.h"
+
+#include <algorithm>
+#include <string>
+
+#include "file/random_access_file_reader.h"
+#include "file/sequence_file_reader.h"
+#include "file/sst_file_manager_impl.h"
+#include "file/writable_file_writer.h"
+#include "rocksdb/env.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Utility function to copy a file up to a specified length
+IOStatus CopyFile(FileSystem* fs, const std::string& source,
+ std::unique_ptr<WritableFileWriter>& dest_writer,
+ uint64_t size, bool use_fsync,
+ const std::shared_ptr<IOTracer>& io_tracer,
+ const Temperature temperature) {
+ FileOptions soptions;
+ IOStatus io_s;
+ std::unique_ptr<SequentialFileReader> src_reader;
+
+ {
+ soptions.temperature = temperature;
+ std::unique_ptr<FSSequentialFile> srcfile;
+ io_s = fs->NewSequentialFile(source, soptions, &srcfile, nullptr);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+
+ if (size == 0) {
+ // default argument means copy everything
+ io_s = fs->GetFileSize(source, IOOptions(), &size, nullptr);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ }
+ src_reader.reset(
+ new SequentialFileReader(std::move(srcfile), source, io_tracer));
+ }
+
+ char buffer[4096];
+ Slice slice;
+ while (size > 0) {
+ size_t bytes_to_read = std::min(sizeof(buffer), static_cast<size_t>(size));
+ // TODO: rate limit copy file
+ io_s = status_to_io_status(
+ src_reader->Read(bytes_to_read, &slice, buffer,
+ Env::IO_TOTAL /* rate_limiter_priority */));
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ if (slice.size() == 0) {
+ return IOStatus::Corruption("file too small");
+ }
+ io_s = dest_writer->Append(slice);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ size -= slice.size();
+ }
+ return dest_writer->Sync(use_fsync);
+}
+
+IOStatus CopyFile(FileSystem* fs, const std::string& source,
+ const std::string& destination, uint64_t size, bool use_fsync,
+ const std::shared_ptr<IOTracer>& io_tracer,
+ const Temperature temperature) {
+ FileOptions options;
+ IOStatus io_s;
+ std::unique_ptr<WritableFileWriter> dest_writer;
+
+ {
+ options.temperature = temperature;
+ std::unique_ptr<FSWritableFile> destfile;
+ io_s = fs->NewWritableFile(destination, options, &destfile, nullptr);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+
+ dest_writer.reset(
+ new WritableFileWriter(std::move(destfile), destination, options));
+ }
+
+ return CopyFile(fs, source, dest_writer, size, use_fsync, io_tracer,
+ temperature);
+}
+
+// Utility function to create a file with the provided contents
+IOStatus CreateFile(FileSystem* fs, const std::string& destination,
+ const std::string& contents, bool use_fsync) {
+ const EnvOptions soptions;
+ IOStatus io_s;
+ std::unique_ptr<WritableFileWriter> dest_writer;
+
+ std::unique_ptr<FSWritableFile> destfile;
+ io_s = fs->NewWritableFile(destination, soptions, &destfile, nullptr);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ dest_writer.reset(
+ new WritableFileWriter(std::move(destfile), destination, soptions));
+ io_s = dest_writer->Append(Slice(contents));
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ return dest_writer->Sync(use_fsync);
+}
+
+Status DeleteDBFile(const ImmutableDBOptions* db_options,
+ const std::string& fname, const std::string& dir_to_sync,
+ const bool force_bg, const bool force_fg) {
+#ifndef ROCKSDB_LITE
+ SstFileManagerImpl* sfm =
+ static_cast<SstFileManagerImpl*>(db_options->sst_file_manager.get());
+ if (sfm && !force_fg) {
+ return sfm->ScheduleFileDeletion(fname, dir_to_sync, force_bg);
+ } else {
+ return db_options->env->DeleteFile(fname);
+ }
+#else
+ (void)dir_to_sync;
+ (void)force_bg;
+ (void)force_fg;
+ // SstFileManager is not supported in ROCKSDB_LITE
+ // Delete file immediately
+ return db_options->env->DeleteFile(fname);
+#endif
+}
+
+// requested_checksum_func_name brings the function name of the checksum
+// generator in checksum_factory. Empty string is permitted, in which case the
+// name of the generator created by the factory is unchecked. When
+// `requested_checksum_func_name` is non-empty, however, the created generator's
+// name must match it, otherwise an `InvalidArgument` error is returned.
+IOStatus GenerateOneFileChecksum(
+ FileSystem* fs, const std::string& file_path,
+ FileChecksumGenFactory* checksum_factory,
+ const std::string& requested_checksum_func_name, std::string* file_checksum,
+ std::string* file_checksum_func_name,
+ size_t verify_checksums_readahead_size, bool allow_mmap_reads,
+ std::shared_ptr<IOTracer>& io_tracer, RateLimiter* rate_limiter,
+ Env::IOPriority rate_limiter_priority) {
+ if (checksum_factory == nullptr) {
+ return IOStatus::InvalidArgument("Checksum factory is invalid");
+ }
+ assert(file_checksum != nullptr);
+ assert(file_checksum_func_name != nullptr);
+
+ FileChecksumGenContext gen_context;
+ gen_context.requested_checksum_func_name = requested_checksum_func_name;
+ gen_context.file_name = file_path;
+ std::unique_ptr<FileChecksumGenerator> checksum_generator =
+ checksum_factory->CreateFileChecksumGenerator(gen_context);
+ if (checksum_generator == nullptr) {
+ std::string msg =
+ "Cannot get the file checksum generator based on the requested "
+ "checksum function name: " +
+ requested_checksum_func_name +
+ " from checksum factory: " + checksum_factory->Name();
+ return IOStatus::InvalidArgument(msg);
+ } else {
+ // For backward compatibility and use in file ingestion clients where there
+ // is no stored checksum function name, `requested_checksum_func_name` can
+ // be empty. If we give the requested checksum function name, we expect it
+ // is the same name of the checksum generator.
+ if (!requested_checksum_func_name.empty() &&
+ checksum_generator->Name() != requested_checksum_func_name) {
+ std::string msg = "Expected file checksum generator named '" +
+ requested_checksum_func_name +
+ "', while the factory created one "
+ "named '" +
+ checksum_generator->Name() + "'";
+ return IOStatus::InvalidArgument(msg);
+ }
+ }
+
+ uint64_t size;
+ IOStatus io_s;
+ std::unique_ptr<RandomAccessFileReader> reader;
+ {
+ std::unique_ptr<FSRandomAccessFile> r_file;
+ io_s = fs->NewRandomAccessFile(file_path, FileOptions(), &r_file, nullptr);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ io_s = fs->GetFileSize(file_path, IOOptions(), &size, nullptr);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ reader.reset(new RandomAccessFileReader(std::move(r_file), file_path,
+ nullptr /*Env*/, io_tracer, nullptr,
+ 0, nullptr, rate_limiter));
+ }
+
+ // Found that 256 KB readahead size provides the best performance, based on
+ // experiments, for auto readahead. Experiment data is in PR #3282.
+ size_t default_max_read_ahead_size = 256 * 1024;
+ size_t readahead_size = (verify_checksums_readahead_size != 0)
+ ? verify_checksums_readahead_size
+ : default_max_read_ahead_size;
+
+ FilePrefetchBuffer prefetch_buffer(readahead_size /* readahead_size */,
+ readahead_size /* max_readahead_size */,
+ !allow_mmap_reads /* enable */);
+
+ Slice slice;
+ uint64_t offset = 0;
+ IOOptions opts;
+ while (size > 0) {
+ size_t bytes_to_read =
+ static_cast<size_t>(std::min(uint64_t{readahead_size}, size));
+ if (!prefetch_buffer.TryReadFromCache(
+ opts, reader.get(), offset, bytes_to_read, &slice,
+ nullptr /* status */, rate_limiter_priority,
+ false /* for_compaction */)) {
+ return IOStatus::Corruption("file read failed");
+ }
+ if (slice.size() == 0) {
+ return IOStatus::Corruption("file too small");
+ }
+ checksum_generator->Update(slice.data(), slice.size());
+ size -= slice.size();
+ offset += slice.size();
+ }
+ checksum_generator->Finalize();
+ *file_checksum = checksum_generator->GetChecksum();
+ *file_checksum_func_name = checksum_generator->Name();
+ return IOStatus::OK();
+}
+
+Status DestroyDir(Env* env, const std::string& dir) {
+ Status s;
+ if (env->FileExists(dir).IsNotFound()) {
+ return s;
+ }
+ std::vector<std::string> files_in_dir;
+ s = env->GetChildren(dir, &files_in_dir);
+ if (s.ok()) {
+ for (auto& file_in_dir : files_in_dir) {
+ std::string path = dir + "/" + file_in_dir;
+ bool is_dir = false;
+ s = env->IsDirectory(path, &is_dir);
+ if (s.ok()) {
+ if (is_dir) {
+ s = DestroyDir(env, path);
+ } else {
+ s = env->DeleteFile(path);
+ }
+ } else if (s.IsNotSupported()) {
+ s = Status::OK();
+ }
+ if (!s.ok()) {
+ // IsDirectory, etc. might not report NotFound
+ if (s.IsNotFound() || env->FileExists(path).IsNotFound()) {
+ // Allow files to be deleted externally
+ s = Status::OK();
+ } else {
+ break;
+ }
+ }
+ }
+ }
+
+ if (s.ok()) {
+ s = env->DeleteDir(dir);
+ // DeleteDir might or might not report NotFound
+ if (!s.ok() && (s.IsNotFound() || env->FileExists(dir).IsNotFound())) {
+ // Allow to be deleted externally
+ s = Status::OK();
+ }
+ }
+ return s;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/file_util.h b/src/rocksdb/file/file_util.h
new file mode 100644
index 000000000..d46a7ba0e
--- /dev/null
+++ b/src/rocksdb/file/file_util.h
@@ -0,0 +1,89 @@
+// 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).
+//
+#pragma once
+#include <string>
+
+#include "file/filename.h"
+#include "options/db_options.h"
+#include "rocksdb/env.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/sst_file_writer.h"
+#include "rocksdb/status.h"
+#include "rocksdb/system_clock.h"
+#include "rocksdb/types.h"
+#include "trace_replay/io_tracer.h"
+
+namespace ROCKSDB_NAMESPACE {
+// use_fsync maps to options.use_fsync, which determines the way that
+// the file is synced after copying.
+extern IOStatus CopyFile(FileSystem* fs, const std::string& source,
+ std::unique_ptr<WritableFileWriter>& dest_writer,
+ uint64_t size, bool use_fsync,
+ const std::shared_ptr<IOTracer>& io_tracer,
+ const Temperature temperature);
+extern IOStatus CopyFile(FileSystem* fs, const std::string& source,
+ const std::string& destination, uint64_t size,
+ bool use_fsync,
+ const std::shared_ptr<IOTracer>& io_tracer,
+ const Temperature temperature);
+inline IOStatus CopyFile(const std::shared_ptr<FileSystem>& fs,
+ const std::string& source,
+ const std::string& destination, uint64_t size,
+ bool use_fsync,
+ const std::shared_ptr<IOTracer>& io_tracer,
+ const Temperature temperature) {
+ return CopyFile(fs.get(), source, destination, size, use_fsync, io_tracer,
+ temperature);
+}
+extern IOStatus CreateFile(FileSystem* fs, const std::string& destination,
+ const std::string& contents, bool use_fsync);
+
+inline IOStatus CreateFile(const std::shared_ptr<FileSystem>& fs,
+ const std::string& destination,
+ const std::string& contents, bool use_fsync) {
+ return CreateFile(fs.get(), destination, contents, use_fsync);
+}
+
+extern Status DeleteDBFile(const ImmutableDBOptions* db_options,
+ const std::string& fname,
+ const std::string& path_to_sync, const bool force_bg,
+ const bool force_fg);
+
+extern IOStatus GenerateOneFileChecksum(
+ FileSystem* fs, const std::string& file_path,
+ FileChecksumGenFactory* checksum_factory,
+ const std::string& requested_checksum_func_name, std::string* file_checksum,
+ std::string* file_checksum_func_name,
+ size_t verify_checksums_readahead_size, bool allow_mmap_reads,
+ std::shared_ptr<IOTracer>& io_tracer, RateLimiter* rate_limiter,
+ Env::IOPriority rate_limiter_priority);
+
+inline IOStatus PrepareIOFromReadOptions(const ReadOptions& ro,
+ SystemClock* clock, IOOptions& opts) {
+ if (ro.deadline.count()) {
+ std::chrono::microseconds now =
+ std::chrono::microseconds(clock->NowMicros());
+ // Ensure there is atleast 1us available. We don't want to pass a value of
+ // 0 as that means no timeout
+ if (now >= ro.deadline) {
+ return IOStatus::TimedOut("Deadline exceeded");
+ }
+ opts.timeout = ro.deadline - now;
+ }
+
+ if (ro.io_timeout.count() &&
+ (!opts.timeout.count() || ro.io_timeout < opts.timeout)) {
+ opts.timeout = ro.io_timeout;
+ }
+
+ opts.rate_limiter_priority = ro.rate_limiter_priority;
+ return IOStatus::OK();
+}
+
+// Test method to delete the input directory and all of its contents.
+// This method is destructive and is meant for use only in tests!!!
+Status DestroyDir(Env* env, const std::string& dir);
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/filename.cc b/src/rocksdb/file/filename.cc
new file mode 100644
index 000000000..1e04c7339
--- /dev/null
+++ b/src/rocksdb/file/filename.cc
@@ -0,0 +1,523 @@
+// 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/filename.h"
+
+#include <ctype.h>
+#include <stdio.h>
+
+#include <cinttypes>
+#include <vector>
+
+#include "file/writable_file_writer.h"
+#include "rocksdb/env.h"
+#include "test_util/sync_point.h"
+#include "util/stop_watch.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const std::string kCurrentFileName = "CURRENT";
+const std::string kOptionsFileNamePrefix = "OPTIONS-";
+const std::string kTempFileNameSuffix = "dbtmp";
+
+static const std::string kRocksDbTFileExt = "sst";
+static const std::string kLevelDbTFileExt = "ldb";
+static const std::string kRocksDBBlobFileExt = "blob";
+static const std::string kArchivalDirName = "archive";
+
+// Given a path, flatten the path name by replacing all chars not in
+// {[0-9,a-z,A-Z,-,_,.]} with _. And append '_LOG\0' at the end.
+// Return the number of chars stored in dest not including the trailing '\0'.
+static size_t GetInfoLogPrefix(const std::string& path, char* dest, int len) {
+ const char suffix[] = "_LOG";
+
+ size_t write_idx = 0;
+ size_t i = 0;
+ size_t src_len = path.size();
+
+ while (i < src_len && write_idx < len - sizeof(suffix)) {
+ if ((path[i] >= 'a' && path[i] <= 'z') ||
+ (path[i] >= '0' && path[i] <= '9') ||
+ (path[i] >= 'A' && path[i] <= 'Z') || path[i] == '-' ||
+ path[i] == '.' || path[i] == '_') {
+ dest[write_idx++] = path[i];
+ } else {
+ if (i > 0) {
+ dest[write_idx++] = '_';
+ }
+ }
+ i++;
+ }
+ assert(sizeof(suffix) <= len - write_idx);
+ // "\0" is automatically added by snprintf
+ snprintf(dest + write_idx, len - write_idx, suffix);
+ write_idx += sizeof(suffix) - 1;
+ return write_idx;
+}
+
+static std::string MakeFileName(uint64_t number, const char* suffix) {
+ char buf[100];
+ snprintf(buf, sizeof(buf), "%06llu.%s",
+ static_cast<unsigned long long>(number), suffix);
+ return buf;
+}
+
+static std::string MakeFileName(const std::string& name, uint64_t number,
+ const char* suffix) {
+ return name + "/" + MakeFileName(number, suffix);
+}
+
+std::string LogFileName(const std::string& name, uint64_t number) {
+ assert(number > 0);
+ return MakeFileName(name, number, "log");
+}
+
+std::string LogFileName(uint64_t number) {
+ assert(number > 0);
+ return MakeFileName(number, "log");
+}
+
+std::string BlobFileName(uint64_t number) {
+ assert(number > 0);
+ return MakeFileName(number, kRocksDBBlobFileExt.c_str());
+}
+
+std::string BlobFileName(const std::string& blobdirname, uint64_t number) {
+ assert(number > 0);
+ return MakeFileName(blobdirname, number, kRocksDBBlobFileExt.c_str());
+}
+
+std::string BlobFileName(const std::string& dbname, const std::string& blob_dir,
+ uint64_t number) {
+ assert(number > 0);
+ return MakeFileName(dbname + "/" + blob_dir, number,
+ kRocksDBBlobFileExt.c_str());
+}
+
+std::string ArchivalDirectory(const std::string& dir) {
+ return dir + "/" + kArchivalDirName;
+}
+std::string ArchivedLogFileName(const std::string& name, uint64_t number) {
+ assert(number > 0);
+ return MakeFileName(name + "/" + kArchivalDirName, number, "log");
+}
+
+std::string MakeTableFileName(const std::string& path, uint64_t number) {
+ return MakeFileName(path, number, kRocksDbTFileExt.c_str());
+}
+
+std::string MakeTableFileName(uint64_t number) {
+ return MakeFileName(number, kRocksDbTFileExt.c_str());
+}
+
+std::string Rocks2LevelTableFileName(const std::string& fullname) {
+ assert(fullname.size() > kRocksDbTFileExt.size() + 1);
+ if (fullname.size() <= kRocksDbTFileExt.size() + 1) {
+ return "";
+ }
+ return fullname.substr(0, fullname.size() - kRocksDbTFileExt.size()) +
+ kLevelDbTFileExt;
+}
+
+uint64_t TableFileNameToNumber(const std::string& name) {
+ uint64_t number = 0;
+ uint64_t base = 1;
+ int pos = static_cast<int>(name.find_last_of('.'));
+ while (--pos >= 0 && name[pos] >= '0' && name[pos] <= '9') {
+ number += (name[pos] - '0') * base;
+ base *= 10;
+ }
+ return number;
+}
+
+std::string TableFileName(const std::vector<DbPath>& db_paths, uint64_t number,
+ uint32_t path_id) {
+ assert(number > 0);
+ std::string path;
+ if (path_id >= db_paths.size()) {
+ path = db_paths.back().path;
+ } else {
+ path = db_paths[path_id].path;
+ }
+ return MakeTableFileName(path, number);
+}
+
+void FormatFileNumber(uint64_t number, uint32_t path_id, char* out_buf,
+ size_t out_buf_size) {
+ if (path_id == 0) {
+ snprintf(out_buf, out_buf_size, "%" PRIu64, number);
+ } else {
+ snprintf(out_buf, out_buf_size,
+ "%" PRIu64
+ "(path "
+ "%" PRIu32 ")",
+ number, path_id);
+ }
+}
+
+std::string DescriptorFileName(uint64_t number) {
+ assert(number > 0);
+ char buf[100];
+ snprintf(buf, sizeof(buf), "MANIFEST-%06llu",
+ static_cast<unsigned long long>(number));
+ return buf;
+}
+
+std::string DescriptorFileName(const std::string& dbname, uint64_t number) {
+ return dbname + "/" + DescriptorFileName(number);
+}
+
+std::string CurrentFileName(const std::string& dbname) {
+ return dbname + "/" + kCurrentFileName;
+}
+
+std::string LockFileName(const std::string& dbname) { return dbname + "/LOCK"; }
+
+std::string TempFileName(const std::string& dbname, uint64_t number) {
+ return MakeFileName(dbname, number, kTempFileNameSuffix.c_str());
+}
+
+InfoLogPrefix::InfoLogPrefix(bool has_log_dir,
+ const std::string& db_absolute_path) {
+ if (!has_log_dir) {
+ const char kInfoLogPrefix[] = "LOG";
+ // "\0" is automatically added to the end
+ snprintf(buf, sizeof(buf), kInfoLogPrefix);
+ prefix = Slice(buf, sizeof(kInfoLogPrefix) - 1);
+ } else {
+ size_t len =
+ GetInfoLogPrefix(NormalizePath(db_absolute_path), buf, sizeof(buf));
+ prefix = Slice(buf, len);
+ }
+}
+
+std::string InfoLogFileName(const std::string& dbname,
+ const std::string& db_path,
+ const std::string& log_dir) {
+ if (log_dir.empty()) {
+ return dbname + "/LOG";
+ }
+
+ InfoLogPrefix info_log_prefix(true, db_path);
+ return log_dir + "/" + info_log_prefix.buf;
+}
+
+// Return the name of the old info log file for "dbname".
+std::string OldInfoLogFileName(const std::string& dbname, uint64_t ts,
+ const std::string& db_path,
+ const std::string& log_dir) {
+ char buf[50];
+ snprintf(buf, sizeof(buf), "%llu", static_cast<unsigned long long>(ts));
+
+ if (log_dir.empty()) {
+ return dbname + "/LOG.old." + buf;
+ }
+
+ InfoLogPrefix info_log_prefix(true, db_path);
+ return log_dir + "/" + info_log_prefix.buf + ".old." + buf;
+}
+
+std::string OptionsFileName(uint64_t file_num) {
+ char buffer[256];
+ snprintf(buffer, sizeof(buffer), "%s%06" PRIu64,
+ kOptionsFileNamePrefix.c_str(), file_num);
+ return buffer;
+}
+std::string OptionsFileName(const std::string& dbname, uint64_t file_num) {
+ return dbname + "/" + OptionsFileName(file_num);
+}
+
+std::string TempOptionsFileName(const std::string& dbname, uint64_t file_num) {
+ char buffer[256];
+ snprintf(buffer, sizeof(buffer), "%s%06" PRIu64 ".%s",
+ kOptionsFileNamePrefix.c_str(), file_num,
+ kTempFileNameSuffix.c_str());
+ return dbname + "/" + buffer;
+}
+
+std::string MetaDatabaseName(const std::string& dbname, uint64_t number) {
+ char buf[100];
+ snprintf(buf, sizeof(buf), "/METADB-%llu",
+ static_cast<unsigned long long>(number));
+ return dbname + buf;
+}
+
+std::string IdentityFileName(const std::string& dbname) {
+ return dbname + "/IDENTITY";
+}
+
+// Owned filenames have the form:
+// dbname/IDENTITY
+// dbname/CURRENT
+// dbname/LOCK
+// dbname/<info_log_name_prefix>
+// dbname/<info_log_name_prefix>.old.[0-9]+
+// dbname/MANIFEST-[0-9]+
+// dbname/[0-9]+.(log|sst|blob)
+// dbname/METADB-[0-9]+
+// dbname/OPTIONS-[0-9]+
+// dbname/OPTIONS-[0-9]+.dbtmp
+// Disregards / at the beginning
+bool ParseFileName(const std::string& fname, uint64_t* number, FileType* type,
+ WalFileType* log_type) {
+ return ParseFileName(fname, number, "", type, log_type);
+}
+
+bool ParseFileName(const std::string& fname, uint64_t* number,
+ const Slice& info_log_name_prefix, FileType* type,
+ WalFileType* log_type) {
+ Slice rest(fname);
+ if (fname.length() > 1 && fname[0] == '/') {
+ rest.remove_prefix(1);
+ }
+ if (rest == "IDENTITY") {
+ *number = 0;
+ *type = kIdentityFile;
+ } else if (rest == "CURRENT") {
+ *number = 0;
+ *type = kCurrentFile;
+ } else if (rest == "LOCK") {
+ *number = 0;
+ *type = kDBLockFile;
+ } else if (info_log_name_prefix.size() > 0 &&
+ rest.starts_with(info_log_name_prefix)) {
+ rest.remove_prefix(info_log_name_prefix.size());
+ if (rest == "" || rest == ".old") {
+ *number = 0;
+ *type = kInfoLogFile;
+ } else if (rest.starts_with(".old.")) {
+ uint64_t ts_suffix;
+ // sizeof also counts the trailing '\0'.
+ rest.remove_prefix(sizeof(".old.") - 1);
+ if (!ConsumeDecimalNumber(&rest, &ts_suffix)) {
+ return false;
+ }
+ *number = ts_suffix;
+ *type = kInfoLogFile;
+ }
+ } else if (rest.starts_with("MANIFEST-")) {
+ rest.remove_prefix(strlen("MANIFEST-"));
+ uint64_t num;
+ if (!ConsumeDecimalNumber(&rest, &num)) {
+ return false;
+ }
+ if (!rest.empty()) {
+ return false;
+ }
+ *type = kDescriptorFile;
+ *number = num;
+ } else if (rest.starts_with("METADB-")) {
+ rest.remove_prefix(strlen("METADB-"));
+ uint64_t num;
+ if (!ConsumeDecimalNumber(&rest, &num)) {
+ return false;
+ }
+ if (!rest.empty()) {
+ return false;
+ }
+ *type = kMetaDatabase;
+ *number = num;
+ } else if (rest.starts_with(kOptionsFileNamePrefix)) {
+ uint64_t ts_suffix;
+ bool is_temp_file = false;
+ rest.remove_prefix(kOptionsFileNamePrefix.size());
+ const std::string kTempFileNameSuffixWithDot =
+ std::string(".") + kTempFileNameSuffix;
+ if (rest.ends_with(kTempFileNameSuffixWithDot)) {
+ rest.remove_suffix(kTempFileNameSuffixWithDot.size());
+ is_temp_file = true;
+ }
+ if (!ConsumeDecimalNumber(&rest, &ts_suffix)) {
+ return false;
+ }
+ *number = ts_suffix;
+ *type = is_temp_file ? kTempFile : kOptionsFile;
+ } else {
+ // Avoid strtoull() to keep filename format independent of the
+ // current locale
+ bool archive_dir_found = false;
+ if (rest.starts_with(kArchivalDirName)) {
+ if (rest.size() <= kArchivalDirName.size()) {
+ return false;
+ }
+ rest.remove_prefix(kArchivalDirName.size() +
+ 1); // Add 1 to remove / also
+ if (log_type) {
+ *log_type = kArchivedLogFile;
+ }
+ archive_dir_found = true;
+ }
+ uint64_t num;
+ if (!ConsumeDecimalNumber(&rest, &num)) {
+ return false;
+ }
+ if (rest.size() <= 1 || rest[0] != '.') {
+ return false;
+ }
+ rest.remove_prefix(1);
+
+ Slice suffix = rest;
+ if (suffix == Slice("log")) {
+ *type = kWalFile;
+ if (log_type && !archive_dir_found) {
+ *log_type = kAliveLogFile;
+ }
+ } else if (archive_dir_found) {
+ return false; // Archive dir can contain only log files
+ } else if (suffix == Slice(kRocksDbTFileExt) ||
+ suffix == Slice(kLevelDbTFileExt)) {
+ *type = kTableFile;
+ } else if (suffix == Slice(kRocksDBBlobFileExt)) {
+ *type = kBlobFile;
+ } else if (suffix == Slice(kTempFileNameSuffix)) {
+ *type = kTempFile;
+ } else {
+ return false;
+ }
+ *number = num;
+ }
+ return true;
+}
+
+IOStatus SetCurrentFile(FileSystem* fs, const std::string& dbname,
+ uint64_t descriptor_number,
+ FSDirectory* dir_contains_current_file) {
+ // Remove leading "dbname/" and add newline to manifest file name
+ std::string manifest = DescriptorFileName(dbname, descriptor_number);
+ Slice contents = manifest;
+ assert(contents.starts_with(dbname + "/"));
+ contents.remove_prefix(dbname.size() + 1);
+ std::string tmp = TempFileName(dbname, descriptor_number);
+ IOStatus s = WriteStringToFile(fs, contents.ToString() + "\n", tmp, true);
+ TEST_SYNC_POINT_CALLBACK("SetCurrentFile:BeforeRename", &s);
+ if (s.ok()) {
+ TEST_KILL_RANDOM_WITH_WEIGHT("SetCurrentFile:0", REDUCE_ODDS2);
+ s = fs->RenameFile(tmp, CurrentFileName(dbname), IOOptions(), nullptr);
+ TEST_KILL_RANDOM_WITH_WEIGHT("SetCurrentFile:1", REDUCE_ODDS2);
+ TEST_SYNC_POINT_CALLBACK("SetCurrentFile:AfterRename", &s);
+ }
+ if (s.ok()) {
+ if (dir_contains_current_file != nullptr) {
+ s = dir_contains_current_file->FsyncWithDirOptions(
+ IOOptions(), nullptr, DirFsyncOptions(CurrentFileName(dbname)));
+ }
+ } else {
+ fs->DeleteFile(tmp, IOOptions(), nullptr)
+ .PermitUncheckedError(); // NOTE: PermitUncheckedError is acceptable
+ // here as we are already handling an error
+ // case, and this is just a best-attempt
+ // effort at some cleanup
+ }
+ return s;
+}
+
+Status SetIdentityFile(Env* env, const std::string& dbname,
+ const std::string& db_id) {
+ std::string id;
+ if (db_id.empty()) {
+ id = env->GenerateUniqueId();
+ } else {
+ id = db_id;
+ }
+ assert(!id.empty());
+ // Reserve the filename dbname/000000.dbtmp for the temporary identity file
+ std::string tmp = TempFileName(dbname, 0);
+ std::string identify_file_name = IdentityFileName(dbname);
+ Status s = WriteStringToFile(env, id, tmp, true);
+ if (s.ok()) {
+ s = env->RenameFile(tmp, identify_file_name);
+ }
+ std::unique_ptr<FSDirectory> dir_obj;
+ if (s.ok()) {
+ s = env->GetFileSystem()->NewDirectory(dbname, IOOptions(), &dir_obj,
+ nullptr);
+ }
+ if (s.ok()) {
+ s = dir_obj->FsyncWithDirOptions(IOOptions(), nullptr,
+ DirFsyncOptions(identify_file_name));
+ }
+
+ // The default Close() could return "NotSupported" and we bypass it
+ // if it is not impelmented. Detailed explanations can be found in
+ // db/db_impl/db_impl.h
+ if (s.ok()) {
+ Status temp_s = dir_obj->Close(IOOptions(), nullptr);
+ if (!temp_s.ok()) {
+ if (temp_s.IsNotSupported()) {
+ temp_s.PermitUncheckedError();
+ } else {
+ s = temp_s;
+ }
+ }
+ }
+ if (!s.ok()) {
+ env->DeleteFile(tmp).PermitUncheckedError();
+ }
+ return s;
+}
+
+IOStatus SyncManifest(const ImmutableDBOptions* db_options,
+ WritableFileWriter* file) {
+ TEST_KILL_RANDOM_WITH_WEIGHT("SyncManifest:0", REDUCE_ODDS2);
+ StopWatch sw(db_options->clock, db_options->stats, MANIFEST_FILE_SYNC_MICROS);
+ return file->Sync(db_options->use_fsync);
+}
+
+Status GetInfoLogFiles(const std::shared_ptr<FileSystem>& fs,
+ const std::string& db_log_dir, const std::string& dbname,
+ std::string* parent_dir,
+ std::vector<std::string>* info_log_list) {
+ assert(parent_dir != nullptr);
+ assert(info_log_list != nullptr);
+ uint64_t number = 0;
+ FileType type = kWalFile;
+
+ if (!db_log_dir.empty()) {
+ *parent_dir = db_log_dir;
+ } else {
+ *parent_dir = dbname;
+ }
+
+ InfoLogPrefix info_log_prefix(!db_log_dir.empty(), dbname);
+
+ std::vector<std::string> file_names;
+ Status s = fs->GetChildren(*parent_dir, IOOptions(), &file_names, nullptr);
+
+ if (!s.ok()) {
+ return s;
+ }
+
+ for (auto& f : file_names) {
+ if (ParseFileName(f, &number, info_log_prefix.prefix, &type) &&
+ (type == kInfoLogFile)) {
+ info_log_list->push_back(f);
+ }
+ }
+ return Status::OK();
+}
+
+std::string NormalizePath(const std::string& path) {
+ std::string dst;
+
+ if (path.length() > 2 && path[0] == kFilePathSeparator &&
+ path[1] == kFilePathSeparator) { // Handle UNC names
+ dst.append(2, kFilePathSeparator);
+ }
+
+ for (auto c : path) {
+ if (!dst.empty() && (c == kFilePathSeparator || c == '/') &&
+ (dst.back() == kFilePathSeparator || dst.back() == '/')) {
+ continue;
+ }
+ dst.push_back(c);
+ }
+ return dst;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/filename.h b/src/rocksdb/file/filename.h
new file mode 100644
index 000000000..2eb125b6a
--- /dev/null
+++ b/src/rocksdb/file/filename.h
@@ -0,0 +1,188 @@
+// 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.
+//
+// File names used by DB code
+
+#pragma once
+#include <stdint.h>
+
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "options/db_options.h"
+#include "port/port.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/options.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/status.h"
+#include "rocksdb/transaction_log.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Env;
+class Directory;
+class SystemClock;
+class WritableFileWriter;
+
+#ifdef OS_WIN
+constexpr char kFilePathSeparator = '\\';
+#else
+constexpr char kFilePathSeparator = '/';
+#endif
+
+// Return the name of the log file with the specified number
+// in the db named by "dbname". The result will be prefixed with
+// "dbname".
+extern std::string LogFileName(const std::string& dbname, uint64_t number);
+
+extern std::string LogFileName(uint64_t number);
+
+extern std::string BlobFileName(uint64_t number);
+
+extern std::string BlobFileName(const std::string& bdirname, uint64_t number);
+
+extern std::string BlobFileName(const std::string& dbname,
+ const std::string& blob_dir, uint64_t number);
+
+extern std::string ArchivalDirectory(const std::string& dbname);
+
+// Return the name of the archived log file with the specified number
+// in the db named by "dbname". The result will be prefixed with "dbname".
+extern std::string ArchivedLogFileName(const std::string& dbname, uint64_t num);
+
+extern std::string MakeTableFileName(const std::string& name, uint64_t number);
+
+extern std::string MakeTableFileName(uint64_t number);
+
+// Return the name of sstable with LevelDB suffix
+// created from RocksDB sstable suffixed name
+extern std::string Rocks2LevelTableFileName(const std::string& fullname);
+
+// the reverse function of MakeTableFileName
+// TODO(yhchiang): could merge this function with ParseFileName()
+extern uint64_t TableFileNameToNumber(const std::string& name);
+
+// Return the name of the sstable with the specified number
+// in the db named by "dbname". The result will be prefixed with
+// "dbname".
+extern std::string TableFileName(const std::vector<DbPath>& db_paths,
+ uint64_t number, uint32_t path_id);
+
+// Sufficient buffer size for FormatFileNumber.
+const size_t kFormatFileNumberBufSize = 38;
+
+extern void FormatFileNumber(uint64_t number, uint32_t path_id, char* out_buf,
+ size_t out_buf_size);
+
+// Return the name of the descriptor file for the db named by
+// "dbname" and the specified incarnation number. The result will be
+// prefixed with "dbname".
+extern std::string DescriptorFileName(const std::string& dbname,
+ uint64_t number);
+
+extern std::string DescriptorFileName(uint64_t number);
+
+extern const std::string kCurrentFileName; // = "CURRENT"
+
+// Return the name of the current file. This file contains the name
+// of the current manifest file. The result will be prefixed with
+// "dbname".
+extern std::string CurrentFileName(const std::string& dbname);
+
+// Return the name of the lock file for the db named by
+// "dbname". The result will be prefixed with "dbname".
+extern std::string LockFileName(const std::string& dbname);
+
+// Return the name of a temporary file owned by the db named "dbname".
+// The result will be prefixed with "dbname".
+extern std::string TempFileName(const std::string& dbname, uint64_t number);
+
+// A helper structure for prefix of info log names.
+struct InfoLogPrefix {
+ char buf[260];
+ Slice prefix;
+ // Prefix with DB absolute path encoded
+ explicit InfoLogPrefix(bool has_log_dir, const std::string& db_absolute_path);
+ // Default Prefix
+ explicit InfoLogPrefix();
+};
+
+// Return the name of the info log file for "dbname".
+extern std::string InfoLogFileName(const std::string& dbname,
+ const std::string& db_path = "",
+ const std::string& log_dir = "");
+
+// Return the name of the old info log file for "dbname".
+extern std::string OldInfoLogFileName(const std::string& dbname, uint64_t ts,
+ const std::string& db_path = "",
+ const std::string& log_dir = "");
+
+extern const std::string kOptionsFileNamePrefix; // = "OPTIONS-"
+extern const std::string kTempFileNameSuffix; // = "dbtmp"
+
+// Return a options file name given the "dbname" and file number.
+// Format: OPTIONS-[number].dbtmp
+extern std::string OptionsFileName(const std::string& dbname,
+ uint64_t file_num);
+extern std::string OptionsFileName(uint64_t file_num);
+
+// Return a temp options file name given the "dbname" and file number.
+// Format: OPTIONS-[number]
+extern std::string TempOptionsFileName(const std::string& dbname,
+ uint64_t file_num);
+
+// Return the name to use for a metadatabase. The result will be prefixed with
+// "dbname".
+extern std::string MetaDatabaseName(const std::string& dbname, uint64_t number);
+
+// Return the name of the Identity file which stores a unique number for the db
+// that will get regenerated if the db loses all its data and is recreated fresh
+// either from a backup-image or empty
+extern std::string IdentityFileName(const std::string& dbname);
+
+// If filename is a rocksdb file, store the type of the file in *type.
+// The number encoded in the filename is stored in *number. If the
+// filename was successfully parsed, returns true. Else return false.
+// info_log_name_prefix is the path of info logs.
+extern bool ParseFileName(const std::string& filename, uint64_t* number,
+ const Slice& info_log_name_prefix, FileType* type,
+ WalFileType* log_type = nullptr);
+// Same as previous function, but skip info log files.
+extern bool ParseFileName(const std::string& filename, uint64_t* number,
+ FileType* type, WalFileType* log_type = nullptr);
+
+// Make the CURRENT file point to the descriptor file with the
+// specified number. On its success and when dir_contains_current_file is not
+// nullptr, the function will fsync the directory containing the CURRENT file
+// when
+extern IOStatus SetCurrentFile(FileSystem* fs, const std::string& dbname,
+ uint64_t descriptor_number,
+ FSDirectory* dir_contains_current_file);
+
+// Make the IDENTITY file for the db
+extern Status SetIdentityFile(Env* env, const std::string& dbname,
+ const std::string& db_id = {});
+
+// Sync manifest file `file`.
+extern IOStatus SyncManifest(const ImmutableDBOptions* db_options,
+ WritableFileWriter* file);
+
+// Return list of file names of info logs in `file_names`.
+// The list only contains file name. The parent directory name is stored
+// in `parent_dir`.
+// `db_log_dir` should be the one as in options.db_log_dir
+extern Status GetInfoLogFiles(const std::shared_ptr<FileSystem>& fs,
+ const std::string& db_log_dir,
+ const std::string& dbname,
+ std::string* parent_dir,
+ std::vector<std::string>* file_names);
+
+extern std::string NormalizePath(const std::string& path);
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/line_file_reader.cc b/src/rocksdb/file/line_file_reader.cc
new file mode 100644
index 000000000..50c415dc6
--- /dev/null
+++ b/src/rocksdb/file/line_file_reader.cc
@@ -0,0 +1,73 @@
+// Copyright (c) Facebook, Inc. and its affiliates. 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).
+
+#include "file/line_file_reader.h"
+
+#include <cstring>
+
+#include "monitoring/iostats_context_imp.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+IOStatus LineFileReader::Create(const std::shared_ptr<FileSystem>& fs,
+ const std::string& fname,
+ const FileOptions& file_opts,
+ std::unique_ptr<LineFileReader>* reader,
+ IODebugContext* dbg,
+ RateLimiter* rate_limiter) {
+ std::unique_ptr<FSSequentialFile> file;
+ IOStatus io_s = fs->NewSequentialFile(fname, file_opts, &file, dbg);
+ if (io_s.ok()) {
+ reader->reset(new LineFileReader(
+ std::move(file), fname, nullptr,
+ std::vector<std::shared_ptr<EventListener>>{}, rate_limiter));
+ }
+ return io_s;
+}
+
+bool LineFileReader::ReadLine(std::string* out,
+ Env::IOPriority rate_limiter_priority) {
+ assert(out);
+ if (!io_status_.ok()) {
+ // Status should be checked (or permit unchecked) any time we return false.
+ io_status_.MustCheck();
+ return false;
+ }
+ out->clear();
+ for (;;) {
+ // Look for line delimiter
+ const char* found = static_cast<const char*>(
+ std::memchr(buf_begin_, '\n', buf_end_ - buf_begin_));
+ if (found) {
+ size_t len = found - buf_begin_;
+ out->append(buf_begin_, len);
+ buf_begin_ += len + /*delim*/ 1;
+ ++line_number_;
+ return true;
+ }
+ if (at_eof_) {
+ io_status_.MustCheck();
+ return false;
+ }
+ // else flush and reload buffer
+ out->append(buf_begin_, buf_end_ - buf_begin_);
+ Slice result;
+ io_status_ =
+ sfr_.Read(buf_.size(), &result, buf_.data(), rate_limiter_priority);
+ IOSTATS_ADD(bytes_read, result.size());
+ if (!io_status_.ok()) {
+ io_status_.MustCheck();
+ return false;
+ }
+ if (result.size() != buf_.size()) {
+ // The obscure way of indicating EOF
+ at_eof_ = true;
+ }
+ buf_begin_ = result.data();
+ buf_end_ = result.data() + result.size();
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/line_file_reader.h b/src/rocksdb/file/line_file_reader.h
new file mode 100644
index 000000000..cc302d311
--- /dev/null
+++ b/src/rocksdb/file/line_file_reader.h
@@ -0,0 +1,60 @@
+// Copyright (c) Facebook, Inc. and its affiliates. 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).
+
+#pragma once
+#include <array>
+
+#include "file/sequence_file_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// A wrapper on top of Env::SequentialFile for reading text lines from a file.
+// Lines are delimited by '\n'. The last line may or may not include a
+// trailing newline. Uses SequentialFileReader internally.
+class LineFileReader {
+ private:
+ std::array<char, 8192> buf_;
+ SequentialFileReader sfr_;
+ IOStatus io_status_;
+ const char* buf_begin_ = buf_.data();
+ const char* buf_end_ = buf_.data();
+ size_t line_number_ = 0;
+ bool at_eof_ = false;
+
+ public:
+ // See SequentialFileReader constructors
+ template <typename... Args>
+ explicit LineFileReader(Args&&... args)
+ : sfr_(std::forward<Args&&>(args)...) {}
+
+ static IOStatus Create(const std::shared_ptr<FileSystem>& fs,
+ const std::string& fname, const FileOptions& file_opts,
+ std::unique_ptr<LineFileReader>* reader,
+ IODebugContext* dbg, RateLimiter* rate_limiter);
+
+ LineFileReader(const LineFileReader&) = delete;
+ LineFileReader& operator=(const LineFileReader&) = delete;
+
+ // Reads another line from the file, returning true on success and saving
+ // the line to `out`, without delimiter, or returning false on failure. You
+ // must check GetStatus() to determine whether the failure was just
+ // end-of-file (OK status) or an I/O error (another status).
+ // The internal rate limiter will be charged at the specified priority.
+ bool ReadLine(std::string* out, Env::IOPriority rate_limiter_priority);
+
+ // Returns the number of the line most recently returned from ReadLine.
+ // Return value is unspecified if ReadLine has returned false due to
+ // I/O error. After ReadLine returns false due to end-of-file, return
+ // value is the last returned line number, or equivalently the total
+ // number of lines returned.
+ size_t GetLineNumber() const { return line_number_; }
+
+ // Returns any error encountered during read. The error is considered
+ // permanent and no retry or recovery is attempted with the same
+ // LineFileReader.
+ const IOStatus& GetStatus() const { return io_status_; }
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/prefetch_test.cc b/src/rocksdb/file/prefetch_test.cc
new file mode 100644
index 000000000..438286bfc
--- /dev/null
+++ b/src/rocksdb/file/prefetch_test.cc
@@ -0,0 +1,2109 @@
+// 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).
+
+#include "db/db_test_util.h"
+#include "file/file_prefetch_buffer.h"
+#include "file/file_util.h"
+#include "rocksdb/file_system.h"
+#include "test_util/sync_point.h"
+#ifdef GFLAGS
+#include "tools/io_tracer_parser_tool.h"
+#endif
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class MockFS;
+
+class MockRandomAccessFile : public FSRandomAccessFileOwnerWrapper {
+ public:
+ MockRandomAccessFile(std::unique_ptr<FSRandomAccessFile>& file,
+ bool support_prefetch, std::atomic_int& prefetch_count)
+ : FSRandomAccessFileOwnerWrapper(std::move(file)),
+ support_prefetch_(support_prefetch),
+ prefetch_count_(prefetch_count) {}
+
+ IOStatus Prefetch(uint64_t offset, size_t n, const IOOptions& options,
+ IODebugContext* dbg) override {
+ if (support_prefetch_) {
+ prefetch_count_.fetch_add(1);
+ return target()->Prefetch(offset, n, options, dbg);
+ } else {
+ return IOStatus::NotSupported("Prefetch not supported");
+ }
+ }
+
+ private:
+ const bool support_prefetch_;
+ std::atomic_int& prefetch_count_;
+};
+
+class MockFS : public FileSystemWrapper {
+ public:
+ explicit MockFS(const std::shared_ptr<FileSystem>& wrapped,
+ bool support_prefetch)
+ : FileSystemWrapper(wrapped), support_prefetch_(support_prefetch) {}
+
+ static const char* kClassName() { return "MockFS"; }
+ const char* Name() const override { return kClassName(); }
+
+ IOStatus NewRandomAccessFile(const std::string& fname,
+ const FileOptions& opts,
+ std::unique_ptr<FSRandomAccessFile>* result,
+ IODebugContext* dbg) override {
+ std::unique_ptr<FSRandomAccessFile> file;
+ IOStatus s;
+ s = target()->NewRandomAccessFile(fname, opts, &file, dbg);
+ result->reset(
+ new MockRandomAccessFile(file, support_prefetch_, prefetch_count_));
+ return s;
+ }
+
+ void ClearPrefetchCount() { prefetch_count_ = 0; }
+
+ bool IsPrefetchCalled() { return prefetch_count_ > 0; }
+
+ int GetPrefetchCount() {
+ return prefetch_count_.load(std::memory_order_relaxed);
+ }
+
+ private:
+ const bool support_prefetch_;
+ std::atomic_int prefetch_count_{0};
+};
+
+class PrefetchTest
+ : public DBTestBase,
+ public ::testing::WithParamInterface<std::tuple<bool, bool>> {
+ public:
+ PrefetchTest() : DBTestBase("prefetch_test", true) {}
+};
+
+INSTANTIATE_TEST_CASE_P(PrefetchTest, PrefetchTest,
+ ::testing::Combine(::testing::Bool(),
+ ::testing::Bool()));
+
+std::string BuildKey(int num, std::string postfix = "") {
+ return "my_key_" + std::to_string(num) + postfix;
+}
+
+TEST_P(PrefetchTest, Basic) {
+ // First param is if the mockFS support_prefetch or not
+ bool support_prefetch =
+ std::get<0>(GetParam()) &&
+ test::IsPrefetchSupported(env_->GetFileSystem(), dbname_);
+
+ // Second param is if directIO is enabled or not
+ bool use_direct_io = std::get<1>(GetParam());
+ const int kNumKeys = 1100;
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), support_prefetch);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+
+ int buff_prefetch_count = 0;
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ // create first key range
+ WriteBatch batch;
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), "value for range 1 key"));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ // create second key range
+ batch.Clear();
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i, "key2"), "value for range 2 key"));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ // delete second key range
+ batch.Clear();
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Delete(BuildKey(i, "key2")));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ // compact database
+ std::string start_key = BuildKey(0);
+ std::string end_key = BuildKey(kNumKeys - 1);
+ Slice least(start_key.data(), start_key.size());
+ Slice greatest(end_key.data(), end_key.size());
+
+ // commenting out the line below causes the example to work correctly
+ ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &least, &greatest));
+
+ if (support_prefetch && !use_direct_io) {
+ // If underline file system supports prefetch, and directIO is not enabled
+ // make sure prefetch() is called and FilePrefetchBuffer is not used.
+ ASSERT_TRUE(fs->IsPrefetchCalled());
+ fs->ClearPrefetchCount();
+ ASSERT_EQ(0, buff_prefetch_count);
+ } else {
+ // If underline file system doesn't support prefetch, or directIO is
+ // enabled, make sure prefetch() is not called and FilePrefetchBuffer is
+ // used.
+ ASSERT_FALSE(fs->IsPrefetchCalled());
+ ASSERT_GT(buff_prefetch_count, 0);
+ buff_prefetch_count = 0;
+ }
+
+ // count the keys
+ {
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ int num_keys = 0;
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ num_keys++;
+ }
+ }
+
+ // Make sure prefetch is called only if file system support prefetch.
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_TRUE(fs->IsPrefetchCalled());
+ fs->ClearPrefetchCount();
+ ASSERT_EQ(0, buff_prefetch_count);
+ } else {
+ ASSERT_FALSE(fs->IsPrefetchCalled());
+ ASSERT_GT(buff_prefetch_count, 0);
+ buff_prefetch_count = 0;
+ }
+ Close();
+}
+
+#ifndef ROCKSDB_LITE
+TEST_P(PrefetchTest, ConfigureAutoMaxReadaheadSize) {
+ // First param is if the mockFS support_prefetch or not
+ bool support_prefetch =
+ std::get<0>(GetParam()) &&
+ test::IsPrefetchSupported(env_->GetFileSystem(), dbname_);
+
+ // Second param is if directIO is enabled or not
+ bool use_direct_io = std::get<1>(GetParam());
+
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), support_prefetch);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ options.disable_auto_compactions = true;
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ table_options.max_auto_readahead_size = 0;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ int buff_prefetch_count = 0;
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ // DB open will create table readers unless we reduce the table cache
+ // capacity. SanitizeOptions will set max_open_files to minimum of 20. Table
+ // cache is allocated with max_open_files - 10 as capacity. So override
+ // max_open_files to 10 so table cache capacity will become 0. This will
+ // prevent file open during DB open and force the file to be opened during
+ // Iteration.
+ SyncPoint::GetInstance()->SetCallBack(
+ "SanitizeOptions::AfterChangeMaxOpenFiles", [&](void* arg) {
+ int* max_open_files = (int*)arg;
+ *max_open_files = 11;
+ });
+
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ Status s = TryReopen(options);
+
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ Random rnd(309);
+ int key_count = 0;
+ const int num_keys_per_level = 100;
+ // Level 0 : Keys in range [0, 99], Level 1:[100, 199], Level 2:[200, 299].
+ for (int level = 2; level >= 0; level--) {
+ key_count = level * num_keys_per_level;
+ for (int i = 0; i < num_keys_per_level; ++i) {
+ ASSERT_OK(Put(Key(key_count++), rnd.RandomString(500)));
+ }
+ ASSERT_OK(Flush());
+ MoveFilesToLevel(level);
+ }
+ Close();
+ std::vector<int> buff_prefectch_level_count = {0, 0, 0};
+ TryReopen(options);
+ {
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ fs->ClearPrefetchCount();
+ buff_prefetch_count = 0;
+
+ for (int level = 2; level >= 0; level--) {
+ key_count = level * num_keys_per_level;
+ switch (level) {
+ case 0:
+ // max_auto_readahead_size is set 0 so data and index blocks are not
+ // prefetched.
+ ASSERT_OK(db_->SetOptions(
+ {{"block_based_table_factory", "{max_auto_readahead_size=0;}"}}));
+ break;
+ case 1:
+ // max_auto_readahead_size is set less than
+ // initial_auto_readahead_size. So readahead_size remains equal to
+ // max_auto_readahead_size.
+ ASSERT_OK(db_->SetOptions({{"block_based_table_factory",
+ "{max_auto_readahead_size=4096;}"}}));
+ break;
+ case 2:
+ ASSERT_OK(db_->SetOptions({{"block_based_table_factory",
+ "{max_auto_readahead_size=65536;}"}}));
+ break;
+ default:
+ assert(false);
+ }
+
+ for (int i = 0; i < num_keys_per_level; ++i) {
+ iter->Seek(Key(key_count++));
+ iter->Next();
+ }
+
+ buff_prefectch_level_count[level] = buff_prefetch_count;
+ if (support_prefetch && !use_direct_io) {
+ if (level == 0) {
+ ASSERT_FALSE(fs->IsPrefetchCalled());
+ } else {
+ ASSERT_TRUE(fs->IsPrefetchCalled());
+ }
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_FALSE(fs->IsPrefetchCalled());
+ if (level == 0) {
+ ASSERT_EQ(buff_prefetch_count, 0);
+ } else {
+ ASSERT_GT(buff_prefetch_count, 0);
+ }
+ buff_prefetch_count = 0;
+ }
+ }
+ }
+
+ if (!support_prefetch) {
+ ASSERT_GT(buff_prefectch_level_count[1], buff_prefectch_level_count[2]);
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+ Close();
+}
+
+TEST_P(PrefetchTest, ConfigureInternalAutoReadaheadSize) {
+ // First param is if the mockFS support_prefetch or not
+ bool support_prefetch =
+ std::get<0>(GetParam()) &&
+ test::IsPrefetchSupported(env_->GetFileSystem(), dbname_);
+
+ // Second param is if directIO is enabled or not
+ bool use_direct_io = std::get<1>(GetParam());
+
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), support_prefetch);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ options.disable_auto_compactions = true;
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ table_options.initial_auto_readahead_size = 0;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ int buff_prefetch_count = 0;
+ // DB open will create table readers unless we reduce the table cache
+ // capacity. SanitizeOptions will set max_open_files to minimum of 20.
+ // Table cache is allocated with max_open_files - 10 as capacity. So
+ // override max_open_files to 10 so table cache capacity will become 0.
+ // This will prevent file open during DB open and force the file to be
+ // opened during Iteration.
+ SyncPoint::GetInstance()->SetCallBack(
+ "SanitizeOptions::AfterChangeMaxOpenFiles", [&](void* arg) {
+ int* max_open_files = (int*)arg;
+ *max_open_files = 11;
+ });
+
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ Status s = TryReopen(options);
+
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ Random rnd(309);
+ int key_count = 0;
+ const int num_keys_per_level = 100;
+ // Level 0 : Keys in range [0, 99], Level 1:[100, 199], Level 2:[200, 299].
+ for (int level = 2; level >= 0; level--) {
+ key_count = level * num_keys_per_level;
+ for (int i = 0; i < num_keys_per_level; ++i) {
+ ASSERT_OK(Put(Key(key_count++), rnd.RandomString(500)));
+ }
+ ASSERT_OK(Flush());
+ MoveFilesToLevel(level);
+ }
+ Close();
+
+ TryReopen(options);
+ {
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ fs->ClearPrefetchCount();
+ buff_prefetch_count = 0;
+ std::vector<int> buff_prefetch_level_count = {0, 0, 0};
+
+ for (int level = 2; level >= 0; level--) {
+ key_count = level * num_keys_per_level;
+ switch (level) {
+ case 0:
+ // initial_auto_readahead_size is set 0 so data and index blocks are
+ // not prefetched.
+ ASSERT_OK(db_->SetOptions({{"block_based_table_factory",
+ "{initial_auto_readahead_size=0;}"}}));
+ break;
+ case 1:
+ // intial_auto_readahead_size and max_auto_readahead_size are set same
+ // so readahead_size remains same.
+ ASSERT_OK(db_->SetOptions({{"block_based_table_factory",
+ "{initial_auto_readahead_size=4096;max_"
+ "auto_readahead_size=4096;}"}}));
+ break;
+ case 2:
+ ASSERT_OK(
+ db_->SetOptions({{"block_based_table_factory",
+ "{initial_auto_readahead_size=65536;}"}}));
+ break;
+ default:
+ assert(false);
+ }
+
+ for (int i = 0; i < num_keys_per_level; ++i) {
+ iter->Seek(Key(key_count++));
+ iter->Next();
+ }
+
+ buff_prefetch_level_count[level] = buff_prefetch_count;
+ if (support_prefetch && !use_direct_io) {
+ if (level == 0) {
+ ASSERT_FALSE(fs->IsPrefetchCalled());
+ } else {
+ ASSERT_TRUE(fs->IsPrefetchCalled());
+ }
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_FALSE(fs->IsPrefetchCalled());
+ if (level == 0) {
+ ASSERT_EQ(buff_prefetch_count, 0);
+ } else {
+ ASSERT_GT(buff_prefetch_count, 0);
+ }
+ buff_prefetch_count = 0;
+ }
+ }
+ if (!support_prefetch) {
+ ASSERT_GT(buff_prefetch_level_count[1], buff_prefetch_level_count[2]);
+ }
+ }
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+ Close();
+}
+
+TEST_P(PrefetchTest, ConfigureNumFilesReadsForReadaheadSize) {
+ // First param is if the mockFS support_prefetch or not
+ bool support_prefetch =
+ std::get<0>(GetParam()) &&
+ test::IsPrefetchSupported(env_->GetFileSystem(), dbname_);
+
+ const int kNumKeys = 2000;
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), support_prefetch);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ // Second param is if directIO is enabled or not
+ bool use_direct_io = std::get<1>(GetParam());
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ table_options.num_file_reads_for_auto_readahead = 0;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+
+ int buff_prefetch_count = 0;
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ std::string start_key = BuildKey(0);
+ std::string end_key = BuildKey(kNumKeys - 1);
+ Slice least(start_key.data(), start_key.size());
+ Slice greatest(end_key.data(), end_key.size());
+
+ ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &least, &greatest));
+
+ Close();
+ TryReopen(options);
+
+ fs->ClearPrefetchCount();
+ buff_prefetch_count = 0;
+
+ {
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ /*
+ * Reseek keys from sequential Data Blocks within same partitioned
+ * index. It will prefetch the data block at the first seek since
+ * num_file_reads_for_auto_readahead = 0. Data Block size is nearly 4076 so
+ * readahead will fetch 8 * 1024 data more initially (2 more data blocks).
+ */
+ iter->Seek(BuildKey(0)); // Prefetch data + index block since
+ // num_file_reads_for_auto_readahead = 0.
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1000)); // In buffer
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1004)); // In buffer
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1008)); // Prefetch Data
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1011)); // In buffer
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1015)); // In buffer
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1019)); // In buffer
+ ASSERT_TRUE(iter->Valid());
+ // Missed 2 blocks but they are already in buffer so no reset.
+ iter->Seek(BuildKey(103)); // Already in buffer.
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1033)); // Prefetch Data.
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 4);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 4);
+ buff_prefetch_count = 0;
+ }
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+ Close();
+}
+#endif // !ROCKSDB_LITE
+
+TEST_P(PrefetchTest, PrefetchWhenReseek) {
+ // First param is if the mockFS support_prefetch or not
+ bool support_prefetch =
+ std::get<0>(GetParam()) &&
+ test::IsPrefetchSupported(env_->GetFileSystem(), dbname_);
+
+ const int kNumKeys = 2000;
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), support_prefetch);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ // Second param is if directIO is enabled or not
+ bool use_direct_io = std::get<1>(GetParam());
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+
+ int buff_prefetch_count = 0;
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ std::string start_key = BuildKey(0);
+ std::string end_key = BuildKey(kNumKeys - 1);
+ Slice least(start_key.data(), start_key.size());
+ Slice greatest(end_key.data(), end_key.size());
+
+ ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &least, &greatest));
+
+ fs->ClearPrefetchCount();
+ buff_prefetch_count = 0;
+
+ {
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ /*
+ * Reseek keys from sequential Data Blocks within same partitioned
+ * index. After 2 sequential reads it will prefetch the data block.
+ * Data Block size is nearly 4076 so readahead will fetch 8 * 1024 data more
+ * initially (2 more data blocks).
+ */
+ iter->Seek(BuildKey(0));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1000));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1004)); // Prefetch Data
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1008));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1011));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1015)); // Prefetch Data
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1019));
+ ASSERT_TRUE(iter->Valid());
+ // Missed 2 blocks but they are already in buffer so no reset.
+ iter->Seek(BuildKey(103)); // Already in buffer.
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1033)); // Prefetch Data
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 3);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 3);
+ buff_prefetch_count = 0;
+ }
+ }
+ {
+ /*
+ * Reseek keys from non sequential data blocks within same partitioned
+ * index. buff_prefetch_count will be 0 in that case.
+ */
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ iter->Seek(BuildKey(0));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1008));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1019));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1033));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1048));
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 0);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 0);
+ buff_prefetch_count = 0;
+ }
+ }
+ {
+ /*
+ * Reesek keys from Single Data Block.
+ */
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ iter->Seek(BuildKey(0));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(10));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(100));
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 0);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 0);
+ buff_prefetch_count = 0;
+ }
+ }
+ {
+ /*
+ * Reseek keys from sequential data blocks to set implicit auto readahead
+ * and prefetch data but after that iterate over different (non sequential)
+ * data blocks which won't prefetch any data further. So buff_prefetch_count
+ * will be 1 for the first one.
+ */
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ iter->Seek(BuildKey(0));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1000));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1004)); // This iteration will prefetch buffer
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1008));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(
+ BuildKey(996)); // Reseek won't prefetch any data and
+ // readahead_size will be initiallized to 8*1024.
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(992));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(989));
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 1);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 1);
+ buff_prefetch_count = 0;
+ }
+
+ // Read sequentially to confirm readahead_size is reset to initial value (2
+ // more data blocks)
+ iter->Seek(BuildKey(1011));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1015));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1019)); // Prefetch Data
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1022));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1026));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(103)); // Prefetch Data
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 2);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 2);
+ buff_prefetch_count = 0;
+ }
+ }
+ {
+ /* Reseek keys from sequential partitioned index block. Since partitioned
+ * index fetch are sequential, buff_prefetch_count will be 1.
+ */
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ iter->Seek(BuildKey(0));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1167));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1334)); // This iteration will prefetch buffer
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1499));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1667));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1847));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1999));
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 1);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 1);
+ buff_prefetch_count = 0;
+ }
+ }
+ {
+ /*
+ * Reseek over different keys from different blocks. buff_prefetch_count is
+ * set 0.
+ */
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ int i = 0;
+ int j = 1000;
+ do {
+ iter->Seek(BuildKey(i));
+ if (!iter->Valid()) {
+ break;
+ }
+ i = i + 100;
+ iter->Seek(BuildKey(j));
+ j = j + 100;
+ } while (i < 1000 && j < kNumKeys && iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 0);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 0);
+ buff_prefetch_count = 0;
+ }
+ }
+ {
+ /* Iterates sequentially over all keys. It will prefetch the buffer.*/
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ }
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 13);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 13);
+ buff_prefetch_count = 0;
+ }
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+ Close();
+}
+
+TEST_P(PrefetchTest, PrefetchWhenReseekwithCache) {
+ // First param is if the mockFS support_prefetch or not
+ bool support_prefetch =
+ std::get<0>(GetParam()) &&
+ test::IsPrefetchSupported(env_->GetFileSystem(), dbname_);
+
+ const int kNumKeys = 2000;
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), support_prefetch);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ // Second param is if directIO is enabled or not
+ bool use_direct_io = std::get<1>(GetParam());
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+
+ BlockBasedTableOptions table_options;
+ std::shared_ptr<Cache> cache = NewLRUCache(4 * 1024 * 1024, 2); // 8MB
+ table_options.block_cache = cache;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+
+ int buff_prefetch_count = 0;
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ std::string start_key = BuildKey(0);
+ std::string end_key = BuildKey(kNumKeys - 1);
+ Slice least(start_key.data(), start_key.size());
+ Slice greatest(end_key.data(), end_key.size());
+
+ ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &least, &greatest));
+
+ fs->ClearPrefetchCount();
+ buff_prefetch_count = 0;
+
+ {
+ /*
+ * Reseek keys from sequential Data Blocks within same partitioned
+ * index. After 2 sequential reads it will prefetch the data block.
+ * Data Block size is nearly 4076 so readahead will fetch 8 * 1024 data more
+ * initially (2 more data blocks).
+ */
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ // Warm up the cache
+ iter->Seek(BuildKey(1011));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1015));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1019));
+ ASSERT_TRUE(iter->Valid());
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 1);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 1);
+ buff_prefetch_count = 0;
+ }
+ }
+ {
+ // After caching, blocks will be read from cache (Sequential blocks)
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ iter->Seek(BuildKey(0));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1000));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1004)); // Prefetch data (not in cache).
+ ASSERT_TRUE(iter->Valid());
+ // Missed one sequential block but next is in already in buffer so readahead
+ // will not be reset.
+ iter->Seek(BuildKey(1011));
+ ASSERT_TRUE(iter->Valid());
+ // Prefetch data but blocks are in cache so no prefetch and reset.
+ iter->Seek(BuildKey(1015));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1019));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1022));
+ ASSERT_TRUE(iter->Valid());
+ // Prefetch data with readahead_size = 4 blocks.
+ iter->Seek(BuildKey(1026));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(103));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1033));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1037));
+ ASSERT_TRUE(iter->Valid());
+
+ if (support_prefetch && !use_direct_io) {
+ ASSERT_EQ(fs->GetPrefetchCount(), 3);
+ fs->ClearPrefetchCount();
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 2);
+ buff_prefetch_count = 0;
+ }
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+ Close();
+}
+
+#ifndef ROCKSDB_LITE
+TEST_P(PrefetchTest, DBIterLevelReadAhead) {
+ const int kNumKeys = 1000;
+ // Set options
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ bool use_direct_io = std::get<0>(GetParam());
+ bool is_adaptive_readahead = std::get<1>(GetParam());
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.statistics = CreateDBStatistics();
+ options.env = env.get();
+
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ int total_keys = 0;
+ for (int j = 0; j < 5; j++) {
+ for (int i = j * kNumKeys; i < (j + 1) * kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ total_keys++;
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+ ASSERT_OK(Flush());
+ }
+ MoveFilesToLevel(2);
+ int buff_prefetch_count = 0;
+ int buff_async_prefetch_count = 0;
+ int readahead_carry_over_count = 0;
+ int num_sst_files = NumTableFilesAtLevel(2);
+ size_t current_readahead_size = 0;
+
+ // Test - Iterate over the keys sequentially.
+ {
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::PrefetchAsyncInternal:Start",
+ [&](void*) { buff_async_prefetch_count++; });
+
+ // The callback checks, since reads are sequential, readahead_size doesn't
+ // start from 8KB when iterator moves to next file and its called
+ // num_sst_files-1 times (excluding for first file).
+ SyncPoint::GetInstance()->SetCallBack(
+ "BlockPrefetcher::SetReadaheadState", [&](void* arg) {
+ readahead_carry_over_count++;
+ size_t readahead_size = *reinterpret_cast<size_t*>(arg);
+ if (readahead_carry_over_count) {
+ ASSERT_GT(readahead_size, 8 * 1024);
+ }
+ });
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::TryReadFromCache", [&](void* arg) {
+ current_readahead_size = *reinterpret_cast<size_t*>(arg);
+ ASSERT_GT(current_readahead_size, 0);
+ });
+
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ReadOptions ro;
+ if (is_adaptive_readahead) {
+ ro.adaptive_readahead = true;
+ ro.async_io = true;
+ }
+
+ ASSERT_OK(options.statistics->Reset());
+
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ int num_keys = 0;
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ ASSERT_OK(iter->status());
+ num_keys++;
+ }
+ ASSERT_EQ(num_keys, total_keys);
+
+ // For index and data blocks.
+ if (is_adaptive_readahead) {
+ ASSERT_EQ(readahead_carry_over_count, 2 * (num_sst_files - 1));
+ ASSERT_GT(buff_async_prefetch_count, 0);
+ } else {
+ ASSERT_GT(buff_prefetch_count, 0);
+ ASSERT_EQ(readahead_carry_over_count, 0);
+ }
+
+ // Check stats to make sure async prefetch is done.
+ {
+ HistogramData async_read_bytes;
+ options.statistics->histogramData(ASYNC_READ_BYTES, &async_read_bytes);
+ if (ro.async_io) {
+ ASSERT_GT(async_read_bytes.count, 0);
+ } else {
+ ASSERT_EQ(async_read_bytes.count, 0);
+ }
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+ }
+ Close();
+}
+#endif //! ROCKSDB_LITE
+
+class PrefetchTest1 : public DBTestBase,
+ public ::testing::WithParamInterface<bool> {
+ public:
+ PrefetchTest1() : DBTestBase("prefetch_test1", true) {}
+};
+
+INSTANTIATE_TEST_CASE_P(PrefetchTest1, PrefetchTest1, ::testing::Bool());
+
+#ifndef ROCKSDB_LITE
+TEST_P(PrefetchTest1, NonSequentialReadsWithAdaptiveReadahead) {
+ const int kNumKeys = 1000;
+ // Set options
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ if (GetParam()) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (GetParam() && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ for (int j = 0; j < 5; j++) {
+ for (int i = j * kNumKeys; i < (j + 1) * kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+ ASSERT_OK(Flush());
+ }
+ MoveFilesToLevel(2);
+
+ int buff_prefetch_count = 0;
+ int set_readahead = 0;
+ size_t readahead_size = 0;
+
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+ SyncPoint::GetInstance()->SetCallBack(
+ "BlockPrefetcher::SetReadaheadState",
+ [&](void* /*arg*/) { set_readahead++; });
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::TryReadFromCache",
+ [&](void* arg) { readahead_size = *reinterpret_cast<size_t*>(arg); });
+
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ {
+ // Iterate until prefetch is done.
+ ReadOptions ro;
+ ro.adaptive_readahead = true;
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+
+ iter->SeekToFirst();
+ ASSERT_TRUE(iter->Valid());
+
+ while (iter->Valid() && buff_prefetch_count == 0) {
+ iter->Next();
+ }
+
+ ASSERT_EQ(readahead_size, 8 * 1024);
+ ASSERT_EQ(buff_prefetch_count, 1);
+ ASSERT_EQ(set_readahead, 0);
+ buff_prefetch_count = 0;
+
+ // Move to last file and check readahead size fallbacks to 8KB. So next
+ // readahead size after prefetch should be 8 * 1024;
+ iter->Seek(BuildKey(4004));
+ ASSERT_TRUE(iter->Valid());
+
+ while (iter->Valid() && buff_prefetch_count == 0) {
+ iter->Next();
+ }
+
+ ASSERT_EQ(readahead_size, 8 * 1024);
+ ASSERT_EQ(set_readahead, 0);
+ ASSERT_EQ(buff_prefetch_count, 1);
+ }
+ Close();
+}
+#endif //! ROCKSDB_LITE
+
+TEST_P(PrefetchTest1, DecreaseReadAheadIfInCache) {
+ const int kNumKeys = 2000;
+ // Set options
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ if (GetParam()) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+
+ options.statistics = CreateDBStatistics();
+ BlockBasedTableOptions table_options;
+ std::shared_ptr<Cache> cache = NewLRUCache(4 * 1024 * 1024, 2); // 8MB
+ table_options.block_cache = cache;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (GetParam() && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ std::string start_key = BuildKey(0);
+ std::string end_key = BuildKey(kNumKeys - 1);
+ Slice least(start_key.data(), start_key.size());
+ Slice greatest(end_key.data(), end_key.size());
+
+ ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &least, &greatest));
+
+ int buff_prefetch_count = 0;
+ size_t current_readahead_size = 0;
+ size_t expected_current_readahead_size = 8 * 1024;
+ size_t decrease_readahead_size = 8 * 1024;
+
+ SyncPoint::GetInstance()->SetCallBack("FilePrefetchBuffer::Prefetch:Start",
+ [&](void*) { buff_prefetch_count++; });
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::TryReadFromCache", [&](void* arg) {
+ current_readahead_size = *reinterpret_cast<size_t*>(arg);
+ });
+
+ SyncPoint::GetInstance()->EnableProcessing();
+ ReadOptions ro;
+ ro.adaptive_readahead = true;
+ {
+ /*
+ * Reseek keys from sequential Data Blocks within same partitioned
+ * index. After 2 sequential reads it will prefetch the data block.
+ * Data Block size is nearly 4076 so readahead will fetch 8 * 1024 data
+ * more initially (2 more data blocks).
+ */
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ // Warm up the cache
+ iter->Seek(BuildKey(1011));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1015));
+ ASSERT_TRUE(iter->Valid());
+ iter->Seek(BuildKey(1019));
+ ASSERT_TRUE(iter->Valid());
+ buff_prefetch_count = 0;
+ }
+
+ {
+ ASSERT_OK(options.statistics->Reset());
+ // After caching, blocks will be read from cache (Sequential blocks)
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ iter->Seek(
+ BuildKey(0)); // In cache so it will decrease the readahead_size.
+ ASSERT_TRUE(iter->Valid());
+ expected_current_readahead_size = std::max(
+ decrease_readahead_size,
+ (expected_current_readahead_size >= decrease_readahead_size
+ ? (expected_current_readahead_size - decrease_readahead_size)
+ : 0));
+
+ iter->Seek(BuildKey(1000)); // Won't prefetch the block.
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(current_readahead_size, expected_current_readahead_size);
+
+ iter->Seek(BuildKey(1004)); // Prefetch the block.
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(current_readahead_size, expected_current_readahead_size);
+ expected_current_readahead_size *= 2;
+
+ iter->Seek(BuildKey(1011));
+ ASSERT_TRUE(iter->Valid());
+
+ // Eligible to Prefetch data (not in buffer) but block is in cache so no
+ // prefetch will happen and will result in decrease in readahead_size.
+ // readahead_size will be 8 * 1024
+ iter->Seek(BuildKey(1015));
+ ASSERT_TRUE(iter->Valid());
+ expected_current_readahead_size = std::max(
+ decrease_readahead_size,
+ (expected_current_readahead_size >= decrease_readahead_size
+ ? (expected_current_readahead_size - decrease_readahead_size)
+ : 0));
+
+ // 1016 is the same block as 1015. So no change in readahead_size.
+ iter->Seek(BuildKey(1016));
+ ASSERT_TRUE(iter->Valid());
+
+ // Prefetch data (not in buffer) but found in cache. So decrease
+ // readahead_size. Since it will 0 after decrementing so readahead_size will
+ // be set to initial value.
+ iter->Seek(BuildKey(1019));
+ ASSERT_TRUE(iter->Valid());
+ expected_current_readahead_size = std::max(
+ decrease_readahead_size,
+ (expected_current_readahead_size >= decrease_readahead_size
+ ? (expected_current_readahead_size - decrease_readahead_size)
+ : 0));
+
+ // Prefetch next sequential data.
+ iter->Seek(BuildKey(1022));
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(current_readahead_size, expected_current_readahead_size);
+ ASSERT_EQ(buff_prefetch_count, 2);
+
+ buff_prefetch_count = 0;
+ }
+ Close();
+}
+
+TEST_P(PrefetchTest1, SeekParallelizationTest) {
+ const int kNumKeys = 2000;
+ // Set options
+ std::shared_ptr<MockFS> fs =
+ std::make_shared<MockFS>(env_->GetFileSystem(), false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ if (GetParam()) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+
+ options.statistics = CreateDBStatistics();
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (GetParam() && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ std::string start_key = BuildKey(0);
+ std::string end_key = BuildKey(kNumKeys - 1);
+ Slice least(start_key.data(), start_key.size());
+ Slice greatest(end_key.data(), end_key.size());
+
+ ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &least, &greatest));
+
+ int buff_prefetch_count = 0;
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::PrefetchAsyncInternal:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ SyncPoint::GetInstance()->EnableProcessing();
+ ReadOptions ro;
+ ro.adaptive_readahead = true;
+ ro.async_io = true;
+
+ {
+ ASSERT_OK(options.statistics->Reset());
+ // Each block contains around 4 keys.
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ iter->Seek(BuildKey(0)); // Prefetch data because of seek parallelization.
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+
+ // New data block. Since num_file_reads in FilePrefetch after this read is
+ // 2, it won't go for prefetching.
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+
+ // Prefetch data.
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+
+ ASSERT_EQ(buff_prefetch_count, 2);
+
+ // Check stats to make sure async prefetch is done.
+ {
+ HistogramData async_read_bytes;
+ options.statistics->histogramData(ASYNC_READ_BYTES, &async_read_bytes);
+ ASSERT_GT(async_read_bytes.count, 0);
+ ASSERT_GT(get_perf_context()->number_async_seek, 0);
+ }
+
+ buff_prefetch_count = 0;
+ }
+ Close();
+}
+
+extern "C" bool RocksDbIOUringEnable() { return true; }
+
+namespace {
+#ifndef ROCKSDB_LITE
+#ifdef GFLAGS
+const int kMaxArgCount = 100;
+const size_t kArgBufferSize = 100000;
+
+void RunIOTracerParserTool(std::string trace_file) {
+ std::vector<std::string> params = {"./io_tracer_parser",
+ "-io_trace_file=" + trace_file};
+
+ char arg_buffer[kArgBufferSize];
+ char* argv[kMaxArgCount];
+ int argc = 0;
+ int cursor = 0;
+ for (const auto& arg : params) {
+ ASSERT_LE(cursor + arg.size() + 1, kArgBufferSize);
+ ASSERT_LE(argc + 1, kMaxArgCount);
+
+ snprintf(arg_buffer + cursor, arg.size() + 1, "%s", arg.c_str());
+
+ argv[argc++] = arg_buffer + cursor;
+ cursor += static_cast<int>(arg.size()) + 1;
+ }
+ ASSERT_EQ(0, ROCKSDB_NAMESPACE::io_tracer_parser(argc, argv));
+}
+#endif // GFLAGS
+#endif // ROCKSDB_LITE
+} // namespace
+
+// Tests the default implementation of ReadAsync API with PosixFileSystem.
+TEST_P(PrefetchTest, ReadAsyncWithPosixFS) {
+ if (mem_env_ || encrypted_env_) {
+ ROCKSDB_GTEST_SKIP("Test requires non-mem or non-encrypted environment");
+ return;
+ }
+
+ const int kNumKeys = 1000;
+ std::shared_ptr<MockFS> fs = std::make_shared<MockFS>(
+ FileSystem::Default(), /*support_prefetch=*/false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ bool use_direct_io = std::get<0>(GetParam());
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ options.statistics = CreateDBStatistics();
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ int total_keys = 0;
+ // Write the keys.
+ {
+ WriteBatch batch;
+ Random rnd(309);
+ for (int j = 0; j < 5; j++) {
+ for (int i = j * kNumKeys; i < (j + 1) * kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ total_keys++;
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+ ASSERT_OK(Flush());
+ }
+ MoveFilesToLevel(2);
+ }
+
+ int buff_prefetch_count = 0;
+ bool read_async_called = false;
+ ReadOptions ro;
+ ro.adaptive_readahead = true;
+ ro.async_io = true;
+
+ if (std::get<1>(GetParam())) {
+ ro.readahead_size = 16 * 1024;
+ }
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::PrefetchAsyncInternal:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "UpdateResults::io_uring_result",
+ [&](void* /*arg*/) { read_async_called = true; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ // Read the keys.
+ {
+ ASSERT_OK(options.statistics->Reset());
+ get_perf_context()->Reset();
+
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ int num_keys = 0;
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ ASSERT_OK(iter->status());
+ num_keys++;
+ }
+
+ ASSERT_EQ(num_keys, total_keys);
+ ASSERT_GT(buff_prefetch_count, 0);
+
+ // Check stats to make sure async prefetch is done.
+ {
+ HistogramData async_read_bytes;
+ options.statistics->histogramData(ASYNC_READ_BYTES, &async_read_bytes);
+ HistogramData prefetched_bytes_discarded;
+ options.statistics->histogramData(PREFETCHED_BYTES_DISCARDED,
+ &prefetched_bytes_discarded);
+
+ // Not all platforms support iouring. In that case, ReadAsync in posix
+ // won't submit async requests.
+ if (read_async_called) {
+ ASSERT_GT(async_read_bytes.count, 0);
+ } else {
+ ASSERT_EQ(async_read_bytes.count, 0);
+ }
+ ASSERT_GT(prefetched_bytes_discarded.count, 0);
+ }
+ ASSERT_EQ(get_perf_context()->number_async_seek, 0);
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ Close();
+}
+
+TEST_P(PrefetchTest, MultipleSeekWithPosixFS) {
+ if (mem_env_ || encrypted_env_) {
+ ROCKSDB_GTEST_SKIP("Test requires non-mem or non-encrypted environment");
+ return;
+ }
+
+ const int kNumKeys = 1000;
+ std::shared_ptr<MockFS> fs = std::make_shared<MockFS>(
+ FileSystem::Default(), /*support_prefetch=*/false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ bool use_direct_io = std::get<0>(GetParam());
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ options.statistics = CreateDBStatistics();
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ int total_keys = 0;
+ // Write the keys.
+ {
+ WriteBatch batch;
+ Random rnd(309);
+ for (int j = 0; j < 5; j++) {
+ for (int i = j * kNumKeys; i < (j + 1) * kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ total_keys++;
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+ ASSERT_OK(Flush());
+ }
+ MoveFilesToLevel(2);
+ }
+
+ int num_keys_first_batch = 0;
+ int num_keys_second_batch = 0;
+ // Calculate number of keys without async_io for correctness validation.
+ {
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ReadOptions()));
+ // First Seek.
+ iter->Seek(BuildKey(450));
+ while (iter->Valid() && num_keys_first_batch < 100) {
+ ASSERT_OK(iter->status());
+ num_keys_first_batch++;
+ iter->Next();
+ }
+ ASSERT_OK(iter->status());
+
+ iter->Seek(BuildKey(942));
+ while (iter->Valid()) {
+ ASSERT_OK(iter->status());
+ num_keys_second_batch++;
+ iter->Next();
+ }
+ ASSERT_OK(iter->status());
+ }
+
+ int buff_prefetch_count = 0;
+ bool read_async_called = false;
+ ReadOptions ro;
+ ro.adaptive_readahead = true;
+ ro.async_io = true;
+
+ if (std::get<1>(GetParam())) {
+ ro.readahead_size = 16 * 1024;
+ }
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::PrefetchAsyncInternal:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "UpdateResults::io_uring_result",
+ [&](void* /*arg*/) { read_async_called = true; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ // Read the keys using seek.
+ {
+ ASSERT_OK(options.statistics->Reset());
+ get_perf_context()->Reset();
+
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ int num_keys = 0;
+ // First Seek.
+ {
+ iter->Seek(BuildKey(450));
+ while (iter->Valid() && num_keys < 100) {
+ ASSERT_OK(iter->status());
+ num_keys++;
+ iter->Next();
+ }
+ ASSERT_OK(iter->status());
+ ASSERT_EQ(num_keys, num_keys_first_batch);
+ // Check stats to make sure async prefetch is done.
+ {
+ HistogramData async_read_bytes;
+ options.statistics->histogramData(ASYNC_READ_BYTES, &async_read_bytes);
+
+ // Not all platforms support iouring. In that case, ReadAsync in posix
+ // won't submit async requests.
+ if (read_async_called) {
+ ASSERT_GT(async_read_bytes.count, 0);
+ ASSERT_GT(get_perf_context()->number_async_seek, 0);
+ } else {
+ ASSERT_EQ(async_read_bytes.count, 0);
+ ASSERT_EQ(get_perf_context()->number_async_seek, 0);
+ }
+ }
+ }
+
+ // Second Seek.
+ {
+ num_keys = 0;
+ ASSERT_OK(options.statistics->Reset());
+ get_perf_context()->Reset();
+
+ iter->Seek(BuildKey(942));
+ while (iter->Valid()) {
+ ASSERT_OK(iter->status());
+ num_keys++;
+ iter->Next();
+ }
+ ASSERT_OK(iter->status());
+ ASSERT_EQ(num_keys, num_keys_second_batch);
+
+ ASSERT_GT(buff_prefetch_count, 0);
+
+ // Check stats to make sure async prefetch is done.
+ {
+ HistogramData async_read_bytes;
+ options.statistics->histogramData(ASYNC_READ_BYTES, &async_read_bytes);
+ HistogramData prefetched_bytes_discarded;
+ options.statistics->histogramData(PREFETCHED_BYTES_DISCARDED,
+ &prefetched_bytes_discarded);
+
+ // Not all platforms support iouring. In that case, ReadAsync in posix
+ // won't submit async requests.
+ if (read_async_called) {
+ ASSERT_GT(async_read_bytes.count, 0);
+ ASSERT_GT(get_perf_context()->number_async_seek, 0);
+ } else {
+ ASSERT_EQ(async_read_bytes.count, 0);
+ ASSERT_EQ(get_perf_context()->number_async_seek, 0);
+ }
+ ASSERT_GT(prefetched_bytes_discarded.count, 0);
+ }
+ }
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+ Close();
+}
+
+TEST_P(PrefetchTest, SeekParallelizationTest1) {
+ if (mem_env_ || encrypted_env_) {
+ ROCKSDB_GTEST_SKIP("Test requires non-mem or non-encrypted environment");
+ return;
+ }
+ const int kNumKeys = 2000;
+ // Set options
+ std::shared_ptr<MockFS> fs = std::make_shared<MockFS>(
+ FileSystem::Default(), /*support_prefetch=*/false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ bool use_direct_io = std::get<0>(GetParam());
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+
+ options.statistics = CreateDBStatistics();
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ WriteBatch batch;
+ Random rnd(309);
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+
+ std::string start_key = BuildKey(0);
+ std::string end_key = BuildKey(kNumKeys - 1);
+ Slice least(start_key.data(), start_key.size());
+ Slice greatest(end_key.data(), end_key.size());
+
+ ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &least, &greatest));
+
+ int buff_prefetch_count = 0;
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::PrefetchAsyncInternal:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ bool read_async_called = false;
+ SyncPoint::GetInstance()->SetCallBack(
+ "UpdateResults::io_uring_result",
+ [&](void* /*arg*/) { read_async_called = true; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ SyncPoint::GetInstance()->EnableProcessing();
+ ReadOptions ro;
+ ro.adaptive_readahead = true;
+ ro.async_io = true;
+
+ if (std::get<1>(GetParam())) {
+ ro.readahead_size = 16 * 1024;
+ }
+
+ {
+ ASSERT_OK(options.statistics->Reset());
+ // Each block contains around 4 keys.
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ iter->Seek(BuildKey(0)); // Prefetch data because of seek parallelization.
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+
+ // New data block. Since num_file_reads in FilePrefetch after this read is
+ // 2, it won't go for prefetching.
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+
+ // Prefetch data.
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+
+ // Check stats to make sure async prefetch is done.
+ {
+ HistogramData async_read_bytes;
+ options.statistics->histogramData(ASYNC_READ_BYTES, &async_read_bytes);
+ // Not all platforms support iouring. In that case, ReadAsync in posix
+ // won't submit async requests.
+ if (read_async_called) {
+ ASSERT_GT(async_read_bytes.count, 0);
+ ASSERT_GT(get_perf_context()->number_async_seek, 0);
+ if (std::get<1>(GetParam())) {
+ ASSERT_EQ(buff_prefetch_count, 1);
+ } else {
+ ASSERT_EQ(buff_prefetch_count, 2);
+ }
+ } else {
+ ASSERT_EQ(async_read_bytes.count, 0);
+ ASSERT_EQ(get_perf_context()->number_async_seek, 0);
+ ASSERT_EQ(buff_prefetch_count, 1);
+ }
+ }
+
+ buff_prefetch_count = 0;
+ }
+ Close();
+}
+
+#ifndef ROCKSDB_LITE
+#ifdef GFLAGS
+TEST_P(PrefetchTest, TraceReadAsyncWithCallbackWrapper) {
+ if (mem_env_ || encrypted_env_) {
+ ROCKSDB_GTEST_SKIP("Test requires non-mem or non-encrypted environment");
+ return;
+ }
+
+ const int kNumKeys = 1000;
+ std::shared_ptr<MockFS> fs = std::make_shared<MockFS>(
+ FileSystem::Default(), /*support_prefetch=*/false);
+ std::unique_ptr<Env> env(new CompositeEnvWrapper(env_, fs));
+
+ bool use_direct_io = std::get<0>(GetParam());
+ Options options = CurrentOptions();
+ options.write_buffer_size = 1024;
+ options.create_if_missing = true;
+ options.compression = kNoCompression;
+ options.env = env.get();
+ options.statistics = CreateDBStatistics();
+ if (use_direct_io) {
+ options.use_direct_reads = true;
+ options.use_direct_io_for_flush_and_compaction = true;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.no_block_cache = true;
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.metadata_block_size = 1024;
+ table_options.index_type =
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ Status s = TryReopen(options);
+ if (use_direct_io && (s.IsNotSupported() || s.IsInvalidArgument())) {
+ // If direct IO is not supported, skip the test
+ return;
+ } else {
+ ASSERT_OK(s);
+ }
+
+ int total_keys = 0;
+ // Write the keys.
+ {
+ WriteBatch batch;
+ Random rnd(309);
+ for (int j = 0; j < 5; j++) {
+ for (int i = j * kNumKeys; i < (j + 1) * kNumKeys; i++) {
+ ASSERT_OK(batch.Put(BuildKey(i), rnd.RandomString(1000)));
+ total_keys++;
+ }
+ ASSERT_OK(db_->Write(WriteOptions(), &batch));
+ ASSERT_OK(Flush());
+ }
+ MoveFilesToLevel(2);
+ }
+
+ int buff_prefetch_count = 0;
+ bool read_async_called = false;
+ ReadOptions ro;
+ ro.adaptive_readahead = true;
+ ro.async_io = true;
+
+ if (std::get<1>(GetParam())) {
+ ro.readahead_size = 16 * 1024;
+ }
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "FilePrefetchBuffer::PrefetchAsyncInternal:Start",
+ [&](void*) { buff_prefetch_count++; });
+
+ SyncPoint::GetInstance()->SetCallBack(
+ "UpdateResults::io_uring_result",
+ [&](void* /*arg*/) { read_async_called = true; });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ // Read the keys.
+ {
+ // Start io_tracing.
+ WriteOptions write_opt;
+ TraceOptions trace_opt;
+ std::unique_ptr<TraceWriter> trace_writer;
+ std::string trace_file_path = dbname_ + "/io_trace_file";
+
+ ASSERT_OK(
+ NewFileTraceWriter(env_, EnvOptions(), trace_file_path, &trace_writer));
+ ASSERT_OK(db_->StartIOTrace(trace_opt, std::move(trace_writer)));
+ ASSERT_OK(options.statistics->Reset());
+
+ auto iter = std::unique_ptr<Iterator>(db_->NewIterator(ro));
+ int num_keys = 0;
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ ASSERT_OK(iter->status());
+ num_keys++;
+ }
+
+ // End the tracing.
+ ASSERT_OK(db_->EndIOTrace());
+ ASSERT_OK(env_->FileExists(trace_file_path));
+
+ ASSERT_EQ(num_keys, total_keys);
+ ASSERT_GT(buff_prefetch_count, 0);
+
+ // Check stats to make sure async prefetch is done.
+ {
+ HistogramData async_read_bytes;
+ options.statistics->histogramData(ASYNC_READ_BYTES, &async_read_bytes);
+ // Not all platforms support iouring. In that case, ReadAsync in posix
+ // won't submit async requests.
+ if (read_async_called) {
+ ASSERT_GT(async_read_bytes.count, 0);
+ } else {
+ ASSERT_EQ(async_read_bytes.count, 0);
+ }
+ }
+
+ // Check the file to see if ReadAsync is logged.
+ RunIOTracerParserTool(trace_file_path);
+ }
+
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ Close();
+}
+#endif // GFLAGS
+
+class FilePrefetchBufferTest : public testing::Test {
+ public:
+ void SetUp() override {
+ SetupSyncPointsToMockDirectIO();
+ env_ = Env::Default();
+ fs_ = FileSystem::Default();
+ test_dir_ = test::PerThreadDBPath("file_prefetch_buffer_test");
+ ASSERT_OK(fs_->CreateDir(test_dir_, IOOptions(), nullptr));
+ }
+
+ void TearDown() override { EXPECT_OK(DestroyDir(env_, test_dir_)); }
+
+ void Write(const std::string& fname, const std::string& content) {
+ std::unique_ptr<FSWritableFile> f;
+ ASSERT_OK(fs_->NewWritableFile(Path(fname), FileOptions(), &f, nullptr));
+ ASSERT_OK(f->Append(content, IOOptions(), nullptr));
+ ASSERT_OK(f->Close(IOOptions(), nullptr));
+ }
+
+ void Read(const std::string& fname, const FileOptions& opts,
+ std::unique_ptr<RandomAccessFileReader>* reader) {
+ std::string fpath = Path(fname);
+ std::unique_ptr<FSRandomAccessFile> f;
+ ASSERT_OK(fs_->NewRandomAccessFile(fpath, opts, &f, nullptr));
+ reader->reset(new RandomAccessFileReader(std::move(f), fpath,
+ env_->GetSystemClock().get()));
+ }
+
+ void AssertResult(const std::string& content,
+ const std::vector<FSReadRequest>& reqs) {
+ for (const auto& r : reqs) {
+ ASSERT_OK(r.status);
+ ASSERT_EQ(r.len, r.result.size());
+ ASSERT_EQ(content.substr(r.offset, r.len), r.result.ToString());
+ }
+ }
+
+ FileSystem* fs() { return fs_.get(); }
+
+ private:
+ Env* env_;
+ std::shared_ptr<FileSystem> fs_;
+ std::string test_dir_;
+
+ std::string Path(const std::string& fname) { return test_dir_ + "/" + fname; }
+};
+
+TEST_F(FilePrefetchBufferTest, SeekWithBlockCacheHit) {
+ std::string fname = "seek-with-block-cache-hit";
+ Random rand(0);
+ std::string content = rand.RandomString(32768);
+ Write(fname, content);
+
+ FileOptions opts;
+ std::unique_ptr<RandomAccessFileReader> r;
+ Read(fname, opts, &r);
+
+ FilePrefetchBuffer fpb(16384, 16384, true, false, false, 0, 0, fs());
+ Slice result;
+ // Simulate a seek of 4096 bytes at offset 0. Due to the readahead settings,
+ // it will do two reads of 4096+8192 and 8192
+ Status s = fpb.PrefetchAsync(IOOptions(), r.get(), 0, 4096, &result);
+ ASSERT_EQ(s, Status::TryAgain());
+ // Simulate a block cache hit
+ fpb.UpdateReadPattern(0, 4096, false);
+ // Now read some data that straddles the two prefetch buffers - offset 8192 to
+ // 16384
+ ASSERT_TRUE(fpb.TryReadFromCacheAsync(IOOptions(), r.get(), 8192, 8192,
+ &result, &s, Env::IOPriority::IO_LOW));
+}
+#endif // ROCKSDB_LITE
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/file/random_access_file_reader.cc b/src/rocksdb/file/random_access_file_reader.cc
new file mode 100644
index 000000000..030cd8d07
--- /dev/null
+++ b/src/rocksdb/file/random_access_file_reader.cc
@@ -0,0 +1,602 @@
+// 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/random_access_file_reader.h"
+
+#include <algorithm>
+#include <mutex>
+
+#include "file/file_util.h"
+#include "monitoring/histogram.h"
+#include "monitoring/iostats_context_imp.h"
+#include "port/port.h"
+#include "table/format.h"
+#include "test_util/sync_point.h"
+#include "util/random.h"
+#include "util/rate_limiter.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+inline void RecordIOStats(Statistics* stats, Temperature file_temperature,
+ bool is_last_level, size_t size) {
+ IOSTATS_ADD(bytes_read, size);
+ // record for last/non-last level
+ if (is_last_level) {
+ RecordTick(stats, LAST_LEVEL_READ_BYTES, size);
+ RecordTick(stats, LAST_LEVEL_READ_COUNT, 1);
+ } else {
+ RecordTick(stats, NON_LAST_LEVEL_READ_BYTES, size);
+ RecordTick(stats, NON_LAST_LEVEL_READ_COUNT, 1);
+ }
+
+ // record for temperature file
+ if (file_temperature != Temperature::kUnknown) {
+ switch (file_temperature) {
+ case Temperature::kHot:
+ IOSTATS_ADD(file_io_stats_by_temperature.hot_file_bytes_read, size);
+ IOSTATS_ADD(file_io_stats_by_temperature.hot_file_read_count, 1);
+ RecordTick(stats, HOT_FILE_READ_BYTES, size);
+ RecordTick(stats, HOT_FILE_READ_COUNT, 1);
+ break;
+ case Temperature::kWarm:
+ IOSTATS_ADD(file_io_stats_by_temperature.warm_file_bytes_read, size);
+ IOSTATS_ADD(file_io_stats_by_temperature.warm_file_read_count, 1);
+ RecordTick(stats, WARM_FILE_READ_BYTES, size);
+ RecordTick(stats, WARM_FILE_READ_COUNT, 1);
+ break;
+ case Temperature::kCold:
+ IOSTATS_ADD(file_io_stats_by_temperature.cold_file_bytes_read, size);
+ IOSTATS_ADD(file_io_stats_by_temperature.cold_file_read_count, 1);
+ RecordTick(stats, COLD_FILE_READ_BYTES, size);
+ RecordTick(stats, COLD_FILE_READ_COUNT, 1);
+ break;
+ default:
+ break;
+ }
+ }
+}
+
+IOStatus RandomAccessFileReader::Create(
+ const std::shared_ptr<FileSystem>& fs, const std::string& fname,
+ const FileOptions& file_opts,
+ std::unique_ptr<RandomAccessFileReader>* reader, IODebugContext* dbg) {
+ std::unique_ptr<FSRandomAccessFile> file;
+ IOStatus io_s = fs->NewRandomAccessFile(fname, file_opts, &file, dbg);
+ if (io_s.ok()) {
+ reader->reset(new RandomAccessFileReader(std::move(file), fname));
+ }
+ return io_s;
+}
+
+IOStatus RandomAccessFileReader::Read(
+ const IOOptions& opts, uint64_t offset, size_t n, Slice* result,
+ char* scratch, AlignedBuf* aligned_buf,
+ Env::IOPriority rate_limiter_priority) const {
+ (void)aligned_buf;
+
+ TEST_SYNC_POINT_CALLBACK("RandomAccessFileReader::Read", nullptr);
+
+ // To be paranoid: modify scratch a little bit, so in case underlying
+ // FileSystem doesn't fill the buffer but return success and `scratch` returns
+ // contains a previous block, returned value will not pass checksum.
+ if (n > 0 && scratch != nullptr) {
+ // This byte might not change anything for direct I/O case, but it's OK.
+ scratch[0]++;
+ }
+
+ IOStatus io_s;
+ uint64_t elapsed = 0;
+ {
+ StopWatch sw(clock_, stats_, hist_type_,
+ (stats_ != nullptr) ? &elapsed : nullptr, true /*overwrite*/,
+ true /*delay_enabled*/);
+ auto prev_perf_level = GetPerfLevel();
+ IOSTATS_TIMER_GUARD(read_nanos);
+ if (use_direct_io()) {
+#ifndef ROCKSDB_LITE
+ size_t alignment = file_->GetRequiredBufferAlignment();
+ size_t aligned_offset =
+ TruncateToPageBoundary(alignment, static_cast<size_t>(offset));
+ size_t offset_advance = static_cast<size_t>(offset) - aligned_offset;
+ size_t read_size =
+ Roundup(static_cast<size_t>(offset + n), alignment) - aligned_offset;
+ AlignedBuffer buf;
+ buf.Alignment(alignment);
+ buf.AllocateNewBuffer(read_size);
+ while (buf.CurrentSize() < read_size) {
+ size_t allowed;
+ if (rate_limiter_priority != Env::IO_TOTAL &&
+ rate_limiter_ != nullptr) {
+ allowed = rate_limiter_->RequestToken(
+ buf.Capacity() - buf.CurrentSize(), buf.Alignment(),
+ rate_limiter_priority, stats_, RateLimiter::OpType::kRead);
+ } else {
+ assert(buf.CurrentSize() == 0);
+ allowed = read_size;
+ }
+ Slice tmp;
+
+ FileOperationInfo::StartTimePoint start_ts;
+ uint64_t orig_offset = 0;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ orig_offset = aligned_offset + buf.CurrentSize();
+ }
+
+ {
+ IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_);
+ // Only user reads are expected to specify a timeout. And user reads
+ // are not subjected to rate_limiter and should go through only
+ // one iteration of this loop, so we don't need to check and adjust
+ // the opts.timeout before calling file_->Read
+ assert(!opts.timeout.count() || allowed == read_size);
+ io_s = file_->Read(aligned_offset + buf.CurrentSize(), allowed, opts,
+ &tmp, buf.Destination(), nullptr);
+ }
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileReadFinish(orig_offset, tmp.size(), start_ts, finish_ts,
+ io_s);
+ if (!io_s.ok()) {
+ NotifyOnIOError(io_s, FileOperationType::kRead, file_name(),
+ tmp.size(), orig_offset);
+ }
+ }
+
+ buf.Size(buf.CurrentSize() + tmp.size());
+ if (!io_s.ok() || tmp.size() < allowed) {
+ break;
+ }
+ }
+ size_t res_len = 0;
+ if (io_s.ok() && offset_advance < buf.CurrentSize()) {
+ res_len = std::min(buf.CurrentSize() - offset_advance, n);
+ if (aligned_buf == nullptr) {
+ buf.Read(scratch, offset_advance, res_len);
+ } else {
+ scratch = buf.BufferStart() + offset_advance;
+ aligned_buf->reset(buf.Release());
+ }
+ }
+ *result = Slice(scratch, res_len);
+#endif // !ROCKSDB_LITE
+ } else {
+ size_t pos = 0;
+ const char* res_scratch = nullptr;
+ while (pos < n) {
+ size_t allowed;
+ if (rate_limiter_priority != Env::IO_TOTAL &&
+ rate_limiter_ != nullptr) {
+ if (rate_limiter_->IsRateLimited(RateLimiter::OpType::kRead)) {
+ sw.DelayStart();
+ }
+ allowed = rate_limiter_->RequestToken(n - pos, 0 /* alignment */,
+ rate_limiter_priority, stats_,
+ RateLimiter::OpType::kRead);
+ if (rate_limiter_->IsRateLimited(RateLimiter::OpType::kRead)) {
+ sw.DelayStop();
+ }
+ } else {
+ allowed = n;
+ }
+ Slice tmp_result;
+
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+
+ {
+ IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_);
+ // Only user reads are expected to specify a timeout. And user reads
+ // are not subjected to rate_limiter and should go through only
+ // one iteration of this loop, so we don't need to check and adjust
+ // the opts.timeout before calling file_->Read
+ assert(!opts.timeout.count() || allowed == n);
+ io_s = file_->Read(offset + pos, allowed, opts, &tmp_result,
+ scratch + pos, nullptr);
+ }
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileReadFinish(offset + pos, tmp_result.size(), start_ts,
+ finish_ts, io_s);
+
+ if (!io_s.ok()) {
+ NotifyOnIOError(io_s, FileOperationType::kRead, file_name(),
+ tmp_result.size(), offset + pos);
+ }
+ }
+#endif
+ if (res_scratch == nullptr) {
+ // we can't simply use `scratch` because reads of mmap'd files return
+ // data in a different buffer.
+ res_scratch = tmp_result.data();
+ } else {
+ // make sure chunks are inserted contiguously into `res_scratch`.
+ assert(tmp_result.data() == res_scratch + pos);
+ }
+ pos += tmp_result.size();
+ if (!io_s.ok() || tmp_result.size() < allowed) {
+ break;
+ }
+ }
+ *result = Slice(res_scratch, io_s.ok() ? pos : 0);
+ }
+ RecordIOStats(stats_, file_temperature_, is_last_level_, result->size());
+ SetPerfLevel(prev_perf_level);
+ }
+ if (stats_ != nullptr && file_read_hist_ != nullptr) {
+ file_read_hist_->Add(elapsed);
+ }
+
+ return io_s;
+}
+
+size_t End(const FSReadRequest& r) {
+ return static_cast<size_t>(r.offset) + r.len;
+}
+
+FSReadRequest Align(const FSReadRequest& r, size_t alignment) {
+ FSReadRequest req;
+ req.offset = static_cast<uint64_t>(
+ TruncateToPageBoundary(alignment, static_cast<size_t>(r.offset)));
+ req.len = Roundup(End(r), alignment) - req.offset;
+ req.scratch = nullptr;
+ return req;
+}
+
+bool TryMerge(FSReadRequest* dest, const FSReadRequest& src) {
+ size_t dest_offset = static_cast<size_t>(dest->offset);
+ size_t src_offset = static_cast<size_t>(src.offset);
+ size_t dest_end = End(*dest);
+ size_t src_end = End(src);
+ if (std::max(dest_offset, src_offset) > std::min(dest_end, src_end)) {
+ return false;
+ }
+ dest->offset = static_cast<uint64_t>(std::min(dest_offset, src_offset));
+ dest->len = std::max(dest_end, src_end) - dest->offset;
+ return true;
+}
+
+IOStatus RandomAccessFileReader::MultiRead(
+ const IOOptions& opts, FSReadRequest* read_reqs, size_t num_reqs,
+ AlignedBuf* aligned_buf, Env::IOPriority rate_limiter_priority) const {
+ (void)aligned_buf; // suppress warning of unused variable in LITE mode
+ assert(num_reqs > 0);
+
+#ifndef NDEBUG
+ for (size_t i = 0; i < num_reqs - 1; ++i) {
+ assert(read_reqs[i].offset <= read_reqs[i + 1].offset);
+ }
+#endif // !NDEBUG
+
+ // To be paranoid modify scratch a little bit, so in case underlying
+ // FileSystem doesn't fill the buffer but return success and `scratch` returns
+ // contains a previous block, returned value will not pass checksum.
+ // This byte might not change anything for direct I/O case, but it's OK.
+ for (size_t i = 0; i < num_reqs; i++) {
+ FSReadRequest& r = read_reqs[i];
+ if (r.len > 0 && r.scratch != nullptr) {
+ r.scratch[0]++;
+ }
+ }
+
+ IOStatus io_s;
+ uint64_t elapsed = 0;
+ {
+ StopWatch sw(clock_, stats_, hist_type_,
+ (stats_ != nullptr) ? &elapsed : nullptr, true /*overwrite*/,
+ true /*delay_enabled*/);
+ auto prev_perf_level = GetPerfLevel();
+ IOSTATS_TIMER_GUARD(read_nanos);
+
+ FSReadRequest* fs_reqs = read_reqs;
+ size_t num_fs_reqs = num_reqs;
+#ifndef ROCKSDB_LITE
+ std::vector<FSReadRequest> aligned_reqs;
+ if (use_direct_io()) {
+ // num_reqs is the max possible size,
+ // this can reduce std::vecector's internal resize operations.
+ aligned_reqs.reserve(num_reqs);
+ // Align and merge the read requests.
+ size_t alignment = file_->GetRequiredBufferAlignment();
+ for (size_t i = 0; i < num_reqs; i++) {
+ const auto& r = Align(read_reqs[i], alignment);
+ if (i == 0) {
+ // head
+ aligned_reqs.push_back(r);
+
+ } else if (!TryMerge(&aligned_reqs.back(), r)) {
+ // head + n
+ aligned_reqs.push_back(r);
+
+ } else {
+ // unused
+ r.status.PermitUncheckedError();
+ }
+ }
+ TEST_SYNC_POINT_CALLBACK("RandomAccessFileReader::MultiRead:AlignedReqs",
+ &aligned_reqs);
+
+ // Allocate aligned buffer and let scratch buffers point to it.
+ size_t total_len = 0;
+ for (const auto& r : aligned_reqs) {
+ total_len += r.len;
+ }
+ AlignedBuffer buf;
+ buf.Alignment(alignment);
+ buf.AllocateNewBuffer(total_len);
+ char* scratch = buf.BufferStart();
+ for (auto& r : aligned_reqs) {
+ r.scratch = scratch;
+ scratch += r.len;
+ }
+
+ aligned_buf->reset(buf.Release());
+ fs_reqs = aligned_reqs.data();
+ num_fs_reqs = aligned_reqs.size();
+ }
+#endif // ROCKSDB_LITE
+
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif // ROCKSDB_LITE
+
+ {
+ IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, clock_);
+ if (rate_limiter_priority != Env::IO_TOTAL && rate_limiter_ != nullptr) {
+ // TODO: ideally we should call `RateLimiter::RequestToken()` for
+ // allowed bytes to multi-read and then consume those bytes by
+ // satisfying as many requests in `MultiRead()` as possible, instead of
+ // what we do here, which can cause burst when the
+ // `total_multi_read_size` is big.
+ size_t total_multi_read_size = 0;
+ assert(fs_reqs != nullptr);
+ for (size_t i = 0; i < num_fs_reqs; ++i) {
+ FSReadRequest& req = fs_reqs[i];
+ total_multi_read_size += req.len;
+ }
+ size_t remaining_bytes = total_multi_read_size;
+ size_t request_bytes = 0;
+ while (remaining_bytes > 0) {
+ request_bytes = std::min(
+ static_cast<size_t>(rate_limiter_->GetSingleBurstBytes()),
+ remaining_bytes);
+ rate_limiter_->Request(request_bytes, rate_limiter_priority,
+ nullptr /* stats */,
+ RateLimiter::OpType::kRead);
+ remaining_bytes -= request_bytes;
+ }
+ }
+ io_s = file_->MultiRead(fs_reqs, num_fs_reqs, opts, nullptr);
+ RecordInHistogram(stats_, MULTIGET_IO_BATCH_SIZE, num_fs_reqs);
+ }
+
+#ifndef ROCKSDB_LITE
+ if (use_direct_io()) {
+ // Populate results in the unaligned read requests.
+ size_t aligned_i = 0;
+ for (size_t i = 0; i < num_reqs; i++) {
+ auto& r = read_reqs[i];
+ if (static_cast<size_t>(r.offset) > End(aligned_reqs[aligned_i])) {
+ aligned_i++;
+ }
+ const auto& fs_r = fs_reqs[aligned_i];
+ r.status = fs_r.status;
+ if (r.status.ok()) {
+ uint64_t offset = r.offset - fs_r.offset;
+ if (fs_r.result.size() <= offset) {
+ // No byte in the read range is returned.
+ r.result = Slice();
+ } else {
+ size_t len = std::min(
+ r.len, static_cast<size_t>(fs_r.result.size() - offset));
+ r.result = Slice(fs_r.scratch + offset, len);
+ }
+ } else {
+ r.result = Slice();
+ }
+ }
+ }
+#endif // ROCKSDB_LITE
+
+ for (size_t i = 0; i < num_reqs; ++i) {
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileReadFinish(read_reqs[i].offset, read_reqs[i].result.size(),
+ start_ts, finish_ts, read_reqs[i].status);
+ }
+ if (!read_reqs[i].status.ok()) {
+ NotifyOnIOError(read_reqs[i].status, FileOperationType::kRead,
+ file_name(), read_reqs[i].result.size(),
+ read_reqs[i].offset);
+ }
+
+#endif // ROCKSDB_LITE
+ RecordIOStats(stats_, file_temperature_, is_last_level_,
+ read_reqs[i].result.size());
+ }
+ SetPerfLevel(prev_perf_level);
+ }
+ if (stats_ != nullptr && file_read_hist_ != nullptr) {
+ file_read_hist_->Add(elapsed);
+ }
+
+ return io_s;
+}
+
+IOStatus RandomAccessFileReader::PrepareIOOptions(const ReadOptions& ro,
+ IOOptions& opts) {
+ if (clock_ != nullptr) {
+ return PrepareIOFromReadOptions(ro, clock_, opts);
+ } else {
+ return PrepareIOFromReadOptions(ro, SystemClock::Default().get(), opts);
+ }
+}
+
+IOStatus RandomAccessFileReader::ReadAsync(
+ FSReadRequest& req, const IOOptions& opts,
+ std::function<void(const FSReadRequest&, void*)> cb, void* cb_arg,
+ void** io_handle, IOHandleDeleter* del_fn, AlignedBuf* aligned_buf) {
+ IOStatus s;
+ // Create a callback and populate info.
+ auto read_async_callback =
+ std::bind(&RandomAccessFileReader::ReadAsyncCallback, this,
+ std::placeholders::_1, std::placeholders::_2);
+ ReadAsyncInfo* read_async_info =
+ new ReadAsyncInfo(cb, cb_arg, clock_->NowMicros());
+
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ read_async_info->fs_start_ts_ = FileOperationInfo::StartNow();
+ }
+#endif
+
+ size_t alignment = file_->GetRequiredBufferAlignment();
+ bool is_aligned = (req.offset & (alignment - 1)) == 0 &&
+ (req.len & (alignment - 1)) == 0 &&
+ (uintptr_t(req.scratch) & (alignment - 1)) == 0;
+ read_async_info->is_aligned_ = is_aligned;
+
+ uint64_t elapsed = 0;
+ if (use_direct_io() && is_aligned == false) {
+ FSReadRequest aligned_req = Align(req, alignment);
+ aligned_req.status.PermitUncheckedError();
+
+ // Allocate aligned buffer.
+ read_async_info->buf_.Alignment(alignment);
+ read_async_info->buf_.AllocateNewBuffer(aligned_req.len);
+
+ // Set rem fields in aligned FSReadRequest.
+ aligned_req.scratch = read_async_info->buf_.BufferStart();
+
+ // Set user provided fields to populate back in callback.
+ read_async_info->user_scratch_ = req.scratch;
+ read_async_info->user_aligned_buf_ = aligned_buf;
+ read_async_info->user_len_ = req.len;
+ read_async_info->user_offset_ = req.offset;
+ read_async_info->user_result_ = req.result;
+
+ assert(read_async_info->buf_.CurrentSize() == 0);
+
+ StopWatch sw(clock_, nullptr /*stats*/, 0 /*hist_type*/, &elapsed,
+ true /*overwrite*/, true /*delay_enabled*/);
+ s = file_->ReadAsync(aligned_req, opts, read_async_callback,
+ read_async_info, io_handle, del_fn, nullptr /*dbg*/);
+ } else {
+ StopWatch sw(clock_, nullptr /*stats*/, 0 /*hist_type*/, &elapsed,
+ true /*overwrite*/, true /*delay_enabled*/);
+ s = file_->ReadAsync(req, opts, read_async_callback, read_async_info,
+ io_handle, del_fn, nullptr /*dbg*/);
+ }
+ RecordTick(stats_, READ_ASYNC_MICROS, elapsed);
+
+// Suppress false positive clang analyzer warnings.
+// Memory is not released if file_->ReadAsync returns !s.ok(), because
+// ReadAsyncCallback is never called in that case. If ReadAsyncCallback is
+// called then ReadAsync should always return IOStatus::OK().
+#ifndef __clang_analyzer__
+ if (!s.ok()) {
+ delete read_async_info;
+ }
+#endif // __clang_analyzer__
+
+ return s;
+}
+
+void RandomAccessFileReader::ReadAsyncCallback(const FSReadRequest& req,
+ void* cb_arg) {
+ ReadAsyncInfo* read_async_info = static_cast<ReadAsyncInfo*>(cb_arg);
+ assert(read_async_info);
+ assert(read_async_info->cb_);
+
+ if (use_direct_io() && read_async_info->is_aligned_ == false) {
+ // Create FSReadRequest with user provided fields.
+ FSReadRequest user_req;
+ user_req.scratch = read_async_info->user_scratch_;
+ user_req.offset = read_async_info->user_offset_;
+ user_req.len = read_async_info->user_len_;
+
+ // Update results in user_req.
+ user_req.result = req.result;
+ user_req.status = req.status;
+
+ read_async_info->buf_.Size(read_async_info->buf_.CurrentSize() +
+ req.result.size());
+
+ size_t offset_advance_len = static_cast<size_t>(
+ /*offset_passed_by_user=*/read_async_info->user_offset_ -
+ /*aligned_offset=*/req.offset);
+
+ size_t res_len = 0;
+ if (req.status.ok() &&
+ offset_advance_len < read_async_info->buf_.CurrentSize()) {
+ res_len =
+ std::min(read_async_info->buf_.CurrentSize() - offset_advance_len,
+ read_async_info->user_len_);
+ if (read_async_info->user_aligned_buf_ == nullptr) {
+ // Copy the data into user's scratch.
+// Clang analyzer assumes that it will take use_direct_io() == false in
+// ReadAsync and use_direct_io() == true in Callback which cannot be true.
+#ifndef __clang_analyzer__
+ read_async_info->buf_.Read(user_req.scratch, offset_advance_len,
+ res_len);
+#endif // __clang_analyzer__
+ } else {
+ // Set aligned_buf provided by user without additional copy.
+ user_req.scratch =
+ read_async_info->buf_.BufferStart() + offset_advance_len;
+ read_async_info->user_aligned_buf_->reset(
+ read_async_info->buf_.Release());
+ }
+ user_req.result = Slice(user_req.scratch, res_len);
+ } else {
+ // Either req.status is not ok or data was not read.
+ user_req.result = Slice();
+ }
+ read_async_info->cb_(user_req, read_async_info->cb_arg_);
+ } else {
+ read_async_info->cb_(req, read_async_info->cb_arg_);
+ }
+
+ // Update stats and notify listeners.
+ if (stats_ != nullptr && file_read_hist_ != nullptr) {
+ // elapsed doesn't take into account delay and overwrite as StopWatch does
+ // in Read.
+ uint64_t elapsed = clock_->NowMicros() - read_async_info->start_time_;
+ file_read_hist_->Add(elapsed);
+ }
+ if (req.status.ok()) {
+ RecordInHistogram(stats_, ASYNC_READ_BYTES, req.result.size());
+ } else if (!req.status.IsAborted()) {
+ RecordTick(stats_, ASYNC_READ_ERROR_COUNT, 1);
+ }
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileReadFinish(req.offset, req.result.size(),
+ read_async_info->fs_start_ts_, finish_ts,
+ req.status);
+ }
+ if (!req.status.ok()) {
+ NotifyOnIOError(req.status, FileOperationType::kRead, file_name(),
+ req.result.size(), req.offset);
+ }
+#endif
+ RecordIOStats(stats_, file_temperature_, is_last_level_, req.result.size());
+ delete read_async_info;
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/random_access_file_reader.h b/src/rocksdb/file/random_access_file_reader.h
new file mode 100644
index 000000000..ea7cfd234
--- /dev/null
+++ b/src/rocksdb/file/random_access_file_reader.h
@@ -0,0 +1,217 @@
+// 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 <sstream>
+#include <string>
+
+#include "env/file_system_tracer.h"
+#include "port/port.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/listener.h"
+#include "rocksdb/options.h"
+#include "rocksdb/rate_limiter.h"
+#include "util/aligned_buffer.h"
+
+namespace ROCKSDB_NAMESPACE {
+class Statistics;
+class HistogramImpl;
+class SystemClock;
+
+using AlignedBuf = std::unique_ptr<char[]>;
+
+// Align the request r according to alignment and return the aligned result.
+FSReadRequest Align(const FSReadRequest& r, size_t alignment);
+
+// Try to merge src to dest if they have overlap.
+//
+// Each request represents an inclusive interval [offset, offset + len].
+// If the intervals have overlap, update offset and len to represent the
+// merged interval, and return true.
+// Otherwise, do nothing and return false.
+bool TryMerge(FSReadRequest* dest, const FSReadRequest& src);
+
+// RandomAccessFileReader is a wrapper on top of FSRandomAccessFile. It is
+// responsible for:
+// - Handling Buffered and Direct reads appropriately.
+// - Rate limiting compaction reads.
+// - Notifying any interested listeners on the completion of a read.
+// - Updating IO stats.
+class RandomAccessFileReader {
+ 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, file_temperature_);
+ info.offset = offset;
+ info.length = length;
+
+ for (auto& listener : listeners_) {
+ listener->OnFileReadFinish(info);
+ }
+ info.status.PermitUncheckedError();
+ }
+
+ void NotifyOnIOError(const IOStatus& io_status, FileOperationType operation,
+ const std::string& file_path, size_t length,
+ uint64_t offset) const {
+ if (listeners_.empty()) {
+ return;
+ }
+ IOErrorInfo io_error_info(io_status, operation, file_path, length, offset);
+
+ for (auto& listener : listeners_) {
+ listener->OnIOError(io_error_info);
+ }
+ io_status.PermitUncheckedError();
+ }
+
+#endif // ROCKSDB_LITE
+
+ bool ShouldNotifyListeners() const { return !listeners_.empty(); }
+
+ FSRandomAccessFilePtr file_;
+ std::string file_name_;
+ SystemClock* clock_;
+ Statistics* stats_;
+ uint32_t hist_type_;
+ HistogramImpl* file_read_hist_;
+ RateLimiter* rate_limiter_;
+ std::vector<std::shared_ptr<EventListener>> listeners_;
+ const Temperature file_temperature_;
+ const bool is_last_level_;
+
+ struct ReadAsyncInfo {
+ ReadAsyncInfo(std::function<void(const FSReadRequest&, void*)> cb,
+ void* cb_arg, uint64_t start_time)
+ : cb_(cb),
+ cb_arg_(cb_arg),
+ start_time_(start_time),
+ user_scratch_(nullptr),
+ user_aligned_buf_(nullptr),
+ user_offset_(0),
+ user_len_(0),
+ is_aligned_(false) {}
+
+ std::function<void(const FSReadRequest&, void*)> cb_;
+ void* cb_arg_;
+ uint64_t start_time_;
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint fs_start_ts_;
+#endif
+ // Below fields stores the parameters passed by caller in case of direct_io.
+ char* user_scratch_;
+ AlignedBuf* user_aligned_buf_;
+ uint64_t user_offset_;
+ size_t user_len_;
+ Slice user_result_;
+ // Used in case of direct_io
+ AlignedBuffer buf_;
+ bool is_aligned_;
+ };
+
+ public:
+ explicit RandomAccessFileReader(
+ std::unique_ptr<FSRandomAccessFile>&& raf, const std::string& _file_name,
+ SystemClock* clock = nullptr,
+ const std::shared_ptr<IOTracer>& io_tracer = nullptr,
+ Statistics* stats = nullptr, uint32_t hist_type = 0,
+ HistogramImpl* file_read_hist = nullptr,
+ RateLimiter* rate_limiter = nullptr,
+ const std::vector<std::shared_ptr<EventListener>>& listeners = {},
+ Temperature file_temperature = Temperature::kUnknown,
+ bool is_last_level = false)
+ : file_(std::move(raf), io_tracer, _file_name),
+ file_name_(std::move(_file_name)),
+ clock_(clock),
+ stats_(stats),
+ hist_type_(hist_type),
+ file_read_hist_(file_read_hist),
+ rate_limiter_(rate_limiter),
+ listeners_(),
+ file_temperature_(file_temperature),
+ is_last_level_(is_last_level) {
+#ifndef ROCKSDB_LITE
+ std::for_each(listeners.begin(), listeners.end(),
+ [this](const std::shared_ptr<EventListener>& e) {
+ if (e->ShouldBeNotifiedOnFileIO()) {
+ listeners_.emplace_back(e);
+ }
+ });
+#else // !ROCKSDB_LITE
+ (void)listeners;
+#endif
+ }
+
+ static IOStatus Create(const std::shared_ptr<FileSystem>& fs,
+ const std::string& fname, const FileOptions& file_opts,
+ std::unique_ptr<RandomAccessFileReader>* reader,
+ IODebugContext* dbg);
+ RandomAccessFileReader(const RandomAccessFileReader&) = delete;
+ RandomAccessFileReader& operator=(const RandomAccessFileReader&) = delete;
+
+ // In non-direct IO mode,
+ // 1. if using mmap, result is stored in a buffer other than scratch;
+ // 2. if not using mmap, result is stored in the buffer starting from scratch.
+ //
+ // In direct IO mode, an aligned buffer is allocated internally.
+ // 1. If aligned_buf is null, then results are copied to the buffer
+ // starting from scratch;
+ // 2. Otherwise, scratch is not used and can be null, the aligned_buf owns
+ // the internally allocated buffer on return, and the result refers to a
+ // region in aligned_buf.
+ //
+ // `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.
+ IOStatus Read(const IOOptions& opts, uint64_t offset, size_t n, Slice* result,
+ char* scratch, AlignedBuf* aligned_buf,
+ Env::IOPriority rate_limiter_priority) const;
+
+ // REQUIRES:
+ // num_reqs > 0, reqs do not overlap, and offsets in reqs are increasing.
+ // In non-direct IO mode, aligned_buf should be null;
+ // In direct IO mode, aligned_buf stores the aligned buffer allocated inside
+ // MultiRead, the result Slices in reqs refer to aligned_buf.
+ //
+ // `rate_limiter_priority` will be used to charge the internal rate limiter.
+ // It is not yet supported so the client must provide the special value
+ // `Env::IO_TOTAL` to bypass the rate limiter.
+ IOStatus MultiRead(const IOOptions& opts, FSReadRequest* reqs,
+ size_t num_reqs, AlignedBuf* aligned_buf,
+ Env::IOPriority rate_limiter_priority) const;
+
+ IOStatus Prefetch(uint64_t offset, size_t n,
+ const Env::IOPriority rate_limiter_priority) const {
+ IOOptions opts;
+ opts.rate_limiter_priority = rate_limiter_priority;
+ return file_->Prefetch(offset, n, opts, nullptr);
+ }
+
+ FSRandomAccessFile* file() { return file_.get(); }
+
+ const std::string& file_name() const { return file_name_; }
+
+ bool use_direct_io() const { return file_->use_direct_io(); }
+
+ IOStatus PrepareIOOptions(const ReadOptions& ro, IOOptions& opts);
+
+ IOStatus ReadAsync(FSReadRequest& req, const IOOptions& opts,
+ std::function<void(const FSReadRequest&, void*)> cb,
+ void* cb_arg, void** io_handle, IOHandleDeleter* del_fn,
+ AlignedBuf* aligned_buf);
+
+ void ReadAsyncCallback(const FSReadRequest& req, void* cb_arg);
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/random_access_file_reader_test.cc b/src/rocksdb/file/random_access_file_reader_test.cc
new file mode 100644
index 000000000..ac0e9e57a
--- /dev/null
+++ b/src/rocksdb/file/random_access_file_reader_test.cc
@@ -0,0 +1,481 @@
+// 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).
+
+#include "file/random_access_file_reader.h"
+
+#include <algorithm>
+
+#include "file/file_util.h"
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/file_system.h"
+#include "test_util/sync_point.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class RandomAccessFileReaderTest : public testing::Test {
+ public:
+ void SetUp() override {
+ SetupSyncPointsToMockDirectIO();
+ env_ = Env::Default();
+ fs_ = FileSystem::Default();
+ test_dir_ = test::PerThreadDBPath("random_access_file_reader_test");
+ ASSERT_OK(fs_->CreateDir(test_dir_, IOOptions(), nullptr));
+ }
+
+ void TearDown() override { EXPECT_OK(DestroyDir(env_, test_dir_)); }
+
+ void Write(const std::string& fname, const std::string& content) {
+ std::unique_ptr<FSWritableFile> f;
+ ASSERT_OK(fs_->NewWritableFile(Path(fname), FileOptions(), &f, nullptr));
+ ASSERT_OK(f->Append(content, IOOptions(), nullptr));
+ ASSERT_OK(f->Close(IOOptions(), nullptr));
+ }
+
+ void Read(const std::string& fname, const FileOptions& opts,
+ std::unique_ptr<RandomAccessFileReader>* reader) {
+ std::string fpath = Path(fname);
+ std::unique_ptr<FSRandomAccessFile> f;
+ ASSERT_OK(fs_->NewRandomAccessFile(fpath, opts, &f, nullptr));
+ reader->reset(new RandomAccessFileReader(std::move(f), fpath,
+ env_->GetSystemClock().get()));
+ }
+
+ void AssertResult(const std::string& content,
+ const std::vector<FSReadRequest>& reqs) {
+ for (const auto& r : reqs) {
+ ASSERT_OK(r.status);
+ ASSERT_EQ(r.len, r.result.size());
+ ASSERT_EQ(content.substr(r.offset, r.len), r.result.ToString());
+ }
+ }
+
+ private:
+ Env* env_;
+ std::shared_ptr<FileSystem> fs_;
+ std::string test_dir_;
+
+ std::string Path(const std::string& fname) { return test_dir_ + "/" + fname; }
+};
+
+// Skip the following tests in lite mode since direct I/O is unsupported.
+#ifndef ROCKSDB_LITE
+
+TEST_F(RandomAccessFileReaderTest, ReadDirectIO) {
+ std::string fname = "read-direct-io";
+ Random rand(0);
+ std::string content = rand.RandomString(kDefaultPageSize);
+ Write(fname, content);
+
+ FileOptions opts;
+ opts.use_direct_reads = true;
+ std::unique_ptr<RandomAccessFileReader> r;
+ Read(fname, opts, &r);
+ ASSERT_TRUE(r->use_direct_io());
+
+ const size_t page_size = r->file()->GetRequiredBufferAlignment();
+ size_t offset = page_size / 2;
+ size_t len = page_size / 3;
+ Slice result;
+ AlignedBuf buf;
+ for (Env::IOPriority rate_limiter_priority : {Env::IO_LOW, Env::IO_TOTAL}) {
+ ASSERT_OK(r->Read(IOOptions(), offset, len, &result, nullptr, &buf,
+ rate_limiter_priority));
+ ASSERT_EQ(result.ToString(), content.substr(offset, len));
+ }
+}
+
+TEST_F(RandomAccessFileReaderTest, MultiReadDirectIO) {
+ std::vector<FSReadRequest> aligned_reqs;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "RandomAccessFileReader::MultiRead:AlignedReqs", [&](void* reqs) {
+ // Copy reqs, since it's allocated on stack inside MultiRead, which will
+ // be deallocated after MultiRead returns.
+ aligned_reqs = *reinterpret_cast<std::vector<FSReadRequest>*>(reqs);
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ // Creates a file with 3 pages.
+ std::string fname = "multi-read-direct-io";
+ Random rand(0);
+ std::string content = rand.RandomString(3 * kDefaultPageSize);
+ Write(fname, content);
+
+ FileOptions opts;
+ opts.use_direct_reads = true;
+ std::unique_ptr<RandomAccessFileReader> r;
+ Read(fname, opts, &r);
+ ASSERT_TRUE(r->use_direct_io());
+
+ const size_t page_size = r->file()->GetRequiredBufferAlignment();
+
+ {
+ // Reads 2 blocks in the 1st page.
+ // The results should be SharedSlices of the same underlying buffer.
+ //
+ // Illustration (each x is a 1/4 page)
+ // First page: xxxx
+ // 1st block: x
+ // 2nd block: xx
+ FSReadRequest r0;
+ r0.offset = 0;
+ r0.len = page_size / 4;
+ r0.scratch = nullptr;
+
+ FSReadRequest r1;
+ r1.offset = page_size / 2;
+ r1.len = page_size / 2;
+ r1.scratch = nullptr;
+
+ std::vector<FSReadRequest> reqs;
+ reqs.push_back(std::move(r0));
+ reqs.push_back(std::move(r1));
+ AlignedBuf aligned_buf;
+ ASSERT_OK(r->MultiRead(IOOptions(), reqs.data(), reqs.size(), &aligned_buf,
+ Env::IO_TOTAL /* rate_limiter_priority */));
+
+ AssertResult(content, reqs);
+
+ // Reads the first page internally.
+ ASSERT_EQ(aligned_reqs.size(), 1);
+ const FSReadRequest& aligned_r = aligned_reqs[0];
+ ASSERT_OK(aligned_r.status);
+ ASSERT_EQ(aligned_r.offset, 0);
+ ASSERT_EQ(aligned_r.len, page_size);
+ }
+
+ {
+ // Reads 3 blocks:
+ // 1st block in the 1st page;
+ // 2nd block from the middle of the 1st page to the middle of the 2nd page;
+ // 3rd block in the 2nd page.
+ // The results should be SharedSlices of the same underlying buffer.
+ //
+ // Illustration (each x is a 1/4 page)
+ // 2 pages: xxxxxxxx
+ // 1st block: x
+ // 2nd block: xxxx
+ // 3rd block: x
+ FSReadRequest r0;
+ r0.offset = 0;
+ r0.len = page_size / 4;
+ r0.scratch = nullptr;
+
+ FSReadRequest r1;
+ r1.offset = page_size / 2;
+ r1.len = page_size;
+ r1.scratch = nullptr;
+
+ FSReadRequest r2;
+ r2.offset = 2 * page_size - page_size / 4;
+ r2.len = page_size / 4;
+ r2.scratch = nullptr;
+
+ std::vector<FSReadRequest> reqs;
+ reqs.push_back(std::move(r0));
+ reqs.push_back(std::move(r1));
+ reqs.push_back(std::move(r2));
+ AlignedBuf aligned_buf;
+ ASSERT_OK(r->MultiRead(IOOptions(), reqs.data(), reqs.size(), &aligned_buf,
+ Env::IO_TOTAL /* rate_limiter_priority */));
+
+ AssertResult(content, reqs);
+
+ // Reads the first two pages in one request internally.
+ ASSERT_EQ(aligned_reqs.size(), 1);
+ const FSReadRequest& aligned_r = aligned_reqs[0];
+ ASSERT_OK(aligned_r.status);
+ ASSERT_EQ(aligned_r.offset, 0);
+ ASSERT_EQ(aligned_r.len, 2 * page_size);
+ }
+
+ {
+ // Reads 3 blocks:
+ // 1st block in the middle of the 1st page;
+ // 2nd block in the middle of the 2nd page;
+ // 3rd block in the middle of the 3rd page.
+ // The results should be SharedSlices of the same underlying buffer.
+ //
+ // Illustration (each x is a 1/4 page)
+ // 3 pages: xxxxxxxxxxxx
+ // 1st block: xx
+ // 2nd block: xx
+ // 3rd block: xx
+ FSReadRequest r0;
+ r0.offset = page_size / 4;
+ r0.len = page_size / 2;
+ r0.scratch = nullptr;
+
+ FSReadRequest r1;
+ r1.offset = page_size + page_size / 4;
+ r1.len = page_size / 2;
+ r1.scratch = nullptr;
+
+ FSReadRequest r2;
+ r2.offset = 2 * page_size + page_size / 4;
+ r2.len = page_size / 2;
+ r2.scratch = nullptr;
+
+ std::vector<FSReadRequest> reqs;
+ reqs.push_back(std::move(r0));
+ reqs.push_back(std::move(r1));
+ reqs.push_back(std::move(r2));
+ AlignedBuf aligned_buf;
+ ASSERT_OK(r->MultiRead(IOOptions(), reqs.data(), reqs.size(), &aligned_buf,
+ Env::IO_TOTAL /* rate_limiter_priority */));
+
+ AssertResult(content, reqs);
+
+ // Reads the first 3 pages in one request internally.
+ ASSERT_EQ(aligned_reqs.size(), 1);
+ const FSReadRequest& aligned_r = aligned_reqs[0];
+ ASSERT_OK(aligned_r.status);
+ ASSERT_EQ(aligned_r.offset, 0);
+ ASSERT_EQ(aligned_r.len, 3 * page_size);
+ }
+
+ {
+ // Reads 2 blocks:
+ // 1st block in the middle of the 1st page;
+ // 2nd block in the middle of the 3rd page.
+ // The results are two different buffers.
+ //
+ // Illustration (each x is a 1/4 page)
+ // 3 pages: xxxxxxxxxxxx
+ // 1st block: xx
+ // 2nd block: xx
+ FSReadRequest r0;
+ r0.offset = page_size / 4;
+ r0.len = page_size / 2;
+ r0.scratch = nullptr;
+
+ FSReadRequest r1;
+ r1.offset = 2 * page_size + page_size / 4;
+ r1.len = page_size / 2;
+ r1.scratch = nullptr;
+
+ std::vector<FSReadRequest> reqs;
+ reqs.push_back(std::move(r0));
+ reqs.push_back(std::move(r1));
+ AlignedBuf aligned_buf;
+ ASSERT_OK(r->MultiRead(IOOptions(), reqs.data(), reqs.size(), &aligned_buf,
+ Env::IO_TOTAL /* rate_limiter_priority */));
+
+ AssertResult(content, reqs);
+
+ // Reads the 1st and 3rd pages in two requests internally.
+ ASSERT_EQ(aligned_reqs.size(), 2);
+ const FSReadRequest& aligned_r0 = aligned_reqs[0];
+ const FSReadRequest& aligned_r1 = aligned_reqs[1];
+ ASSERT_OK(aligned_r0.status);
+ ASSERT_EQ(aligned_r0.offset, 0);
+ ASSERT_EQ(aligned_r0.len, page_size);
+ ASSERT_OK(aligned_r1.status);
+ ASSERT_EQ(aligned_r1.offset, 2 * page_size);
+ ASSERT_EQ(aligned_r1.len, page_size);
+ }
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+#endif // ROCKSDB_LITE
+
+TEST(FSReadRequest, Align) {
+ FSReadRequest r;
+ r.offset = 2000;
+ r.len = 2000;
+ r.scratch = nullptr;
+ ASSERT_OK(r.status);
+
+ FSReadRequest aligned_r = Align(r, 1024);
+ ASSERT_OK(r.status);
+ ASSERT_OK(aligned_r.status);
+ ASSERT_EQ(aligned_r.offset, 1024);
+ ASSERT_EQ(aligned_r.len, 3072);
+}
+
+TEST(FSReadRequest, TryMerge) {
+ // reverse means merging dest into src.
+ for (bool reverse : {true, false}) {
+ {
+ // dest: [ ]
+ // src: [ ]
+ FSReadRequest dest;
+ dest.offset = 0;
+ dest.len = 10;
+ dest.scratch = nullptr;
+ ASSERT_OK(dest.status);
+
+ FSReadRequest src;
+ src.offset = 15;
+ src.len = 10;
+ src.scratch = nullptr;
+ ASSERT_OK(src.status);
+
+ if (reverse) {
+ std::swap(dest, src);
+ }
+ ASSERT_FALSE(TryMerge(&dest, src));
+ ASSERT_OK(dest.status);
+ ASSERT_OK(src.status);
+ }
+
+ {
+ // dest: [ ]
+ // src: [ ]
+ FSReadRequest dest;
+ dest.offset = 0;
+ dest.len = 10;
+ dest.scratch = nullptr;
+ ASSERT_OK(dest.status);
+
+ FSReadRequest src;
+ src.offset = 10;
+ src.len = 10;
+ src.scratch = nullptr;
+ ASSERT_OK(src.status);
+
+ if (reverse) {
+ std::swap(dest, src);
+ }
+ ASSERT_TRUE(TryMerge(&dest, src));
+ ASSERT_EQ(dest.offset, 0);
+ ASSERT_EQ(dest.len, 20);
+ ASSERT_OK(dest.status);
+ ASSERT_OK(src.status);
+ }
+
+ {
+ // dest: [ ]
+ // src: [ ]
+ FSReadRequest dest;
+ dest.offset = 0;
+ dest.len = 10;
+ dest.scratch = nullptr;
+ ASSERT_OK(dest.status);
+
+ FSReadRequest src;
+ src.offset = 5;
+ src.len = 10;
+ src.scratch = nullptr;
+ ASSERT_OK(src.status);
+
+ if (reverse) {
+ std::swap(dest, src);
+ }
+ ASSERT_TRUE(TryMerge(&dest, src));
+ ASSERT_EQ(dest.offset, 0);
+ ASSERT_EQ(dest.len, 15);
+ ASSERT_OK(dest.status);
+ ASSERT_OK(src.status);
+ }
+
+ {
+ // dest: [ ]
+ // src: [ ]
+ FSReadRequest dest;
+ dest.offset = 0;
+ dest.len = 10;
+ dest.scratch = nullptr;
+ ASSERT_OK(dest.status);
+
+ FSReadRequest src;
+ src.offset = 5;
+ src.len = 5;
+ src.scratch = nullptr;
+ ASSERT_OK(src.status);
+
+ if (reverse) {
+ std::swap(dest, src);
+ }
+ ASSERT_TRUE(TryMerge(&dest, src));
+ ASSERT_EQ(dest.offset, 0);
+ ASSERT_EQ(dest.len, 10);
+ ASSERT_OK(dest.status);
+ ASSERT_OK(src.status);
+ }
+
+ {
+ // dest: [ ]
+ // src: [ ]
+ FSReadRequest dest;
+ dest.offset = 0;
+ dest.len = 10;
+ dest.scratch = nullptr;
+ ASSERT_OK(dest.status);
+
+ FSReadRequest src;
+ src.offset = 5;
+ src.len = 1;
+ src.scratch = nullptr;
+ ASSERT_OK(src.status);
+
+ if (reverse) std::swap(dest, src);
+ ASSERT_TRUE(TryMerge(&dest, src));
+ ASSERT_EQ(dest.offset, 0);
+ ASSERT_EQ(dest.len, 10);
+ ASSERT_OK(dest.status);
+ ASSERT_OK(src.status);
+ }
+
+ {
+ // dest: [ ]
+ // src: [ ]
+ FSReadRequest dest;
+ dest.offset = 0;
+ dest.len = 10;
+ dest.scratch = nullptr;
+ ASSERT_OK(dest.status);
+
+ FSReadRequest src;
+ src.offset = 0;
+ src.len = 10;
+ src.scratch = nullptr;
+ ASSERT_OK(src.status);
+
+ if (reverse) std::swap(dest, src);
+ ASSERT_TRUE(TryMerge(&dest, src));
+ ASSERT_EQ(dest.offset, 0);
+ ASSERT_EQ(dest.len, 10);
+ ASSERT_OK(dest.status);
+ ASSERT_OK(src.status);
+ }
+
+ {
+ // dest: [ ]
+ // src: [ ]
+ FSReadRequest dest;
+ dest.offset = 0;
+ dest.len = 10;
+ dest.scratch = nullptr;
+ ASSERT_OK(dest.status);
+
+ FSReadRequest src;
+ src.offset = 0;
+ src.len = 5;
+ src.scratch = nullptr;
+ ASSERT_OK(src.status);
+
+ if (reverse) std::swap(dest, src);
+ ASSERT_TRUE(TryMerge(&dest, src));
+ ASSERT_EQ(dest.offset, 0);
+ ASSERT_EQ(dest.len, 10);
+ ASSERT_OK(dest.status);
+ ASSERT_OK(src.status);
+ }
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/file/read_write_util.cc b/src/rocksdb/file/read_write_util.cc
new file mode 100644
index 000000000..3617a35e3
--- /dev/null
+++ b/src/rocksdb/file/read_write_util.cc
@@ -0,0 +1,33 @@
+// 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/read_write_util.h"
+
+#include <sstream>
+
+#include "test_util/sync_point.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+IOStatus NewWritableFile(FileSystem* fs, const std::string& fname,
+ std::unique_ptr<FSWritableFile>* result,
+ const FileOptions& options) {
+ TEST_SYNC_POINT_CALLBACK("NewWritableFile::FileOptions.temperature",
+ const_cast<Temperature*>(&options.temperature));
+ IOStatus s = fs->NewWritableFile(fname, options, result, nullptr);
+ TEST_KILL_RANDOM_WITH_WEIGHT("NewWritableFile:0", REDUCE_ODDS2);
+ return s;
+}
+
+#ifndef NDEBUG
+bool IsFileSectorAligned(const size_t off, size_t sector_size) {
+ return off % sector_size == 0;
+}
+#endif // NDEBUG
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/read_write_util.h b/src/rocksdb/file/read_write_util.h
new file mode 100644
index 000000000..9f034b705
--- /dev/null
+++ b/src/rocksdb/file/read_write_util.h
@@ -0,0 +1,31 @@
+// 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 "file/sequence_file_reader.h"
+#include "rocksdb/env.h"
+#include "rocksdb/file_system.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Returns a WritableFile.
+//
+// env : the Env.
+// fname : the file name.
+// result : output arg. A WritableFile based on `fname` returned.
+// options : the Env Options.
+extern IOStatus NewWritableFile(FileSystem* fs, const std::string& fname,
+ std::unique_ptr<FSWritableFile>* result,
+ const FileOptions& options);
+
+#ifndef NDEBUG
+bool IsFileSectorAligned(const size_t off, size_t sector_size);
+#endif // NDEBUG
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/readahead_file_info.h b/src/rocksdb/file/readahead_file_info.h
new file mode 100644
index 000000000..f0208bf2d
--- /dev/null
+++ b/src/rocksdb/file/readahead_file_info.h
@@ -0,0 +1,33 @@
+// 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).
+
+#pragma once
+
+#include <cstddef>
+#include <cstdint>
+
+#include "rocksdb/rocksdb_namespace.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// struct ReadaheadFileInfo contains readahead information that is passed from
+// one file to another file per level during iterations. This information helps
+// iterators to carry forward the internal automatic prefetching readahead value
+// to next file during sequential reads instead of starting from the scratch.
+
+struct ReadaheadFileInfo {
+ struct ReadaheadInfo {
+ size_t readahead_size = 0;
+ int64_t num_file_reads = 0;
+ };
+
+ // Used by Data block iterators to update readahead info.
+ ReadaheadInfo data_block_readahead_info;
+
+ // Used by Index block iterators to update readahead info.
+ ReadaheadInfo index_block_readahead_info;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/readahead_raf.cc b/src/rocksdb/file/readahead_raf.cc
new file mode 100644
index 000000000..6d346432e
--- /dev/null
+++ b/src/rocksdb/file/readahead_raf.cc
@@ -0,0 +1,169 @@
+// 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/readahead_raf.h"
+
+#include <algorithm>
+#include <mutex>
+
+#include "file/read_write_util.h"
+#include "rocksdb/file_system.h"
+#include "util/aligned_buffer.h"
+#include "util/rate_limiter.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace {
+class ReadaheadRandomAccessFile : public FSRandomAccessFile {
+ public:
+ ReadaheadRandomAccessFile(std::unique_ptr<FSRandomAccessFile>&& file,
+ size_t readahead_size)
+ : file_(std::move(file)),
+ alignment_(file_->GetRequiredBufferAlignment()),
+ readahead_size_(Roundup(readahead_size, alignment_)),
+ buffer_(),
+ buffer_offset_(0) {
+ buffer_.Alignment(alignment_);
+ buffer_.AllocateNewBuffer(readahead_size_);
+ }
+
+ ReadaheadRandomAccessFile(const ReadaheadRandomAccessFile&) = delete;
+
+ ReadaheadRandomAccessFile& operator=(const ReadaheadRandomAccessFile&) =
+ delete;
+
+ IOStatus Read(uint64_t offset, size_t n, const IOOptions& options,
+ Slice* result, char* scratch,
+ IODebugContext* dbg) const override {
+ // Read-ahead only make sense if we have some slack left after reading
+ if (n + alignment_ >= readahead_size_) {
+ return file_->Read(offset, n, options, result, scratch, dbg);
+ }
+
+ std::unique_lock<std::mutex> lk(lock_);
+
+ size_t cached_len = 0;
+ // Check if there is a cache hit, meaning that [offset, offset + n) is
+ // either completely or partially in the buffer. If it's completely cached,
+ // including end of file case when offset + n is greater than EOF, then
+ // return.
+ if (TryReadFromCache(offset, n, &cached_len, scratch) &&
+ (cached_len == n || buffer_.CurrentSize() < readahead_size_)) {
+ // We read exactly what we needed, or we hit end of file - return.
+ *result = Slice(scratch, cached_len);
+ return IOStatus::OK();
+ }
+ size_t advanced_offset = static_cast<size_t>(offset + cached_len);
+ // In the case of cache hit advanced_offset is already aligned, means that
+ // chunk_offset equals to advanced_offset
+ size_t chunk_offset = TruncateToPageBoundary(alignment_, advanced_offset);
+
+ IOStatus s = ReadIntoBuffer(chunk_offset, readahead_size_, options, dbg);
+ if (s.ok()) {
+ // The data we need is now in cache, so we can safely read it
+ size_t remaining_len;
+ TryReadFromCache(advanced_offset, n - cached_len, &remaining_len,
+ scratch + cached_len);
+ *result = Slice(scratch, cached_len + remaining_len);
+ }
+ return s;
+ }
+
+ IOStatus Prefetch(uint64_t offset, size_t n, const IOOptions& options,
+ IODebugContext* dbg) override {
+ if (n < readahead_size_) {
+ // Don't allow smaller prefetches than the configured `readahead_size_`.
+ // `Read()` assumes a smaller prefetch buffer indicates EOF was reached.
+ return IOStatus::OK();
+ }
+
+ std::unique_lock<std::mutex> lk(lock_);
+
+ size_t offset_ = static_cast<size_t>(offset);
+ size_t prefetch_offset = TruncateToPageBoundary(alignment_, offset_);
+ if (prefetch_offset == buffer_offset_) {
+ return IOStatus::OK();
+ }
+ return ReadIntoBuffer(prefetch_offset,
+ Roundup(offset_ + n, alignment_) - prefetch_offset,
+ options, dbg);
+ }
+
+ size_t GetUniqueId(char* id, size_t max_size) const override {
+ return file_->GetUniqueId(id, max_size);
+ }
+
+ void Hint(AccessPattern pattern) override { file_->Hint(pattern); }
+
+ IOStatus InvalidateCache(size_t offset, size_t length) override {
+ std::unique_lock<std::mutex> lk(lock_);
+ buffer_.Clear();
+ return file_->InvalidateCache(offset, length);
+ }
+
+ bool use_direct_io() const override { return file_->use_direct_io(); }
+
+ private:
+ // Tries to read from buffer_ n bytes starting at offset. If anything was read
+ // from the cache, it sets cached_len to the number of bytes actually read,
+ // copies these number of bytes to scratch and returns true.
+ // If nothing was read sets cached_len to 0 and returns false.
+ bool TryReadFromCache(uint64_t offset, size_t n, size_t* cached_len,
+ char* scratch) const {
+ if (offset < buffer_offset_ ||
+ offset >= buffer_offset_ + buffer_.CurrentSize()) {
+ *cached_len = 0;
+ return false;
+ }
+ uint64_t offset_in_buffer = offset - buffer_offset_;
+ *cached_len = std::min(
+ buffer_.CurrentSize() - static_cast<size_t>(offset_in_buffer), n);
+ memcpy(scratch, buffer_.BufferStart() + offset_in_buffer, *cached_len);
+ return true;
+ }
+
+ // Reads into buffer_ the next n bytes from file_ starting at offset.
+ // Can actually read less if EOF was reached.
+ // Returns the status of the read operastion on the file.
+ IOStatus ReadIntoBuffer(uint64_t offset, size_t n, const IOOptions& options,
+ IODebugContext* dbg) const {
+ if (n > buffer_.Capacity()) {
+ n = buffer_.Capacity();
+ }
+ assert(IsFileSectorAligned(offset, alignment_));
+ assert(IsFileSectorAligned(n, alignment_));
+ Slice result;
+ IOStatus s =
+ file_->Read(offset, n, options, &result, buffer_.BufferStart(), dbg);
+ if (s.ok()) {
+ buffer_offset_ = offset;
+ buffer_.Size(result.size());
+ assert(result.size() == 0 || buffer_.BufferStart() == result.data());
+ }
+ return s;
+ }
+
+ const std::unique_ptr<FSRandomAccessFile> file_;
+ const size_t alignment_;
+ const size_t readahead_size_;
+
+ mutable std::mutex lock_;
+ // The buffer storing the prefetched data
+ mutable AlignedBuffer buffer_;
+ // The offset in file_, corresponding to data stored in buffer_
+ mutable uint64_t buffer_offset_;
+};
+} // namespace
+
+std::unique_ptr<FSRandomAccessFile> NewReadaheadRandomAccessFile(
+ std::unique_ptr<FSRandomAccessFile>&& file, size_t readahead_size) {
+ std::unique_ptr<FSRandomAccessFile> result(
+ new ReadaheadRandomAccessFile(std::move(file), readahead_size));
+ return result;
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/readahead_raf.h b/src/rocksdb/file/readahead_raf.h
new file mode 100644
index 000000000..dfaf2b4fa
--- /dev/null
+++ b/src/rocksdb/file/readahead_raf.h
@@ -0,0 +1,29 @@
+// 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 <memory>
+
+#include "rocksdb/rocksdb_namespace.h"
+
+namespace ROCKSDB_NAMESPACE {
+class FSRandomAccessFile;
+// This file provides the following main abstractions:
+// SequentialFileReader : wrapper over Env::SequentialFile
+// RandomAccessFileReader : wrapper over Env::RandomAccessFile
+// WritableFileWriter : wrapper over Env::WritableFile
+// In addition, it also exposed NewReadaheadRandomAccessFile, NewWritableFile,
+// and ReadOneLine primitives.
+
+// NewReadaheadRandomAccessFile provides a wrapper over RandomAccessFile to
+// always prefetch additional data with every read. This is mainly used in
+// Compaction Table Readers.
+std::unique_ptr<FSRandomAccessFile> NewReadaheadRandomAccessFile(
+ std::unique_ptr<FSRandomAccessFile>&& file, size_t readahead_size);
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/sequence_file_reader.cc b/src/rocksdb/file/sequence_file_reader.cc
new file mode 100644
index 000000000..d51d5be46
--- /dev/null
+++ b/src/rocksdb/file/sequence_file_reader.cc
@@ -0,0 +1,328 @@
+// 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/sequence_file_reader.h"
+
+#include <algorithm>
+#include <mutex>
+
+#include "file/read_write_util.h"
+#include "monitoring/histogram.h"
+#include "monitoring/iostats_context_imp.h"
+#include "port/port.h"
+#include "test_util/sync_point.h"
+#include "util/aligned_buffer.h"
+#include "util/random.h"
+#include "util/rate_limiter.h"
+
+namespace ROCKSDB_NAMESPACE {
+IOStatus SequentialFileReader::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) {
+ std::unique_ptr<FSSequentialFile> file;
+ IOStatus io_s = fs->NewSequentialFile(fname, file_opts, &file, dbg);
+ if (io_s.ok()) {
+ reader->reset(new SequentialFileReader(std::move(file), fname, nullptr, {},
+ rate_limiter));
+ }
+ return io_s;
+}
+
+IOStatus SequentialFileReader::Read(size_t n, Slice* result, char* scratch,
+ Env::IOPriority rate_limiter_priority) {
+ IOStatus io_s;
+ if (use_direct_io()) {
+#ifndef ROCKSDB_LITE
+ //
+ // |-offset_advance-|---bytes returned--|
+ // |----------------------buf size-------------------------|
+ // | | | |
+ // aligned offset offset + n Roundup(offset + n,
+ // offset alignment)
+ //
+ size_t offset = offset_.fetch_add(n);
+ size_t alignment = file_->GetRequiredBufferAlignment();
+ size_t aligned_offset = TruncateToPageBoundary(alignment, offset);
+ size_t offset_advance = offset - aligned_offset;
+ size_t size = Roundup(offset + n, alignment) - aligned_offset;
+ size_t r = 0;
+ AlignedBuffer buf;
+ buf.Alignment(alignment);
+ buf.AllocateNewBuffer(size);
+
+ while (buf.CurrentSize() < size) {
+ size_t allowed;
+ if (rate_limiter_priority != Env::IO_TOTAL && rate_limiter_ != nullptr) {
+ allowed = rate_limiter_->RequestToken(
+ buf.Capacity() - buf.CurrentSize(), buf.Alignment(),
+ rate_limiter_priority, nullptr /* stats */,
+ RateLimiter::OpType::kRead);
+ } else {
+ assert(buf.CurrentSize() == 0);
+ allowed = size;
+ }
+
+ Slice tmp;
+ uint64_t orig_offset = 0;
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ orig_offset = aligned_offset + buf.CurrentSize();
+ start_ts = FileOperationInfo::StartNow();
+ }
+ io_s = file_->PositionedRead(aligned_offset + buf.CurrentSize(), allowed,
+ IOOptions(), &tmp, buf.Destination(),
+ nullptr /* dbg */);
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileReadFinish(orig_offset, tmp.size(), start_ts, finish_ts,
+ io_s);
+ }
+ buf.Size(buf.CurrentSize() + tmp.size());
+ if (!io_s.ok() || tmp.size() < allowed) {
+ break;
+ }
+ }
+
+ if (io_s.ok() && offset_advance < buf.CurrentSize()) {
+ r = buf.Read(scratch, offset_advance,
+ std::min(buf.CurrentSize() - offset_advance, n));
+ }
+ *result = Slice(scratch, r);
+#endif // !ROCKSDB_LITE
+ } else {
+ // To be paranoid, modify scratch a little bit, so in case underlying
+ // FileSystem doesn't fill the buffer but return success and `scratch`
+ // returns contains a previous block, returned value will not pass
+ // checksum.
+ // It's hard to find useful byte for direct I/O case, so we skip it.
+ if (n > 0 && scratch != nullptr) {
+ scratch[0]++;
+ }
+
+ size_t read = 0;
+ while (read < n) {
+ size_t allowed;
+ if (rate_limiter_priority != Env::IO_TOTAL && rate_limiter_ != nullptr) {
+ allowed = rate_limiter_->RequestToken(
+ n - read, 0 /* alignment */, rate_limiter_priority,
+ nullptr /* stats */, RateLimiter::OpType::kRead);
+ } else {
+ allowed = n;
+ }
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+ Slice tmp;
+ io_s = file_->Read(allowed, IOOptions(), &tmp, scratch + read,
+ nullptr /* dbg */);
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ size_t offset = offset_.fetch_add(tmp.size());
+ NotifyOnFileReadFinish(offset, tmp.size(), start_ts, finish_ts, io_s);
+ }
+#endif
+ read += tmp.size();
+ if (!io_s.ok() || tmp.size() < allowed) {
+ break;
+ }
+ }
+ *result = Slice(scratch, read);
+ }
+ IOSTATS_ADD(bytes_read, result->size());
+ return io_s;
+}
+
+IOStatus SequentialFileReader::Skip(uint64_t n) {
+#ifndef ROCKSDB_LITE
+ if (use_direct_io()) {
+ offset_ += static_cast<size_t>(n);
+ return IOStatus::OK();
+ }
+#endif // !ROCKSDB_LITE
+ return file_->Skip(n);
+}
+
+namespace {
+// This class wraps a SequentialFile, exposing same API, with the differenece
+// of being able to prefetch up to readahead_size bytes and then serve them
+// from memory, avoiding the entire round-trip if, for example, the data for the
+// file is actually remote.
+class ReadaheadSequentialFile : public FSSequentialFile {
+ public:
+ ReadaheadSequentialFile(std::unique_ptr<FSSequentialFile>&& file,
+ size_t readahead_size)
+ : file_(std::move(file)),
+ alignment_(file_->GetRequiredBufferAlignment()),
+ readahead_size_(Roundup(readahead_size, alignment_)),
+ buffer_(),
+ buffer_offset_(0),
+ read_offset_(0) {
+ buffer_.Alignment(alignment_);
+ buffer_.AllocateNewBuffer(readahead_size_);
+ }
+
+ ReadaheadSequentialFile(const ReadaheadSequentialFile&) = delete;
+
+ ReadaheadSequentialFile& operator=(const ReadaheadSequentialFile&) = delete;
+
+ IOStatus Read(size_t n, const IOOptions& opts, Slice* result, char* scratch,
+ IODebugContext* dbg) override {
+ std::unique_lock<std::mutex> lk(lock_);
+
+ size_t cached_len = 0;
+ // Check if there is a cache hit, meaning that [offset, offset + n) is
+ // either completely or partially in the buffer. If it's completely cached,
+ // including end of file case when offset + n is greater than EOF, then
+ // return.
+ if (TryReadFromCache(n, &cached_len, scratch) &&
+ (cached_len == n || buffer_.CurrentSize() < readahead_size_)) {
+ // We read exactly what we needed, or we hit end of file - return.
+ *result = Slice(scratch, cached_len);
+ return IOStatus::OK();
+ }
+ n -= cached_len;
+
+ IOStatus s;
+ // Read-ahead only make sense if we have some slack left after reading
+ if (n + alignment_ >= readahead_size_) {
+ s = file_->Read(n, opts, result, scratch + cached_len, dbg);
+ if (s.ok()) {
+ read_offset_ += result->size();
+ *result = Slice(scratch, cached_len + result->size());
+ }
+ buffer_.Clear();
+ return s;
+ }
+
+ s = ReadIntoBuffer(readahead_size_, opts, dbg);
+ if (s.ok()) {
+ // The data we need is now in cache, so we can safely read it
+ size_t remaining_len;
+ TryReadFromCache(n, &remaining_len, scratch + cached_len);
+ *result = Slice(scratch, cached_len + remaining_len);
+ }
+ return s;
+ }
+
+ IOStatus Skip(uint64_t n) override {
+ std::unique_lock<std::mutex> lk(lock_);
+ IOStatus s = IOStatus::OK();
+ // First check if we need to skip already cached data
+ if (buffer_.CurrentSize() > 0) {
+ // Do we need to skip beyond cached data?
+ if (read_offset_ + n >= buffer_offset_ + buffer_.CurrentSize()) {
+ // Yes. Skip whaterver is in memory and adjust offset accordingly
+ n -= buffer_offset_ + buffer_.CurrentSize() - read_offset_;
+ read_offset_ = buffer_offset_ + buffer_.CurrentSize();
+ } else {
+ // No. The entire section to be skipped is entirely i cache.
+ read_offset_ += n;
+ n = 0;
+ }
+ }
+ if (n > 0) {
+ // We still need to skip more, so call the file API for skipping
+ s = file_->Skip(n);
+ if (s.ok()) {
+ read_offset_ += n;
+ }
+ buffer_.Clear();
+ }
+ return s;
+ }
+
+ IOStatus PositionedRead(uint64_t offset, size_t n, const IOOptions& opts,
+ Slice* result, char* scratch,
+ IODebugContext* dbg) override {
+ return file_->PositionedRead(offset, n, opts, result, scratch, dbg);
+ }
+
+ IOStatus InvalidateCache(size_t offset, size_t length) override {
+ std::unique_lock<std::mutex> lk(lock_);
+ buffer_.Clear();
+ return file_->InvalidateCache(offset, length);
+ }
+
+ bool use_direct_io() const override { return file_->use_direct_io(); }
+
+ private:
+ // Tries to read from buffer_ n bytes. If anything was read from the cache, it
+ // sets cached_len to the number of bytes actually read, copies these number
+ // of bytes to scratch and returns true.
+ // If nothing was read sets cached_len to 0 and returns false.
+ bool TryReadFromCache(size_t n, size_t* cached_len, char* scratch) {
+ if (read_offset_ < buffer_offset_ ||
+ read_offset_ >= buffer_offset_ + buffer_.CurrentSize()) {
+ *cached_len = 0;
+ return false;
+ }
+ uint64_t offset_in_buffer = read_offset_ - buffer_offset_;
+ *cached_len = std::min(
+ buffer_.CurrentSize() - static_cast<size_t>(offset_in_buffer), n);
+ memcpy(scratch, buffer_.BufferStart() + offset_in_buffer, *cached_len);
+ read_offset_ += *cached_len;
+ return true;
+ }
+
+ // Reads into buffer_ the next n bytes from file_.
+ // Can actually read less if EOF was reached.
+ // Returns the status of the read operastion on the file.
+ IOStatus ReadIntoBuffer(size_t n, const IOOptions& opts,
+ IODebugContext* dbg) {
+ if (n > buffer_.Capacity()) {
+ n = buffer_.Capacity();
+ }
+ assert(IsFileSectorAligned(n, alignment_));
+ Slice result;
+ IOStatus s = file_->Read(n, opts, &result, buffer_.BufferStart(), dbg);
+ if (s.ok()) {
+ buffer_offset_ = read_offset_;
+ buffer_.Size(result.size());
+ assert(result.size() == 0 || buffer_.BufferStart() == result.data());
+ }
+ return s;
+ }
+
+ const std::unique_ptr<FSSequentialFile> file_;
+ const size_t alignment_;
+ const size_t readahead_size_;
+
+ std::mutex lock_;
+ // The buffer storing the prefetched data
+ AlignedBuffer buffer_;
+ // The offset in file_, corresponding to data stored in buffer_
+ uint64_t buffer_offset_;
+ // The offset up to which data was read from file_. In fact, it can be larger
+ // than the actual file size, since the file_->Skip(n) call doesn't return the
+ // actual number of bytes that were skipped, which can be less than n.
+ // This is not a problemm since read_offset_ is monotonically increasing and
+ // its only use is to figure out if next piece of data should be read from
+ // buffer_ or file_ directly.
+ uint64_t read_offset_;
+};
+} // namespace
+
+std::unique_ptr<FSSequentialFile>
+SequentialFileReader::NewReadaheadSequentialFile(
+ std::unique_ptr<FSSequentialFile>&& file, size_t readahead_size) {
+ if (file->GetRequiredBufferAlignment() >= readahead_size) {
+ // Short-circuit and return the original file if readahead_size is
+ // too small and hence doesn't make sense to be used for prefetching.
+ return std::move(file);
+ }
+ std::unique_ptr<FSSequentialFile> result(
+ new ReadaheadSequentialFile(std::move(file), readahead_size));
+ return result;
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/sequence_file_reader.h b/src/rocksdb/file/sequence_file_reader.h
new file mode 100644
index 000000000..baea10eb7
--- /dev/null
+++ b/src/rocksdb/file/sequence_file_reader.h
@@ -0,0 +1,129 @@
+// 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
diff --git a/src/rocksdb/file/sst_file_manager_impl.cc b/src/rocksdb/file/sst_file_manager_impl.cc
new file mode 100644
index 000000000..7053e6a07
--- /dev/null
+++ b/src/rocksdb/file/sst_file_manager_impl.cc
@@ -0,0 +1,525 @@
+// 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).
+
+#include "file/sst_file_manager_impl.h"
+
+#include <cinttypes>
+#include <vector>
+
+#include "db/db_impl/db_impl.h"
+#include "logging/logging.h"
+#include "port/port.h"
+#include "rocksdb/env.h"
+#include "rocksdb/sst_file_manager.h"
+#include "test_util/sync_point.h"
+#include "util/mutexlock.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+#ifndef ROCKSDB_LITE
+SstFileManagerImpl::SstFileManagerImpl(
+ const std::shared_ptr<SystemClock>& clock,
+ const std::shared_ptr<FileSystem>& fs,
+ const std::shared_ptr<Logger>& logger, int64_t rate_bytes_per_sec,
+ double max_trash_db_ratio, uint64_t bytes_max_delete_chunk)
+ : clock_(clock),
+ fs_(fs),
+ logger_(logger),
+ total_files_size_(0),
+ compaction_buffer_size_(0),
+ cur_compactions_reserved_size_(0),
+ max_allowed_space_(0),
+ delete_scheduler_(clock_.get(), fs_.get(), rate_bytes_per_sec,
+ logger.get(), this, max_trash_db_ratio,
+ bytes_max_delete_chunk),
+ cv_(&mu_),
+ closing_(false),
+ bg_thread_(nullptr),
+ reserved_disk_buffer_(0),
+ free_space_trigger_(0),
+ cur_instance_(nullptr) {}
+
+SstFileManagerImpl::~SstFileManagerImpl() {
+ Close();
+ bg_err_.PermitUncheckedError();
+}
+
+void SstFileManagerImpl::Close() {
+ {
+ MutexLock l(&mu_);
+ if (closing_) {
+ return;
+ }
+ closing_ = true;
+ cv_.SignalAll();
+ }
+ if (bg_thread_) {
+ bg_thread_->join();
+ }
+}
+
+Status SstFileManagerImpl::OnAddFile(const std::string& file_path) {
+ uint64_t file_size;
+ Status s = fs_->GetFileSize(file_path, IOOptions(), &file_size, nullptr);
+ if (s.ok()) {
+ MutexLock l(&mu_);
+ OnAddFileImpl(file_path, file_size);
+ }
+ TEST_SYNC_POINT_CALLBACK("SstFileManagerImpl::OnAddFile",
+ const_cast<std::string*>(&file_path));
+ return s;
+}
+
+Status SstFileManagerImpl::OnAddFile(const std::string& file_path,
+ uint64_t file_size) {
+ MutexLock l(&mu_);
+ OnAddFileImpl(file_path, file_size);
+ TEST_SYNC_POINT_CALLBACK("SstFileManagerImpl::OnAddFile",
+ const_cast<std::string*>(&file_path));
+ return Status::OK();
+}
+
+Status SstFileManagerImpl::OnDeleteFile(const std::string& file_path) {
+ {
+ MutexLock l(&mu_);
+ OnDeleteFileImpl(file_path);
+ }
+ TEST_SYNC_POINT_CALLBACK("SstFileManagerImpl::OnDeleteFile",
+ const_cast<std::string*>(&file_path));
+ return Status::OK();
+}
+
+void SstFileManagerImpl::OnCompactionCompletion(Compaction* c) {
+ MutexLock l(&mu_);
+ uint64_t size_added_by_compaction = 0;
+ for (size_t i = 0; i < c->num_input_levels(); i++) {
+ for (size_t j = 0; j < c->num_input_files(i); j++) {
+ FileMetaData* filemeta = c->input(i, j);
+ size_added_by_compaction += filemeta->fd.GetFileSize();
+ }
+ }
+ cur_compactions_reserved_size_ -= size_added_by_compaction;
+}
+
+Status SstFileManagerImpl::OnMoveFile(const std::string& old_path,
+ const std::string& new_path,
+ uint64_t* file_size) {
+ {
+ MutexLock l(&mu_);
+ if (file_size != nullptr) {
+ *file_size = tracked_files_[old_path];
+ }
+ OnAddFileImpl(new_path, tracked_files_[old_path]);
+ OnDeleteFileImpl(old_path);
+ }
+ TEST_SYNC_POINT("SstFileManagerImpl::OnMoveFile");
+ return Status::OK();
+}
+
+void SstFileManagerImpl::SetMaxAllowedSpaceUsage(uint64_t max_allowed_space) {
+ MutexLock l(&mu_);
+ max_allowed_space_ = max_allowed_space;
+}
+
+void SstFileManagerImpl::SetCompactionBufferSize(
+ uint64_t compaction_buffer_size) {
+ MutexLock l(&mu_);
+ compaction_buffer_size_ = compaction_buffer_size;
+}
+
+bool SstFileManagerImpl::IsMaxAllowedSpaceReached() {
+ MutexLock l(&mu_);
+ if (max_allowed_space_ <= 0) {
+ return false;
+ }
+ return total_files_size_ >= max_allowed_space_;
+}
+
+bool SstFileManagerImpl::IsMaxAllowedSpaceReachedIncludingCompactions() {
+ MutexLock l(&mu_);
+ if (max_allowed_space_ <= 0) {
+ return false;
+ }
+ return total_files_size_ + cur_compactions_reserved_size_ >=
+ max_allowed_space_;
+}
+
+bool SstFileManagerImpl::EnoughRoomForCompaction(
+ ColumnFamilyData* cfd, const std::vector<CompactionInputFiles>& inputs,
+ const Status& bg_error) {
+ MutexLock l(&mu_);
+ uint64_t size_added_by_compaction = 0;
+ // First check if we even have the space to do the compaction
+ for (size_t i = 0; i < inputs.size(); i++) {
+ for (size_t j = 0; j < inputs[i].size(); j++) {
+ FileMetaData* filemeta = inputs[i][j];
+ size_added_by_compaction += filemeta->fd.GetFileSize();
+ }
+ }
+
+ // Update cur_compactions_reserved_size_ so concurrent compaction
+ // don't max out space
+ size_t needed_headroom = cur_compactions_reserved_size_ +
+ size_added_by_compaction + compaction_buffer_size_;
+ if (max_allowed_space_ != 0 &&
+ (needed_headroom + total_files_size_ > max_allowed_space_)) {
+ return false;
+ }
+
+ // Implement more aggressive checks only if this DB instance has already
+ // seen a NoSpace() error. This is tin order to contain a single potentially
+ // misbehaving DB instance and prevent it from slowing down compactions of
+ // other DB instances
+ if (bg_error.IsNoSpace() && CheckFreeSpace()) {
+ auto fn =
+ TableFileName(cfd->ioptions()->cf_paths, inputs[0][0]->fd.GetNumber(),
+ inputs[0][0]->fd.GetPathId());
+ uint64_t free_space = 0;
+ Status s = fs_->GetFreeSpace(fn, IOOptions(), &free_space, nullptr);
+ s.PermitUncheckedError(); // TODO: Check the status
+ // needed_headroom is based on current size reserved by compactions,
+ // minus any files created by running compactions as they would count
+ // against the reserved size. If user didn't specify any compaction
+ // buffer, add reserved_disk_buffer_ that's calculated by default so the
+ // compaction doesn't end up leaving nothing for logs and flush SSTs
+ if (compaction_buffer_size_ == 0) {
+ needed_headroom += reserved_disk_buffer_;
+ }
+ if (free_space < needed_headroom + size_added_by_compaction) {
+ // We hit the condition of not enough disk space
+ ROCKS_LOG_ERROR(logger_,
+ "free space [%" PRIu64
+ " bytes] is less than "
+ "needed headroom [%" ROCKSDB_PRIszt " bytes]\n",
+ free_space, needed_headroom);
+ return false;
+ }
+ }
+
+ cur_compactions_reserved_size_ += size_added_by_compaction;
+ // Take a snapshot of cur_compactions_reserved_size_ for when we encounter
+ // a NoSpace error.
+ free_space_trigger_ = cur_compactions_reserved_size_;
+ return true;
+}
+
+uint64_t SstFileManagerImpl::GetCompactionsReservedSize() {
+ MutexLock l(&mu_);
+ return cur_compactions_reserved_size_;
+}
+
+uint64_t SstFileManagerImpl::GetTotalSize() {
+ MutexLock l(&mu_);
+ return total_files_size_;
+}
+
+std::unordered_map<std::string, uint64_t>
+SstFileManagerImpl::GetTrackedFiles() {
+ MutexLock l(&mu_);
+ return tracked_files_;
+}
+
+int64_t SstFileManagerImpl::GetDeleteRateBytesPerSecond() {
+ return delete_scheduler_.GetRateBytesPerSecond();
+}
+
+void SstFileManagerImpl::SetDeleteRateBytesPerSecond(int64_t delete_rate) {
+ return delete_scheduler_.SetRateBytesPerSecond(delete_rate);
+}
+
+double SstFileManagerImpl::GetMaxTrashDBRatio() {
+ return delete_scheduler_.GetMaxTrashDBRatio();
+}
+
+void SstFileManagerImpl::SetMaxTrashDBRatio(double r) {
+ return delete_scheduler_.SetMaxTrashDBRatio(r);
+}
+
+uint64_t SstFileManagerImpl::GetTotalTrashSize() {
+ return delete_scheduler_.GetTotalTrashSize();
+}
+
+void SstFileManagerImpl::ReserveDiskBuffer(uint64_t size,
+ const std::string& path) {
+ MutexLock l(&mu_);
+
+ reserved_disk_buffer_ += size;
+ if (path_.empty()) {
+ path_ = path;
+ }
+}
+
+void SstFileManagerImpl::ClearError() {
+ while (true) {
+ MutexLock l(&mu_);
+
+ if (error_handler_list_.empty() || closing_) {
+ return;
+ }
+
+ uint64_t free_space = 0;
+ Status s = fs_->GetFreeSpace(path_, IOOptions(), &free_space, nullptr);
+ free_space = max_allowed_space_ > 0
+ ? std::min(max_allowed_space_, free_space)
+ : free_space;
+ if (s.ok()) {
+ // In case of multi-DB instances, some of them may have experienced a
+ // soft error and some a hard error. In the SstFileManagerImpl, a hard
+ // error will basically override previously reported soft errors. Once
+ // we clear the hard error, we don't keep track of previous errors for
+ // now
+ if (bg_err_.severity() == Status::Severity::kHardError) {
+ if (free_space < reserved_disk_buffer_) {
+ ROCKS_LOG_ERROR(logger_,
+ "free space [%" PRIu64
+ " bytes] is less than "
+ "required disk buffer [%" PRIu64 " bytes]\n",
+ free_space, reserved_disk_buffer_);
+ ROCKS_LOG_ERROR(logger_, "Cannot clear hard error\n");
+ s = Status::NoSpace();
+ }
+ } else if (bg_err_.severity() == Status::Severity::kSoftError) {
+ if (free_space < free_space_trigger_) {
+ ROCKS_LOG_WARN(logger_,
+ "free space [%" PRIu64
+ " bytes] is less than "
+ "free space for compaction trigger [%" PRIu64
+ " bytes]\n",
+ free_space, free_space_trigger_);
+ ROCKS_LOG_WARN(logger_, "Cannot clear soft error\n");
+ s = Status::NoSpace();
+ }
+ }
+ }
+
+ // Someone could have called CancelErrorRecovery() and the list could have
+ // become empty, so check again here
+ if (s.ok()) {
+ assert(!error_handler_list_.empty());
+ auto error_handler = error_handler_list_.front();
+ // Since we will release the mutex, set cur_instance_ to signal to the
+ // shutdown thread, if it calls // CancelErrorRecovery() the meantime,
+ // to indicate that this DB instance is busy. The DB instance is
+ // guaranteed to not be deleted before RecoverFromBGError() returns,
+ // since the ErrorHandler::recovery_in_prog_ flag would be true
+ cur_instance_ = error_handler;
+ mu_.Unlock();
+ s = error_handler->RecoverFromBGError();
+ TEST_SYNC_POINT("SstFileManagerImpl::ErrorCleared");
+ mu_.Lock();
+ // The DB instance might have been deleted while we were
+ // waiting for the mutex, so check cur_instance_ to make sure its
+ // still non-null
+ if (cur_instance_) {
+ // Check for error again, since the instance may have recovered but
+ // immediately got another error. If that's the case, and the new
+ // error is also a NoSpace() non-fatal error, leave the instance in
+ // the list
+ Status err = cur_instance_->GetBGError();
+ if (s.ok() && err.subcode() == IOStatus::SubCode::kNoSpace &&
+ err.severity() < Status::Severity::kFatalError) {
+ s = err;
+ }
+ cur_instance_ = nullptr;
+ }
+
+ if (s.ok() || s.IsShutdownInProgress() ||
+ (!s.ok() && s.severity() >= Status::Severity::kFatalError)) {
+ // If shutdown is in progress, abandon this handler instance
+ // and continue with the others
+ error_handler_list_.pop_front();
+ }
+ }
+
+ if (!error_handler_list_.empty()) {
+ // If there are more instances to be recovered, reschedule after 5
+ // seconds
+ int64_t wait_until = clock_->NowMicros() + 5000000;
+ cv_.TimedWait(wait_until);
+ }
+
+ // Check again for error_handler_list_ empty, as a DB instance shutdown
+ // could have removed it from the queue while we were in timed wait
+ if (error_handler_list_.empty()) {
+ ROCKS_LOG_INFO(logger_, "Clearing error\n");
+ bg_err_ = Status::OK();
+ return;
+ }
+ }
+}
+
+void SstFileManagerImpl::StartErrorRecovery(ErrorHandler* handler,
+ Status bg_error) {
+ MutexLock l(&mu_);
+ if (bg_error.severity() == Status::Severity::kSoftError) {
+ if (bg_err_.ok()) {
+ // Setting bg_err_ basically means we're in degraded mode
+ // Assume that all pending compactions will fail similarly. The trigger
+ // for clearing this condition is set to current compaction reserved
+ // size, so we stop checking disk space available in
+ // EnoughRoomForCompaction once this much free space is available
+ bg_err_ = bg_error;
+ }
+ } else if (bg_error.severity() == Status::Severity::kHardError) {
+ bg_err_ = bg_error;
+ } else {
+ assert(false);
+ }
+
+ // If this is the first instance of this error, kick of a thread to poll
+ // and recover from this condition
+ if (error_handler_list_.empty()) {
+ error_handler_list_.push_back(handler);
+ // Release lock before calling join. Its ok to do so because
+ // error_handler_list_ is now non-empty, so no other invocation of this
+ // function will execute this piece of code
+ mu_.Unlock();
+ if (bg_thread_) {
+ bg_thread_->join();
+ }
+ // Start a new thread. The previous one would have exited.
+ bg_thread_.reset(new port::Thread(&SstFileManagerImpl::ClearError, this));
+ mu_.Lock();
+ } else {
+ // Check if this DB instance is already in the list
+ for (auto iter = error_handler_list_.begin();
+ iter != error_handler_list_.end(); ++iter) {
+ if ((*iter) == handler) {
+ return;
+ }
+ }
+ error_handler_list_.push_back(handler);
+ }
+}
+
+bool SstFileManagerImpl::CancelErrorRecovery(ErrorHandler* handler) {
+ MutexLock l(&mu_);
+
+ if (cur_instance_ == handler) {
+ // This instance is currently busy attempting to recover
+ // Nullify it so the recovery thread doesn't attempt to access it again
+ cur_instance_ = nullptr;
+ return false;
+ }
+
+ for (auto iter = error_handler_list_.begin();
+ iter != error_handler_list_.end(); ++iter) {
+ if ((*iter) == handler) {
+ error_handler_list_.erase(iter);
+ return true;
+ }
+ }
+ return false;
+}
+
+Status SstFileManagerImpl::ScheduleFileDeletion(const std::string& file_path,
+ const std::string& path_to_sync,
+ const bool force_bg) {
+ TEST_SYNC_POINT_CALLBACK("SstFileManagerImpl::ScheduleFileDeletion",
+ const_cast<std::string*>(&file_path));
+ return delete_scheduler_.DeleteFile(file_path, path_to_sync, force_bg);
+}
+
+void SstFileManagerImpl::WaitForEmptyTrash() {
+ delete_scheduler_.WaitForEmptyTrash();
+}
+
+void SstFileManagerImpl::OnAddFileImpl(const std::string& file_path,
+ uint64_t file_size) {
+ auto tracked_file = tracked_files_.find(file_path);
+ if (tracked_file != tracked_files_.end()) {
+ // File was added before, we will just update the size
+ total_files_size_ -= tracked_file->second;
+ total_files_size_ += file_size;
+ cur_compactions_reserved_size_ -= file_size;
+ } else {
+ total_files_size_ += file_size;
+ }
+ tracked_files_[file_path] = file_size;
+}
+
+void SstFileManagerImpl::OnDeleteFileImpl(const std::string& file_path) {
+ auto tracked_file = tracked_files_.find(file_path);
+ if (tracked_file == tracked_files_.end()) {
+ // File is not tracked
+ return;
+ }
+
+ total_files_size_ -= tracked_file->second;
+ tracked_files_.erase(tracked_file);
+}
+
+SstFileManager* NewSstFileManager(Env* env, std::shared_ptr<Logger> info_log,
+ std::string trash_dir,
+ int64_t rate_bytes_per_sec,
+ bool delete_existing_trash, Status* status,
+ double max_trash_db_ratio,
+ uint64_t bytes_max_delete_chunk) {
+ const auto& fs = env->GetFileSystem();
+ return NewSstFileManager(env, fs, info_log, trash_dir, rate_bytes_per_sec,
+ delete_existing_trash, status, max_trash_db_ratio,
+ bytes_max_delete_chunk);
+}
+
+SstFileManager* NewSstFileManager(Env* env, std::shared_ptr<FileSystem> fs,
+ std::shared_ptr<Logger> info_log,
+ const std::string& trash_dir,
+ int64_t rate_bytes_per_sec,
+ bool delete_existing_trash, Status* status,
+ double max_trash_db_ratio,
+ uint64_t bytes_max_delete_chunk) {
+ const auto& clock = env->GetSystemClock();
+ SstFileManagerImpl* res =
+ new SstFileManagerImpl(clock, fs, info_log, rate_bytes_per_sec,
+ max_trash_db_ratio, bytes_max_delete_chunk);
+
+ // trash_dir is deprecated and not needed anymore, but if user passed it
+ // we will still remove files in it.
+ Status s = Status::OK();
+ if (delete_existing_trash && trash_dir != "") {
+ std::vector<std::string> files_in_trash;
+ s = fs->GetChildren(trash_dir, IOOptions(), &files_in_trash, nullptr);
+ if (s.ok()) {
+ for (const std::string& trash_file : files_in_trash) {
+ std::string path_in_trash = trash_dir + "/" + trash_file;
+ res->OnAddFile(path_in_trash);
+ Status file_delete =
+ res->ScheduleFileDeletion(path_in_trash, trash_dir);
+ if (s.ok() && !file_delete.ok()) {
+ s = file_delete;
+ }
+ }
+ }
+ }
+
+ if (status) {
+ *status = s;
+ } else {
+ // No one passed us a Status, so they must not care about the error...
+ s.PermitUncheckedError();
+ }
+
+ return res;
+}
+
+#else
+
+SstFileManager* NewSstFileManager(Env* /*env*/,
+ std::shared_ptr<Logger> /*info_log*/,
+ std::string /*trash_dir*/,
+ int64_t /*rate_bytes_per_sec*/,
+ bool /*delete_existing_trash*/,
+ Status* status, double /*max_trash_db_ratio*/,
+ uint64_t /*bytes_max_delete_chunk*/) {
+ if (status) {
+ *status =
+ Status::NotSupported("SstFileManager is not supported in ROCKSDB_LITE");
+ }
+ return nullptr;
+}
+
+#endif // ROCKSDB_LITE
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/sst_file_manager_impl.h b/src/rocksdb/file/sst_file_manager_impl.h
new file mode 100644
index 000000000..548eb57f8
--- /dev/null
+++ b/src/rocksdb/file/sst_file_manager_impl.h
@@ -0,0 +1,195 @@
+// 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).
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+
+#include <string>
+
+#include "db/compaction/compaction.h"
+#include "file/delete_scheduler.h"
+#include "port/port.h"
+#include "rocksdb/sst_file_manager.h"
+
+namespace ROCKSDB_NAMESPACE {
+class ErrorHandler;
+class FileSystem;
+class SystemClock;
+class Logger;
+
+// SstFileManager is used to track SST and blob files in the DB and control
+// their deletion rate. All SstFileManager public functions are thread-safe.
+class SstFileManagerImpl : public SstFileManager {
+ public:
+ explicit SstFileManagerImpl(const std::shared_ptr<SystemClock>& clock,
+ const std::shared_ptr<FileSystem>& fs,
+ const std::shared_ptr<Logger>& logger,
+ int64_t rate_bytes_per_sec,
+ double max_trash_db_ratio,
+ uint64_t bytes_max_delete_chunk);
+
+ ~SstFileManagerImpl();
+
+ // DB will call OnAddFile whenever a new sst/blob file is added.
+ Status OnAddFile(const std::string& file_path);
+
+ // Overload where size of the file is provided by the caller rather than
+ // queried from the filesystem. This is an optimization.
+ Status OnAddFile(const std::string& file_path, uint64_t file_size);
+
+ // DB will call OnDeleteFile whenever a sst/blob file is deleted.
+ Status OnDeleteFile(const std::string& file_path);
+
+ // DB will call OnMoveFile whenever a sst/blob file is move to a new path.
+ Status OnMoveFile(const std::string& old_path, const std::string& new_path,
+ uint64_t* file_size = nullptr);
+
+ // Update the maximum allowed space that should be used by RocksDB, if
+ // the total size of the SST and blob files exceeds max_allowed_space, writes
+ // to RocksDB will fail.
+ //
+ // Setting max_allowed_space to 0 will disable this feature, maximum allowed
+ // space will be infinite (Default value).
+ //
+ // thread-safe.
+ void SetMaxAllowedSpaceUsage(uint64_t max_allowed_space) override;
+
+ void SetCompactionBufferSize(uint64_t compaction_buffer_size) override;
+
+ // Return true if the total size of SST and blob files exceeded the maximum
+ // allowed space usage.
+ //
+ // thread-safe.
+ bool IsMaxAllowedSpaceReached() override;
+
+ bool IsMaxAllowedSpaceReachedIncludingCompactions() override;
+
+ // Returns true is there is enough (approximate) space for the specified
+ // compaction. Space is approximate because this function conservatively
+ // estimates how much space is currently being used by compactions (i.e.
+ // if a compaction has started, this function bumps the used space by
+ // the full compaction size).
+ bool EnoughRoomForCompaction(ColumnFamilyData* cfd,
+ const std::vector<CompactionInputFiles>& inputs,
+ const Status& bg_error);
+
+ // Bookkeeping so total_file_sizes_ goes back to normal after compaction
+ // finishes
+ void OnCompactionCompletion(Compaction* c);
+
+ uint64_t GetCompactionsReservedSize();
+
+ // Return the total size of all tracked files.
+ uint64_t GetTotalSize() override;
+
+ // Return a map containing all tracked files and there corresponding sizes.
+ std::unordered_map<std::string, uint64_t> GetTrackedFiles() override;
+
+ // Return delete rate limit in bytes per second.
+ virtual int64_t GetDeleteRateBytesPerSecond() override;
+
+ // Update the delete rate limit in bytes per second.
+ virtual void SetDeleteRateBytesPerSecond(int64_t delete_rate) override;
+
+ // Return trash/DB size ratio where new files will be deleted immediately
+ virtual double GetMaxTrashDBRatio() override;
+
+ // Update trash/DB size ratio where new files will be deleted immediately
+ virtual void SetMaxTrashDBRatio(double ratio) override;
+
+ // Return the total size of trash files
+ uint64_t GetTotalTrashSize() override;
+
+ // Called by each DB instance using this sst file manager to reserve
+ // disk buffer space for recovery from out of space errors
+ void ReserveDiskBuffer(uint64_t buffer, const std::string& path);
+
+ // Set a flag upon encountering disk full. May enqueue the ErrorHandler
+ // instance for background polling and recovery
+ void StartErrorRecovery(ErrorHandler* db, Status bg_error);
+
+ // Remove the given Errorhandler instance from the recovery queue. Its
+ // not guaranteed
+ bool CancelErrorRecovery(ErrorHandler* db);
+
+ // Mark file as trash and schedule it's deletion. If force_bg is set, it
+ // forces the file to be deleting in the background regardless of DB size,
+ // except when rate limited delete is disabled
+ virtual Status ScheduleFileDeletion(const std::string& file_path,
+ const std::string& dir_to_sync,
+ const bool force_bg = false);
+
+ // Wait for all files being deleteing in the background to finish or for
+ // destructor to be called.
+ virtual void WaitForEmptyTrash();
+
+ DeleteScheduler* delete_scheduler() { return &delete_scheduler_; }
+
+ // Stop the error recovery background thread. This should be called only
+ // once in the object's lifetime, and before the destructor
+ void Close();
+
+ void SetStatisticsPtr(const std::shared_ptr<Statistics>& stats) override {
+ stats_ = stats;
+ delete_scheduler_.SetStatisticsPtr(stats);
+ }
+
+ private:
+ // REQUIRES: mutex locked
+ void OnAddFileImpl(const std::string& file_path, uint64_t file_size);
+ // REQUIRES: mutex locked
+ void OnDeleteFileImpl(const std::string& file_path);
+
+ void ClearError();
+ bool CheckFreeSpace() {
+ return bg_err_.severity() == Status::Severity::kSoftError;
+ }
+
+ std::shared_ptr<SystemClock> clock_;
+ std::shared_ptr<FileSystem> fs_;
+ std::shared_ptr<Logger> logger_;
+ // Mutex to protect tracked_files_, total_files_size_
+ port::Mutex mu_;
+ // The summation of the sizes of all files in tracked_files_ map
+ uint64_t total_files_size_;
+ // Compactions should only execute if they can leave at least
+ // this amount of buffer space for logs and flushes
+ uint64_t compaction_buffer_size_;
+ // Estimated size of the current ongoing compactions
+ uint64_t cur_compactions_reserved_size_;
+ // A map containing all tracked files and there sizes
+ // file_path => file_size
+ std::unordered_map<std::string, uint64_t> tracked_files_;
+ // The maximum allowed space (in bytes) for sst and blob files.
+ uint64_t max_allowed_space_;
+ // DeleteScheduler used to throttle file deletion.
+ DeleteScheduler delete_scheduler_;
+ port::CondVar cv_;
+ // Flag to force error recovery thread to exit
+ bool closing_;
+ // Background error recovery thread
+ std::unique_ptr<port::Thread> bg_thread_;
+ // A path in the filesystem corresponding to this SFM. This is used for
+ // calling Env::GetFreeSpace. Posix requires a path in the filesystem
+ std::string path_;
+ // Save the current background error
+ Status bg_err_;
+ // Amount of free disk headroom before allowing recovery from hard errors
+ uint64_t reserved_disk_buffer_;
+ // For soft errors, amount of free disk space before we can allow
+ // compactions to run full throttle. If disk space is below this trigger,
+ // compactions will be gated by free disk space > input size
+ uint64_t free_space_trigger_;
+ // List of database error handler instances tracked by this SstFileManager.
+ std::list<ErrorHandler*> error_handler_list_;
+ // Pointer to ErrorHandler instance that is currently processing recovery
+ ErrorHandler* cur_instance_;
+ std::shared_ptr<Statistics> stats_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/file/writable_file_writer.cc b/src/rocksdb/file/writable_file_writer.cc
new file mode 100644
index 000000000..3afc51c56
--- /dev/null
+++ b/src/rocksdb/file/writable_file_writer.cc
@@ -0,0 +1,1025 @@
+// 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/writable_file_writer.h"
+
+#include <algorithm>
+#include <mutex>
+
+#include "db/version_edit.h"
+#include "monitoring/histogram.h"
+#include "monitoring/iostats_context_imp.h"
+#include "port/port.h"
+#include "rocksdb/io_status.h"
+#include "rocksdb/system_clock.h"
+#include "test_util/sync_point.h"
+#include "util/crc32c.h"
+#include "util/random.h"
+#include "util/rate_limiter.h"
+
+namespace ROCKSDB_NAMESPACE {
+IOStatus WritableFileWriter::Create(const std::shared_ptr<FileSystem>& fs,
+ const std::string& fname,
+ const FileOptions& file_opts,
+ std::unique_ptr<WritableFileWriter>* writer,
+ IODebugContext* dbg) {
+ if (file_opts.use_direct_writes &&
+ 0 == file_opts.writable_file_max_buffer_size) {
+ return IOStatus::InvalidArgument(
+ "Direct write requires writable_file_max_buffer_size > 0");
+ }
+ std::unique_ptr<FSWritableFile> file;
+ IOStatus io_s = fs->NewWritableFile(fname, file_opts, &file, dbg);
+ if (io_s.ok()) {
+ writer->reset(new WritableFileWriter(std::move(file), fname, file_opts));
+ }
+ return io_s;
+}
+
+IOStatus WritableFileWriter::Append(const Slice& data, uint32_t crc32c_checksum,
+ Env::IOPriority op_rate_limiter_priority) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+
+ const char* src = data.data();
+ size_t left = data.size();
+ IOStatus s;
+ pending_sync_ = true;
+
+ TEST_KILL_RANDOM_WITH_WEIGHT("WritableFileWriter::Append:0", REDUCE_ODDS2);
+
+ // Calculate the checksum of appended data
+ UpdateFileChecksum(data);
+
+ {
+ IOOptions io_options;
+ io_options.rate_limiter_priority =
+ WritableFileWriter::DecideRateLimiterPriority(
+ writable_file_->GetIOPriority(), op_rate_limiter_priority);
+ IOSTATS_TIMER_GUARD(prepare_write_nanos);
+ TEST_SYNC_POINT("WritableFileWriter::Append:BeforePrepareWrite");
+ writable_file_->PrepareWrite(static_cast<size_t>(GetFileSize()), left,
+ io_options, nullptr);
+ }
+
+ // See whether we need to enlarge the buffer to avoid the flush
+ if (buf_.Capacity() - buf_.CurrentSize() < left) {
+ for (size_t cap = buf_.Capacity();
+ cap < max_buffer_size_; // There is still room to increase
+ cap *= 2) {
+ // See whether the next available size is large enough.
+ // Buffer will never be increased to more than max_buffer_size_.
+ size_t desired_capacity = std::min(cap * 2, max_buffer_size_);
+ if (desired_capacity - buf_.CurrentSize() >= left ||
+ (use_direct_io() && desired_capacity == max_buffer_size_)) {
+ buf_.AllocateNewBuffer(desired_capacity, true);
+ break;
+ }
+ }
+ }
+
+ // Flush only when buffered I/O
+ if (!use_direct_io() && (buf_.Capacity() - buf_.CurrentSize()) < left) {
+ if (buf_.CurrentSize() > 0) {
+ s = Flush(op_rate_limiter_priority);
+ if (!s.ok()) {
+ set_seen_error();
+ return s;
+ }
+ }
+ assert(buf_.CurrentSize() == 0);
+ }
+
+ if (perform_data_verification_ && buffered_data_with_checksum_ &&
+ crc32c_checksum != 0) {
+ // Since we want to use the checksum of the input data, we cannot break it
+ // into several pieces. We will only write them in the buffer when buffer
+ // size is enough. Otherwise, we will directly write it down.
+ if (use_direct_io() || (buf_.Capacity() - buf_.CurrentSize()) >= left) {
+ if ((buf_.Capacity() - buf_.CurrentSize()) >= left) {
+ size_t appended = buf_.Append(src, left);
+ if (appended != left) {
+ s = IOStatus::Corruption("Write buffer append failure");
+ }
+ buffered_data_crc32c_checksum_ = crc32c::Crc32cCombine(
+ buffered_data_crc32c_checksum_, crc32c_checksum, appended);
+ } else {
+ while (left > 0) {
+ size_t appended = buf_.Append(src, left);
+ buffered_data_crc32c_checksum_ =
+ crc32c::Extend(buffered_data_crc32c_checksum_, src, appended);
+ left -= appended;
+ src += appended;
+
+ if (left > 0) {
+ s = Flush(op_rate_limiter_priority);
+ if (!s.ok()) {
+ break;
+ }
+ }
+ }
+ }
+ } else {
+ assert(buf_.CurrentSize() == 0);
+ buffered_data_crc32c_checksum_ = crc32c_checksum;
+ s = WriteBufferedWithChecksum(src, left, op_rate_limiter_priority);
+ }
+ } else {
+ // In this case, either we do not need to do the data verification or
+ // caller does not provide the checksum of the data (crc32c_checksum = 0).
+ //
+ // We never write directly to disk with direct I/O on.
+ // or we simply use it for its original purpose to accumulate many small
+ // chunks
+ if (use_direct_io() || (buf_.Capacity() >= left)) {
+ while (left > 0) {
+ size_t appended = buf_.Append(src, left);
+ if (perform_data_verification_ && buffered_data_with_checksum_) {
+ buffered_data_crc32c_checksum_ =
+ crc32c::Extend(buffered_data_crc32c_checksum_, src, appended);
+ }
+ left -= appended;
+ src += appended;
+
+ if (left > 0) {
+ s = Flush(op_rate_limiter_priority);
+ if (!s.ok()) {
+ break;
+ }
+ }
+ }
+ } else {
+ // Writing directly to file bypassing the buffer
+ assert(buf_.CurrentSize() == 0);
+ if (perform_data_verification_ && buffered_data_with_checksum_) {
+ buffered_data_crc32c_checksum_ = crc32c::Value(src, left);
+ s = WriteBufferedWithChecksum(src, left, op_rate_limiter_priority);
+ } else {
+ s = WriteBuffered(src, left, op_rate_limiter_priority);
+ }
+ }
+ }
+
+ TEST_KILL_RANDOM("WritableFileWriter::Append:1");
+ if (s.ok()) {
+ uint64_t cur_size = filesize_.load(std::memory_order_acquire);
+ filesize_.store(cur_size + data.size(), std::memory_order_release);
+ } else {
+ set_seen_error();
+ }
+ return s;
+}
+
+IOStatus WritableFileWriter::Pad(const size_t pad_bytes,
+ Env::IOPriority op_rate_limiter_priority) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+ assert(pad_bytes < kDefaultPageSize);
+ size_t left = pad_bytes;
+ size_t cap = buf_.Capacity() - buf_.CurrentSize();
+ size_t pad_start = buf_.CurrentSize();
+
+ // Assume pad_bytes is small compared to buf_ capacity. So we always
+ // use buf_ rather than write directly to file in certain cases like
+ // Append() does.
+ while (left) {
+ size_t append_bytes = std::min(cap, left);
+ buf_.PadWith(append_bytes, 0);
+ left -= append_bytes;
+ if (left > 0) {
+ IOStatus s = Flush(op_rate_limiter_priority);
+ if (!s.ok()) {
+ set_seen_error();
+ return s;
+ }
+ }
+ cap = buf_.Capacity() - buf_.CurrentSize();
+ }
+ pending_sync_ = true;
+ uint64_t cur_size = filesize_.load(std::memory_order_acquire);
+ filesize_.store(cur_size + pad_bytes, std::memory_order_release);
+ if (perform_data_verification_) {
+ buffered_data_crc32c_checksum_ =
+ crc32c::Extend(buffered_data_crc32c_checksum_,
+ buf_.BufferStart() + pad_start, pad_bytes);
+ }
+ return IOStatus::OK();
+}
+
+IOStatus WritableFileWriter::Close() {
+ if (seen_error()) {
+ IOStatus interim;
+ if (writable_file_.get() != nullptr) {
+ interim = writable_file_->Close(IOOptions(), nullptr);
+ writable_file_.reset();
+ }
+ if (interim.ok()) {
+ return IOStatus::IOError(
+ "File is closed but data not flushed as writer has previous error.");
+ } else {
+ return interim;
+ }
+ }
+
+ // Do not quit immediately on failure the file MUST be closed
+
+ // Possible to close it twice now as we MUST close
+ // in __dtor, simply flushing is not enough
+ // Windows when pre-allocating does not fill with zeros
+ // also with unbuffered access we also set the end of data.
+ if (writable_file_.get() == nullptr) {
+ return IOStatus::OK();
+ }
+
+ IOStatus s;
+ s = Flush(); // flush cache to OS
+
+ IOStatus interim;
+ IOOptions io_options;
+ io_options.rate_limiter_priority = writable_file_->GetIOPriority();
+ // In direct I/O mode we write whole pages so
+ // we need to let the file know where data ends.
+ if (use_direct_io()) {
+ {
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+ uint64_t filesz = filesize_.load(std::memory_order_acquire);
+ interim = writable_file_->Truncate(filesz, io_options, nullptr);
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileTruncateFinish(start_ts, finish_ts, s);
+ if (!interim.ok()) {
+ NotifyOnIOError(interim, FileOperationType::kTruncate, file_name(),
+ filesz);
+ }
+ }
+#endif
+ }
+ if (interim.ok()) {
+ {
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+ interim = writable_file_->Fsync(io_options, nullptr);
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileSyncFinish(start_ts, finish_ts, s,
+ FileOperationType::kFsync);
+ if (!interim.ok()) {
+ NotifyOnIOError(interim, FileOperationType::kFsync, file_name());
+ }
+ }
+#endif
+ }
+ }
+ if (!interim.ok() && s.ok()) {
+ s = interim;
+ }
+ }
+
+ TEST_KILL_RANDOM("WritableFileWriter::Close:0");
+ {
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+ interim = writable_file_->Close(io_options, nullptr);
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = FileOperationInfo::FinishNow();
+ NotifyOnFileCloseFinish(start_ts, finish_ts, s);
+ if (!interim.ok()) {
+ NotifyOnIOError(interim, FileOperationType::kClose, file_name());
+ }
+ }
+#endif
+ }
+ if (!interim.ok() && s.ok()) {
+ s = interim;
+ }
+
+ writable_file_.reset();
+ TEST_KILL_RANDOM("WritableFileWriter::Close:1");
+
+ if (s.ok()) {
+ if (checksum_generator_ != nullptr && !checksum_finalized_) {
+ checksum_generator_->Finalize();
+ checksum_finalized_ = true;
+ }
+ } else {
+ set_seen_error();
+ }
+
+ return s;
+}
+
+// write out the cached data to the OS cache or storage if direct I/O
+// enabled
+IOStatus WritableFileWriter::Flush(Env::IOPriority op_rate_limiter_priority) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+
+ IOStatus s;
+ TEST_KILL_RANDOM_WITH_WEIGHT("WritableFileWriter::Flush:0", REDUCE_ODDS2);
+
+ if (buf_.CurrentSize() > 0) {
+ if (use_direct_io()) {
+#ifndef ROCKSDB_LITE
+ if (pending_sync_) {
+ if (perform_data_verification_ && buffered_data_with_checksum_) {
+ s = WriteDirectWithChecksum(op_rate_limiter_priority);
+ } else {
+ s = WriteDirect(op_rate_limiter_priority);
+ }
+ }
+#endif // !ROCKSDB_LITE
+ } else {
+ if (perform_data_verification_ && buffered_data_with_checksum_) {
+ s = WriteBufferedWithChecksum(buf_.BufferStart(), buf_.CurrentSize(),
+ op_rate_limiter_priority);
+ } else {
+ s = WriteBuffered(buf_.BufferStart(), buf_.CurrentSize(),
+ op_rate_limiter_priority);
+ }
+ }
+ if (!s.ok()) {
+ set_seen_error();
+ return s;
+ }
+ }
+
+ {
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+ IOOptions io_options;
+ io_options.rate_limiter_priority =
+ WritableFileWriter::DecideRateLimiterPriority(
+ writable_file_->GetIOPriority(), op_rate_limiter_priority);
+ s = writable_file_->Flush(io_options, nullptr);
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = std::chrono::steady_clock::now();
+ NotifyOnFileFlushFinish(start_ts, finish_ts, s);
+ if (!s.ok()) {
+ NotifyOnIOError(s, FileOperationType::kFlush, file_name());
+ }
+ }
+#endif
+ }
+
+ if (!s.ok()) {
+ set_seen_error();
+ return s;
+ }
+
+ // sync OS cache to disk for every bytes_per_sync_
+ // TODO: give log file and sst file different options (log
+ // files could be potentially cached in OS for their whole
+ // life time, thus we might not want to flush at all).
+
+ // We try to avoid sync to the last 1MB of data. For two reasons:
+ // (1) avoid rewrite the same page that is modified later.
+ // (2) for older version of OS, write can block while writing out
+ // the page.
+ // Xfs does neighbor page flushing outside of the specified ranges. We
+ // need to make sure sync range is far from the write offset.
+ if (!use_direct_io() && bytes_per_sync_) {
+ const uint64_t kBytesNotSyncRange =
+ 1024 * 1024; // recent 1MB is not synced.
+ const uint64_t kBytesAlignWhenSync = 4 * 1024; // Align 4KB.
+ uint64_t cur_size = filesize_.load(std::memory_order_acquire);
+ if (cur_size > kBytesNotSyncRange) {
+ uint64_t offset_sync_to = cur_size - kBytesNotSyncRange;
+ offset_sync_to -= offset_sync_to % kBytesAlignWhenSync;
+ assert(offset_sync_to >= last_sync_size_);
+ if (offset_sync_to > 0 &&
+ offset_sync_to - last_sync_size_ >= bytes_per_sync_) {
+ s = RangeSync(last_sync_size_, offset_sync_to - last_sync_size_);
+ if (!s.ok()) {
+ set_seen_error();
+ }
+ last_sync_size_ = offset_sync_to;
+ }
+ }
+ }
+
+ return s;
+}
+
+std::string WritableFileWriter::GetFileChecksum() {
+ if (checksum_generator_ != nullptr) {
+ assert(checksum_finalized_);
+ return checksum_generator_->GetChecksum();
+ } else {
+ return kUnknownFileChecksum;
+ }
+}
+
+const char* WritableFileWriter::GetFileChecksumFuncName() const {
+ if (checksum_generator_ != nullptr) {
+ return checksum_generator_->Name();
+ } else {
+ return kUnknownFileChecksumFuncName;
+ }
+}
+
+IOStatus WritableFileWriter::Sync(bool use_fsync) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+
+ IOStatus s = Flush();
+ if (!s.ok()) {
+ set_seen_error();
+ return s;
+ }
+ TEST_KILL_RANDOM("WritableFileWriter::Sync:0");
+ if (!use_direct_io() && pending_sync_) {
+ s = SyncInternal(use_fsync);
+ if (!s.ok()) {
+ set_seen_error();
+ return s;
+ }
+ }
+ TEST_KILL_RANDOM("WritableFileWriter::Sync:1");
+ pending_sync_ = false;
+ return IOStatus::OK();
+}
+
+IOStatus WritableFileWriter::SyncWithoutFlush(bool use_fsync) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+ if (!writable_file_->IsSyncThreadSafe()) {
+ return IOStatus::NotSupported(
+ "Can't WritableFileWriter::SyncWithoutFlush() because "
+ "WritableFile::IsSyncThreadSafe() is false");
+ }
+ TEST_SYNC_POINT("WritableFileWriter::SyncWithoutFlush:1");
+ IOStatus s = SyncInternal(use_fsync);
+ TEST_SYNC_POINT("WritableFileWriter::SyncWithoutFlush:2");
+ if (!s.ok()) {
+#ifndef NDEBUG
+ sync_without_flush_called_ = true;
+#endif // NDEBUG
+ set_seen_error();
+ }
+ return s;
+}
+
+IOStatus WritableFileWriter::SyncInternal(bool use_fsync) {
+ // Caller is supposed to check seen_error_
+ IOStatus s;
+ IOSTATS_TIMER_GUARD(fsync_nanos);
+ TEST_SYNC_POINT("WritableFileWriter::SyncInternal:0");
+ auto prev_perf_level = GetPerfLevel();
+
+ IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_);
+
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+
+ IOOptions io_options;
+ io_options.rate_limiter_priority = writable_file_->GetIOPriority();
+ if (use_fsync) {
+ s = writable_file_->Fsync(io_options, nullptr);
+ } else {
+ s = writable_file_->Sync(io_options, nullptr);
+ }
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = std::chrono::steady_clock::now();
+ NotifyOnFileSyncFinish(
+ start_ts, finish_ts, s,
+ use_fsync ? FileOperationType::kFsync : FileOperationType::kSync);
+ if (!s.ok()) {
+ NotifyOnIOError(
+ s, (use_fsync ? FileOperationType::kFsync : FileOperationType::kSync),
+ file_name());
+ }
+ }
+#endif
+ SetPerfLevel(prev_perf_level);
+
+ // The caller will be responsible to call set_seen_error() if s is not OK.
+ return s;
+}
+
+IOStatus WritableFileWriter::RangeSync(uint64_t offset, uint64_t nbytes) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+
+ IOSTATS_TIMER_GUARD(range_sync_nanos);
+ TEST_SYNC_POINT("WritableFileWriter::RangeSync:0");
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+#endif
+ IOOptions io_options;
+ io_options.rate_limiter_priority = writable_file_->GetIOPriority();
+ IOStatus s = writable_file_->RangeSync(offset, nbytes, io_options, nullptr);
+ if (!s.ok()) {
+ set_seen_error();
+ }
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = std::chrono::steady_clock::now();
+ NotifyOnFileRangeSyncFinish(offset, nbytes, start_ts, finish_ts, s);
+ if (!s.ok()) {
+ NotifyOnIOError(s, FileOperationType::kRangeSync, file_name(), nbytes,
+ offset);
+ }
+ }
+#endif
+ return s;
+}
+
+// This method writes to disk the specified data and makes use of the rate
+// limiter if available
+IOStatus WritableFileWriter::WriteBuffered(
+ const char* data, size_t size, Env::IOPriority op_rate_limiter_priority) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+
+ IOStatus s;
+ assert(!use_direct_io());
+ const char* src = data;
+ size_t left = size;
+ DataVerificationInfo v_info;
+ char checksum_buf[sizeof(uint32_t)];
+ Env::IOPriority rate_limiter_priority_used =
+ WritableFileWriter::DecideRateLimiterPriority(
+ writable_file_->GetIOPriority(), op_rate_limiter_priority);
+ IOOptions io_options;
+ io_options.rate_limiter_priority = rate_limiter_priority_used;
+
+ while (left > 0) {
+ size_t allowed = left;
+ if (rate_limiter_ != nullptr &&
+ rate_limiter_priority_used != Env::IO_TOTAL) {
+ allowed = rate_limiter_->RequestToken(left, 0 /* alignment */,
+ rate_limiter_priority_used, stats_,
+ RateLimiter::OpType::kWrite);
+ }
+
+ {
+ IOSTATS_TIMER_GUARD(write_nanos);
+ TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend");
+
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ uint64_t old_size = writable_file_->GetFileSize(io_options, nullptr);
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ old_size = next_write_offset_;
+ }
+#endif
+ {
+ auto prev_perf_level = GetPerfLevel();
+
+ IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_);
+ if (perform_data_verification_) {
+ Crc32cHandoffChecksumCalculation(src, allowed, checksum_buf);
+ v_info.checksum = Slice(checksum_buf, sizeof(uint32_t));
+ s = writable_file_->Append(Slice(src, allowed), io_options, v_info,
+ nullptr);
+ } else {
+ s = writable_file_->Append(Slice(src, allowed), io_options, nullptr);
+ }
+ if (!s.ok()) {
+ // If writable_file_->Append() failed, then the data may or may not
+ // exist in the underlying memory buffer, OS page cache, remote file
+ // system's buffer, etc. If WritableFileWriter keeps the data in
+ // buf_, then a future Close() or write retry may send the data to
+ // the underlying file again. If the data does exist in the
+ // underlying buffer and gets written to the file eventually despite
+ // returning error, the file may end up with two duplicate pieces of
+ // data. Therefore, clear the buf_ at the WritableFileWriter layer
+ // and let caller determine error handling.
+ buf_.Size(0);
+ buffered_data_crc32c_checksum_ = 0;
+ }
+ SetPerfLevel(prev_perf_level);
+ }
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = std::chrono::steady_clock::now();
+ NotifyOnFileWriteFinish(old_size, allowed, start_ts, finish_ts, s);
+ if (!s.ok()) {
+ NotifyOnIOError(s, FileOperationType::kAppend, file_name(), allowed,
+ old_size);
+ }
+ }
+#endif
+ if (!s.ok()) {
+ set_seen_error();
+ return s;
+ }
+ }
+
+ IOSTATS_ADD(bytes_written, allowed);
+ TEST_KILL_RANDOM("WritableFileWriter::WriteBuffered:0");
+
+ left -= allowed;
+ src += allowed;
+ uint64_t cur_size = flushed_size_.load(std::memory_order_acquire);
+ flushed_size_.store(cur_size + allowed, std::memory_order_release);
+ }
+ buf_.Size(0);
+ buffered_data_crc32c_checksum_ = 0;
+ if (!s.ok()) {
+ set_seen_error();
+ }
+ return s;
+}
+
+IOStatus WritableFileWriter::WriteBufferedWithChecksum(
+ const char* data, size_t size, Env::IOPriority op_rate_limiter_priority) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+
+ IOStatus s;
+ assert(!use_direct_io());
+ assert(perform_data_verification_ && buffered_data_with_checksum_);
+ const char* src = data;
+ size_t left = size;
+ DataVerificationInfo v_info;
+ char checksum_buf[sizeof(uint32_t)];
+ Env::IOPriority rate_limiter_priority_used =
+ WritableFileWriter::DecideRateLimiterPriority(
+ writable_file_->GetIOPriority(), op_rate_limiter_priority);
+ IOOptions io_options;
+ io_options.rate_limiter_priority = rate_limiter_priority_used;
+ // Check how much is allowed. Here, we loop until the rate limiter allows to
+ // write the entire buffer.
+ // TODO: need to be improved since it sort of defeats the purpose of the rate
+ // limiter
+ size_t data_size = left;
+ if (rate_limiter_ != nullptr && rate_limiter_priority_used != Env::IO_TOTAL) {
+ while (data_size > 0) {
+ size_t tmp_size;
+ tmp_size = rate_limiter_->RequestToken(data_size, buf_.Alignment(),
+ rate_limiter_priority_used, stats_,
+ RateLimiter::OpType::kWrite);
+ data_size -= tmp_size;
+ }
+ }
+
+ {
+ IOSTATS_TIMER_GUARD(write_nanos);
+ TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend");
+
+#ifndef ROCKSDB_LITE
+ FileOperationInfo::StartTimePoint start_ts;
+ uint64_t old_size = writable_file_->GetFileSize(io_options, nullptr);
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ old_size = next_write_offset_;
+ }
+#endif
+ {
+ auto prev_perf_level = GetPerfLevel();
+
+ IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, clock_);
+
+ EncodeFixed32(checksum_buf, buffered_data_crc32c_checksum_);
+ v_info.checksum = Slice(checksum_buf, sizeof(uint32_t));
+ s = writable_file_->Append(Slice(src, left), io_options, v_info, nullptr);
+ SetPerfLevel(prev_perf_level);
+ }
+#ifndef ROCKSDB_LITE
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = std::chrono::steady_clock::now();
+ NotifyOnFileWriteFinish(old_size, left, start_ts, finish_ts, s);
+ if (!s.ok()) {
+ NotifyOnIOError(s, FileOperationType::kAppend, file_name(), left,
+ old_size);
+ }
+ }
+#endif
+ if (!s.ok()) {
+ // If writable_file_->Append() failed, then the data may or may not
+ // exist in the underlying memory buffer, OS page cache, remote file
+ // system's buffer, etc. If WritableFileWriter keeps the data in
+ // buf_, then a future Close() or write retry may send the data to
+ // the underlying file again. If the data does exist in the
+ // underlying buffer and gets written to the file eventually despite
+ // returning error, the file may end up with two duplicate pieces of
+ // data. Therefore, clear the buf_ at the WritableFileWriter layer
+ // and let caller determine error handling.
+ buf_.Size(0);
+ buffered_data_crc32c_checksum_ = 0;
+ set_seen_error();
+ return s;
+ }
+ }
+
+ IOSTATS_ADD(bytes_written, left);
+ TEST_KILL_RANDOM("WritableFileWriter::WriteBuffered:0");
+
+ // Buffer write is successful, reset the buffer current size to 0 and reset
+ // the corresponding checksum value
+ buf_.Size(0);
+ buffered_data_crc32c_checksum_ = 0;
+ uint64_t cur_size = flushed_size_.load(std::memory_order_acquire);
+ flushed_size_.store(cur_size + left, std::memory_order_release);
+ if (!s.ok()) {
+ set_seen_error();
+ }
+ return s;
+}
+
+void WritableFileWriter::UpdateFileChecksum(const Slice& data) {
+ if (checksum_generator_ != nullptr) {
+ checksum_generator_->Update(data.data(), data.size());
+ }
+}
+
+// Currently, crc32c checksum is used to calculate the checksum value of the
+// content in the input buffer for handoff. In the future, the checksum might be
+// calculated from the existing crc32c checksums of the in WAl and Manifest
+// records, or even SST file blocks.
+// TODO: effectively use the existing checksum of the data being writing to
+// generate the crc32c checksum instead of a raw calculation.
+void WritableFileWriter::Crc32cHandoffChecksumCalculation(const char* data,
+ size_t size,
+ char* buf) {
+ uint32_t v_crc32c = crc32c::Extend(0, data, size);
+ EncodeFixed32(buf, v_crc32c);
+}
+
+// This flushes the accumulated data in the buffer. We pad data with zeros if
+// necessary to the whole page.
+// However, during automatic flushes padding would not be necessary.
+// We always use RateLimiter if available. We move (Refit) any buffer bytes
+// that are left over the
+// whole number of pages to be written again on the next flush because we can
+// only write on aligned
+// offsets.
+#ifndef ROCKSDB_LITE
+IOStatus WritableFileWriter::WriteDirect(
+ Env::IOPriority op_rate_limiter_priority) {
+ if (seen_error()) {
+ assert(false);
+
+ return IOStatus::IOError("Writer has previous error.");
+ }
+
+ assert(use_direct_io());
+ IOStatus s;
+ const size_t alignment = buf_.Alignment();
+ assert((next_write_offset_ % alignment) == 0);
+
+ // Calculate whole page final file advance if all writes succeed
+ const size_t file_advance =
+ TruncateToPageBoundary(alignment, buf_.CurrentSize());
+
+ // Calculate the leftover tail, we write it here padded with zeros BUT we
+ // will write it again in the future either on Close() OR when the current
+ // whole page fills out.
+ const size_t leftover_tail = buf_.CurrentSize() - file_advance;
+
+ // Round up and pad
+ buf_.PadToAlignmentWith(0);
+
+ const char* src = buf_.BufferStart();
+ uint64_t write_offset = next_write_offset_;
+ size_t left = buf_.CurrentSize();
+ DataVerificationInfo v_info;
+ char checksum_buf[sizeof(uint32_t)];
+ Env::IOPriority rate_limiter_priority_used =
+ WritableFileWriter::DecideRateLimiterPriority(
+ writable_file_->GetIOPriority(), op_rate_limiter_priority);
+ IOOptions io_options;
+ io_options.rate_limiter_priority = rate_limiter_priority_used;
+
+ while (left > 0) {
+ // Check how much is allowed
+ size_t size = left;
+ if (rate_limiter_ != nullptr &&
+ rate_limiter_priority_used != Env::IO_TOTAL) {
+ size = rate_limiter_->RequestToken(left, buf_.Alignment(),
+ rate_limiter_priority_used, stats_,
+ RateLimiter::OpType::kWrite);
+ }
+
+ {
+ IOSTATS_TIMER_GUARD(write_nanos);
+ TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend");
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+ // direct writes must be positional
+ if (perform_data_verification_) {
+ Crc32cHandoffChecksumCalculation(src, size, checksum_buf);
+ v_info.checksum = Slice(checksum_buf, sizeof(uint32_t));
+ s = writable_file_->PositionedAppend(Slice(src, size), write_offset,
+ io_options, v_info, nullptr);
+ } else {
+ s = writable_file_->PositionedAppend(Slice(src, size), write_offset,
+ io_options, nullptr);
+ }
+
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = std::chrono::steady_clock::now();
+ NotifyOnFileWriteFinish(write_offset, size, start_ts, finish_ts, s);
+ if (!s.ok()) {
+ NotifyOnIOError(s, FileOperationType::kPositionedAppend, file_name(),
+ size, write_offset);
+ }
+ }
+ if (!s.ok()) {
+ buf_.Size(file_advance + leftover_tail);
+ set_seen_error();
+ return s;
+ }
+ }
+
+ IOSTATS_ADD(bytes_written, size);
+ left -= size;
+ src += size;
+ write_offset += size;
+ uint64_t cur_size = flushed_size_.load(std::memory_order_acquire);
+ flushed_size_.store(cur_size + size, std::memory_order_release);
+ assert((next_write_offset_ % alignment) == 0);
+ }
+
+ if (s.ok()) {
+ // Move the tail to the beginning of the buffer
+ // This never happens during normal Append but rather during
+ // explicit call to Flush()/Sync() or Close()
+ buf_.RefitTail(file_advance, leftover_tail);
+ // This is where we start writing next time which may or not be
+ // the actual file size on disk. They match if the buffer size
+ // is a multiple of whole pages otherwise filesize_ is leftover_tail
+ // behind
+ next_write_offset_ += file_advance;
+ } else {
+ set_seen_error();
+ }
+ return s;
+}
+
+IOStatus WritableFileWriter::WriteDirectWithChecksum(
+ Env::IOPriority op_rate_limiter_priority) {
+ if (seen_error()) {
+ return AssertFalseAndGetStatusForPrevError();
+ }
+
+ assert(use_direct_io());
+ assert(perform_data_verification_ && buffered_data_with_checksum_);
+ IOStatus s;
+ const size_t alignment = buf_.Alignment();
+ assert((next_write_offset_ % alignment) == 0);
+
+ // Calculate whole page final file advance if all writes succeed
+ const size_t file_advance =
+ TruncateToPageBoundary(alignment, buf_.CurrentSize());
+
+ // Calculate the leftover tail, we write it here padded with zeros BUT we
+ // will write it again in the future either on Close() OR when the current
+ // whole page fills out.
+ const size_t leftover_tail = buf_.CurrentSize() - file_advance;
+
+ // Round up, pad, and combine the checksum.
+ size_t last_cur_size = buf_.CurrentSize();
+ buf_.PadToAlignmentWith(0);
+ size_t padded_size = buf_.CurrentSize() - last_cur_size;
+ const char* padded_start = buf_.BufferStart() + last_cur_size;
+ uint32_t padded_checksum = crc32c::Value(padded_start, padded_size);
+ buffered_data_crc32c_checksum_ = crc32c::Crc32cCombine(
+ buffered_data_crc32c_checksum_, padded_checksum, padded_size);
+
+ const char* src = buf_.BufferStart();
+ uint64_t write_offset = next_write_offset_;
+ size_t left = buf_.CurrentSize();
+ DataVerificationInfo v_info;
+ char checksum_buf[sizeof(uint32_t)];
+
+ Env::IOPriority rate_limiter_priority_used =
+ WritableFileWriter::DecideRateLimiterPriority(
+ writable_file_->GetIOPriority(), op_rate_limiter_priority);
+ IOOptions io_options;
+ io_options.rate_limiter_priority = rate_limiter_priority_used;
+ // Check how much is allowed. Here, we loop until the rate limiter allows to
+ // write the entire buffer.
+ // TODO: need to be improved since it sort of defeats the purpose of the rate
+ // limiter
+ size_t data_size = left;
+ if (rate_limiter_ != nullptr && rate_limiter_priority_used != Env::IO_TOTAL) {
+ while (data_size > 0) {
+ size_t size;
+ size = rate_limiter_->RequestToken(data_size, buf_.Alignment(),
+ rate_limiter_priority_used, stats_,
+ RateLimiter::OpType::kWrite);
+ data_size -= size;
+ }
+ }
+
+ {
+ IOSTATS_TIMER_GUARD(write_nanos);
+ TEST_SYNC_POINT("WritableFileWriter::Flush:BeforeAppend");
+ FileOperationInfo::StartTimePoint start_ts;
+ if (ShouldNotifyListeners()) {
+ start_ts = FileOperationInfo::StartNow();
+ }
+ // direct writes must be positional
+ EncodeFixed32(checksum_buf, buffered_data_crc32c_checksum_);
+ v_info.checksum = Slice(checksum_buf, sizeof(uint32_t));
+ s = writable_file_->PositionedAppend(Slice(src, left), write_offset,
+ io_options, v_info, nullptr);
+
+ if (ShouldNotifyListeners()) {
+ auto finish_ts = std::chrono::steady_clock::now();
+ NotifyOnFileWriteFinish(write_offset, left, start_ts, finish_ts, s);
+ if (!s.ok()) {
+ NotifyOnIOError(s, FileOperationType::kPositionedAppend, file_name(),
+ left, write_offset);
+ }
+ }
+ if (!s.ok()) {
+ // In this case, we do not change buffered_data_crc32c_checksum_ because
+ // it still aligns with the data in the buffer.
+ buf_.Size(file_advance + leftover_tail);
+ buffered_data_crc32c_checksum_ =
+ crc32c::Value(buf_.BufferStart(), buf_.CurrentSize());
+ set_seen_error();
+ return s;
+ }
+ }
+
+ IOSTATS_ADD(bytes_written, left);
+ assert((next_write_offset_ % alignment) == 0);
+ uint64_t cur_size = flushed_size_.load(std::memory_order_acquire);
+ flushed_size_.store(cur_size + left, std::memory_order_release);
+
+ if (s.ok()) {
+ // Move the tail to the beginning of the buffer
+ // This never happens during normal Append but rather during
+ // explicit call to Flush()/Sync() or Close(). Also the buffer checksum will
+ // recalculated accordingly.
+ buf_.RefitTail(file_advance, leftover_tail);
+ // Adjust the checksum value to align with the data in the buffer
+ buffered_data_crc32c_checksum_ =
+ crc32c::Value(buf_.BufferStart(), buf_.CurrentSize());
+ // This is where we start writing next time which may or not be
+ // the actual file size on disk. They match if the buffer size
+ // is a multiple of whole pages otherwise filesize_ is leftover_tail
+ // behind
+ next_write_offset_ += file_advance;
+ } else {
+ set_seen_error();
+ }
+ return s;
+}
+#endif // !ROCKSDB_LITE
+Env::IOPriority WritableFileWriter::DecideRateLimiterPriority(
+ Env::IOPriority writable_file_io_priority,
+ Env::IOPriority op_rate_limiter_priority) {
+ if (writable_file_io_priority == Env::IO_TOTAL &&
+ op_rate_limiter_priority == Env::IO_TOTAL) {
+ return Env::IO_TOTAL;
+ } else if (writable_file_io_priority == Env::IO_TOTAL) {
+ return op_rate_limiter_priority;
+ } else if (op_rate_limiter_priority == Env::IO_TOTAL) {
+ return writable_file_io_priority;
+ } else {
+ return op_rate_limiter_priority;
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/file/writable_file_writer.h b/src/rocksdb/file/writable_file_writer.h
new file mode 100644
index 000000000..b3985eb20
--- /dev/null
+++ b/src/rocksdb/file/writable_file_writer.h
@@ -0,0 +1,336 @@
+// 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 "db/version_edit.h"
+#include "env/file_system_tracer.h"
+#include "port/port.h"
+#include "rocksdb/file_checksum.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/io_status.h"
+#include "rocksdb/listener.h"
+#include "rocksdb/rate_limiter.h"
+#include "test_util/sync_point.h"
+#include "util/aligned_buffer.h"
+
+namespace ROCKSDB_NAMESPACE {
+class Statistics;
+class SystemClock;
+
+// WritableFileWriter is a wrapper on top of Env::WritableFile. It provides
+// facilities to:
+// - Handle Buffered and Direct writes.
+// - Rate limit writes.
+// - Flush and Sync the data to the underlying filesystem.
+// - Notify any interested listeners on the completion of a write.
+// - Update IO stats.
+class WritableFileWriter {
+ private:
+#ifndef ROCKSDB_LITE
+ void NotifyOnFileWriteFinish(
+ uint64_t offset, size_t length,
+ const FileOperationInfo::StartTimePoint& start_ts,
+ const FileOperationInfo::FinishTimePoint& finish_ts,
+ const IOStatus& io_status) {
+ FileOperationInfo info(FileOperationType::kWrite, file_name_, start_ts,
+ finish_ts, io_status, temperature_);
+ info.offset = offset;
+ info.length = length;
+
+ for (auto& listener : listeners_) {
+ listener->OnFileWriteFinish(info);
+ }
+ info.status.PermitUncheckedError();
+ }
+ void NotifyOnFileFlushFinish(
+ const FileOperationInfo::StartTimePoint& start_ts,
+ const FileOperationInfo::FinishTimePoint& finish_ts,
+ const IOStatus& io_status) {
+ FileOperationInfo info(FileOperationType::kFlush, file_name_, start_ts,
+ finish_ts, io_status, temperature_);
+
+ for (auto& listener : listeners_) {
+ listener->OnFileFlushFinish(info);
+ }
+ info.status.PermitUncheckedError();
+ }
+ void NotifyOnFileSyncFinish(
+ const FileOperationInfo::StartTimePoint& start_ts,
+ const FileOperationInfo::FinishTimePoint& finish_ts,
+ const IOStatus& io_status,
+ FileOperationType type = FileOperationType::kSync) {
+ FileOperationInfo info(type, file_name_, start_ts, finish_ts, io_status,
+ temperature_);
+
+ for (auto& listener : listeners_) {
+ listener->OnFileSyncFinish(info);
+ }
+ info.status.PermitUncheckedError();
+ }
+ void NotifyOnFileRangeSyncFinish(
+ uint64_t offset, size_t length,
+ const FileOperationInfo::StartTimePoint& start_ts,
+ const FileOperationInfo::FinishTimePoint& finish_ts,
+ const IOStatus& io_status) {
+ FileOperationInfo info(FileOperationType::kRangeSync, file_name_, start_ts,
+ finish_ts, io_status, temperature_);
+ info.offset = offset;
+ info.length = length;
+
+ for (auto& listener : listeners_) {
+ listener->OnFileRangeSyncFinish(info);
+ }
+ info.status.PermitUncheckedError();
+ }
+ void NotifyOnFileTruncateFinish(
+ const FileOperationInfo::StartTimePoint& start_ts,
+ const FileOperationInfo::FinishTimePoint& finish_ts,
+ const IOStatus& io_status) {
+ FileOperationInfo info(FileOperationType::kTruncate, file_name_, start_ts,
+ finish_ts, io_status, temperature_);
+
+ for (auto& listener : listeners_) {
+ listener->OnFileTruncateFinish(info);
+ }
+ info.status.PermitUncheckedError();
+ }
+ void NotifyOnFileCloseFinish(
+ const FileOperationInfo::StartTimePoint& start_ts,
+ const FileOperationInfo::FinishTimePoint& finish_ts,
+ const IOStatus& io_status) {
+ FileOperationInfo info(FileOperationType::kClose, file_name_, start_ts,
+ finish_ts, io_status, temperature_);
+
+ for (auto& listener : listeners_) {
+ listener->OnFileCloseFinish(info);
+ }
+ info.status.PermitUncheckedError();
+ }
+
+ void NotifyOnIOError(const IOStatus& io_status, FileOperationType operation,
+ const std::string& file_path, size_t length = 0,
+ uint64_t offset = 0) {
+ if (listeners_.empty()) {
+ return;
+ }
+ IOErrorInfo io_error_info(io_status, operation, file_path, length, offset);
+ for (auto& listener : listeners_) {
+ listener->OnIOError(io_error_info);
+ }
+ io_error_info.io_status.PermitUncheckedError();
+ }
+#endif // ROCKSDB_LITE
+
+ bool ShouldNotifyListeners() const { return !listeners_.empty(); }
+ void UpdateFileChecksum(const Slice& data);
+ void Crc32cHandoffChecksumCalculation(const char* data, size_t size,
+ char* buf);
+
+ std::string file_name_;
+ FSWritableFilePtr writable_file_;
+ SystemClock* clock_;
+ AlignedBuffer buf_;
+ size_t max_buffer_size_;
+ // Actually written data size can be used for truncate
+ // not counting padding data
+ std::atomic<uint64_t> filesize_;
+ std::atomic<uint64_t> flushed_size_;
+#ifndef ROCKSDB_LITE
+ // This is necessary when we use unbuffered access
+ // and writes must happen on aligned offsets
+ // so we need to go back and write that page again
+ uint64_t next_write_offset_;
+#endif // ROCKSDB_LITE
+ bool pending_sync_;
+ std::atomic<bool> seen_error_;
+#ifndef NDEBUG
+ // SyncWithoutFlush() is the function that is allowed to be called
+ // concurrently with other function. One of the concurrent call
+ // could set seen_error_, and the other one would hit assertion
+ // in debug mode.
+ std::atomic<bool> sync_without_flush_called_ = false;
+#endif // NDEBUG
+ uint64_t last_sync_size_;
+ uint64_t bytes_per_sync_;
+ RateLimiter* rate_limiter_;
+ Statistics* stats_;
+ std::vector<std::shared_ptr<EventListener>> listeners_;
+ std::unique_ptr<FileChecksumGenerator> checksum_generator_;
+ bool checksum_finalized_;
+ bool perform_data_verification_;
+ uint32_t buffered_data_crc32c_checksum_;
+ bool buffered_data_with_checksum_;
+#ifndef ROCKSDB_LITE
+ Temperature temperature_;
+#endif // ROCKSDB_LITE
+
+ public:
+ WritableFileWriter(
+ std::unique_ptr<FSWritableFile>&& file, const std::string& _file_name,
+ const FileOptions& options, SystemClock* clock = nullptr,
+ const std::shared_ptr<IOTracer>& io_tracer = nullptr,
+ Statistics* stats = nullptr,
+ const std::vector<std::shared_ptr<EventListener>>& listeners = {},
+ FileChecksumGenFactory* file_checksum_gen_factory = nullptr,
+ bool perform_data_verification = false,
+ bool buffered_data_with_checksum = false)
+ : file_name_(_file_name),
+ writable_file_(std::move(file), io_tracer, _file_name),
+ clock_(clock),
+ buf_(),
+ max_buffer_size_(options.writable_file_max_buffer_size),
+ filesize_(0),
+ flushed_size_(0),
+#ifndef ROCKSDB_LITE
+ next_write_offset_(0),
+#endif // ROCKSDB_LITE
+ pending_sync_(false),
+ seen_error_(false),
+ last_sync_size_(0),
+ bytes_per_sync_(options.bytes_per_sync),
+ rate_limiter_(options.rate_limiter),
+ stats_(stats),
+ listeners_(),
+ checksum_generator_(nullptr),
+ checksum_finalized_(false),
+ perform_data_verification_(perform_data_verification),
+ buffered_data_crc32c_checksum_(0),
+ buffered_data_with_checksum_(buffered_data_with_checksum) {
+#ifndef ROCKSDB_LITE
+ temperature_ = options.temperature;
+#endif // ROCKSDB_LITE
+ assert(!use_direct_io() || max_buffer_size_ > 0);
+ TEST_SYNC_POINT_CALLBACK("WritableFileWriter::WritableFileWriter:0",
+ reinterpret_cast<void*>(max_buffer_size_));
+ buf_.Alignment(writable_file_->GetRequiredBufferAlignment());
+ buf_.AllocateNewBuffer(std::min((size_t)65536, max_buffer_size_));
+#ifndef ROCKSDB_LITE
+ std::for_each(listeners.begin(), listeners.end(),
+ [this](const std::shared_ptr<EventListener>& e) {
+ if (e->ShouldBeNotifiedOnFileIO()) {
+ listeners_.emplace_back(e);
+ }
+ });
+#else // !ROCKSDB_LITE
+ (void)listeners;
+#endif
+ if (file_checksum_gen_factory != nullptr) {
+ FileChecksumGenContext checksum_gen_context;
+ checksum_gen_context.file_name = _file_name;
+ checksum_generator_ =
+ file_checksum_gen_factory->CreateFileChecksumGenerator(
+ checksum_gen_context);
+ }
+ }
+
+ static IOStatus Create(const std::shared_ptr<FileSystem>& fs,
+ const std::string& fname, const FileOptions& file_opts,
+ std::unique_ptr<WritableFileWriter>* writer,
+ IODebugContext* dbg);
+ WritableFileWriter(const WritableFileWriter&) = delete;
+
+ WritableFileWriter& operator=(const WritableFileWriter&) = delete;
+
+ ~WritableFileWriter() {
+ auto s = Close();
+ s.PermitUncheckedError();
+ }
+
+ std::string file_name() const { return file_name_; }
+
+ // When this Append API is called, if the crc32c_checksum is not provided, we
+ // will calculate the checksum internally.
+ IOStatus Append(const Slice& data, uint32_t crc32c_checksum = 0,
+ Env::IOPriority op_rate_limiter_priority = Env::IO_TOTAL);
+
+ IOStatus Pad(const size_t pad_bytes,
+ Env::IOPriority op_rate_limiter_priority = Env::IO_TOTAL);
+
+ IOStatus Flush(Env::IOPriority op_rate_limiter_priority = Env::IO_TOTAL);
+
+ IOStatus Close();
+
+ IOStatus Sync(bool use_fsync);
+
+ // Sync only the data that was already Flush()ed. Safe to call concurrently
+ // with Append() and Flush(). If !writable_file_->IsSyncThreadSafe(),
+ // returns NotSupported status.
+ IOStatus SyncWithoutFlush(bool use_fsync);
+
+ uint64_t GetFileSize() const {
+ return filesize_.load(std::memory_order_acquire);
+ }
+
+ // Returns the size of data flushed to the underlying `FSWritableFile`.
+ // Expected to match `writable_file()->GetFileSize()`.
+ // The return value can serve as a lower-bound for the amount of data synced
+ // by a future call to `SyncWithoutFlush()`.
+ uint64_t GetFlushedSize() const {
+ return flushed_size_.load(std::memory_order_acquire);
+ }
+
+ IOStatus InvalidateCache(size_t offset, size_t length) {
+ return writable_file_->InvalidateCache(offset, length);
+ }
+
+ FSWritableFile* writable_file() const { return writable_file_.get(); }
+
+ bool use_direct_io() { return writable_file_->use_direct_io(); }
+
+ bool BufferIsEmpty() { return buf_.CurrentSize() == 0; }
+
+ void TEST_SetFileChecksumGenerator(
+ FileChecksumGenerator* checksum_generator) {
+ checksum_generator_.reset(checksum_generator);
+ }
+
+ std::string GetFileChecksum();
+
+ const char* GetFileChecksumFuncName() const;
+
+ bool seen_error() const {
+ return seen_error_.load(std::memory_order_relaxed);
+ }
+ // For options of relaxed consistency, users might hope to continue
+ // operating on the file after an error happens.
+ void reset_seen_error() {
+ seen_error_.store(false, std::memory_order_relaxed);
+ }
+ void set_seen_error() { seen_error_.store(true, std::memory_order_relaxed); }
+
+ IOStatus AssertFalseAndGetStatusForPrevError() {
+ // This should only happen if SyncWithoutFlush() was called.
+ assert(sync_without_flush_called_);
+ return IOStatus::IOError("Writer has previous error.");
+ }
+
+ private:
+ // Decide the Rate Limiter priority.
+ static Env::IOPriority DecideRateLimiterPriority(
+ Env::IOPriority writable_file_io_priority,
+ Env::IOPriority op_rate_limiter_priority);
+
+ // Used when os buffering is OFF and we are writing
+ // DMA such as in Direct I/O mode
+#ifndef ROCKSDB_LITE
+ IOStatus WriteDirect(Env::IOPriority op_rate_limiter_priority);
+ IOStatus WriteDirectWithChecksum(Env::IOPriority op_rate_limiter_priority);
+#endif // !ROCKSDB_LITE
+ // Normal write.
+ IOStatus WriteBuffered(const char* data, size_t size,
+ Env::IOPriority op_rate_limiter_priority);
+ IOStatus WriteBufferedWithChecksum(const char* data, size_t size,
+ Env::IOPriority op_rate_limiter_priority);
+ IOStatus RangeSync(uint64_t offset, uint64_t nbytes);
+ IOStatus SyncInternal(bool use_fsync);
+};
+} // namespace ROCKSDB_NAMESPACE