Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env

Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.

Test Plan: Run all existing unit tests.

Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor

Reviewed By: igor

Subscribers: leveldb, dhruba

Differential Revision: https://reviews.facebook.net/D42321
main
sdong 9 years ago
parent 5ec829bc4f
commit 6e9fbeb27c
  1. 38
      db/builder.cc
  2. 4
      db/builder.h
  3. 22
      db/compaction_job.cc
  4. 5
      db/compaction_job_test.cc
  5. 84
      db/db_impl.cc
  6. 13
      db/db_test.cc
  7. 4
      db/fault_injection_test.cc
  8. 9
      db/filename.cc
  9. 4
      db/filename.h
  10. 5
      db/flush_job_test.cc
  11. 3
      db/log_reader.cc
  12. 8
      db/log_reader.h
  13. 52
      db/log_test.cc
  14. 6
      db/log_writer.cc
  15. 10
      db/log_writer.h
  16. 8
      db/plain_table_db_test.cc
  17. 15
      db/repair.cc
  18. 5
      db/table_cache.cc
  19. 42
      db/table_properties_collector_test.cc
  20. 19
      db/transaction_log_impl.cc
  21. 3
      db/transaction_log_impl.h
  22. 88
      db/version_set.cc
  23. 5
      db/wal_manager.cc
  24. 9
      db/wal_manager_test.cc
  25. 23
      include/rocksdb/env.h
  26. 8
      include/rocksdb/table.h
  27. 12
      port/port_posix.cc
  28. 3
      port/port_posix.h
  29. 39
      port/win/env_win.cc
  30. 1
      src.mk
  31. 7
      table/adaptive_table_factory.cc
  32. 13
      table/adaptive_table_factory.h
  33. 8
      table/block_based_table_builder.cc
  34. 2
      table/block_based_table_builder.h
  35. 4
      table/block_based_table_factory.cc
  36. 8
      table/block_based_table_factory.h
  37. 20
      table/block_based_table_reader.cc
  38. 5
      table/block_based_table_reader.h
  39. 3
      table/cuckoo_table_builder.cc
  40. 15
      table/cuckoo_table_builder.h
  41. 114
      table/cuckoo_table_builder_test.cc
  42. 9
      table/cuckoo_table_factory.cc
  43. 13
      table/cuckoo_table_factory.h
  44. 3
      table/cuckoo_table_reader.cc
  45. 14
      table/cuckoo_table_reader.h
  46. 66
      table/cuckoo_table_reader_test.cc
  47. 11
      table/format.cc
  48. 5
      table/format.h
  49. 12
      table/meta_blocks.cc
  50. 13
      table/meta_blocks.h
  51. 9
      table/mock_table.cc
  52. 13
      table/mock_table.h
  53. 10
      table/plain_table_builder.cc
  54. 11
      table/plain_table_builder.h
  55. 13
      table/plain_table_factory.cc
  56. 13
      table/plain_table_factory.h
  57. 6
      table/plain_table_key_coding.cc
  58. 2
      table/plain_table_key_coding.h
  59. 4
      table/plain_table_reader.cc
  60. 9
      table/plain_table_reader.h
  61. 15
      table/table_reader_bench.cc
  62. 48
      table/table_test.cc
  63. 3
      util/db_test_util.h
  64. 238
      util/env_posix.cc
  65. 7
      util/env_test.cc
  66. 225
      util/file_reader_writer.cc
  67. 109
      util/file_reader_writer.h
  68. 14
      util/file_util.cc
  69. 14
      util/ldb_cmd.cc
  70. 8
      util/sst_dump_test.cc
  71. 18
      util/sst_dump_tool.cc
  72. 7
      util/sst_dump_tool_imp.h
  73. 15
      util/sync_point.cc
  74. 28
      util/sync_point.h
  75. 16
      util/testutil.cc
  76. 8
      util/testutil.h
  77. 32
      utilities/backupable/backupable_db.cc
  78. 6
      utilities/backupable/backupable_db_test.cc

