Blob DB: enable readahead for garbage collection (#3648)

Summary:
Enable readahead for blob DB garbage collection, which should improve GC performance a little bit.
Closes https://github.com/facebook/rocksdb/pull/3648

Differential Revision: D7383791

Pulled By: yiwu-arbug

fbshipit-source-id: 642b3327f7105eca85986d3fb2d8f960a3d83cf1
main
Yi Wu 6 years ago committed by Facebook Github Bot
parent 2729dd72ad
commit a71e467381
  1. 2
      utilities/blob_db/blob_db_impl.cc
  2. 3
      utilities/blob_db/blob_dump_tool.cc
  3. 12
      utilities/blob_db/blob_file.cc
  4. 2
      utilities/blob_db/blob_file.h
  5. 6
      utilities/blob_db/blob_log_reader.cc
  6. 5
      utilities/blob_db/blob_log_reader.h

@ -1434,7 +1434,7 @@ Status BlobDBImpl::GCFileAndUpdateLSM(const std::shared_ptr<BlobFile>& bfptr,
uint64_t now = EpochNow();
std::shared_ptr<Reader> reader =
bfptr->OpenSequentialReader(env_, db_options_, env_options_);
bfptr->OpenRandomAccessReader(env_, db_options_, env_options_);
if (!reader) {
ROCKS_LOG_ERROR(db_options_.info_log,
"File sequential reader could not be opened",

@ -19,6 +19,7 @@
#include "rocksdb/env.h"
#include "table/format.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
#include "util/string_util.h"
namespace rocksdb {
@ -31,6 +32,7 @@ Status BlobDumpTool::Run(const std::string& filename, DisplayType show_key,
DisplayType show_blob,
DisplayType show_uncompressed_blob,
bool show_summary) {
constexpr size_t kReadaheadSize = 2 * 1024 * 1024;
Status s;
Env* env = Env::Default();
s = env->FileExists(filename);
@ -47,6 +49,7 @@ Status BlobDumpTool::Run(const std::string& filename, DisplayType show_key,
if (!s.ok()) {
return s;
}
file = NewReadaheadRandomAccessFile(std::move(file), kReadaheadSize);
if (file_size == 0) {
return Status::Corruption("File is empty.");
}

@ -80,19 +80,21 @@ std::string BlobFile::PathName() const {
return BlobFileName(path_to_dir_, file_number_);
}
std::shared_ptr<Reader> BlobFile::OpenSequentialReader(
std::shared_ptr<Reader> BlobFile::OpenRandomAccessReader(
Env* env, const DBOptions& db_options,
const EnvOptions& env_options) const {
std::unique_ptr<SequentialFile> sfile;
constexpr size_t kReadaheadSize = 2 * 1024 * 1024;
std::unique_ptr<RandomAccessFile> sfile;
std::string path_name(PathName());
Status s = env->NewSequentialFile(path_name, &sfile, env_options);
Status s = env->NewRandomAccessFile(path_name, &sfile, env_options);
if (!s.ok()) {
// report something here.
return nullptr;
}
sfile = NewReadaheadRandomAccessFile(std::move(sfile), kReadaheadSize);
std::unique_ptr<SequentialFileReader> sfile_reader;
sfile_reader.reset(new SequentialFileReader(std::move(sfile), path_name));
std::unique_ptr<RandomAccessFileReader> sfile_reader;
sfile_reader.reset(new RandomAccessFileReader(std::move(sfile), path_name));
std::shared_ptr<Reader> log_reader = std::make_shared<Reader>(
std::move(sfile_reader), db_options.env, db_options.statistics.get());

@ -181,7 +181,7 @@ class BlobFile {
Status ReadMetadata(Env* env, const EnvOptions& env_options);
private:
std::shared_ptr<Reader> OpenSequentialReader(
std::shared_ptr<Reader> OpenRandomAccessReader(
Env* env, const DBOptions& db_options,
const EnvOptions& env_options) const;

@ -16,7 +16,7 @@
namespace rocksdb {
namespace blob_db {
Reader::Reader(unique_ptr<SequentialFileReader>&& file_reader, Env* env,
Reader::Reader(unique_ptr<RandomAccessFileReader>&& file_reader, Env* env,
Statistics* statistics)
: file_(std::move(file_reader)),
env_(env),
@ -27,7 +27,7 @@ Reader::Reader(unique_ptr<SequentialFileReader>&& file_reader, Env* env,
Status Reader::ReadSlice(uint64_t size, Slice* slice, std::string* buf) {
StopWatch read_sw(env_, statistics_, BLOB_DB_BLOB_FILE_READ_MICROS);
buf->reserve(size);
Status s = file_->Read(size, slice, &(*buf)[0]);
Status s = file_->Read(next_byte_, size, slice, &(*buf)[0]);
next_byte_ += size;
if (!s.ok()) {
return s;
@ -76,13 +76,11 @@ Status Reader::ReadRecord(BlobLogRecord* record, ReadLevel level,
switch (level) {
case kReadHeader:
file_->Skip(record->key_size + record->value_size);
next_byte_ += kb_size;
break;
case kReadHeaderKey:
s = ReadSlice(record->key_size, &record->key, &record->key_buf);
file_->Skip(record->value_size);
next_byte_ += record->value_size;
break;

@ -14,6 +14,7 @@
#include "rocksdb/slice.h"
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
#include "util/file_reader_writer.h"
#include "utilities/blob_db/blob_log_format.h"
namespace rocksdb {
@ -39,7 +40,7 @@ class Reader {
// Create a reader that will return log records from "*file".
// "*file" must remain live while this Reader is in use.
Reader(std::unique_ptr<SequentialFileReader>&& file_reader, Env* env,
Reader(std::unique_ptr<RandomAccessFileReader>&& file_reader, Env* env,
Statistics* statistics);
~Reader() = default;
@ -66,7 +67,7 @@ class Reader {
uint64_t GetNextByte() const { return next_byte_; }
private:
const std::unique_ptr<SequentialFileReader> file_;
const std::unique_ptr<RandomAccessFileReader> file_;
Env* env_;
Statistics* statistics_;

Loading…
Cancel
Save