@ -21,6 +21,7 @@
#include "rocksdb/options.h"
#include "rocksdb/table.h"
#include "table/block_based_table_builder.h"
#include "util/file_reader_writer.h"
#include "util/iostats_context_imp.h"
#include "util/thread_status_util.h"
#include "util/stop_watch.h"
@ -34,7 +35,7 @@ TableBuilder* NewTableBuilder(
const InternalKeyComparator& internal_comparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
WritableFile* file, const CompressionType compression_type,
WritableFileWriter* file, const CompressionType compression_type,
const CompressionOptions& compression_opts, const bool skip_filters) {
return ioptions.table_factory->NewTableBuilder(
TableBuilderOptions(ioptions, internal_comparator,
@ -72,16 +73,22 @@ Status BuildTable(
std::string fname = TableFileName(ioptions.db_paths, meta->fd.GetNumber(),
meta->fd.GetPathId());
if (iter->Valid()) {
unique_ptr<WritableFile> file;
s = env->NewWritableFile(fname, &file, env_options);
if (!s.ok()) {
return s;
}
file->SetIOPriority(io_priority);
TableBuilder* builder;
unique_ptr<WritableFileWriter> file_writer;
{
unique_ptr<WritableFile> file;
s = env->NewWritableFile(fname, &file, env_options);
if (!s.ok()) {
return s;
}
file->SetIOPriority(io_priority);
TableBuilder* builder = NewTableBuilder(
ioptions, internal_comparator, int_tbl_prop_collector_factories,
file.get(), compression, compression_opts);
file_writer.reset(new WritableFileWriter(std::move(file), env_options));
builder = NewTableBuilder(
ioptions, internal_comparator, int_tbl_prop_collector_factories,
file_writer.get(), compression, compression_opts);
}
{
// the first key is the smallest key
@ -232,16 +239,11 @@ Status BuildTable(
// Finish and check for file errors
if (s.ok() && !ioptions.disable_data_sync) {
if (ioptions.use_fsync) {
StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS);
s = file->Fsync();
} else {
StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS);
s = file->Sync();
}
StopWatch sw(env, ioptions.statistics, TABLE_SYNC_MICROS);
file_writer->Sync(ioptions.use_fsync);
}
if (s.ok()) {
s = file->Close();
s = file_writer->Close();
}
if (s.ok()) {

@ -29,14 +29,14 @@ class Iterator;
class TableCache;
class VersionEdit;
class TableBuilder;
class WritableFile;
class WritableFileWriter;
TableBuilder* NewTableBuilder(
const ImmutableCFOptions& options,
const InternalKeyComparator& internal_comparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
WritableFile* file, const CompressionType compression_type,
WritableFileWriter* file, const CompressionType compression_type,
const CompressionOptions& compression_opts,
const bool skip_filters = false);

@ -44,6 +44,7 @@
#include "table/table_builder.h"
#include "table/two_level_iterator.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
#include "util/logging.h"
#include "util/log_buffer.h"
#include "util/mutexlock.h"
@ -71,7 +72,7 @@ struct CompactionJob::CompactionState {
std::vector<Output> outputs;
// State kept for output being generated
std::unique_ptr<WritableFile> outfile;
std::unique_ptr<WritableFileWriter> outfile;
std::unique_ptr<TableBuilder> builder;
uint64_t total_bytes;
@ -662,13 +663,8 @@ Status CompactionJob::FinishCompactionOutputFile(const Status& input_status) {
// Finish and check for file errors
if (s.ok() && !db_options_.disableDataSync) {
if (db_options_.use_fsync) {
StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
s = compact_->outfile->Fsync();
} else {
StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
s = compact_->outfile->Sync();
}
StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
s = compact_->outfile->Sync(db_options_.use_fsync);
}
if (s.ok()) {
s = compact_->outfile->Close();
@ -799,10 +795,10 @@ Status CompactionJob::OpenCompactionOutputFile() {
// no need to lock because VersionSet::next_file_number_ is atomic
uint64_t file_number = versions_->NewFileNumber();
// Make the output file
unique_ptr<WritableFile> writable_file;
std::string fname = TableFileName(db_options_.db_paths, file_number,
compact_->compaction->output_path_id());
Status s = env_->NewWritableFile(fname, &compact_->outfile, env_options_);
Status s = env_->NewWritableFile(fname, &writable_file, env_options_);
if (!s.ok()) {
Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
"[%s] [JOB %d] OpenCompactionOutputFiles for table #%" PRIu64
@ -820,9 +816,11 @@ Status CompactionJob::OpenCompactionOutputFile() {
out.smallest_seqno = out.largest_seqno = 0;
compact_->outputs.push_back(out);
compact_->outfile->SetIOPriority(Env::IO_LOW);
compact_->outfile->SetPreallocationBlockSize(
writable_file->SetIOPriority(Env::IO_LOW);
writable_file->SetPreallocationBlockSize(
static_cast<size_t>(compact_->compaction->OutputFilePreallocationSize()));
compact_->outfile.reset(
new WritableFileWriter(std::move(writable_file), env_options_));
ColumnFamilyData* cfd = compact_->compaction->column_family_data();
bool skip_filters = false;

@ -13,6 +13,7 @@
#include "rocksdb/cache.h"
#include "rocksdb/options.h"
#include "rocksdb/db.h"
#include "util/file_reader_writer.h"
#include "util/string_util.h"
#include "util/testharness.h"
#include "util/testutil.h"
@ -166,8 +167,10 @@ class CompactionJobTest : public testing::Test {
Status s = env_->NewWritableFile(
manifest, &file, env_->OptimizeForManifestWrite(env_options_));
ASSERT_OK(s);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), EnvOptions()));
{
log::Writer log(std::move(file));
log::Writer log(std::move(file_writer));
std::string record;
new_db.EncodeTo(&record);
s = log.AddRecord(record);

@ -72,6 +72,7 @@
#include "util/compression.h"
#include "util/crc32c.h"
#include "util/db_info_dumper.h"
#include "util/file_reader_writer.h"
#include "util/file_util.h"
#include "util/hash_skiplist_rep.h"
#include "util/hash_linklist_rep.h"
@ -384,18 +385,22 @@ Status DBImpl::NewDB() {
new_db.SetNextFile(2);
new_db.SetLastSequence(0);
Status s;
Log(InfoLogLevel::INFO_LEVEL,
db_options_.info_log, "Creating manifest 1 \n");
const std::string manifest = DescriptorFileName(dbname_, 1);
unique_ptr<WritableFile> file;
Status s = env_->NewWritableFile(
manifest, &file, env_->OptimizeForManifestWrite(env_options_));
if (!s.ok()) {
return s;
}
file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
{
log::Writer log(std::move(file));
unique_ptr<WritableFile> file;
EnvOptions env_options = env_->OptimizeForManifestWrite(env_options_);
s = env_->NewWritableFile(manifest, &file, env_options);
if (!s.ok()) {
return s;
}
file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options));
log::Writer log(std::move(file_writer));
std::string record;
new_db.EncodeTo(&record);
s = log.AddRecord(record);
@ -1013,17 +1018,21 @@ Status DBImpl::RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
versions_->MarkFileNumberUsedDuringRecovery(log_number);
// Open the log file
std::string fname = LogFileName(db_options_.wal_dir, log_number);
unique_ptr<SequentialFile> file;
status = env_->NewSequentialFile(fname, &file, env_options_);
if (!status.ok()) {
MaybeIgnoreError(&status);
unique_ptr<SequentialFileReader> file_reader;
{
unique_ptr<SequentialFile> file;
status = env_->NewSequentialFile(fname, &file, env_options_);
if (!status.ok()) {
return status;
} else {
// Fail with one log file, but that's ok.
// Try next one.
continue;
MaybeIgnoreError(&status);
if (!status.ok()) {
return status;
} else {
// Fail with one log file, but that's ok.
// Try next one.
continue;
}
}
file_reader.reset(new SequentialFileReader(std::move(file)));
}
// Create the log reader.
@ -1042,7 +1051,7 @@ Status DBImpl::RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
// paranoid_checks==false so that corruptions cause entire commits
// to be skipped instead of propagating bad information (like overly
// large sequence numbers).
log::Reader reader(std::move(file), &reporter, true /*checksum*/,
log::Reader reader(std::move(file_reader), &reporter, true /*checksum*/,
0 /*initial_offset*/);
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
"Recovering log #%" PRIu64 " mode %d skip-recovery %d", log_number,
@ -3490,11 +3499,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
if (status.ok() && write_options.sync) {
RecordTick(stats_, WAL_FILE_SYNCED);
StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
if (db_options_.use_fsync) {
status = log_->file()->Fsync();
} else {
status = log_->file()->Sync();
}
status = log_->file()->Sync(db_options_.use_fsync);
if (status.ok() && !log_dir_synced_) {
// We only sync WAL directory the first time WAL syncing is
// requested, so that in case users never turn on WAL sync,
@ -3624,15 +3629,19 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
Status s;
{
if (creating_new_log) {
EnvOptions opt_env_opt =
env_->OptimizeForLogWrite(env_options_, db_options_);
s = env_->NewWritableFile(
LogFileName(db_options_.wal_dir, new_log_number), &lfile,
env_->OptimizeForLogWrite(env_options_, db_options_));
opt_env_opt);
if (s.ok()) {
// Our final size should be less than write_buffer_size
// (compression, etc) but err on the side of caution.
lfile->SetPreallocationBlockSize(
1.1 * mutable_cf_options.write_buffer_size);
new_log = new log::Writer(std::move(lfile));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(lfile), opt_env_opt));
new_log = new log::Writer(std::move(file_writer));
log_dir_synced_ = false;
}
}
@ -4031,12 +4040,18 @@ Status DBImpl::CheckConsistency() {
Status DBImpl::GetDbIdentity(std::string& identity) const {
std::string idfilename = IdentityFileName(dbname_);
unique_ptr<SequentialFile> idfile;
const EnvOptions soptions;
Status s = env_->NewSequentialFile(idfilename, &idfile, soptions);
if (!s.ok()) {
return s;
unique_ptr<SequentialFileReader> id_file_reader;
Status s;
{
unique_ptr<SequentialFile> idfile;
s = env_->NewSequentialFile(idfilename, &idfile, soptions);
if (!s.ok()) {
return s;
}
id_file_reader.reset(new SequentialFileReader(std::move(idfile)));
}
uint64_t file_size;
s = env_->GetFileSize(idfilename, &file_size);
if (!s.ok()) {
@ -4044,7 +4059,7 @@ Status DBImpl::GetDbIdentity(std::string& identity) const {
}
char* buffer = reinterpret_cast<char*>(alloca(file_size));
Slice id;
s = idfile->Read(static_cast<size_t>(file_size), &id, buffer);
s = id_file_reader->Read(static_cast<size_t>(file_size), &id, buffer);
if (!s.ok()) {
return s;
}
@ -4176,14 +4191,17 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname,
uint64_t new_log_number = impl->versions_->NewFileNumber();
unique_ptr<WritableFile> lfile;
EnvOptions soptions(db_options);
EnvOptions opt_env_options =
impl->db_options_.env->OptimizeForLogWrite(soptions, impl->db_options_);
s = impl->db_options_.env->NewWritableFile(
LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
impl->db_options_.env->OptimizeForLogWrite(soptions,
impl->db_options_));
opt_env_options);
if (s.ok()) {
lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
impl->logfile_number_ = new_log_number;
impl->log_.reset(new log::Writer(std::move(lfile)));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(lfile), opt_env_options));
impl->log_.reset(new log::Writer(std::move(file_writer)));
// set column family handles
for (auto cf : column_families) {

@ -52,6 +52,7 @@
#include "table/mock_table.h"
#include "table/plain_table_factory.h"
#include "util/db_test_util.h"
#include "util/file_reader_writer.h"
#include "util/hash.h"
#include "util/hash_linklist_rep.h"
#include "utilities/merge_operators.h"
@ -6008,7 +6009,9 @@ class RecoveryTestHelper {
std::string fname = LogFileName(test->dbname_, current_log_number);
unique_ptr<WritableFile> file;
ASSERT_OK(db_options.env->NewWritableFile(fname, &file, env_options));
current_log_writer.reset(new log::Writer(std::move(file)));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options));
current_log_writer.reset(new log::Writer(std::move(file_writer)));
for (int i = 0; i < kKeysPerWALFile; i++) {
std::string key = "key" + ToString(count++);
@ -7231,8 +7234,7 @@ TEST_F(DBTest, RateLimitingTest) {
}
elapsed = env_->NowMicros() - start;
Close();
ASSERT_TRUE(options.rate_limiter->GetTotalBytesThrough() ==
env_->bytes_written_);
ASSERT_EQ(options.rate_limiter->GetTotalBytesThrough(), env_->bytes_written_);
double ratio = env_->bytes_written_ * 1000000 / elapsed / raw_rate;
fprintf(stderr, "write rate ratio = %.2lf, expected 0.7\n", ratio);
ASSERT_TRUE(ratio < 0.8);
@ -7251,11 +7253,10 @@ TEST_F(DBTest, RateLimitingTest) {
}
elapsed = env_->NowMicros() - start;
Close();
ASSERT_TRUE(options.rate_limiter->GetTotalBytesThrough() ==
env_->bytes_written_);
ASSERT_EQ(options.rate_limiter->GetTotalBytesThrough(), env_->bytes_written_);
ratio = env_->bytes_written_ * 1000000 / elapsed / raw_rate;
fprintf(stderr, "write rate ratio = %.2lf, expected 0.5\n", ratio);
ASSERT_TRUE(ratio < 0.6);
ASSERT_LT(ratio, 0.6);
}
namespace {

@ -130,7 +130,7 @@ struct FileState {
} // anonymous namespace
// A wrapper around WritableFile which informs another Env whenever this file
// A wrapper around WritableFileWriter* file
// is written to or sync'ed.
class TestWritableFile : public WritableFile {
public:
@ -197,7 +197,7 @@ class FaultInjectionTestEnv : public EnvWrapper {
Status s = target()->NewWritableFile(fname, result, soptions);
if (s.ok()) {
result->reset(new TestWritableFile(fname, std::move(*result), this));
// WritableFile doesn't append to files, so if the same file is opened
// WritableFileWriter* file is opened
// again then it will be truncated - so forget our saved state.
UntrackFile(fname);
MutexLock l(&mutex_);

@ -18,6 +18,7 @@
#include <vector>
#include "db/dbformat.h"
#include "rocksdb/env.h"
#include "util/file_reader_writer.h"
#include "util/logging.h"
#include "util/stop_watch.h"
@ -328,15 +329,13 @@ Status SetIdentityFile(Env* env, const std::string& dbname) {
return s;
}
Status SyncManifest(Env* env, const DBOptions* db_options, WritableFile* file) {
Status SyncManifest(Env* env, const DBOptions* db_options,
WritableFileWriter* file) {
if (db_options->disableDataSync) {
return Status::OK();
} else if (db_options->use_fsync) {
StopWatch sw(env, db_options->statistics.get(), MANIFEST_FILE_SYNC_MICROS);
return file->Fsync();
} else {
StopWatch sw(env, db_options->statistics.get(), MANIFEST_FILE_SYNC_MICROS);
return file->Sync();
return file->Sync(db_options->use_fsync);
}
}

@ -25,7 +25,7 @@ namespace rocksdb {
class Env;
class Directory;
class WritableFile;
class WritableFileWriter;
enum FileType {
kLogFile,
@ -140,6 +140,6 @@ extern Status SetIdentityFile(Env* env, const std::string& dbname);
// Sync manifest file `file`.
extern Status SyncManifest(Env* env, const DBOptions* db_options,
WritableFile* file);
WritableFileWriter* file);
} // namespace rocksdb

@ -11,6 +11,7 @@
#include "db/version_set.h"
#include "db/writebuffer.h"
#include "rocksdb/cache.h"
#include "util/file_reader_writer.h"
#include "util/string_util.h"
#include "util/testharness.h"
#include "util/testutil.h"
@ -56,8 +57,10 @@ class FlushJobTest : public testing::Test {
Status s = env_->NewWritableFile(
manifest, &file, env_->OptimizeForManifestWrite(env_options_));
ASSERT_OK(s);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), EnvOptions()));
{
log::Writer log(std::move(file));
log::Writer log(std::move(file_writer));
std::string record;
new_db.EncodeTo(&record);
s = log.AddRecord(record);

@ -13,6 +13,7 @@
#include "rocksdb/env.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
namespace log {
@ -20,7 +21,7 @@ namespace log {
Reader::Reporter::~Reporter() {
}
Reader::Reader(unique_ptr<SequentialFile>&& _file, Reporter* reporter,
Reader::Reader(unique_ptr<SequentialFileReader>&& _file, Reporter* reporter,
bool checksum, uint64_t initial_offset)
: file_(std::move(_file)),
reporter_(reporter),

@ -17,7 +17,7 @@
namespace rocksdb {
class SequentialFile;
class SequentialFileReader;
using std::unique_ptr;
namespace log {
@ -51,7 +51,7 @@ class Reader {
//
// The Reader will start reading at the first record located at physical
// position >= initial_offset within the file.
Reader(unique_ptr<SequentialFile>&& file, Reporter* reporter,
Reader(unique_ptr<SequentialFileReader>&& file, Reporter* reporter,
bool checksum, uint64_t initial_offset);
~Reader();
@ -81,10 +81,10 @@ class Reader {
// block that was partially read.
void UnmarkEOF();
SequentialFile* file() { return file_.get(); }
SequentialFileReader* file() { return file_.get(); }
private:
const unique_ptr<SequentialFile> file_;
const unique_ptr<SequentialFileReader> file_;
Reporter* const reporter_;
bool const checksum_;
char* const backing_store_;

@ -12,8 +12,10 @@
#include "rocksdb/env.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/file_reader_writer.h"
#include "util/random.h"
#include "util/testharness.h"
#include "util/testutil.h"
namespace rocksdb {
namespace log {
@ -163,26 +165,27 @@ class LogTest : public testing::Test {
};
std::string& dest_contents() {
auto dest = dynamic_cast<StringDest*>(writer_.file());
auto dest = dynamic_cast<StringDest*>(writer_.file()->writable_file());
assert(dest);
return dest->contents_;
}
const std::string& dest_contents() const {
auto dest = dynamic_cast<const StringDest*>(writer_.file());
auto dest =
dynamic_cast<const StringDest*>(writer_.file()->writable_file());
assert(dest);
return dest->contents_;
}
void reset_source_contents() {
auto src = dynamic_cast<StringSource*>(reader_.file());
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
assert(src);
src->contents_ = dest_contents();
}
Slice reader_contents_;
unique_ptr<StringDest> dest_holder_;
unique_ptr<StringSource> source_holder_;
unique_ptr<WritableFileWriter> dest_holder_;
unique_ptr<SequentialFileReader> source_holder_;
ReportCollector report_;
Writer writer_;
Reader reader_;
@ -192,13 +195,15 @@ class LogTest : public testing::Test {
static uint64_t initial_offset_last_record_offsets_[];
public:
LogTest() : reader_contents_(),
dest_holder_(new StringDest(reader_contents_)),
source_holder_(new StringSource(reader_contents_)),
writer_(std::move(dest_holder_)),
reader_(std::move(source_holder_), &report_, true/*checksum*/,
0/*initial_offset*/) {
}
LogTest()
: reader_contents_(),
dest_holder_(
test::GetWritableFileWriter(new StringDest(reader_contents_))),
source_holder_(
test::GetSequentialFileReader(new StringSource(reader_contents_))),
writer_(std::move(dest_holder_)),
reader_(std::move(source_holder_), &report_, true /*checksum*/,
0 /*initial_offset*/) {}
void Write(const std::string& msg) {
writer_.AddRecord(Slice(msg));
@ -227,7 +232,7 @@ class LogTest : public testing::Test {
}
void ShrinkSize(int bytes) {
auto dest = dynamic_cast<StringDest*>(writer_.file());
auto dest = dynamic_cast<StringDest*>(writer_.file()->writable_file());
assert(dest);
dest->Drop(bytes);
}
@ -240,7 +245,7 @@ class LogTest : public testing::Test {
}
void ForceError(size_t position = 0) {
auto src = dynamic_cast<StringSource*>(reader_.file());
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
src->force_error_ = true;
src->force_error_position_ = position;
}
@ -254,13 +259,13 @@ class LogTest : public testing::Test {
}
void ForceEOF(size_t position = 0) {
auto src = dynamic_cast<StringSource*>(reader_.file());
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
src->force_eof_ = true;
src->force_eof_position_ = position;
}
void UnmarkEOF() {
auto src = dynamic_cast<StringSource*>(reader_.file());
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
src->returned_partial_ = false;
reader_.UnmarkEOF();
}
@ -288,10 +293,11 @@ class LogTest : public testing::Test {
void CheckOffsetPastEndReturnsNoRecords(uint64_t offset_past_end) {
WriteInitialOffsetLog();
unique_ptr<StringSource> source(new StringSource(reader_contents_));
unique_ptr<SequentialFileReader> file_reader(
test::GetSequentialFileReader(new StringSource(reader_contents_)));
unique_ptr<Reader> offset_reader(
new Reader(std::move(source), &report_, true/*checksum*/,
WrittenBytes() + offset_past_end));
new Reader(std::move(file_reader), &report_, true /*checksum*/,
WrittenBytes() + offset_past_end));
Slice record;
std::string scratch;
ASSERT_TRUE(!offset_reader->ReadRecord(&record, &scratch));
@ -300,10 +306,10 @@ class LogTest : public testing::Test {
void CheckInitialOffsetRecord(uint64_t initial_offset,
int expected_record_offset) {
WriteInitialOffsetLog();
unique_ptr<StringSource> source(new StringSource(reader_contents_));
unique_ptr<Reader> offset_reader(
new Reader(std::move(source), &report_, true/*checksum*/,
initial_offset));
unique_ptr<SequentialFileReader> file_reader(
test::GetSequentialFileReader(new StringSource(reader_contents_)));
unique_ptr<Reader> offset_reader(new Reader(
std::move(file_reader), &report_, true /*checksum*/, initial_offset));
Slice record;
std::string scratch;
ASSERT_TRUE(offset_reader->ReadRecord(&record, &scratch));

@ -13,13 +13,13 @@
#include "rocksdb/env.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
namespace log {
Writer::Writer(unique_ptr<WritableFile>&& dest)
: dest_(std::move(dest)),
block_offset_(0) {
Writer::Writer(unique_ptr<WritableFileWriter>&& dest)
: dest_(std::move(dest)), block_offset_(0) {
for (int i = 0; i <= kMaxRecordType; i++) {
char t = static_cast<char>(i);
type_crc_[i] = crc32c::Value(&t, 1);

@ -16,7 +16,7 @@
namespace rocksdb {
class WritableFile;
class WritableFileWriter;
using std::unique_ptr;
@ -61,16 +61,16 @@ class Writer {
// Create a writer that will append data to "*dest".
// "*dest" must be initially empty.
// "*dest" must remain live while this Writer is in use.
explicit Writer(unique_ptr<WritableFile>&& dest);
explicit Writer(unique_ptr<WritableFileWriter>&& dest);
~Writer();
Status AddRecord(const Slice& slice);
WritableFile* file() { return dest_.get(); }
const WritableFile* file() const { return dest_.get(); }
WritableFileWriter* file() { return dest_.get(); }
const WritableFileWriter* file() const { return dest_.get(); }
private:
unique_ptr<WritableFile> dest_;
unique_ptr<WritableFileWriter> dest_;
int block_offset_; // Current offset in block
// crc32c values for all supported record types. These are

@ -198,10 +198,9 @@ class TestPlainTableReader : public PlainTableReader {
int bloom_bits_per_key, double hash_table_ratio,
size_t index_sparseness,
const TableProperties* table_properties,
unique_ptr<RandomAccessFile>&& file,
unique_ptr<RandomAccessFileReader>&& file,
const ImmutableCFOptions& ioptions,
bool* expect_bloom_not_match,
bool store_index_in_file)
bool* expect_bloom_not_match, bool store_index_in_file)
: PlainTableReader(ioptions, std::move(file), env_options, icomparator,
encoding_type, file_size, table_properties),
expect_bloom_not_match_(expect_bloom_not_match) {
@ -257,7 +256,8 @@ class TestPlainTableFactory : public PlainTableFactory {
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table) const override {
TableProperties* props = nullptr;
auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber,

@ -81,6 +81,7 @@
#include "rocksdb/env.h"
#include "rocksdb/options.h"
#include "rocksdb/immutable_options.h"
#include "util/file_reader_writer.h"
#include "util/scoped_arena_iterator.h"
namespace rocksdb {
@ -236,6 +237,8 @@ class Repairer {
if (!status.ok()) {
return status;
}
unique_ptr<SequentialFileReader> lfile_reader(
new SequentialFileReader(std::move(lfile)));
// Create the log reader.
LogReporter reporter;
@ -246,8 +249,8 @@ class Repairer {
// corruptions cause entire commits to be skipped instead of
// propagating bad information (like overly large sequence
// numbers).
log::Reader reader(std::move(lfile), &reporter, true /*enable checksum*/,
0/*initial_offset*/);
log::Reader reader(std::move(lfile_reader), &reporter,
true /*enable checksum*/, 0 /*initial_offset*/);
// Read all the records and add to a memtable
std::string scratch;
@ -378,8 +381,8 @@ class Repairer {
Status WriteDescriptor() {
std::string tmp = TempFileName(dbname_, 1);
unique_ptr<WritableFile> file;
Status status = env_->NewWritableFile(
tmp, &file, env_->OptimizeForManifestWrite(env_options_));
EnvOptions env_options = env_->OptimizeForManifestWrite(env_options_);
Status status = env_->NewWritableFile(tmp, &file, env_options);
if (!status.ok()) {
return status;
}
@ -407,7 +410,9 @@ class Repairer {
//fprintf(stderr, "NewDescriptor:\n%s\n", edit_.DebugString().c_str());
{
log::Writer log(std::move(file));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options));
log::Writer log(std::move(file_writer));
std::string record;
edit_->EncodeTo(&record);
status = log.AddRecord(record);

@ -18,6 +18,7 @@
#include "table/table_reader.h"
#include "table/get_context.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
#include "util/perf_context_imp.h"
#include "util/stop_watch.h"
@ -99,8 +100,10 @@ Status TableCache::FindTable(const EnvOptions& env_options,
file->Hint(RandomAccessFile::RANDOM);
}
StopWatch sw(ioptions_.env, ioptions_.statistics, TABLE_OPEN_IO_MICROS);
std::unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(file)));
s = ioptions_.table_factory->NewTableReader(
ioptions_, env_options, internal_comparator, std::move(file),
ioptions_, env_options, internal_comparator, std::move(file_reader),
fd.GetFileSize(), &table_reader);
}

@ -18,6 +18,7 @@
#include "table/plain_table_factory.h"
#include "table/table_builder.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
#include "util/testharness.h"
#include "util/testutil.h"
@ -33,7 +34,7 @@ class TablePropertiesTest : public testing::Test,
// TODO(kailiu) the following classes should be moved to some more general
// places, so that other tests can also make use of them.
// `FakeWritableFile` and `FakeRandomeAccessFile` bypass the real file system
// `FakeWritableFileWriter* file system
// and therefore enable us to quickly setup the tests.
class FakeWritableFile : public WritableFile {
public:
@ -96,9 +97,11 @@ void MakeBuilder(const Options& options, const ImmutableCFOptions& ioptions,
const InternalKeyComparator& internal_comparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
std::unique_ptr<FakeWritableFile>* writable,
std::unique_ptr<WritableFileWriter>* writable,
std::unique_ptr<TableBuilder>* builder) {
writable->reset(new FakeWritableFile);
unique_ptr<WritableFile> wf(new FakeWritableFile);
writable->reset(new WritableFileWriter(std::move(wf), EnvOptions()));
builder->reset(NewTableBuilder(
ioptions, internal_comparator, int_tbl_prop_collector_factories,
writable->get(), options.compression, options.compression_opts));
@ -289,7 +292,7 @@ void TestCustomizedTablePropertiesCollector(
// -- Step 1: build table
std::unique_ptr<TableBuilder> builder;
std::unique_ptr<FakeWritableFile> writable;
std::unique_ptr<WritableFileWriter> writer;
const ImmutableCFOptions ioptions(options);
std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
int_tbl_prop_collector_factories;
@ -300,7 +303,7 @@ void TestCustomizedTablePropertiesCollector(
GetIntTblPropCollectorFactory(options, &int_tbl_prop_collector_factories);
}
MakeBuilder(options, ioptions, internal_comparator,
&int_tbl_prop_collector_factories, &writable, &builder);
&int_tbl_prop_collector_factories, &writer, &builder);
SequenceNumber seqNum = 0U;
for (const auto& kv : kvs) {
@ -310,18 +313,17 @@ void TestCustomizedTablePropertiesCollector(
builder->Add(ikey.Encode(), kv.second);
}
ASSERT_OK(builder->Finish());
writer->Flush();
// -- Step 2: Read properties
FakeRandomeAccessFile readable(writable->contents());
FakeWritableFile* fwf =
static_cast<FakeWritableFile*>(writer->writable_file());
std::unique_ptr<RandomAccessFileReader> fake_file_reader(
test::GetRandomAccessFileReader(
new FakeRandomeAccessFile(fwf->contents())));
TableProperties* props;
Status s = ReadTableProperties(
&readable,
writable->contents().size(),
magic_number,
Env::Default(),
nullptr,
&props
);
Status s = ReadTableProperties(fake_file_reader.get(), fwf->contents().size(),
magic_number, Env::Default(), nullptr, &props);
std::unique_ptr<TableProperties> props_guard(props);
ASSERT_OK(s);
@ -414,7 +416,7 @@ void TestInternalKeyPropertiesCollector(
};
std::unique_ptr<TableBuilder> builder;
std::unique_ptr<FakeWritableFile> writable;
std::unique_ptr<WritableFileWriter> writable;
Options options;
test::PlainInternalKeyComparator pikc(options.comparator);
@ -449,12 +451,16 @@ void TestInternalKeyPropertiesCollector(
}
ASSERT_OK(builder->Finish());
writable->Flush();
FakeRandomeAccessFile readable(writable->contents());
FakeWritableFile* fwf =
static_cast<FakeWritableFile*>(writable->writable_file());
unique_ptr<RandomAccessFileReader> reader(test::GetRandomAccessFileReader(
new FakeRandomeAccessFile(fwf->contents())));
TableProperties* props;
Status s =
ReadTableProperties(&readable, writable->contents().size(),
magic_number, Env::Default(), nullptr, &props);
ReadTableProperties(reader.get(), fwf->contents().size(), magic_number,
Env::Default(), nullptr, &props);
ASSERT_OK(s);
std::unique_ptr<TableProperties> props_guard(props);

@ -11,6 +11,7 @@
#include <inttypes.h>
#include "db/transaction_log_impl.h"
#include "db/write_batch_internal.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
@ -40,23 +41,27 @@ TransactionLogIteratorImpl::TransactionLogIteratorImpl(
}
Status TransactionLogIteratorImpl::OpenLogFile(
const LogFile* logFile,
unique_ptr<SequentialFile>* file) {
const LogFile* logFile, unique_ptr<SequentialFileReader>* file_reader) {
Env* env = options_->env;
unique_ptr<SequentialFile> file;
Status s;
if (logFile->Type() == kArchivedLogFile) {
std::string fname = ArchivedLogFileName(dir_, logFile->LogNumber());
return env->NewSequentialFile(fname, file, soptions_);
s = env->NewSequentialFile(fname, &file, soptions_);
} else {
std::string fname = LogFileName(dir_, logFile->LogNumber());
Status s = env->NewSequentialFile(fname, file, soptions_);
s = env->NewSequentialFile(fname, &file, soptions_);
if (!s.ok()) {
// If cannot open file in DB directory.
// Try the archive dir, as it could have moved in the meanwhile.
fname = ArchivedLogFileName(dir_, logFile->LogNumber());
s = env->NewSequentialFile(fname, file, soptions_);
s = env->NewSequentialFile(fname, &file, soptions_);
}
return s;
}
if (s.ok()) {
file_reader->reset(new SequentialFileReader(std::move(file)));
}
return s;
}
BatchResult TransactionLogIteratorImpl::GetBatch() {
@ -251,7 +256,7 @@ void TransactionLogIteratorImpl::UpdateCurrentWriteBatch(const Slice& record) {
}
Status TransactionLogIteratorImpl::OpenLogReader(const LogFile* logFile) {
unique_ptr<SequentialFile> file;
unique_ptr<SequentialFileReader> file;
Status s = OpenLogFile(logFile, &file);
if (!s.ok()) {
return s;

@ -84,7 +84,8 @@ class TransactionLogIteratorImpl : public TransactionLogIterator {
size_t currentFileIndex_;
std::unique_ptr<WriteBatch> currentBatch_;
unique_ptr<log::Reader> currentLogReader_;
Status OpenLogFile(const LogFile* logFile, unique_ptr<SequentialFile>* file);
Status OpenLogFile(const LogFile* logFile,
unique_ptr<SequentialFileReader>* file);
struct LogReporter : public log::Reader::Reporter {
Env* env;

@ -42,6 +42,7 @@
#include "table/meta_blocks.h"
#include "table/get_context.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
#include "util/logging.h"
#include "util/stop_watch.h"
#include "util/sync_point.h"
@ -567,10 +568,12 @@ Status Version::GetTableProperties(std::shared_ptr<const TableProperties>* tp,
TableProperties* raw_table_properties;
// By setting the magic number to kInvalidTableMagicNumber, we can by
// pass the magic number check in the footer.
std::unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(file)));
s = ReadTableProperties(
file.get(), file_meta->fd.GetFileSize(),
Footer::kInvalidTableMagicNumber /* table's magic number */,
vset_->env_, ioptions->info_log, &raw_table_properties);
file_reader.get(), file_meta->fd.GetFileSize(),
Footer::kInvalidTableMagicNumber /* table's magic number */, vset_->env_,
ioptions->info_log, &raw_table_properties);
if (!s.ok()) {
return s;
}
@ -1912,13 +1915,17 @@ Status VersionSet::LogAndApply(ColumnFamilyData* column_family_data,
Log(InfoLogLevel::INFO_LEVEL, db_options_->info_log,
"Creating manifest %" PRIu64 "\n", pending_manifest_file_number_);
unique_ptr<WritableFile> descriptor_file;
EnvOptions opt_env_opts = env_->OptimizeForManifestWrite(env_options_);
s = env_->NewWritableFile(
DescriptorFileName(dbname_, pending_manifest_file_number_),
&descriptor_file, env_->OptimizeForManifestWrite(env_options_));
&descriptor_file, opt_env_opts);
if (s.ok()) {
descriptor_file->SetPreallocationBlockSize(
db_options_->manifest_preallocation_size);
descriptor_log_.reset(new log::Writer(std::move(descriptor_file)));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(descriptor_file), opt_env_opts));
descriptor_log_.reset(new log::Writer(std::move(file_writer)));
s = WriteSnapshot(descriptor_log_.get());
}
}
@ -2132,11 +2139,16 @@ Status VersionSet::Recover(
manifest_filename.c_str());
manifest_filename = dbname_ + "/" + manifest_filename;
unique_ptr<SequentialFile> manifest_file;
s = env_->NewSequentialFile(manifest_filename, &manifest_file,
env_options_);
if (!s.ok()) {
return s;
unique_ptr<SequentialFileReader> manifest_file_reader;
{
unique_ptr<SequentialFile> manifest_file;
s = env_->NewSequentialFile(manifest_filename, &manifest_file,
env_options_);
if (!s.ok()) {
return s;
}
manifest_file_reader.reset(
new SequentialFileReader(std::move(manifest_file)));
}
uint64_t current_manifest_file_size;
s = env_->GetFileSize(manifest_filename, &current_manifest_file_size);
@ -2170,8 +2182,8 @@ Status VersionSet::Recover(
{
VersionSet::LogReporter reporter;
reporter.status = &s;
log::Reader reader(std::move(manifest_file), &reporter, true /*checksum*/,
0 /*initial_offset*/);
log::Reader reader(std::move(manifest_file_reader), &reporter,
true /*checksum*/, 0 /*initial_offset*/);
Slice record;
std::string scratch;
while (reader.ReadRecord(&record, &scratch) && s.ok()) {
@ -2405,18 +2417,23 @@ Status VersionSet::ListColumnFamilies(std::vector<std::string>* column_families,
current.resize(current.size() - 1);
std::string dscname = dbname + "/" + current;
unique_ptr<SequentialFileReader> file_reader;
{
unique_ptr<SequentialFile> file;
s = env->NewSequentialFile(dscname, &file, soptions);
if (!s.ok()) {
return s;
}
file_reader.reset(new SequentialFileReader(std::move(file)));
}
std::map<uint32_t, std::string> column_family_names;
// default column family is always implicitly there
column_family_names.insert({0, kDefaultColumnFamilyName});
VersionSet::LogReporter reporter;
reporter.status = &s;
log::Reader reader(std::move(file), &reporter, true /*checksum*/,
log::Reader reader(std::move(file_reader), &reporter, true /*checksum*/,
0 /*initial_offset*/);
Slice record;
std::string scratch;
@ -2542,10 +2559,15 @@ Status VersionSet::ReduceNumberOfLevels(const std::string& dbname,
Status VersionSet::DumpManifest(Options& options, std::string& dscname,
bool verbose, bool hex, bool json) {
// Open the specified manifest file.
unique_ptr<SequentialFile> file;
Status s = options.env->NewSequentialFile(dscname, &file, env_options_);
if (!s.ok()) {
return s;
unique_ptr<SequentialFileReader> file_reader;
Status s;
{
unique_ptr<SequentialFile> file;
s = options.env->NewSequentialFile(dscname, &file, env_options_);
if (!s.ok()) {
return s;
}
file_reader.reset(new SequentialFileReader(std::move(file)));
}
bool have_prev_log_number = false;
@ -2569,8 +2591,8 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname,
{
VersionSet::LogReporter reporter;
reporter.status = &s;
log::Reader reader(std::move(file), &reporter, true/*checksum*/,
0/*initial_offset*/);
log::Reader reader(std::move(file_reader), &reporter, true /*checksum*/,
0 /*initial_offset*/);
Slice record;
std::string scratch;
while (reader.ReadRecord(&record, &scratch) && s.ok()) {
@ -2664,7 +2686,7 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname,
}
}
}
file.reset();
file_reader.reset();
if (s.ok()) {
if (!have_next_file) {
@ -2806,17 +2828,23 @@ bool VersionSet::ManifestContains(uint64_t manifest_file_num,
std::string fname = DescriptorFileName(dbname_, manifest_file_num);
Log(InfoLogLevel::INFO_LEVEL, db_options_->info_log,
"ManifestContains: checking %s\n", fname.c_str());
unique_ptr<SequentialFile> file;
Status s = env_->NewSequentialFile(fname, &file, env_options_);
if (!s.ok()) {
Log(InfoLogLevel::INFO_LEVEL, db_options_->info_log,
"ManifestContains: %s\n", s.ToString().c_str());
Log(InfoLogLevel::INFO_LEVEL, db_options_->info_log,
"ManifestContains: is unable to reopen the manifest file %s",
fname.c_str());
return false;
unique_ptr<SequentialFileReader> file_reader;
Status s;
{
unique_ptr<SequentialFile> file;
s = env_->NewSequentialFile(fname, &file, env_options_);
if (!s.ok()) {
Log(InfoLogLevel::INFO_LEVEL, db_options_->info_log,
"ManifestContains: %s\n", s.ToString().c_str());
Log(InfoLogLevel::INFO_LEVEL, db_options_->info_log,
"ManifestContains: is unable to reopen the manifest file %s",
fname.c_str());
return false;
}
file_reader.reset(new SequentialFileReader(std::move(file)));
}
log::Reader reader(std::move(file), nullptr, true/*checksum*/, 0);
log::Reader reader(std::move(file_reader), nullptr, true /*checksum*/, 0);
Slice r;
std::string scratch;
bool result = false;

@ -28,6 +28,7 @@
#include "rocksdb/options.h"
#include "rocksdb/write_batch.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
#include "util/logging.h"
#include "util/mutexlock.h"
#include "util/sync_point.h"
@ -430,6 +431,8 @@ Status WalManager::ReadFirstLine(const std::string& fname,
std::unique_ptr<SequentialFile> file;
Status status = env_->NewSequentialFile(fname, &file, env_options_);
unique_ptr<SequentialFileReader> file_reader(
new SequentialFileReader(std::move(file)));
if (!status.ok()) {
return status;
@ -441,7 +444,7 @@ Status WalManager::ReadFirstLine(const std::string& fname,
reporter.fname = fname.c_str();
reporter.status = &status;
reporter.ignore_error = !db_options_.paranoid_checks;
log::Reader reader(std::move(file), &reporter, true /*checksum*/,
log::Reader reader(std::move(file_reader), &reporter, true /*checksum*/,
0 /*initial_offset*/);
std::string scratch;
Slice record;

@ -14,6 +14,7 @@
#include "db/column_family.h"
#include "db/version_set.h"
#include "db/writebuffer.h"
#include "util/file_reader_writer.h"
#include "util/mock_env.h"
#include "util/string_util.h"
#include "util/testharness.h"
@ -72,7 +73,9 @@ class WalManagerTest : public testing::Test {
std::string fname = ArchivedLogFileName(dbname_, current_log_number_);
unique_ptr<WritableFile> file;
ASSERT_OK(env_->NewWritableFile(fname, &file, env_options_));
current_log_writer_.reset(new log::Writer(std::move(file)));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options_));
current_log_writer_.reset(new log::Writer(std::move(file_writer)));
}
void CreateArchiveLogs(int num_logs, int entries_per_log) {
@ -120,7 +123,9 @@ TEST_F(WalManagerTest, ReadFirstRecordCache) {
ASSERT_OK(wal_manager_->TEST_ReadFirstRecord(kAliveLogFile, 1, &s));
ASSERT_EQ(s, 0U);
log::Writer writer(std::move(file));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), EnvOptions()));
log::Writer writer(std::move(file_writer));
WriteBatch batch;
batch.Put("foo", "bar");
WriteBatchInternal::SetSequence(&batch, 10);

@ -465,6 +465,8 @@ class WritableFile {
io_priority_ = pri;
}
virtual Env::IOPriority GetIOPriority() { return io_priority_; }
/*
* Get the size of valid data in the file.
*/
@ -501,7 +503,14 @@ class WritableFile {
return Status::NotSupported("InvalidateCache not supported.");
}
protected:
// Sync a file range with disk.
// offset is the starting byte of the file range to be synchronized.
// nbytes specifies the length of the range to be synchronized.
// This asks the OS to initiate flushing the cached data to disk,
// without waiting for completion.
// Default implementation does nothing.
virtual Status RangeSync(off_t offset, off_t nbytes) { return Status::OK(); }
// PrepareWrite performs any necessary preparation for a write
// before the write actually occurs. This allows for pre-allocation
// of space on devices where it can result in less file
@ -526,6 +535,7 @@ class WritableFile {
}
}
protected:
/*
* Pre-allocate space for a file.
*/
@ -533,16 +543,6 @@ class WritableFile {
return Status::OK();
}
// Sync a file range with disk.
// offset is the starting byte of the file range to be synchronized.
// nbytes specifies the length of the range to be synchronized.
// This asks the OS to initiate flushing the cached data to disk,
// without waiting for completion.
// Default implementation does nothing.
virtual Status RangeSync(off_t offset, off_t nbytes) {
return Status::OK();
}
size_t preallocation_block_size() { return preallocation_block_size_; }
private:
@ -893,6 +893,7 @@ class WritableFileWrapper : public WritableFile {
void SetIOPriority(Env::IOPriority pri) override {
target_->SetIOPriority(pri);
}
Env::IOPriority GetIOPriority() override { return target_->GetIOPriority(); }
uint64_t GetFileSize() override { return target_->GetFileSize(); }
void GetPreallocationStatus(size_t* block_size,
size_t* last_allocated_block) override {

@ -34,7 +34,7 @@ class RandomAccessFile;
struct TableBuilderOptions;
class TableBuilder;
class TableReader;
class WritableFile;
class WritableFileWriter;
struct EnvOptions;
struct Options;
@ -315,6 +315,8 @@ extern TableFactory* NewCuckooTableFactory(
#endif // ROCKSDB_LITE
class RandomAccessFileReader;
// A base class for table factories.
class TableFactory {
public:
@ -348,7 +350,7 @@ class TableFactory {
virtual Status NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const = 0;
// Return a table builder to write to a file for this table type.
@ -372,7 +374,7 @@ class TableFactory {
// to use in this table.
virtual TableBuilder* NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const = 0;
WritableFileWriter* file) const = 0;
// Sanitizes the specified DB Options and ColumnFamilyOptions.
//

@ -9,11 +9,13 @@
#include "port/port_posix.h"
#include <stdio.h>
#include <assert.h>
#include <errno.h>
#include <sys/time.h>
#include <signal.h>
#include <stdio.h>
#include <string.h>
#include <sys/time.h>
#include <unistd.h>
#include <cstdlib>
#include "util/logging.h"
@ -133,5 +135,11 @@ void InitOnce(OnceType* once, void (*initializer)()) {
PthreadCall("once", pthread_once(once, initializer));
}
void Crash(const std::string& srcfile, int srcline) {
fprintf(stdout, "Crashing at %s:%d\n", srcfile.c_str(), srcline);
fflush(stdout);
kill(getpid(), SIGTERM);
}
} // namespace port
} // namespace rocksdb

@ -53,7 +53,7 @@
#if defined(OS_MACOSX) || defined(OS_SOLARIS) || defined(OS_FREEBSD) ||\
defined(OS_NETBSD) || defined(OS_OPENBSD) || defined(OS_DRAGONFLYBSD) ||\
defined(OS_ANDROID) || defined(OS_CYGWIN)
defined(OS_ANDROID) || defined(CYGWIN)
// Use fread/fwrite/fflush on platforms without _unlocked variants
#define fread_unlocked fread
#define fwrite_unlocked fwrite
@ -150,6 +150,7 @@ extern void InitOnce(OnceType* once, void (*initializer)());
#define PREFETCH(addr, rw, locality) __builtin_prefetch(addr, rw, locality)
extern void Crash(const std::string& srcfile, int srcline);
} // namespace port
} // namespace rocksdb

@ -29,6 +29,7 @@
#include "util/random.h"
#include "util/iostats_context_imp.h"
#include "util/rate_limiter.h"
#include "util/sync_point.h"
#include "util/thread_status_updater.h"
#include "util/thread_status_util.h"
@ -36,10 +37,6 @@
#include <Rpc.h> // For UUID generation
#include <Windows.h>
// This is only set from db_stress.cc and for testing only.
// If non-zero, kill at various points in source code with probability 1/this
int rocksdb_kill_odds = 0;
namespace rocksdb {
std::string GetWindowsErrSz(DWORD err) {
@ -90,40 +87,6 @@ inline void PrintThreadInfo(size_t thread_id, size_t terminatingId) {
// returns the ID of the current process
inline int current_process_id() { return _getpid(); }
#ifdef NDEBUG
// empty in release build
#define TEST_KILL_RANDOM(rocksdb_kill_odds)
#else
// Kill the process with probablity 1/odds for testing.
void TestKillRandom(int odds, const std::string& srcfile, int srcline) {
time_t curtime = time(nullptr);
Random r((uint32_t)curtime);
assert(odds > 0);
bool crash = r.OneIn(odds);
if (crash) {
fprintf(stdout, "Crashing at %s:%d\n", srcfile.c_str(), srcline);
fflush(stdout);
std::string* p_str = nullptr;
p_str->c_str();
}
}
// To avoid crashing always at some frequently executed codepaths (during
// kill random test), use this factor to reduce odds
#define REDUCE_ODDS 2
#define REDUCE_ODDS2 4
#define TEST_KILL_RANDOM(rocksdb_kill_odds) \
{ \
if (rocksdb_kill_odds > 0) { \
TestKillRandom(rocksdb_kill_odds, __FILE__, __LINE__); \
} \
}
#endif
// RAII helpers for HANDLEs
const auto CloseHandleFunc = [](HANDLE h) { ::CloseHandle(h); };
typedef std::unique_ptr<void, decltype(CloseHandleFunc)> UniqueCloseHandlePtr;

@ -88,6 +88,7 @@ LIB_SOURCES = \
util/env_hdfs.cc \
util/env_posix.cc \
util/file_util.cc \
util/file_reader_writer.cc \
util/filter_policy.cc \
util/hash.cc \
util/hash_cuckoo_rep.cc \

@ -41,8 +41,9 @@ extern const uint64_t kCuckooTableMagicNumber;
Status AdaptiveTableFactory::NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& icomp, unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, unique_ptr<TableReader>* table) const {
const InternalKeyComparator& icomp,
unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const {
Footer footer;
auto s = ReadFooterFromFile(file.get(), file_size, &footer);
if (!s.ok()) {
@ -66,7 +67,7 @@ Status AdaptiveTableFactory::NewTableReader(
TableBuilder* AdaptiveTableFactory::NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const {
WritableFileWriter* file) const {
return table_factory_to_write_->NewTableBuilder(table_builder_options, file);
}

@ -33,15 +33,16 @@ class AdaptiveTableFactory : public TableFactory {
const char* Name() const override { return "AdaptiveTableFactory"; }
Status NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const override;
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table) const override;
TableBuilder* NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const override;
WritableFileWriter* file) const override;
// Sanitizes the specified DB Options.
Status SanitizeOptions(const DBOptions& db_opts,

@ -437,7 +437,7 @@ struct BlockBasedTableBuilder::Rep {
const ImmutableCFOptions ioptions;
const BlockBasedTableOptions table_options;
const InternalKeyComparator& internal_comparator;
WritableFile* file;
WritableFileWriter* file;
uint64_t offset = 0;
Status status;
BlockBuilder data_block;
@ -467,7 +467,7 @@ struct BlockBasedTableBuilder::Rep {
const InternalKeyComparator& icomparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
WritableFile* f, const CompressionType _compression_type,
WritableFileWriter* f, const CompressionType _compression_type,
const CompressionOptions& _compression_opts, const bool skip_filters)
: ioptions(_ioptions),
table_options(table_opt),
@ -502,7 +502,7 @@ BlockBasedTableBuilder::BlockBasedTableBuilder(
const InternalKeyComparator& internal_comparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
WritableFile* file, const CompressionType compression_type,
WritableFileWriter* file, const CompressionType compression_type,
const CompressionOptions& compression_opts, const bool skip_filters) {
BlockBasedTableOptions sanitized_table_options(table_options);
if (sanitized_table_options.format_version == 0 &&
@ -524,7 +524,7 @@ BlockBasedTableBuilder::BlockBasedTableBuilder(
}
if (table_options.block_cache_compressed.get() != nullptr) {
BlockBasedTable::GenerateCachePrefix(
table_options.block_cache_compressed.get(), file,
table_options.block_cache_compressed.get(), file->writable_file(),
&rep_->compressed_cache_key_prefix[0],
&rep_->compressed_cache_key_prefix_size);
}

@ -40,7 +40,7 @@ class BlockBasedTableBuilder : public TableBuilder {
const InternalKeyComparator& internal_comparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
WritableFile* file, const CompressionType compression_type,
WritableFileWriter* file, const CompressionType compression_type,
const CompressionOptions& compression_opts, const bool skip_filters);
// REQUIRES: Either Finish() or Abandon() has been called.

@ -44,7 +44,7 @@ BlockBasedTableFactory::BlockBasedTableFactory(
Status BlockBasedTableFactory::NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader, const bool prefetch_enabled) const {
return BlockBasedTable::Open(ioptions, soptions, table_options_,
internal_comparator, std::move(file), file_size,
@ -53,7 +53,7 @@ Status BlockBasedTableFactory::NewTableReader(
TableBuilder* BlockBasedTableFactory::NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const {
WritableFileWriter* file) const {
auto table_builder = new BlockBasedTableBuilder(
table_builder_options.ioptions, table_options_,
table_builder_options.internal_comparator,

@ -36,7 +36,8 @@ class BlockBasedTableFactory : public TableFactory {
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table_reader) const override {
return NewTableReader(ioptions, soptions, internal_comparator,
std::move(file), file_size, table_reader,
@ -48,13 +49,14 @@ class BlockBasedTableFactory : public TableFactory {
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table_reader,
bool prefetch_index_and_filter) const;
TableBuilder* NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const override;
WritableFileWriter* file) const override;
// Sanitizes the specified DB Options.
Status SanitizeOptions(const DBOptions& db_opts,

@ -37,6 +37,7 @@
#include "table/get_context.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
#include "util/perf_context_imp.h"
#include "util/stop_watch.h"
#include "util/string_util.h"
@ -62,7 +63,7 @@ const size_t kMaxCacheKeyPrefixSize __attribute__((unused)) =
// The only relevant option is options.verify_checksums for now.
// On failure return non-OK.
// On success fill *result and return OK - caller owns *result
Status ReadBlockFromFile(RandomAccessFile* file, const Footer& footer,
Status ReadBlockFromFile(RandomAccessFileReader* file, const Footer& footer,
const ReadOptions& options, const BlockHandle& handle,
std::unique_ptr<Block>* result, Env* env,
bool do_uncompress = true) {
@ -167,7 +168,7 @@ class BinarySearchIndexReader : public IndexReader {
// `BinarySearchIndexReader`.
// On success, index_reader will be populated; otherwise it will remain
// unmodified.
static Status Create(RandomAccessFile* file, const Footer& footer,
static Status Create(RandomAccessFileReader* file, const Footer& footer,
const BlockHandle& index_handle, Env* env,
const Comparator* comparator,
IndexReader** index_reader) {
@ -212,8 +213,8 @@ class BinarySearchIndexReader : public IndexReader {
class HashIndexReader : public IndexReader {
public:
static Status Create(const SliceTransform* hash_key_extractor,
const Footer& footer, RandomAccessFile* file, Env* env,
const Comparator* comparator,
const Footer& footer, RandomAccessFileReader* file,
Env* env, const Comparator* comparator,
const BlockHandle& index_handle,
Iterator* meta_index_iter, IndexReader** index_reader,
bool hash_index_allow_collision) {
@ -347,7 +348,7 @@ struct BlockBasedTable::Rep {
const FilterPolicy* const filter_policy;
const InternalKeyComparator& internal_comparator;
Status status;
unique_ptr<RandomAccessFile> file;
unique_ptr<RandomAccessFileReader> file;
char cache_key_prefix[kMaxCacheKeyPrefixSize];
size_t cache_key_prefix_size = 0;
char compressed_cache_key_prefix[kMaxCacheKeyPrefixSize];
@ -405,13 +406,12 @@ void BlockBasedTable::SetupCacheKeyPrefix(Rep* rep) {
rep->cache_key_prefix_size = 0;
rep->compressed_cache_key_prefix_size = 0;
if (rep->table_options.block_cache != nullptr) {
GenerateCachePrefix(rep->table_options.block_cache.get(), rep->file.get(),
&rep->cache_key_prefix[0],
&rep->cache_key_prefix_size);
GenerateCachePrefix(rep->table_options.block_cache.get(), rep->file->file(),
&rep->cache_key_prefix[0], &rep->cache_key_prefix_size);
}
if (rep->table_options.block_cache_compressed != nullptr) {
GenerateCachePrefix(rep->table_options.block_cache_compressed.get(),
rep->file.get(), &rep->compressed_cache_key_prefix[0],
rep->file->file(), &rep->compressed_cache_key_prefix[0],
&rep->compressed_cache_key_prefix_size);
}
}
@ -469,7 +469,7 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table_reader,
const bool prefetch_index_and_filter) {

@ -21,6 +21,7 @@
#include "table/table_reader.h"
#include "table/table_properties_internal.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
@ -69,8 +70,8 @@ class BlockBasedTable : public TableReader {
const EnvOptions& env_options,
const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_key_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size, unique_ptr<TableReader>* table_reader,
bool prefetch_index_and_filter = true);
bool PrefixMayMatch(const Slice& internal_key);

@ -20,6 +20,7 @@
#include "table/format.h"
#include "table/meta_blocks.h"
#include "util/autovector.h"
#include "util/file_reader_writer.h"
#include "util/random.h"
#include "util/string_util.h"
@ -47,7 +48,7 @@ const std::string CuckooTablePropertyNames::kUserKeyLength =
extern const uint64_t kCuckooTableMagicNumber = 0x926789d0c5f17873ull;
CuckooTableBuilder::CuckooTableBuilder(
WritableFile* file, double max_hash_table_ratio,
WritableFileWriter* file, double max_hash_table_ratio,
uint32_t max_num_hash_table, uint32_t max_search_depth,
const Comparator* user_comparator, uint32_t cuckoo_block_size,
bool use_module_hash, bool identity_as_first_hash,

@ -21,12 +21,13 @@ namespace rocksdb {
class CuckooTableBuilder: public TableBuilder {
public:
CuckooTableBuilder(
WritableFile* file, double max_hash_table_ratio,
uint32_t max_num_hash_func, uint32_t max_search_depth,
const Comparator* user_comparator, uint32_t cuckoo_block_size,
bool use_module_hash, bool identity_as_first_hash,
uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t));
CuckooTableBuilder(WritableFileWriter* file, double max_hash_table_ratio,
uint32_t max_num_hash_func, uint32_t max_search_depth,
const Comparator* user_comparator,
uint32_t cuckoo_block_size, bool use_module_hash,
bool identity_as_first_hash,
uint64_t (*get_slice_hash)(const Slice&, uint32_t,
uint64_t));
// REQUIRES: Either Finish() or Abandon() has been called.
~CuckooTableBuilder() {}
@ -82,7 +83,7 @@ class CuckooTableBuilder: public TableBuilder {
inline Slice GetValue(uint64_t idx) const;
uint32_t num_hash_func_;
WritableFile* file_;
WritableFileWriter* file_;
const double max_hash_table_ratio_;
const uint32_t max_num_hash_func_;
const uint32_t max_search_depth_;

@ -10,6 +10,7 @@
#include "table/meta_blocks.h"
#include "table/cuckoo_table_builder.h"
#include "util/file_reader_writer.h"
#include "util/testharness.h"
#include "util/testutil.h"
@ -48,8 +49,11 @@ class CuckooBuilderTest : public testing::Test {
// Assert Table Properties.
TableProperties* props = nullptr;
ASSERT_OK(ReadTableProperties(read_file.get(), read_file_size,
kCuckooTableMagicNumber, env_, nullptr, &props));
unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(read_file)));
ASSERT_OK(ReadTableProperties(file_reader.get(), read_file_size,
kCuckooTableMagicNumber, env_, nullptr,
&props));
// Check unused bucket.
std::string unused_key = props->user_collected_properties[
CuckooTablePropertyNames::kEmptyKey];
@ -90,8 +94,8 @@ class CuckooBuilderTest : public testing::Test {
size_t bucket_size = expected_unused_bucket.size();
for (uint32_t i = 0; i < table_size + cuckoo_block_size - 1; ++i) {
Slice read_slice;
ASSERT_OK(read_file->Read(i*bucket_size, bucket_size,
&read_slice, nullptr));
ASSERT_OK(file_reader->Read(i * bucket_size, bucket_size, &read_slice,
nullptr));
size_t key_idx =
std::find(expected_locations.begin(), expected_locations.end(), i) -
expected_locations.begin();
@ -104,7 +108,7 @@ class CuckooBuilderTest : public testing::Test {
}
}
for (auto key_found : keys_found) {
// Check that all keys were found.
// Check that all keys wereReader found.
ASSERT_TRUE(key_found);
}
}
@ -133,12 +137,15 @@ TEST_F(CuckooBuilderTest, SuccessWithEmptyFile) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/EmptyFile";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
4, 100, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, 4, 100,
BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
ASSERT_EQ(0UL, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
CheckFileContents({}, {}, {}, "", 2, 2, false);
}
@ -165,8 +172,11 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionFullKey) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/NoCollisionFullKey";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(keys[i]), Slice(values[i]));
@ -176,7 +186,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionFullKey) {
size_t bucket_size = keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = GetInternalKey("key00", true);
@ -209,8 +219,11 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionFullKey) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/WithCollisionFullKey";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(keys[i]), Slice(values[i]));
@ -220,7 +233,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionFullKey) {
size_t bucket_size = keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = GetInternalKey("key00", true);
@ -254,9 +267,11 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionAndCuckooBlock) {
uint32_t cuckoo_block_size = 2;
fname = test::TmpDir() + "/WithCollisionFullKey2";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), cuckoo_block_size,
false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), cuckoo_block_size,
false, false, GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(keys[i]), Slice(values[i]));
@ -266,7 +281,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionAndCuckooBlock) {
size_t bucket_size = keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = GetInternalKey("key00", true);
@ -304,8 +319,11 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKey) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/WithCollisionPathFullKey";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(keys[i]), Slice(values[i]));
@ -315,7 +333,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKey) {
size_t bucket_size = keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = GetInternalKey("key00", true);
@ -350,8 +368,11 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKeyAndCuckooBlock) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/WithCollisionPathFullKeyAndCuckooBlock";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), 2, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 2, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(keys[i]), Slice(values[i]));
@ -361,7 +382,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKeyAndCuckooBlock) {
size_t bucket_size = keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = GetInternalKey("key00", true);
@ -389,8 +410,11 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionUserKey) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/NoCollisionUserKey";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(GetInternalKey(user_keys[i], true)), Slice(values[i]));
@ -400,7 +424,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionUserKey) {
size_t bucket_size = user_keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = "key00";
@ -429,8 +453,11 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionUserKey) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/WithCollisionUserKey";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(GetInternalKey(user_keys[i], true)), Slice(values[i]));
@ -440,7 +467,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionUserKey) {
size_t bucket_size = user_keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = "key00";
@ -471,8 +498,11 @@ TEST_F(CuckooBuilderTest, WithCollisionPathUserKey) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/WithCollisionPathUserKey";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 2, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
2, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(GetInternalKey(user_keys[i], true)), Slice(values[i]));
@ -482,7 +512,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathUserKey) {
size_t bucket_size = user_keys[0].size() + values[0].size();
ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
ASSERT_OK(builder.Finish());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
std::string expected_unused_bucket = "key00";
@ -512,8 +542,11 @@ TEST_F(CuckooBuilderTest, FailWhenCollisionPathTooLong) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/WithCollisionPathUserKey";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 2, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
2, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t i = 0; i < user_keys.size(); i++) {
builder.Add(Slice(GetInternalKey(user_keys[i], false)), Slice("value"));
@ -521,7 +554,7 @@ TEST_F(CuckooBuilderTest, FailWhenCollisionPathTooLong) {
ASSERT_OK(builder.status());
}
ASSERT_TRUE(builder.Finish().IsNotSupported());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
}
TEST_F(CuckooBuilderTest, FailWhenSameKeyInserted) {
@ -536,8 +569,11 @@ TEST_F(CuckooBuilderTest, FailWhenSameKeyInserted) {
unique_ptr<WritableFile> writable_file;
fname = test::TmpDir() + "/FailWhenSameKeyInserted";
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
CuckooTableBuilder builder(writable_file.get(), kHashTableRatio,
num_hash_fun, 100, BytewiseComparator(), 1, false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash);
ASSERT_OK(builder.status());
builder.Add(Slice(GetInternalKey(user_key, false)), Slice("value1"));
@ -548,7 +584,7 @@ TEST_F(CuckooBuilderTest, FailWhenSameKeyInserted) {
ASSERT_OK(builder.status());
ASSERT_TRUE(builder.Finish().IsNotSupported());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
}
} // namespace rocksdb

@ -12,9 +12,10 @@
namespace rocksdb {
Status CuckooTableFactory::NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options, const InternalKeyComparator& icomp,
std::unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
Status CuckooTableFactory::NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& icomp,
std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
std::unique_ptr<TableReader>* table) const {
std::unique_ptr<CuckooTableReader> new_reader(new CuckooTableReader(ioptions,
std::move(file), file_size, icomp.user_comparator(), nullptr));
@ -27,7 +28,7 @@ Status CuckooTableFactory::NewTableReader(const ImmutableCFOptions& ioptions,
TableBuilder* CuckooTableFactory::NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const {
WritableFileWriter* file) const {
// Ignore the skipFIlters flag. Does not apply to this file format
//

@ -55,15 +55,16 @@ class CuckooTableFactory : public TableFactory {
const char* Name() const override { return "CuckooTable"; }
Status NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const override;
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table) const override;
TableBuilder* NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const override;
WritableFileWriter* file) const override;
// Sanitizes the specified DB Options.
Status SanitizeOptions(const DBOptions& db_opts,

@ -33,8 +33,7 @@ extern const uint64_t kCuckooTableMagicNumber;
CuckooTableReader::CuckooTableReader(
const ImmutableCFOptions& ioptions,
std::unique_ptr<RandomAccessFile>&& file,
uint64_t file_size,
std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
const Comparator* comparator,
uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t))
: file_(std::move(file)),

@ -18,6 +18,7 @@
#include "rocksdb/env.h"
#include "rocksdb/options.h"
#include "table/table_reader.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
@ -26,12 +27,11 @@ class TableReader;
class CuckooTableReader: public TableReader {
public:
CuckooTableReader(
const ImmutableCFOptions& ioptions,
std::unique_ptr<RandomAccessFile>&& file,
uint64_t file_size,
const Comparator* user_comparator,
uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t));
CuckooTableReader(const ImmutableCFOptions& ioptions,
std::unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size, const Comparator* user_comparator,
uint64_t (*get_slice_hash)(const Slice&, uint32_t,
uint64_t));
~CuckooTableReader() {}
std::shared_ptr<const TableProperties> GetTableProperties() const override {
@ -57,7 +57,7 @@ class CuckooTableReader: public TableReader {
private:
friend class CuckooTableIterator;
void LoadAllKeys(std::vector<std::pair<Slice, uint32_t>>* key_to_bucket_id);
std::unique_ptr<RandomAccessFile> file_;
std::unique_ptr<RandomAccessFileReader> file_;
Slice file_data_;
bool is_last_level_;
bool identity_as_first_hash_;

@ -62,7 +62,6 @@ uint64_t GetSliceHash(const Slice& s, uint32_t index,
uint64_t max_num_buckets) {
return hash_map[s.ToString()][index];
}
} // namespace
class CuckooReaderTest : public testing::Test {
@ -94,9 +93,11 @@ class CuckooReaderTest : public testing::Test {
const Comparator* ucomp = BytewiseComparator()) {
std::unique_ptr<WritableFile> writable_file;
ASSERT_OK(env->NewWritableFile(fname, &writable_file, env_options));
CuckooTableBuilder builder(
writable_file.get(), 0.9, kNumHashFunc, 100, ucomp, 2,
false, false, GetSliceHash);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), env_options));
CuckooTableBuilder builder(file_writer.get(), 0.9, kNumHashFunc, 100, ucomp,
2, false, false, GetSliceHash);
ASSERT_OK(builder.status());
for (uint32_t key_idx = 0; key_idx < num_items; ++key_idx) {
builder.Add(Slice(keys[key_idx]), Slice(values[key_idx]));
@ -106,18 +107,16 @@ class CuckooReaderTest : public testing::Test {
ASSERT_OK(builder.Finish());
ASSERT_EQ(num_items, builder.NumEntries());
file_size = builder.FileSize();
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
// Check reader now.
std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(read_file)));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader(
ioptions,
std::move(read_file),
file_size,
ucomp,
GetSliceHash);
CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucomp,
GetSliceHash);
ASSERT_OK(reader.status());
// Assume no merge/deletion
for (uint32_t i = 0; i < num_items; ++i) {
@ -141,13 +140,11 @@ class CuckooReaderTest : public testing::Test {
void CheckIterator(const Comparator* ucomp = BytewiseComparator()) {
std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(read_file)));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader(
ioptions,
std::move(read_file),
file_size,
ucomp,
GetSliceHash);
CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucomp,
GetSliceHash);
ASSERT_OK(reader.status());
Iterator* it = reader.NewIterator(ReadOptions(), nullptr);
ASSERT_OK(it->status());
@ -321,13 +318,11 @@ TEST_F(CuckooReaderTest, WhenKeyNotFound) {
CreateCuckooFileAndCheckReader();
std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(read_file)));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader(
ioptions,
std::move(read_file),
file_size,
ucmp,
GetSliceHash);
CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucmp,
GetSliceHash);
ASSERT_OK(reader.status());
// Search for a key with colliding hash values.
std::string not_found_user_key = "key" + NumToStr(num_items);
@ -406,10 +401,11 @@ void WriteFile(const std::vector<std::string>& keys,
std::unique_ptr<WritableFile> writable_file;
ASSERT_OK(env->NewWritableFile(fname, &writable_file, env_options));
CuckooTableBuilder builder(
writable_file.get(), hash_ratio,
64, 1000, test::Uint64Comparator(), 5,
false, FLAGS_identity_as_first_hash, nullptr);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), env_options));
CuckooTableBuilder builder(file_writer.get(), hash_ratio, 64, 1000,
test::Uint64Comparator(), 5, false,
FLAGS_identity_as_first_hash, nullptr);
ASSERT_OK(builder.status());
for (uint64_t key_idx = 0; key_idx < num; ++key_idx) {
// Value is just a part of key.
@ -419,17 +415,18 @@ void WriteFile(const std::vector<std::string>& keys,
}
ASSERT_OK(builder.Finish());
ASSERT_EQ(num, builder.NumEntries());
ASSERT_OK(writable_file->Close());
ASSERT_OK(file_writer->Close());
uint64_t file_size;
env->GetFileSize(fname, &file_size);
std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(read_file)));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader(
ioptions, std::move(read_file), file_size,
test::Uint64Comparator(), nullptr);
CuckooTableReader reader(ioptions, std::move(file_reader), file_size,
test::Uint64Comparator(), nullptr);
ASSERT_OK(reader.status());
ReadOptions r_options;
std::string value;
@ -455,11 +452,12 @@ void ReadKeys(uint64_t num, uint32_t batch_size) {
env->GetFileSize(fname, &file_size);
std::unique_ptr<RandomAccessFile> read_file;
ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options));
unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(read_file)));
const ImmutableCFOptions ioptions(options);
CuckooTableReader reader(
ioptions, std::move(read_file), file_size, test::Uint64Comparator(),
nullptr);
CuckooTableReader reader(ioptions, std::move(file_reader), file_size,
test::Uint64Comparator(), nullptr);
ASSERT_OK(reader.status());
const UserCollectedProperties user_props =
reader.GetTableProperties()->user_collected_properties;

@ -17,6 +17,7 @@
#include "util/coding.h"
#include "util/compression.h"
#include "util/crc32c.h"
#include "util/file_reader_writer.h"
#include "util/perf_context_imp.h"
#include "util/string_util.h"
#include "util/xxhash.h"
@ -210,7 +211,7 @@ std::string Footer::ToString() const {
return result;
}
Status ReadFooterFromFile(RandomAccessFile* file, uint64_t file_size,
Status ReadFooterFromFile(RandomAccessFileReader* file, uint64_t file_size,
Footer* footer, uint64_t enforce_table_magic_number) {
if (file_size < Footer::kMinEncodedLength) {
return Status::Corruption("file is too short to be an sstable");
@ -249,9 +250,9 @@ namespace {
// Read a block and check its CRC
// contents is the result of reading.
// According to the implementation of file->Read, contents may not point to buf
Status ReadBlock(RandomAccessFile* file, const Footer& footer,
const ReadOptions& options, const BlockHandle& handle,
Slice* contents, /* result of reading */ char* buf) {
Status ReadBlock(RandomAccessFileReader* file, const Footer& footer,
const ReadOptions& options, const BlockHandle& handle,
Slice* contents, /* result of reading */ char* buf) {
size_t n = static_cast<size_t>(handle.size());
Status s;
@ -299,7 +300,7 @@ Status ReadBlock(RandomAccessFile* file, const Footer& footer,
} // namespace
Status ReadBlockContents(RandomAccessFile* file, const Footer& footer,
Status ReadBlockContents(RandomAccessFileReader* file, const Footer& footer,
const ReadOptions& options, const BlockHandle& handle,
BlockContents* contents, Env* env,
bool decompression_requested) {

@ -166,7 +166,7 @@ class Footer {
// Read the footer from file
// If enforce_table_magic_number != 0, ReadFooterFromFile() will return
// corruption if table_magic number is not equal to enforce_table_magic_number
Status ReadFooterFromFile(RandomAccessFile* file, uint64_t file_size,
Status ReadFooterFromFile(RandomAccessFileReader* file, uint64_t file_size,
Footer* footer,
uint64_t enforce_table_magic_number = 0);
@ -205,7 +205,8 @@ struct BlockContents {
// Read the block identified by "handle" from "file". On failure
// return non-OK. On success fill *result and return OK.
extern Status ReadBlockContents(RandomAccessFile* file, const Footer& footer,
extern Status ReadBlockContents(RandomAccessFileReader* file,
const Footer& footer,
const ReadOptions& options,
const BlockHandle& handle,
BlockContents* contents, Env* env,

@ -129,9 +129,9 @@ bool NotifyCollectTableCollectorsOnFinish(
return all_succeeded;
}
Status ReadProperties(const Slice &handle_value, RandomAccessFile *file,
const Footer &footer, Env *env, Logger *logger,
TableProperties **table_properties) {
Status ReadProperties(const Slice& handle_value, RandomAccessFileReader* file,
const Footer& footer, Env* env, Logger* logger,
TableProperties** table_properties) {
assert(table_properties);
Slice v = handle_value;
@ -217,7 +217,7 @@ Status ReadProperties(const Slice &handle_value, RandomAccessFile *file,
return s;
}
Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size,
Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
uint64_t table_magic_number, Env* env,
Logger* info_log, TableProperties** properties) {
// -- Read metaindex block
@ -271,7 +271,7 @@ Status FindMetaBlock(Iterator* meta_index_iter,
}
}
Status FindMetaBlock(RandomAccessFile* file, uint64_t file_size,
Status FindMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
uint64_t table_magic_number, Env* env,
const std::string& meta_block_name,
BlockHandle* block_handle) {
@ -298,7 +298,7 @@ Status FindMetaBlock(RandomAccessFile* file, uint64_t file_size,
return FindMetaBlock(meta_iter.get(), meta_block_name, block_handle);
}
Status ReadMetaBlock(RandomAccessFile* file, uint64_t file_size,
Status ReadMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
uint64_t table_magic_number, Env* env,
const std::string& meta_block_name,
BlockContents* contents) {

@ -107,26 +107,25 @@ bool NotifyCollectTableCollectorsOnFinish(
// @returns a status to indicate if the operation succeeded. On success,
// *table_properties will point to a heap-allocated TableProperties
// object, otherwise value of `table_properties` will not be modified.
Status ReadProperties(const Slice &handle_value, RandomAccessFile *file,
const Footer &footer, Env *env, Logger *logger,
TableProperties **table_properties);
Status ReadProperties(const Slice& handle_value, RandomAccessFileReader* file,
const Footer& footer, Env* env, Logger* logger,
TableProperties** table_properties);
// Directly read the properties from the properties block of a plain table.
// @returns a status to indicate if the operation succeeded. On success,
// *table_properties will point to a heap-allocated TableProperties
// object, otherwise value of `table_properties` will not be modified.
Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size,
Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
uint64_t table_magic_number, Env* env,
Logger* info_log, TableProperties** properties);
// Find the meta block from the meta index block.
Status FindMetaBlock(Iterator* meta_index_iter,
const std::string& meta_block_name,
BlockHandle* block_handle);
// Find the meta block
Status FindMetaBlock(RandomAccessFile* file, uint64_t file_size,
Status FindMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
uint64_t table_magic_number, Env* env,
const std::string& meta_block_name,
BlockHandle* block_handle);
@ -134,7 +133,7 @@ Status FindMetaBlock(RandomAccessFile* file, uint64_t file_size,
// Read the specified meta block with name meta_block_name
// from `file` and initialize `contents` with contents of this block.
// Return Status::OK in case of success.
Status ReadMetaBlock(RandomAccessFile* file, uint64_t file_size,
Status ReadMetaBlock(RandomAccessFileReader* file, uint64_t file_size,
uint64_t table_magic_number, Env* env,
const std::string& meta_block_name,
BlockContents* contents);

@ -11,6 +11,7 @@
#include "db/dbformat.h"
#include "port/port.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
namespace mock {
@ -45,7 +46,7 @@ MockTableFactory::MockTableFactory() : next_id_(1) {}
Status MockTableFactory::NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_key,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const {
uint32_t id = GetIDFromFile(file.get());
@ -63,8 +64,8 @@ Status MockTableFactory::NewTableReader(
TableBuilder* MockTableFactory::NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const {
uint32_t id = GetAndWriteNextID(file);
WritableFileWriter* file) const {
uint32_t id = GetAndWriteNextID(file->writable_file());
return new MockTableBuilder(id, &file_system_);
}
@ -90,7 +91,7 @@ uint32_t MockTableFactory::GetAndWriteNextID(WritableFile* file) const {
return next_id;
}
uint32_t MockTableFactory::GetIDFromFile(RandomAccessFile* file) const {
uint32_t MockTableFactory::GetIDFromFile(RandomAccessFileReader* file) const {
char buf[4];
Slice result;
file->Read(0, 4, &result, buf);

@ -140,13 +140,14 @@ class MockTableFactory : public TableFactory {
MockTableFactory();
const char* Name() const override { return "MockTable"; }
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_key,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const override;
const EnvOptions& env_options,
const InternalKeyComparator& internal_key,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table_reader) const override;
TableBuilder* NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const override;
WritableFileWriter* file) const override;
// This function will directly create mock table instead of going through
// MockTableBuilder. MockFileContents has to have a format of <internal_key,
@ -171,7 +172,7 @@ class MockTableFactory : public TableFactory {
private:
uint32_t GetAndWriteNextID(WritableFile* file) const;
uint32_t GetIDFromFile(RandomAccessFile* file) const;
uint32_t GetIDFromFile(RandomAccessFileReader* file) const;
mutable MockTableFileSystem file_system_;
mutable std::atomic<uint32_t> next_id_;

@ -26,6 +26,7 @@
#include "table/meta_blocks.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/file_reader_writer.h"
#include "util/stop_watch.h"
namespace rocksdb {
@ -35,11 +36,8 @@ namespace {
// a utility that helps writing block content to the file
// @offset will advance if @block_contents was successfully written.
// @block_handle the block handle this particular block.
Status WriteBlock(
const Slice& block_contents,
WritableFile* file,
uint64_t* offset,
BlockHandle* block_handle) {
Status WriteBlock(const Slice& block_contents, WritableFileWriter* file,
uint64_t* offset, BlockHandle* block_handle) {
block_handle->set_offset(*offset);
block_handle->set_size(block_contents.size());
Status s = file->Append(block_contents);
@ -62,7 +60,7 @@ PlainTableBuilder::PlainTableBuilder(
const ImmutableCFOptions& ioptions,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
WritableFile* file, uint32_t user_key_len, EncodingType encoding_type,
WritableFileWriter* file, uint32_t user_key_len, EncodingType encoding_type,
size_t index_sparseness, uint32_t bloom_bits_per_key, uint32_t num_probes,
size_t huge_page_tlb_size, double hash_table_ratio,
bool store_index_in_file)

@ -34,10 +34,11 @@ class PlainTableBuilder: public TableBuilder {
const ImmutableCFOptions& ioptions,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
WritableFile* file, uint32_t user_key_size, EncodingType encoding_type,
size_t index_sparseness, uint32_t bloom_bits_per_key,
uint32_t num_probes = 6, size_t huge_page_tlb_size = 0,
double hash_table_ratio = 0, bool store_index_in_file = false);
WritableFileWriter* file, uint32_t user_key_size,
EncodingType encoding_type, size_t index_sparseness,
uint32_t bloom_bits_per_key, uint32_t num_probes = 6,
size_t huge_page_tlb_size = 0, double hash_table_ratio = 0,
bool store_index_in_file = false);
// REQUIRES: Either Finish() or Abandon() has been called.
~PlainTableBuilder();
@ -82,7 +83,7 @@ class PlainTableBuilder: public TableBuilder {
BloomBlockBuilder bloom_block_;
std::unique_ptr<PlainTableIndexBuilder> index_builder_;
WritableFile* file_;
WritableFileWriter* file_;
uint64_t offset_ = 0;
uint32_t bloom_bits_per_key_;
size_t huge_page_tlb_size_;

@ -14,12 +14,11 @@
namespace rocksdb {
Status PlainTableFactory::NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& icomp,
unique_ptr<RandomAccessFile>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table) const {
Status PlainTableFactory::NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& icomp,
unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const {
return PlainTableReader::Open(ioptions, env_options, icomp, std::move(file),
file_size, table, bloom_bits_per_key_,
hash_table_ratio_, index_sparseness_,
@ -28,7 +27,7 @@ Status PlainTableFactory::NewTableReader(const ImmutableCFOptions& ioptions,
TableBuilder* PlainTableFactory::NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const {
WritableFileWriter* file) const {
// Ignore the skip_filters flag. PlainTable format is optimized for small
// in-memory dbs. The skip_filters optimization is not useful for plain
// tables

@ -153,14 +153,15 @@ class PlainTableFactory : public TableFactory {
full_scan_mode_(options.full_scan_mode),
store_index_in_file_(options.store_index_in_file) {}
const char* Name() const override { return "PlainTable"; }
Status NewTableReader(
const ImmutableCFOptions& options, const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table) const override;
Status NewTableReader(const ImmutableCFOptions& options,
const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table) const override;
TableBuilder* NewTableBuilder(
const TableBuilderOptions& table_builder_options,
WritableFile* file) const override;
WritableFileWriter* file) const override;
std::string GetPrintableTableOptions() const override;

@ -6,8 +6,9 @@
#ifndef ROCKSDB_LITE
#include "table/plain_table_key_coding.h"
#include "table/plain_table_factory.h"
#include "db/dbformat.h"
#include "table/plain_table_factory.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
@ -64,7 +65,8 @@ const char* DecodeSize(const char* offset, const char* limit,
}
} // namespace
Status PlainTableKeyEncoder::AppendKey(const Slice& key, WritableFile* file,
Status PlainTableKeyEncoder::AppendKey(const Slice& key,
WritableFileWriter* file,
uint64_t* offset, char* meta_bytes_buf,
size_t* meta_bytes_buf_size) {
ParsedInternalKey parsed_key;

@ -34,7 +34,7 @@ class PlainTableKeyEncoder {
// meta_bytes_buf: buffer for extra meta bytes
// meta_bytes_buf_size: offset to append extra meta bytes. Will be updated
// if meta_bytes_buf is updated.
Status AppendKey(const Slice& key, WritableFile* file, uint64_t* offset,
Status AppendKey(const Slice& key, WritableFileWriter* file, uint64_t* offset,
char* meta_bytes_buf, size_t* meta_bytes_buf_size);
// Return actual encoding type to be picked

@ -90,7 +90,7 @@ class PlainTableIterator : public Iterator {
extern const uint64_t kPlainTableMagicNumber;
PlainTableReader::PlainTableReader(const ImmutableCFOptions& ioptions,
unique_ptr<RandomAccessFile>&& file,
unique_ptr<RandomAccessFileReader>&& file,
const EnvOptions& storage_options,
const InternalKeyComparator& icomparator,
EncodingType encoding_type,
@ -115,7 +115,7 @@ PlainTableReader::~PlainTableReader() {
Status PlainTableReader::Open(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table_reader,
const int bloom_bits_per_key,

@ -22,6 +22,7 @@
#include "table/plain_table_index.h"
#include "util/arena.h"
#include "util/dynamic_bloom.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
@ -56,8 +57,8 @@ class PlainTableReader: public TableReader {
static Status Open(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table,
unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size, unique_ptr<TableReader>* table,
const int bloom_bits_per_key, double hash_table_ratio,
size_t index_sparseness, size_t huge_page_tlb_size,
bool full_scan_mode);
@ -83,7 +84,7 @@ class PlainTableReader: public TableReader {
}
PlainTableReader(const ImmutableCFOptions& ioptions,
unique_ptr<RandomAccessFile>&& file,
unique_ptr<RandomAccessFileReader>&& file,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
EncodingType encoding_type, uint64_t file_size,
@ -134,7 +135,7 @@ class PlainTableReader: public TableReader {
Arena arena_;
const ImmutableCFOptions& ioptions_;
unique_ptr<RandomAccessFile> file_;
unique_ptr<RandomAccessFileReader> file_;
uint64_t file_size_;
std::shared_ptr<const TableProperties> table_properties_;

@ -22,6 +22,7 @@ int main() {
#include "table/plain_table_factory.h"
#include "table/table_builder.h"
#include "table/get_context.h"
#include "util/file_reader_writer.h"
#include "util/histogram.h"
#include "util/testharness.h"
#include "util/testutil.h"
@ -90,11 +91,14 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
std::vector<std::unique_ptr<IntTblPropCollectorFactory> >
int_tbl_prop_collector_factories;
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options));
tb = opts.table_factory->NewTableBuilder(
TableBuilderOptions(ioptions, ikc, &int_tbl_prop_collector_factories,
CompressionType::kNoCompression,
CompressionOptions(), false),
file.get());
file_writer.get());
} else {
s = DB::Open(opts, dbname, &db);
ASSERT_OK(s);
@ -119,13 +123,16 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
}
unique_ptr<TableReader> table_reader;
unique_ptr<RandomAccessFile> raf;
if (!through_db) {
unique_ptr<RandomAccessFile> raf;
s = env->NewRandomAccessFile(file_name, &raf, env_options);
uint64_t file_size;
env->GetFileSize(file_name, &file_size);
s = opts.table_factory->NewTableReader(
ioptions, env_options, ikc, std::move(raf), file_size, &table_reader);
unique_ptr<RandomAccessFileReader> file_reader(
new RandomAccessFileReader(std::move(raf)));
s = opts.table_factory->NewTableReader(ioptions, env_options, ikc,
std::move(file_reader), file_size,
&table_reader);
}
Random rnd(301);

@ -347,7 +347,7 @@ class TableConstructor: public Constructor {
const InternalKeyComparator& internal_comparator,
const KVMap& kv_map) override {
Reset();
sink_.reset(new StringSink());
file_writer_.reset(test::GetWritableFileWriter(new StringSink()));
unique_ptr<TableBuilder> builder;
std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
int_tbl_prop_collector_factories;
@ -355,7 +355,7 @@ class TableConstructor: public Constructor {
TableBuilderOptions(ioptions, internal_comparator,
&int_tbl_prop_collector_factories,
options.compression, CompressionOptions(), false),
sink_.get()));
file_writer_.get()));
for (const auto kv : kv_map) {
if (convert_to_internal_key_) {
@ -369,17 +369,18 @@ class TableConstructor: public Constructor {
EXPECT_TRUE(builder->status().ok());
}
Status s = builder->Finish();
file_writer_->Flush();
EXPECT_TRUE(s.ok()) << s.ToString();
EXPECT_EQ(sink_->contents().size(), builder->FileSize());
EXPECT_EQ(GetSink()->contents().size(), builder->FileSize());
// Open the table
uniq_id_ = cur_uniq_id_++;
source_.reset(new StringSource(sink_->contents(), uniq_id_,
ioptions.allow_mmap_reads));
file_reader_.reset(test::GetRandomAccessFileReader(new StringSource(
GetSink()->contents(), uniq_id_, ioptions.allow_mmap_reads)));
return ioptions.table_factory->NewTableReader(
ioptions, soptions, internal_comparator, std::move(source_),
sink_->contents().size(), &table_reader_);
ioptions, soptions, internal_comparator, std::move(file_reader_),
GetSink()->contents().size(), &table_reader_);
}
virtual Iterator* NewIterator() const override {
@ -397,12 +398,11 @@ class TableConstructor: public Constructor {
}
virtual Status Reopen(const ImmutableCFOptions& ioptions) {
source_.reset(
new StringSource(sink_->contents(), uniq_id_,
ioptions.allow_mmap_reads));
file_reader_.reset(test::GetRandomAccessFileReader(new StringSource(
GetSink()->contents(), uniq_id_, ioptions.allow_mmap_reads)));
return ioptions.table_factory->NewTableReader(
ioptions, soptions, *last_internal_key_, std::move(source_),
sink_->contents().size(), &table_reader_);
ioptions, soptions, *last_internal_key_, std::move(file_reader_),
GetSink()->contents().size(), &table_reader_);
}
virtual TableReader* GetTableReader() {
@ -417,13 +417,17 @@ class TableConstructor: public Constructor {
void Reset() {
uniq_id_ = 0;
table_reader_.reset();
sink_.reset();
source_.reset();
file_writer_.reset();
file_reader_.reset();
}
StringSink* GetSink() {
return static_cast<StringSink*>(file_writer_->writable_file());
}
uint64_t uniq_id_;
unique_ptr<StringSink> sink_;
unique_ptr<StringSource> source_;
unique_ptr<WritableFileWriter> file_writer_;
unique_ptr<RandomAccessFileReader> file_reader_;
unique_ptr<TableReader> table_reader_;
bool convert_to_internal_key_;
@ -1766,6 +1770,8 @@ TEST_F(PlainTableTest, BasicPlainTableProperties) {
PlainTableFactory factory(plain_table_options);
StringSink sink;
unique_ptr<WritableFileWriter> file_writer(
test::GetWritableFileWriter(new StringSink()));
Options options;
const ImmutableCFOptions ioptions(options);
InternalKeyComparator ikc(options.comparator);
@ -1774,7 +1780,7 @@ TEST_F(PlainTableTest, BasicPlainTableProperties) {
std::unique_ptr<TableBuilder> builder(factory.NewTableBuilder(
TableBuilderOptions(ioptions, ikc, &int_tbl_prop_collector_factories,
kNoCompression, CompressionOptions(), false),
&sink));
file_writer.get()));
for (char c = 'a'; c <= 'z'; ++c) {
std::string key(8, c);
@ -1783,11 +1789,15 @@ TEST_F(PlainTableTest, BasicPlainTableProperties) {
builder->Add(key, value);
}
ASSERT_OK(builder->Finish());
file_writer->Flush();
StringSource source(sink.contents(), 72242, true);
StringSink* ss = static_cast<StringSink*>(file_writer->writable_file());
unique_ptr<RandomAccessFileReader> file_reader(
test::GetRandomAccessFileReader(
new StringSource(ss->contents(), 72242, true)));
TableProperties* props = nullptr;
auto s = ReadTableProperties(&source, sink.contents().size(),
auto s = ReadTableProperties(file_reader.get(), ss->contents().size(),
kPlainTableMagicNumber, Env::Default(), nullptr,
&props);
std::unique_ptr<TableProperties> props_guard(props);

@ -170,6 +170,9 @@ class SpecialEnv : public EnvWrapper {
void SetIOPriority(Env::IOPriority pri) override {
base_->SetIOPriority(pri);
}
Env::IOPriority GetIOPriority() override {
return base_->GetIOPriority();
}
};
class ManifestFile : public WritableFile {
public:

@ -40,7 +40,6 @@
#include "util/posix_logger.h"
#include "util/random.h"
#include "util/iostats_context_imp.h"
#include "util/rate_limiter.h"
#include "util/sync_point.h"
#include "util/thread_status_updater.h"
#include "util/thread_status_util.h"
@ -74,9 +73,6 @@
#define POSIX_FADV_DONTNEED 4 /* [MC1] dont need these pages */
#endif
// This is only set from db_stress.cc and for testing only.
// If non-zero, kill at various points in source code with probability 1/this
int rocksdb_kill_odds = 0;
namespace rocksdb {
@ -104,39 +100,6 @@ static Status IOError(const std::string& context, int err_number) {
return Status::IOError(context, strerror(err_number));
}
#ifdef NDEBUG
// empty in release build
#define TEST_KILL_RANDOM(rocksdb_kill_odds)
#else
// Kill the process with probablity 1/odds for testing.
static void TestKillRandom(int odds, const std::string& srcfile,
int srcline) {
time_t curtime = time(nullptr);
Random r((uint32_t)curtime);
assert(odds > 0);
bool crash = r.OneIn(odds);
if (crash) {
fprintf(stdout, "Crashing at %s:%d\n", srcfile.c_str(), srcline);
fflush(stdout);
kill(getpid(), SIGTERM);
}
}
// To avoid crashing always at some frequently executed codepaths (during
// kill random test), use this factor to reduce odds
#define REDUCE_ODDS 2
#define REDUCE_ODDS2 4
#define TEST_KILL_RANDOM(rocksdb_kill_odds) { \
if (rocksdb_kill_odds > 0) { \
TestKillRandom(rocksdb_kill_odds, __FILE__, __LINE__); \
} \
}
#endif
#if defined(OS_LINUX)
namespace {
static size_t GetUniqueIdFromFile(int fd, char* id, size_t max_size) {
@ -188,7 +151,6 @@ class PosixSequentialFile: public SequentialFile {
do {
r = fread_unlocked(scratch, 1, n, file_);
} while (r == 0 && ferror(file_) && errno == EINTR);
IOSTATS_ADD(bytes_read, r);
*result = Slice(scratch, r);
if (r < n) {
if (feof(file_)) {
@ -252,10 +214,7 @@ class PosixRandomAccessFile: public RandomAccessFile {
size_t left = n;
char* ptr = scratch;
while (left > 0) {
{
IOSTATS_TIMER_GUARD(read_nanos);
r = pread(fd_, ptr, left, static_cast<off_t>(offset));
}
r = pread(fd_, ptr, left, static_cast<off_t>(offset));
if (r <= 0) {
if (errno == EINTR) {
@ -268,7 +227,6 @@ class PosixRandomAccessFile: public RandomAccessFile {
left -= r;
}
IOSTATS_ADD_IF_POSITIVE(bytes_read, n - left);
*result = Slice(scratch, (r < 0) ? 0 : n - left);
if (r < 0) {
// An error: return a non-ok status
@ -458,7 +416,6 @@ class PosixMmapFile : public WritableFile {
if (ptr == MAP_FAILED) {
return Status::IOError("MMap failed on " + filename_);
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
base_ = reinterpret_cast<char*>(ptr);
@ -482,8 +439,7 @@ class PosixMmapFile : public WritableFile {
limit_(nullptr),
dst_(nullptr),
last_sync_(nullptr),
file_offset_(0),
pending_sync_(false) {
file_offset_(0) {
#ifdef ROCKSDB_FALLOCATE_PRESENT
fallocate_with_keep_size_ = options.fallocate_with_keep_size;
#endif
@ -501,8 +457,6 @@ class PosixMmapFile : public WritableFile {
virtual Status Append(const Slice& data) override {
const char* src = data.data();
size_t left = data.size();
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS);
PrepareWrite(static_cast<size_t>(GetFileSize()), left);
while (left > 0) {
assert(base_ <= dst_);
assert(dst_ <= limit_);
@ -521,12 +475,10 @@ class PosixMmapFile : public WritableFile {
size_t n = (left <= avail) ? left : avail;
memcpy(dst_, src, n);
IOSTATS_ADD(bytes_written, n);
dst_ += n;
src += n;
left -= n;
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
return Status::OK();
}
@ -534,8 +486,6 @@ class PosixMmapFile : public WritableFile {
Status s;
size_t unused = limit_ - dst_;
TEST_KILL_RANDOM(rocksdb_kill_odds);
s = UnmapCurrentRegion();
if (!s.ok()) {
s = IOError(filename_, errno);
@ -546,8 +496,6 @@ class PosixMmapFile : public WritableFile {
}
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
if (close(fd_) < 0) {
if (s.ok()) {
s = IOError(filename_, errno);
@ -561,22 +509,15 @@ class PosixMmapFile : public WritableFile {
}
virtual Status Flush() override {
TEST_KILL_RANDOM(rocksdb_kill_odds);
return Status::OK();
}
virtual Status Sync() override {
Status s;
if (pending_sync_) {
// Some unmapped data was not synced
TEST_KILL_RANDOM(rocksdb_kill_odds);
pending_sync_ = false;
if (fdatasync(fd_) < 0) {
s = IOError(filename_, errno);
}
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS);
}
if (dst_ > last_sync_) {
// Find the beginnings of the pages that contain the first and last
@ -588,7 +529,6 @@ class PosixMmapFile : public WritableFile {
if (msync(base_ + p1, p2 - p1 + page_size_, MS_SYNC) < 0) {
s = IOError(filename_, errno);
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
}
return s;
@ -598,15 +538,10 @@ class PosixMmapFile : public WritableFile {
* Flush data as well as metadata to stable storage.
*/
virtual Status Fsync() override {
if (pending_sync_) {
// Some unmapped data was not synced
TEST_KILL_RANDOM(rocksdb_kill_odds);
pending_sync_ = false;
if (fsync(fd_) < 0) {
return IOError(filename_, errno);
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
}
// This invocation to Sync will not issue the call to
// fdatasync because pending_sync_ has already been cleared.
return Sync();
@ -638,7 +573,6 @@ class PosixMmapFile : public WritableFile {
#ifdef ROCKSDB_FALLOCATE_PRESENT
virtual Status Allocate(off_t offset, off_t len) override {
TEST_KILL_RANDOM(rocksdb_kill_odds);
IOSTATS_TIMER_GUARD(allocate_nanos);
int alloc_status = fallocate(
fd_, fallocate_with_keep_size_ ? FALLOC_FL_KEEP_SIZE : 0, offset, len);
if (alloc_status == 0) {
@ -655,33 +589,14 @@ class PosixWritableFile : public WritableFile {
private:
const std::string filename_;
int fd_;
size_t cursize_; // current size of cached data in buf_
size_t capacity_; // max size of buf_
unique_ptr<char[]> buf_; // a buffer to cache writes
uint64_t filesize_;
bool pending_sync_;
bool pending_fsync_;
uint64_t last_sync_size_;
uint64_t bytes_per_sync_;
#ifdef ROCKSDB_FALLOCATE_PRESENT
bool fallocate_with_keep_size_;
#endif
RateLimiter* rate_limiter_;
public:
PosixWritableFile(const std::string& fname, int fd, size_t capacity,
const EnvOptions& options)
: filename_(fname),
fd_(fd),
cursize_(0),
capacity_(capacity),
buf_(new char[capacity]),
filesize_(0),
pending_sync_(false),
pending_fsync_(false),
last_sync_size_(0),
bytes_per_sync_(options.bytes_per_sync),
rate_limiter_(options.rate_limiter) {
PosixWritableFile(const std::string& fname, int fd, const EnvOptions& options)
: filename_(fname), fd_(fd), filesize_(0) {
#ifdef ROCKSDB_FALLOCATE_PRESENT
fallocate_with_keep_size_ = options.fallocate_with_keep_size;
#endif
@ -698,64 +613,23 @@ class PosixWritableFile : public WritableFile {
const char* src = data.data();
size_t left = data.size();
Status s;
pending_sync_ = true;
pending_fsync_ = true;
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS2);
PrepareWrite(static_cast<size_t>(GetFileSize()), left);
// if there is no space in the cache, then flush
if (cursize_ + left > capacity_) {
s = Flush();
if (!s.ok()) {
return s;
}
// Increase the buffer size, but capped at 1MB
if (capacity_ < (1<<20)) {
capacity_ *= 2;
buf_.reset(new char[capacity_]);
}
assert(cursize_ == 0);
}
// if the write fits into the cache, then write to cache
// otherwise do a write() syscall to write to OS buffers.
if (cursize_ + left <= capacity_) {
memcpy(buf_.get()+cursize_, src, left);
cursize_ += left;
} else {
while (left != 0) {
ssize_t done;
size_t size = RequestToken(left);
{
IOSTATS_TIMER_GUARD(write_nanos);
done = write(fd_, src, size);
}
ssize_t done = write(fd_, src, left);
if (done < 0) {
if (errno == EINTR) {
continue;
}
return IOError(filename_, errno);
}
IOSTATS_ADD(bytes_written, done);
TEST_KILL_RANDOM(rocksdb_kill_odds);
left -= done;
src += done;
}
}
filesize_ += data.size();
filesize_ += data.size();
return Status::OK();
}
virtual Status Close() override {
Status s;
s = Flush(); // flush cache to OS
if (!s.ok()) {
return s;
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
size_t block_size;
size_t last_allocated_block;
@ -793,68 +667,20 @@ class PosixWritableFile : public WritableFile {
// write out the cached data to the OS cache
virtual Status Flush() override {
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS2);
size_t left = cursize_;
char* src = buf_.get();
while (left != 0) {
ssize_t done;
size_t size = RequestToken(left);
{
IOSTATS_TIMER_GUARD(write_nanos);
done = write(fd_, src, size);
}
if (done < 0) {
if (errno == EINTR) {
continue;
}
return IOError(filename_, errno);
}
IOSTATS_ADD(bytes_written, done);
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS2);
left -= done;
src += done;
}
cursize_ = 0;
// 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).
if (bytes_per_sync_ &&
filesize_ - last_sync_size_ >= bytes_per_sync_) {
RangeSync(last_sync_size_, filesize_ - last_sync_size_);
last_sync_size_ = filesize_;
}
return Status::OK();
}
virtual Status Sync() override {
Status s = Flush();
if (!s.ok()) {
return s;
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
if (pending_sync_ && fdatasync(fd_) < 0) {
if (fdatasync(fd_) < 0) {
return IOError(filename_, errno);
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
pending_sync_ = false;
return Status::OK();
}
virtual Status Fsync() override {
Status s = Flush();
if (!s.ok()) {
return s;
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
if (pending_fsync_ && fsync(fd_) < 0) {
if (fsync(fd_) < 0) {
return IOError(filename_, errno);
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
pending_fsync_ = false;
pending_sync_ = false;
return Status::OK();
}
@ -876,8 +702,8 @@ class PosixWritableFile : public WritableFile {
#ifdef ROCKSDB_FALLOCATE_PRESENT
virtual Status Allocate(off_t offset, off_t len) override {
TEST_KILL_RANDOM(rocksdb_kill_odds);
int alloc_status;
IOSTATS_TIMER_GUARD(allocate_nanos);
int alloc_status;
alloc_status = fallocate(
fd_, fallocate_with_keep_size_ ? FALLOC_FL_KEEP_SIZE : 0, offset, len);
if (alloc_status == 0) {
@ -888,7 +714,6 @@ class PosixWritableFile : public WritableFile {
}
virtual Status RangeSync(off_t offset, off_t nbytes) override {
IOSTATS_TIMER_GUARD(range_sync_nanos);
if (sync_file_range(fd_, offset, nbytes, SYNC_FILE_RANGE_WRITE) == 0) {
return Status::OK();
} else {
@ -899,34 +724,19 @@ class PosixWritableFile : public WritableFile {
return GetUniqueIdFromFile(fd_, id, max_size);
}
#endif
private:
inline size_t RequestToken(size_t bytes) {
if (rate_limiter_ && io_priority_ < Env::IO_TOTAL) {
bytes = std::min(bytes,
static_cast<size_t>(rate_limiter_->GetSingleBurstBytes()));
rate_limiter_->Request(bytes, io_priority_);
}
return bytes;
}
};
class PosixRandomRWFile : public RandomRWFile {
private:
const std::string filename_;
int fd_;
bool pending_sync_;
bool pending_fsync_;
#ifdef ROCKSDB_FALLOCATE_PRESENT
bool fallocate_with_keep_size_;
#endif
public:
PosixRandomRWFile(const std::string& fname, int fd, const EnvOptions& options)
: filename_(fname),
fd_(fd),
pending_sync_(false),
pending_fsync_(false) {
: filename_(fname), fd_(fd) {
#ifdef ROCKSDB_FALLOCATE_PRESENT
fallocate_with_keep_size_ = options.fallocate_with_keep_size;
#endif
@ -943,22 +753,15 @@ class PosixRandomRWFile : public RandomRWFile {
const char* src = data.data();
size_t left = data.size();
Status s;
pending_sync_ = true;
pending_fsync_ = true;
while (left != 0) {
ssize_t done;
{
IOSTATS_TIMER_GUARD(write_nanos);
done = pwrite(fd_, src, left, offset);
}
ssize_t done = pwrite(fd_, src, left, offset);
if (done < 0) {
if (errno == EINTR) {
continue;
}
return IOError(filename_, errno);
}
IOSTATS_ADD(bytes_written, done);
left -= done;
src += done;
@ -975,11 +778,7 @@ class PosixRandomRWFile : public RandomRWFile {
size_t left = n;
char* ptr = scratch;
while (left > 0) {
{
IOSTATS_TIMER_GUARD(read_nanos);
r = pread(fd_, ptr, left, static_cast<off_t>(offset));
}
r = pread(fd_, ptr, left, static_cast<off_t>(offset));
if (r <= 0) {
if (errno == EINTR) {
continue;
@ -990,7 +789,6 @@ class PosixRandomRWFile : public RandomRWFile {
offset += r;
left -= r;
}
IOSTATS_ADD_IF_POSITIVE(bytes_read, n - left);
*result = Slice(scratch, (r < 0) ? 0 : n - left);
if (r < 0) {
s = IOError(filename_, errno);
@ -1008,25 +806,21 @@ class PosixRandomRWFile : public RandomRWFile {
}
virtual Status Sync() override {
if (pending_sync_ && fdatasync(fd_) < 0) {
if (fdatasync(fd_) < 0) {
return IOError(filename_, errno);
}
pending_sync_ = false;
return Status::OK();
}
virtual Status Fsync() override {
if (pending_fsync_ && fsync(fd_) < 0) {
if (fsync(fd_) < 0) {
return IOError(filename_, errno);
}
pending_fsync_ = false;
pending_sync_ = false;
return Status::OK();
}
#ifdef ROCKSDB_FALLOCATE_PRESENT
virtual Status Allocate(off_t offset, off_t len) override {
TEST_KILL_RANDOM(rocksdb_kill_odds);
IOSTATS_TIMER_GUARD(allocate_nanos);
int alloc_status = fallocate(
fd_, fallocate_with_keep_size_ ? FALLOC_FL_KEEP_SIZE : 0, offset, len);
@ -1216,9 +1010,7 @@ class PosixEnv : public Env {
EnvOptions no_mmap_writes_options = options;
no_mmap_writes_options.use_mmap_writes = false;
result->reset(
new PosixWritableFile(fname, fd, 65536, no_mmap_writes_options)
);
result->reset(new PosixWritableFile(fname, fd, no_mmap_writes_options));
}
}
return s;

@ -664,6 +664,7 @@ TEST_F(EnvPosixTest, AllocateTest) {
size_t kPageSize = 4096;
std::string data(1024 * 1024, 'a');
wfile->SetPreallocationBlockSize(kPreallocateSize);
wfile->PrepareWrite(wfile->GetFileSize(), data.size());
ASSERT_OK(wfile->Append(Slice(data)));
ASSERT_OK(wfile->Flush());
@ -974,18 +975,22 @@ TEST_F(EnvPosixTest, Preallocation) {
ASSERT_EQ(last_allocated_block, 0UL);
// Small write should preallocate one block
srcfile->Append("test");
std::string str = "test";
srcfile->PrepareWrite(srcfile->GetFileSize(), str.size());
srcfile->Append(str);
srcfile->GetPreallocationStatus(&block_size, &last_allocated_block);
ASSERT_EQ(last_allocated_block, 1UL);
// Write an entire preallocation block, make sure we increased by two.
std::string buf(block_size, ' ');
srcfile->PrepareWrite(srcfile->GetFileSize(), buf.size());
srcfile->Append(buf);
srcfile->GetPreallocationStatus(&block_size, &last_allocated_block);
ASSERT_EQ(last_allocated_block, 2UL);
// Write five more blocks at once, ensure we're where we need to be.
buf = std::string(block_size * 5, ' ');
srcfile->PrepareWrite(srcfile->GetFileSize(), buf.size());
srcfile->Append(buf);
srcfile->GetPreallocationStatus(&block_size, &last_allocated_block);
ASSERT_EQ(last_allocated_block, 7UL);

@ -0,0 +1,225 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same 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 "util/file_reader_writer.h"
#include <algorithm>
#include "port/port.h"
#include "util/iostats_context_imp.h"
#include "util/random.h"
#include "util/rate_limiter.h"
#include "util/sync_point.h"
namespace rocksdb {
Status SequentialFileReader::Read(size_t n, Slice* result, char* scratch) {
Status s = file_->Read(n, result, scratch);
IOSTATS_ADD(bytes_read, result->size());
return s;
}
Status SequentialFileReader::Skip(uint64_t n) { return file_->Skip(n); }
Status RandomAccessFileReader::Read(uint64_t offset, size_t n, Slice* result,
char* scratch) const {
IOSTATS_TIMER_GUARD(read_nanos);
Status s = file_->Read(offset, n, result, scratch);
IOSTATS_ADD_IF_POSITIVE(bytes_read, result->size());
return s;
}
Status WritableFileWriter::Append(const Slice& data) {
const char* src = data.data();
size_t left = data.size();
Status s;
pending_sync_ = true;
pending_fsync_ = true;
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS2);
writable_file_->PrepareWrite(static_cast<size_t>(GetFileSize()), left);
// if there is no space in the cache, then flush
if (cursize_ + left > capacity_) {
s = Flush();
if (!s.ok()) {
return s;
}
// Increase the buffer size, but capped at 1MB
if (capacity_ < (1 << 20)) {
capacity_ *= 2;
buf_.reset(new char[capacity_]);
}
assert(cursize_ == 0);
}
// if the write fits into the cache, then write to cache
// otherwise do a write() syscall to write to OS buffers.
if (cursize_ + left <= capacity_) {
memcpy(buf_.get() + cursize_, src, left);
cursize_ += left;
} else {
while (left != 0) {
size_t size = RequestToken(left);
{
IOSTATS_TIMER_GUARD(write_nanos);
s = writable_file_->Append(Slice(src, size));
if (!s.ok()) {
return s;
}
}
IOSTATS_ADD(bytes_written, size);
TEST_KILL_RANDOM(rocksdb_kill_odds);
left -= size;
src += size;
}
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
filesize_ += data.size();
return Status::OK();
}
Status WritableFileWriter::Close() {
Status s;
s = Flush(); // flush cache to OS
if (!s.ok()) {
return s;
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
return writable_file_->Close();
}
// write out the cached data to the OS cache
Status WritableFileWriter::Flush() {
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS2);
size_t left = cursize_;
char* src = buf_.get();
while (left != 0) {
size_t size = RequestToken(left);
{
IOSTATS_TIMER_GUARD(write_nanos);
Status s = writable_file_->Append(Slice(src, size));
if (!s.ok()) {
return s;
}
}
IOSTATS_ADD(bytes_written, size);
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS2);
left -= size;
src += size;
}
cursize_ = 0;
writable_file_->Flush();
// 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).
if (bytes_per_sync_ && filesize_ - last_sync_size_ >= bytes_per_sync_) {
writable_file_->RangeSync(last_sync_size_, filesize_ - last_sync_size_);
last_sync_size_ = filesize_;
}
return Status::OK();
}
Status WritableFileWriter::Sync(bool use_fsync) {
Status s = Flush();
if (!s.ok()) {
return s;
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
if (pending_sync_) {
if (use_fsync) {
s = writable_file_->Fsync();
} else {
s = writable_file_->Sync();
}
if (!s.ok()) {
return s;
}
}
TEST_KILL_RANDOM(rocksdb_kill_odds);
pending_sync_ = false;
if (use_fsync) {
pending_fsync_ = false;
}
return Status::OK();
}
Status WritableFileWriter::RangeSync(off_t offset, off_t nbytes) {
IOSTATS_TIMER_GUARD(range_sync_nanos);
return writable_file_->RangeSync(offset, nbytes);
}
size_t WritableFileWriter::RequestToken(size_t bytes) {
Env::IOPriority io_priority;
if (rate_limiter_&&(io_priority = writable_file_->GetIOPriority()) <
Env::IO_TOTAL) {
bytes = std::min(bytes,
static_cast<size_t>(rate_limiter_->GetSingleBurstBytes()));
rate_limiter_->Request(bytes, io_priority);
}
return bytes;
}
Status RandomRWFileAccessor::Write(uint64_t offset, const Slice& data) {
Status s;
pending_sync_ = true;
pending_fsync_ = true;
{
IOSTATS_TIMER_GUARD(write_nanos);
s = random_rw_file_->Write(offset, data);
if (!s.ok()) {
return s;
}
}
IOSTATS_ADD(bytes_written, data.size());
return s;
}
Status RandomRWFileAccessor::Read(uint64_t offset, size_t n, Slice* result,
char* scratch) const {
Status s;
{
IOSTATS_TIMER_GUARD(read_nanos);
s = random_rw_file_->Read(offset, n, result, scratch);
if (!s.ok()) {
return s;
}
}
IOSTATS_ADD_IF_POSITIVE(bytes_read, result->size());
return s;
}
Status RandomRWFileAccessor::Close() { return random_rw_file_->Close(); }
Status RandomRWFileAccessor::Sync(bool use_fsync) {
Status s;
if (pending_sync_) {
if (use_fsync) {
s = random_rw_file_->Fsync();
} else {
s = random_rw_file_->Sync();
}
if (!s.ok()) {
return s;
}
}
if (use_fsync) {
pending_fsync_ = false;
}
pending_sync_ = false;
return s;
}
} // namespace rocksdb

@ -0,0 +1,109 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same 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 "rocksdb/env.h"
namespace rocksdb {
class SequentialFileReader {
private:
std::unique_ptr<SequentialFile> file_;
public:
explicit SequentialFileReader(std::unique_ptr<SequentialFile>&& _file)
: file_(std::move(_file)) {}
Status Read(size_t n, Slice* result, char* scratch);
Status Skip(uint64_t n);
SequentialFile* file() { return file_.get(); }
};
class RandomAccessFileReader : public RandomAccessFile {
private:
std::unique_ptr<RandomAccessFile> file_;
public:
explicit RandomAccessFileReader(std::unique_ptr<RandomAccessFile>&& raf)
: file_(std::move(raf)) {}
Status Read(uint64_t offset, size_t n, Slice* result, char* scratch) const;
RandomAccessFile* file() { return file_.get(); }
};
// Use posix write to write data to a file.
class WritableFileWriter {
private:
std::unique_ptr<WritableFile> writable_file_;
size_t cursize_; // current size of cached data in buf_
size_t capacity_; // max size of buf_
unique_ptr<char[]> buf_; // a buffer to cache writes
uint64_t filesize_;
bool pending_sync_;
bool pending_fsync_;
uint64_t last_sync_size_;
uint64_t bytes_per_sync_;
RateLimiter* rate_limiter_;
public:
explicit WritableFileWriter(std::unique_ptr<WritableFile>&& file,
const EnvOptions& options)
: writable_file_(std::move(file)),
cursize_(0),
capacity_(65536),
buf_(new char[capacity_]),
filesize_(0),
pending_sync_(false),
pending_fsync_(false),
last_sync_size_(0),
bytes_per_sync_(options.bytes_per_sync),
rate_limiter_(options.rate_limiter) {}
~WritableFileWriter() { Flush(); }
Status Append(const Slice& data);
Status Flush();
Status Close();
Status Sync(bool use_fsync);
uint64_t GetFileSize() { return filesize_; }
Status InvalidateCache(size_t offset, size_t length) {
return writable_file_->InvalidateCache(offset, length);
}
WritableFile* writable_file() const { return writable_file_.get(); }
private:
Status RangeSync(off_t offset, off_t nbytes);
size_t RequestToken(size_t bytes);
};
class RandomRWFileAccessor {
private:
std::unique_ptr<RandomRWFile> random_rw_file_;
bool pending_sync_;
bool pending_fsync_;
public:
explicit RandomRWFileAccessor(std::unique_ptr<RandomRWFile>&& f)
: random_rw_file_(std::move(f)),
pending_sync_(false),
pending_fsync_(false) {}
Status Write(uint64_t offset, const Slice& data);
Status Read(uint64_t offset, size_t n, Slice* result, char* scratch) const;
Status Close();
Status Sync(bool use_fsync);
};
} // namespace rocksdb

@ -8,6 +8,7 @@
#include "util/file_util.h"
#include "rocksdb/env.h"
#include "db/filename.h"
#include "util/file_reader_writer.h"
namespace rocksdb {
@ -15,8 +16,12 @@ namespace rocksdb {
Status CopyFile(Env* env, const std::string& source,
const std::string& destination, uint64_t size) {
const EnvOptions soptions;
unique_ptr<SequentialFile> srcfile;
Status s;
unique_ptr<SequentialFileReader> src_reader;
unique_ptr<WritableFileWriter> dest_writer;
{
unique_ptr<SequentialFile> srcfile;
s = env->NewSequentialFile(source, &srcfile, soptions);
unique_ptr<WritableFile> destfile;
if (s.ok()) {
@ -33,6 +38,9 @@ Status CopyFile(Env* env, const std::string& source,
return s;
}
}
src_reader.reset(new SequentialFileReader(std::move(srcfile)));
dest_writer.reset(new WritableFileWriter(std::move(destfile), soptions));
}
char buffer[4096];
Slice slice;
@ -40,13 +48,13 @@ Status CopyFile(Env* env, const std::string& source,
uint64_t bytes_to_read =
std::min(static_cast<uint64_t>(sizeof(buffer)), size);
if (s.ok()) {
s = srcfile->Read(bytes_to_read, &slice, buffer);
s = src_reader->Read(bytes_to_read, &slice, buffer);
}
if (s.ok()) {
if (slice.size() == 0) {
return Status::Corruption("file too small");
}
s = destfile->Append(slice);
s = dest_writer->Append(slice);
}
if (!s.ok()) {
return s;

@ -1408,10 +1408,18 @@ class InMemoryHandler : public WriteBatch::Handler {
void DumpWalFile(std::string wal_file, bool print_header, bool print_values,
LDBCommandExecuteResult* exec_state) {
unique_ptr<SequentialFile> file;
Env* env_ = Env::Default();
EnvOptions soptions;
Status status = env_->NewSequentialFile(wal_file, &file, soptions);
unique_ptr<SequentialFileReader> wal_file_reader;
Status status;
{
unique_ptr<SequentialFile> file;
status = env_->NewSequentialFile(wal_file, &file, soptions);
if (status.ok()) {
wal_file_reader.reset(new SequentialFileReader(std::move(file)));
}
}
if (!status.ok()) {
if (exec_state) {
*exec_state = LDBCommandExecuteResult::Failed("Failed to open WAL file " +
@ -1422,7 +1430,7 @@ void DumpWalFile(std::string wal_file, bool print_header, bool print_values,
}
} else {
StdErrReporter reporter;
log::Reader reader(move(file), &reporter, true, 0);
log::Reader reader(move(wal_file_reader), &reporter, true, 0);
string scratch;
WriteBatch batch;
Slice record;

@ -13,6 +13,7 @@
#include "rocksdb/filter_policy.h"
#include "table/block_based_table_factory.h"
#include "table/table_builder.h"
#include "util/file_reader_writer.h"
#include "util/testharness.h"
#include "util/testutil.h"
@ -53,12 +54,13 @@ void createSST(const std::string& file_name,
opts.table_factory = tf;
std::vector<std::unique_ptr<IntTblPropCollectorFactory> >
int_tbl_prop_collector_factories;
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), EnvOptions()));
tb.reset(opts.table_factory->NewTableBuilder(
TableBuilderOptions(imoptions, ikc, &int_tbl_prop_collector_factories,
CompressionType::kNoCompression, CompressionOptions(),
false),
file.get()));
file_writer.get()));
// Populate slightly more than 1K keys
uint32_t num_keys = 1024;
@ -66,7 +68,7 @@ void createSST(const std::string& file_name,
tb->Add(MakeKey(i), MakeValue(i));
}
tb->Finish();
file->Close();
file_writer->Close();
}
void cleanup(const std::string& file_name) {

@ -24,7 +24,6 @@ SstFileReader::SstFileReader(const std::string& file_path,
output_hex_(output_hex), ioptions_(options_),
internal_comparator_(BytewiseComparator()) {
fprintf(stdout, "Process %s\n", file_path.c_str());
init_result_ = GetTableReader(file_name_);
}
@ -41,10 +40,13 @@ Status SstFileReader::GetTableReader(const std::string& file_path) {
unique_ptr<RandomAccessFile> file;
uint64_t file_size;
Status s = options_.env->NewRandomAccessFile(file_path, &file_, soptions_);
Status s = options_.env->NewRandomAccessFile(file_path, &file, soptions_);
if (s.ok()) {
s = options_.env->GetFileSize(file_path, &file_size);
}
file_.reset(new RandomAccessFileReader(std::move(file)));
if (s.ok()) {
s = ReadFooterFromFile(file_.get(), file_size, &footer);
}
@ -56,7 +58,8 @@ Status SstFileReader::GetTableReader(const std::string& file_path) {
if (magic_number == kPlainTableMagicNumber ||
magic_number == kLegacyPlainTableMagicNumber) {
soptions_.use_mmap_reads = true;
options_.env->NewRandomAccessFile(file_path, &file_, soptions_);
options_.env->NewRandomAccessFile(file_path, &file, soptions_);
file_.reset(new RandomAccessFileReader(std::move(file)));
}
options_.comparator = &internal_comparator_;
// For old sst format, ReadTableProperties might fail but file can be read
@ -68,16 +71,15 @@ Status SstFileReader::GetTableReader(const std::string& file_path) {
}
if (s.ok()) {
s = NewTableReader(ioptions_, soptions_, internal_comparator_,
std::move(file_), file_size, &table_reader_);
s = NewTableReader(ioptions_, soptions_, internal_comparator_, file_size,
&table_reader_);
}
return s;
}
Status SstFileReader::NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
const InternalKeyComparator& internal_comparator, uint64_t file_size,
unique_ptr<TableReader>* table_reader) {
// We need to turn off pre-fetching of index and filter nodes for
// BlockBasedTable
@ -108,7 +110,7 @@ Status SstFileReader::DumpTable(const std::string& out_filename) {
}
Status SstFileReader::ReadTableProperties(uint64_t table_magic_number,
RandomAccessFile* file,
RandomAccessFileReader* file,
uint64_t file_size) {
TableProperties* table_properties = nullptr;
Status s = rocksdb::ReadTableProperties(file, file_size, table_magic_number,

@ -28,6 +28,7 @@
#include "table/format.h"
#include "table/meta_blocks.h"
#include "table/plain_table_factory.h"
#include "util/file_reader_writer.h"
#include "util/ldb_cmd.h"
#include "util/random.h"
#include "util/testharness.h"
@ -56,7 +57,7 @@ class SstFileReader {
// Get the TableReader implementation for the sst file
Status GetTableReader(const std::string& file_path);
Status ReadTableProperties(uint64_t table_magic_number,
RandomAccessFile* file, uint64_t file_size);
RandomAccessFileReader* file, uint64_t file_size);
Status SetTableOptionsByMagicNumber(uint64_t table_magic_number);
Status SetOldTableOptions();
@ -65,7 +66,7 @@ class SstFileReader {
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& soptions,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
uint64_t file_size,
unique_ptr<TableReader>* table_reader);
std::string file_name_;
@ -76,7 +77,7 @@ class SstFileReader {
Status init_result_;
unique_ptr<TableReader> table_reader_;
unique_ptr<RandomAccessFile> file_;
unique_ptr<RandomAccessFileReader> file_;
// options_ and internal_comparator_ will also be used in
// ReadSequential internally (specifically, seek-related operations)
Options options_;

@ -4,10 +4,25 @@
// of patent rights can be found in the PATENTS file in the same directory.
#include "util/sync_point.h"
#include "port/port.h"
#include "util/random.h"
int rocksdb_kill_odds = 0;
#ifndef NDEBUG
namespace rocksdb {
void TestKillRandom(int odds, const std::string& srcfile, int srcline) {
time_t curtime = time(nullptr);
Random r((uint32_t)curtime);
assert(odds > 0);
bool crash = r.OneIn(odds);
if (crash) {
port::Crash(srcfile, srcline);
}
}
SyncPoint* SyncPoint::GetInstance() {
static SyncPoint sync_point;
return &sync_point;

@ -4,6 +4,7 @@
// of patent rights can be found in the PATENTS file in the same directory.
#pragma once
#include <assert.h>
#include <condition_variable>
#include <mutex>
#include <string>
@ -11,6 +12,33 @@
#include <unordered_map>
#include <vector>
// This is only set from db_stress.cc and for testing only.
// If non-zero, kill at various points in source code with probability 1/this
extern int rocksdb_kill_odds;
#ifdef NDEBUG
// empty in release build
#define TEST_KILL_RANDOM(rocksdb_kill_odds)
#else
namespace rocksdb {
// Kill the process with probablity 1/odds for testing.
extern void TestKillRandom(int odds, const std::string& srcfile, int srcline);
// To avoid crashing always at some frequently executed codepaths (during
// kill random test), use this factor to reduce odds
#define REDUCE_ODDS 2
#define REDUCE_ODDS2 4
#define TEST_KILL_RANDOM(rocksdb_kill_odds) \
{ \
if (rocksdb_kill_odds > 0) { \
TestKillRandom(rocksdb_kill_odds, __FILE__, __LINE__); \
} \
}
} // namespace rocksdb
#endif
#ifdef NDEBUG
#define TEST_SYNC_POINT(x)
#define TEST_SYNC_POINT_CALLBACK(x, y)

@ -10,6 +10,7 @@
#include "util/testutil.h"
#include "port/port.h"
#include "util/file_reader_writer.h"
#include "util/random.h"
namespace rocksdb {
@ -107,5 +108,20 @@ const Comparator* Uint64Comparator() {
return uint64comp;
}
WritableFileWriter* GetWritableFileWriter(WritableFile* wf) {
unique_ptr<WritableFile> file(wf);
return new WritableFileWriter(std::move(file), EnvOptions());
}
RandomAccessFileReader* GetRandomAccessFileReader(RandomAccessFile* raf) {
unique_ptr<RandomAccessFile> file(raf);
return new RandomAccessFileReader(std::move(file));
}
SequentialFileReader* GetSequentialFileReader(SequentialFile* se) {
unique_ptr<SequentialFile> file(se);
return new SequentialFileReader(std::move(file));
}
} // namespace test
} // namespace rocksdb

@ -19,6 +19,9 @@
#include "util/random.h"
namespace rocksdb {
class SequentialFile;
class SequentialFileReader;
namespace test {
// Store in *dst a random string of length "len" and return a Slice that
@ -159,6 +162,11 @@ class VectorIterator : public Iterator {
std::vector<std::string> values_;
size_t current_;
};
extern WritableFileWriter* GetWritableFileWriter(WritableFile* wf);
extern RandomAccessFileReader* GetRandomAccessFileReader(RandomAccessFile* raf);
extern SequentialFileReader* GetSequentialFileReader(SequentialFile* se);
} // namespace test
} // namespace rocksdb

@ -14,6 +14,7 @@
#include "util/channel.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/file_reader_writer.h"
#include "util/logging.h"
#include "util/string_util.h"
#include "rocksdb/transaction_log.h"
@ -1105,7 +1106,10 @@ Status BackupEngineImpl::GetLatestBackupFileContents(uint32_t* latest_backup) {
char buf[11];
Slice data;
s = file->Read(10, &data, buf);
unique_ptr<SequentialFileReader> file_reader(
new SequentialFileReader(std::move(file)));
s = file_reader->Read(10, &data, buf);
if (!s.ok() || data.size() == 0) {
return s.ok() ? Status::Corruption("Latest backup file corrupted") : s;
}
@ -1137,14 +1141,16 @@ Status BackupEngineImpl::PutLatestBackupFileContents(uint32_t latest_backup) {
return s;
}
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options));
char file_contents[10];
int len = sprintf(file_contents, "%u\n", latest_backup);
s = file->Append(Slice(file_contents, len));
s = file_writer->Append(Slice(file_contents, len));
if (s.ok() && options_.sync) {
file->Sync();
file_writer->Sync(false);
}
if (s.ok()) {
s = file->Close();
s = file_writer->Close();
}
if (s.ok()) {
// atomically replace real file with new tmp
@ -1187,6 +1193,10 @@ Status BackupEngineImpl::CopyFile(
return s;
}
unique_ptr<WritableFileWriter> dest_writer(
new WritableFileWriter(std::move(dst_file), env_options));
unique_ptr<SequentialFileReader> src_reader(
new SequentialFileReader(std::move(src_file)));
unique_ptr<char[]> buf(new char[copy_file_buffer_size_]);
Slice data;
@ -1196,7 +1206,7 @@ Status BackupEngineImpl::CopyFile(
}
size_t buffer_to_read = (copy_file_buffer_size_ < size_limit) ?
copy_file_buffer_size_ : size_limit;
s = src_file->Read(buffer_to_read, &data, buf.get());
s = src_reader->Read(buffer_to_read, &data, buf.get());
size_limit -= data.size();
if (!s.ok()) {
@ -1210,14 +1220,14 @@ Status BackupEngineImpl::CopyFile(
*checksum_value = crc32c::Extend(*checksum_value, data.data(),
data.size());
}
s = dst_file->Append(data);
s = dest_writer->Append(data);
if (rate_limiter != nullptr) {
rate_limiter->ReportAndWait(data.size());
}
} while (s.ok() && data.size() > 0 && size_limit > 0);
if (s.ok() && sync) {
s = dst_file->Sync();
s = dest_writer->Sync(false);
}
return s;
@ -1358,6 +1368,8 @@ Status BackupEngineImpl::CalculateChecksum(const std::string& src, Env* src_env,
return s;
}
unique_ptr<SequentialFileReader> src_reader(
new SequentialFileReader(std::move(src_file)));
std::unique_ptr<char[]> buf(new char[copy_file_buffer_size_]);
Slice data;
@ -1367,7 +1379,7 @@ Status BackupEngineImpl::CalculateChecksum(const std::string& src, Env* src_env,
}
size_t buffer_to_read = (copy_file_buffer_size_ < size_limit) ?
copy_file_buffer_size_ : size_limit;
s = src_file->Read(buffer_to_read, &data, buf.get());
s = src_reader->Read(buffer_to_read, &data, buf.get());
if (!s.ok()) {
return s;
@ -1522,9 +1534,11 @@ Status BackupEngineImpl::BackupMeta::LoadFromFile(
return s;
}
unique_ptr<SequentialFileReader> backup_meta_reader(
new SequentialFileReader(std::move(backup_meta_file)));
unique_ptr<char[]> buf(new char[max_backup_meta_file_size_ + 1]);
Slice data;
s = backup_meta_file->Read(max_backup_meta_file_size_, &data, buf.get());
s = backup_meta_reader->Read(max_backup_meta_file_size_, &data, buf.get());
if (!s.ok() || data.size() == max_backup_meta_file_size_) {
return s.ok() ? Status::Corruption("File size too big") : s;

@ -16,6 +16,7 @@
#include "rocksdb/types.h"
#include "rocksdb/transaction_log.h"
#include "rocksdb/utilities/backupable_db.h"
#include "util/file_reader_writer.h"
#include "util/testharness.h"
#include "util/random.h"
#include "util/mutexlock.h"
@ -292,11 +293,12 @@ class FileManager : public EnvWrapper {
if (!s.ok()) {
return s;
}
RandomRWFileAccessor accessor(std::move(file));
for (uint64_t i = 0; s.ok() && i < bytes_to_corrupt; ++i) {
std::string tmp;
// write one random byte to a random position
s = file->Write(rnd_.Next() % size, test::RandomString(&rnd_, 1, &tmp));
s = accessor.Write(rnd_.Next() % size,
test::RandomString(&rnd_, 1, &tmp));
}
return s;
}

Loading…
Cancel
Save