Add path to WritableFileWriter. (#4039)

Summary:
We want to sample the file I/O issued by RocksDB and report the function calls. This requires us to include the file paths otherwise it's hard to tell what has been going on.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4039

Differential Revision: D8670178

Pulled By: riversand963

fbshipit-source-id: 97ee806d1c583a2983e28e213ee764dc6ac28f7a
main
Yanqin Jin 6 years ago committed by Facebook Github Bot
parent f1f5ba085f
commit bb5dcea98e
  1. 4
      db/builder.cc
  2. 6
      db/c.cc
  3. 4
      db/compaction_iterator.cc
  4. 5
      db/compaction_iterator_test.cc
  5. 5
      db/compaction_job.cc
  6. 2
      db/compaction_job_test.cc
  7. 3
      db/db_impl.cc
  8. 2
      db/db_impl.h
  9. 14
      db/db_impl_open.cc
  10. 16
      db/db_impl_write.cc
  11. 2
      db/db_wal_test.cc
  12. 2
      db/flush_job_test.cc
  13. 5
      db/log_test.cc
  14. 4
      db/range_del_aggregator.cc
  15. 5
      db/snapshot_checker.h
  16. 3
      db/table_properties_collector_test.cc
  17. 11
      db/version_set.cc
  18. 2
      db/version_set_test.cc
  19. 4
      db/wal_manager_test.cc
  20. 2
      include/rocksdb/slice.h
  21. 2
      options/options_parser.cc
  22. 23
      port/win/env_win.cc
  23. 5
      port/win/env_win.h
  24. 2
      table/block_builder.h
  25. 8
      table/block_test.cc
  26. 22
      table/cuckoo_table_builder_test.cc
  27. 4
      table/cuckoo_table_reader_test.cc
  28. 3
      table/data_block_hash_index_test.cc
  29. 4
      table/index_builder.cc
  30. 12
      table/index_builder.h
  31. 2
      table/mock_table.cc
  32. 2
      table/sst_file_writer.cc
  33. 3
      table/table_reader_bench.cc
  34. 14
      table/table_test.cc
  35. 9
      tools/db_stress.cc
  36. 13
      tools/sst_dump_test.cc
  37. 3
      tools/sst_dump_tool.cc
  38. 4
      tools/trace_analyzer_test.cc
  39. 7
      tools/trace_analyzer_tool.cc
  40. 3
      util/coding.h
  41. 7
      util/file_reader_writer.h
  42. 13
      util/file_reader_writer_test.cc
  43. 6
      util/file_util.cc
  44. 5
      util/testutil.cc
  45. 3
      util/testutil.h
  46. 2
      utilities/backupable/backupable_db.cc
  47. 6
      utilities/blob_db/blob_db_impl.cc
  48. 3
      utilities/env_mirror.cc
  49. 3
      utilities/simulator_cache/sim_cache.cc
  50. 3
      utilities/trace/file_trace_reader_writer.cc

@ -121,8 +121,8 @@ Status BuildTable(
file->SetIOPriority(io_priority);
file->SetWriteLifeTimeHint(write_hint);
file_writer.reset(new WritableFileWriter(std::move(file), env_options,
ioptions.statistics));
file_writer.reset(new WritableFileWriter(
std::move(file), fname, env_options, ioptions.statistics));
builder = NewTableBuilder(
ioptions, mutable_cf_options, internal_comparator,
int_tbl_prop_collector_factories, column_family_id,

@ -2433,11 +2433,13 @@ void rocksdb_options_set_max_write_buffer_number_to_maintain(
opt->rep.max_write_buffer_number_to_maintain = n;
}
void rocksdb_options_set_enable_pipelined_write(rocksdb_options_t* opt, unsigned char v) {
void rocksdb_options_set_enable_pipelined_write(rocksdb_options_t* opt,
unsigned char v) {
opt->rep.enable_pipelined_write = v;
}
void rocksdb_options_set_max_subcompactions(rocksdb_options_t* opt, uint32_t n) {
void rocksdb_options_set_max_subcompactions(rocksdb_options_t* opt,
uint32_t n) {
opt->rep.max_subcompactions = n;
}

@ -18,8 +18,8 @@ CompactionIterator::CompactionIterator(
SequenceNumber earliest_write_conflict_snapshot,
const SnapshotChecker* snapshot_checker, Env* env,
bool report_detailed_time, bool expect_valid_internal_key,
RangeDelAggregator* range_del_agg,
const Compaction* compaction, const CompactionFilter* compaction_filter,
RangeDelAggregator* range_del_agg, const Compaction* compaction,
const CompactionFilter* compaction_filter,
const std::atomic<bool>* shutting_down,
const SequenceNumber preserve_deletes_seqnum)
: CompactionIterator(

@ -247,9 +247,8 @@ class CompactionIteratorTest : public testing::TestWithParam<bool> {
c_iter_.reset(new CompactionIterator(
iter_.get(), cmp_, merge_helper_.get(), last_sequence, &snapshots_,
earliest_write_conflict_snapshot, snapshot_checker_.get(),
Env::Default(), false /* report_detailed_time */,
false, range_del_agg_.get(), std::move(compaction), filter,
&shutting_down_));
Env::Default(), false /* report_detailed_time */, false,
range_del_agg_.get(), std::move(compaction), filter, &shutting_down_));
}
void AddSnapshot(SequenceNumber snapshot,

@ -1464,8 +1464,9 @@ Status CompactionJob::OpenCompactionOutputFile(
writable_file->SetWriteLifeTimeHint(write_hint_);
writable_file->SetPreallocationBlockSize(static_cast<size_t>(
sub_compact->compaction->OutputFilePreallocationSize()));
sub_compact->outfile.reset(new WritableFileWriter(
std::move(writable_file), env_options_, db_options_.statistics.get()));
sub_compact->outfile.reset(
new WritableFileWriter(std::move(writable_file), fname, env_options_,
db_options_.statistics.get()));
// If the Column family flag is to only optimize filters for hits,
// we can skip creating filters if this is the bottommost_level where

@ -205,7 +205,7 @@ class CompactionJobTest : public testing::Test {
manifest, &file, env_->OptimizeForManifestWrite(env_options_));
ASSERT_OK(s);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options_));
new WritableFileWriter(std::move(file), manifest, env_options_));
{
log::Writer log(std::move(file_writer), 0, false);
std::string record;

@ -3143,7 +3143,8 @@ Status DBImpl::TraceIteratorSeek(const uint32_t& cf_id, const Slice& key) {
return s;
}
Status DBImpl::TraceIteratorSeekForPrev(const uint32_t& cf_id, const Slice& key) {
Status DBImpl::TraceIteratorSeekForPrev(const uint32_t& cf_id,
const Slice& key) {
Status s;
if (tracer_) {
InstrumentedMutexLock lock(&trace_mutex_);

@ -22,9 +22,9 @@
#include "db/column_family.h"
#include "db/compaction_job.h"
#include "db/dbformat.h"
#include "db/external_sst_file_ingestion_job.h"
#include "db/error_handler.h"
#include "db/event_helpers.h"
#include "db/external_sst_file_ingestion_job.h"
#include "db/flush_job.h"
#include "db/flush_scheduler.h"
#include "db/internal_stats.h"

@ -232,7 +232,7 @@ Status DBImpl::NewDB() {
file->SetPreallocationBlockSize(
immutable_db_options_.manifest_preallocation_size);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options));
new WritableFileWriter(std::move(file), manifest, env_options));
log::Writer log(std::move(file_writer), 0, false);
std::string record;
new_db.EncodeTo(&record);
@ -1075,10 +1075,10 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname,
impl->immutable_db_options_.env->OptimizeForLogWrite(
soptions, BuildDBOptions(impl->immutable_db_options_,
impl->mutable_db_options_));
s = NewWritableFile(
impl->immutable_db_options_.env,
LogFileName(impl->immutable_db_options_.wal_dir, new_log_number),
&lfile, opt_env_options);
std::string log_fname =
LogFileName(impl->immutable_db_options_.wal_dir, new_log_number);
s = NewWritableFile(impl->immutable_db_options_.env, log_fname, &lfile,
opt_env_options);
if (s.ok()) {
lfile->SetWriteLifeTimeHint(write_hint);
lfile->SetPreallocationBlockSize(
@ -1086,8 +1086,8 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname,
{
InstrumentedMutexLock wl(&impl->log_write_mutex_);
impl->logfile_number_ = new_log_number;
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(lfile), opt_env_options));
unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
std::move(lfile), log_fname, opt_env_options));
impl->logs_.emplace_back(
new_log_number,
new log::Writer(

@ -1319,6 +1319,8 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context,
auto write_hint = CalculateWALWriteHint();
mutex_.Unlock();
{
std::string log_fname =
LogFileName(immutable_db_options_.wal_dir, new_log_number);
if (creating_new_log) {
EnvOptions opt_env_opt =
env_->OptimizeForLogWrite(env_options_, db_options);
@ -1326,14 +1328,12 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context,
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"reusing log %" PRIu64 " from recycle list\n",
recycle_log_number);
s = env_->ReuseWritableFile(
LogFileName(immutable_db_options_.wal_dir, new_log_number),
LogFileName(immutable_db_options_.wal_dir, recycle_log_number),
&lfile, opt_env_opt);
std::string old_log_fname =
LogFileName(immutable_db_options_.wal_dir, recycle_log_number);
s = env_->ReuseWritableFile(log_fname, old_log_fname, &lfile,
opt_env_opt);
} else {
s = NewWritableFile(
env_, LogFileName(immutable_db_options_.wal_dir, new_log_number),
&lfile, opt_env_opt);
s = NewWritableFile(env_, log_fname, &lfile, opt_env_opt);
}
if (s.ok()) {
// Our final size should be less than write_buffer_size
@ -1344,7 +1344,7 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context,
lfile->SetPreallocationBlockSize(preallocate_block_size);
lfile->SetWriteLifeTimeHint(write_hint);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(lfile), opt_env_opt));
new WritableFileWriter(std::move(lfile), log_fname, opt_env_opt));
new_log = new log::Writer(
std::move(file_writer), new_log_number,
immutable_db_options_.recycle_log_file_num > 0, manual_wal_flush_);

@ -815,7 +815,7 @@ class RecoveryTestHelper {
unique_ptr<WritableFile> file;
ASSERT_OK(db_options.env->NewWritableFile(fname, &file, env_options));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options));
new WritableFileWriter(std::move(file), fname, env_options));
current_log_writer.reset(
new log::Writer(std::move(file_writer), current_log_number,
db_options.recycle_log_file_num > 0));

@ -62,7 +62,7 @@ class FlushJobTest : public testing::Test {
manifest, &file, env_->OptimizeForManifestWrite(env_options_));
ASSERT_OK(s);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), EnvOptions()));
new WritableFileWriter(std::move(file), manifest, EnvOptions()));
{
log::Writer log(std::move(file_writer), 0, false);
std::string record;

@ -159,7 +159,7 @@ class LogTest : public ::testing::TestWithParam<int> {
LogTest()
: reader_contents_(),
dest_holder_(test::GetWritableFileWriter(
new test::StringSink(&reader_contents_))),
new test::StringSink(&reader_contents_), "" /* don't care */)),
source_holder_(test::GetSequentialFileReader(
new StringSource(reader_contents_), "" /* file name */)),
writer_(std::move(dest_holder_), 123, GetParam()),
@ -718,7 +718,8 @@ TEST_P(LogTest, Recycle) {
Write("xxxxxxxxxxxxxxxx");
}
unique_ptr<WritableFileWriter> dest_holder(test::GetWritableFileWriter(
new test::OverwritingStringSink(get_reader_contents())));
new test::OverwritingStringSink(get_reader_contents()),
"" /* don't care */));
Writer recycle_writer(std::move(dest_holder), 123, true);
recycle_writer.AddRecord(Slice("foooo"));
recycle_writer.AddRecord(Slice("bar"));

@ -76,7 +76,9 @@ class UncollapsedRangeDelMap : public RangeDelMap {
return false;
}
void AddTombstone(RangeTombstone tombstone) override { rep_.emplace(tombstone); }
void AddTombstone(RangeTombstone tombstone) override {
rep_.emplace(tombstone);
}
size_t Size() const override { return rep_.size(); }

@ -19,8 +19,9 @@ class SnapshotChecker {
class DisableGCSnapshotChecker : public SnapshotChecker {
public:
virtual ~DisableGCSnapshotChecker() {}
virtual bool IsInSnapshot(SequenceNumber /*sequence*/,
SequenceNumber /*snapshot_sequence*/) const override {
virtual bool IsInSnapshot(
SequenceNumber /*sequence*/,
SequenceNumber /*snapshot_sequence*/) const override {
// By returning false, we prevent all the values from being GCed
return false;
}

@ -46,7 +46,8 @@ void MakeBuilder(const Options& options, const ImmutableCFOptions& ioptions,
std::unique_ptr<WritableFileWriter>* writable,
std::unique_ptr<TableBuilder>* builder) {
unique_ptr<WritableFile> wf(new test::StringSink);
writable->reset(new WritableFileWriter(std::move(wf), EnvOptions()));
writable->reset(
new WritableFileWriter(std::move(wf), "" /* don't care */, EnvOptions()));
int unknown_level = -1;
builder->reset(NewTableBuilder(
ioptions, moptions, internal_comparator, int_tbl_prop_collector_factories,

@ -2901,16 +2901,17 @@ Status VersionSet::ProcessManifestWrites(
// create new manifest file
ROCKS_LOG_INFO(db_options_->info_log, "Creating manifest %" PRIu64 "\n",
pending_manifest_file_number_);
std::string descriptor_fname =
DescriptorFileName(dbname_, pending_manifest_file_number_);
unique_ptr<WritableFile> descriptor_file;
s = NewWritableFile(
env_, DescriptorFileName(dbname_, pending_manifest_file_number_),
&descriptor_file, opt_env_opts);
s = NewWritableFile(env_, descriptor_fname, &descriptor_file,
opt_env_opts);
if (s.ok()) {
descriptor_file->SetPreallocationBlockSize(
db_options_->manifest_preallocation_size);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(descriptor_file), opt_env_opts));
unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
std::move(descriptor_file), descriptor_fname, opt_env_opts));
descriptor_log_.reset(
new log::Writer(std::move(file_writer), 0, false));
s = WriteSnapshot(descriptor_log_.get());

@ -566,7 +566,7 @@ class ManifestWriterTest : public testing::Test {
manifest, &file, env_->OptimizeForManifestWrite(env_options_));
ASSERT_OK(s);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options_));
new WritableFileWriter(std::move(file), manifest, env_options_));
{
log::Writer log(std::move(file_writer), 0, false);
std::string record;

@ -79,7 +79,7 @@ class WalManagerTest : public testing::Test {
unique_ptr<WritableFile> file;
ASSERT_OK(env_->NewWritableFile(fname, &file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), env_options_));
new WritableFileWriter(std::move(file), fname, env_options_));
current_log_writer_.reset(new log::Writer(std::move(file_writer), 0, false));
}
@ -130,7 +130,7 @@ TEST_F(WalManagerTest, ReadFirstRecordCache) {
ASSERT_EQ(s, 0U);
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), EnvOptions()));
new WritableFileWriter(std::move(file), path, EnvOptions()));
log::Writer writer(std::move(file_writer), 1,
db_options_.recycle_log_file_num > 0);
WriteBatch batch;

@ -48,7 +48,7 @@ class Slice {
#ifdef __cpp_lib_string_view
// Create a slice that refers to the same contents as "sv"
/* implicit */
Slice(std::string_view sv) : data_(sv.data()), size_(sv.size()) { }
Slice(std::string_view sv) : data_(sv.data()), size_(sv.size()) {}
#endif
// Create a slice that refers to s[0,strlen(s)-1]

@ -49,7 +49,7 @@ Status PersistRocksDBOptions(const DBOptions& db_opt,
return s;
}
unique_ptr<WritableFileWriter> writable;
writable.reset(new WritableFileWriter(std::move(wf), EnvOptions(),
writable.reset(new WritableFileWriter(std::move(wf), file_name, EnvOptions(),
nullptr /* statistics */));
std::string options_file_content;

@ -716,30 +716,27 @@ Status WinEnvIO::LinkFile(const std::string& src,
return result;
}
Status WinEnvIO::NumFileLinks(const std::string& fname,
uint64_t* count) {
Status WinEnvIO::NumFileLinks(const std::string& fname, uint64_t* count) {
Status s;
HANDLE handle = ::CreateFileA(fname.c_str(), 0,
FILE_SHARE_DELETE | FILE_SHARE_READ | FILE_SHARE_WRITE,
NULL,
OPEN_EXISTING,
FILE_FLAG_BACKUP_SEMANTICS,
NULL);
HANDLE handle = ::CreateFileA(
fname.c_str(), 0, FILE_SHARE_DELETE | FILE_SHARE_READ | FILE_SHARE_WRITE,
NULL, OPEN_EXISTING, FILE_FLAG_BACKUP_SEMANTICS, NULL);
if (INVALID_HANDLE_VALUE == handle) {
auto lastError = GetLastError();
s = IOErrorFromWindowsError(
"NumFileLinks: " + fname, lastError);
s = IOErrorFromWindowsError("NumFileLinks: " + fname, lastError);
return s;
}
UniqueCloseHandlePtr handle_guard(handle, CloseHandleFunc);
FILE_STANDARD_INFO standard_info;
if (0 != GetFileInformationByHandleEx(handle, FileStandardInfo,
&standard_info, sizeof(standard_info))) {
if (0 != GetFileInformationByHandleEx(handle, FileStandardInfo,
&standard_info,
sizeof(standard_info))) {
*count = standard_info.NumberOfLinks;
} else {
auto lastError = GetLastError();
s = IOErrorFromWindowsError("GetFileInformationByHandleEx: " + fname, lastError);
s = IOErrorFromWindowsError("GetFileInformationByHandleEx: " + fname,
lastError);
}
return s;
}

@ -145,7 +145,7 @@ public:
const std::string& target);
virtual Status NumFileLinks(const std::string& /*fname*/,
uint64_t* /*count*/);
uint64_t* /*count*/);
virtual Status AreFilesSame(const std::string& first,
const std::string& second, bool* res);
@ -271,8 +271,7 @@ public:
Status LinkFile(const std::string& src,
const std::string& target) override;
Status NumFileLinks(const std::string& fname,
uint64_t* count) override;
Status NumFileLinks(const std::string& fname, uint64_t* count) override;
Status AreFilesSame(const std::string& first,
const std::string& second, bool* res) override;

@ -60,7 +60,7 @@ class BlockBuilder {
private:
const int block_restart_interval_;
//TODO(myabandeh): put it into a separate IndexBlockBuilder
// TODO(myabandeh): put it into a separate IndexBlockBuilder
const bool use_delta_encoding_;
// Refer to BlockIter::DecodeCurrentValue for format of delta encoded values
const bool use_value_delta_encoding_;

@ -70,10 +70,10 @@ void GenerateRandomKVs(std::vector<std::string> *keys,
// Same as GenerateRandomKVs but the values are BlockHandle
void GenerateRandomKBHs(std::vector<std::string> *keys,
std::vector<BlockHandle> *values, const int from,
const int len, const int step = 1,
const int padding_size = 0,
const int keys_share_prefix = 1) {
std::vector<BlockHandle> *values, const int from,
const int len, const int step = 1,
const int padding_size = 0,
const int keys_share_prefix = 1) {
Random rnd(302);
uint64_t offset = 0;

@ -156,7 +156,7 @@ TEST_F(CuckooBuilderTest, SuccessWithEmptyFile) {
fname = test::PerThreadDBPath("EmptyFile");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, 4, 100,
BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -192,7 +192,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionFullKey) {
fname = test::PerThreadDBPath("NoCollisionFullKey");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -240,7 +240,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionFullKey) {
fname = test::PerThreadDBPath("WithCollisionFullKey");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -289,7 +289,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionAndCuckooBlock) {
fname = test::PerThreadDBPath("WithCollisionFullKey2");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(
file_writer.get(), kHashTableRatio, num_hash_fun, 100,
BytewiseComparator(), cuckoo_block_size, false, false, GetSliceHash,
@ -342,7 +342,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKey) {
fname = test::PerThreadDBPath("WithCollisionPathFullKey");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -392,7 +392,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKeyAndCuckooBlock) {
fname = test::PerThreadDBPath("WithCollisionPathFullKeyAndCuckooBlock");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 2, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -435,7 +435,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionUserKey) {
fname = test::PerThreadDBPath("NoCollisionUserKey");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -479,7 +479,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionUserKey) {
fname = test::PerThreadDBPath("WithCollisionUserKey");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -525,7 +525,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathUserKey) {
fname = test::PerThreadDBPath("WithCollisionPathUserKey");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
2, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -570,7 +570,7 @@ TEST_F(CuckooBuilderTest, FailWhenCollisionPathTooLong) {
fname = test::PerThreadDBPath("WithCollisionPathUserKey");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
2, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,
@ -598,7 +598,7 @@ TEST_F(CuckooBuilderTest, FailWhenSameKeyInserted) {
fname = test::PerThreadDBPath("FailWhenSameKeyInserted");
ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), EnvOptions()));
new WritableFileWriter(std::move(writable_file), fname, EnvOptions()));
CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
100, BytewiseComparator(), 1, false, false,
GetSliceHash, 0 /* column_family_id */,

@ -96,7 +96,7 @@ class CuckooReaderTest : public testing::Test {
std::unique_ptr<WritableFile> writable_file;
ASSERT_OK(env->NewWritableFile(fname, &writable_file, env_options));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), env_options));
new WritableFileWriter(std::move(writable_file), fname, env_options));
CuckooTableBuilder builder(
file_writer.get(), 0.9, kNumHashFunc, 100, ucomp, 2, false, false,
@ -412,7 +412,7 @@ void WriteFile(const std::vector<std::string>& keys,
std::unique_ptr<WritableFile> writable_file;
ASSERT_OK(env->NewWritableFile(fname, &writable_file, env_options));
unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(writable_file), env_options));
new WritableFileWriter(std::move(writable_file), fname, env_options));
CuckooTableBuilder builder(
file_writer.get(), hash_ratio, 64, 1000, test::Uint64Comparator(), 5,
false, FLAGS_identity_as_first_hash, nullptr, 0 /* column_family_id */,

@ -553,7 +553,8 @@ void TestBoundary(InternalKey& ik1, std::string& v1, InternalKey& ik2,
EnvOptions soptions;
soptions.use_mmap_reads = ioptions.allow_mmap_reads;
file_writer.reset(test::GetWritableFileWriter(new test::StringSink()));
file_writer.reset(
test::GetWritableFileWriter(new test::StringSink(), "" /* don't care */));
unique_ptr<TableBuilder> builder;
std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
int_tbl_prop_collector_factories;

@ -210,7 +210,5 @@ Status PartitionedIndexBuilder::Finish(
}
}
size_t PartitionedIndexBuilder::NumPartitions() const {
return partition_cnt_;
}
size_t PartitionedIndexBuilder::NumPartitions() const { return partition_cnt_; }
} // namespace rocksdb

@ -181,9 +181,7 @@ class ShortenedIndexBuilder : public IndexBuilder {
return Status::OK();
}
virtual size_t IndexSize() const override {
return index_size_;
}
virtual size_t IndexSize() const override { return index_size_; }
virtual bool seperator_is_key_plus_seq() override {
return seperator_is_key_plus_seq_;
@ -353,12 +351,8 @@ class PartitionedIndexBuilder : public IndexBuilder {
IndexBlocks* index_blocks,
const BlockHandle& last_partition_block_handle) override;
virtual size_t IndexSize() const override {
return index_size_;
}
size_t TopLevelIndexSize(uint64_t) const {
return top_level_index_size_;
}
virtual size_t IndexSize() const override { return index_size_; }
size_t TopLevelIndexSize(uint64_t) const { return top_level_index_size_; }
size_t NumPartitions() const;
inline bool ShouldCutFilterBlock() {

@ -93,7 +93,7 @@ Status MockTableFactory::CreateMockTable(Env* env, const std::string& fname,
return s;
}
WritableFileWriter file_writer(std::move(file), EnvOptions());
WritableFileWriter file_writer(std::move(file), fname, EnvOptions());
uint32_t id = GetAndWriteNextID(&file_writer);
file_system_.files.insert({id, std::move(file_contents)});

@ -238,7 +238,7 @@ Status SstFileWriter::Open(const std::string& file_path) {
nullptr /* compression_dict */, r->skip_filters, r->column_family_name,
unknown_level);
r->file_writer.reset(
new WritableFileWriter(std::move(sst_file), r->env_options));
new WritableFileWriter(std::move(sst_file), file_path, r->env_options));
// TODO(tec) : If table_factory is using compressed block cache, we will
// be adding the external sst file blocks into it, which is wasteful.

@ -94,7 +94,8 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
std::vector<std::unique_ptr<IntTblPropCollectorFactory> >
int_tbl_prop_collector_factories;
file_writer.reset(new WritableFileWriter(std::move(file), env_options));
file_writer.reset(
new WritableFileWriter(std::move(file), file_name, env_options));
int unknown_level = -1;
tb = opts.table_factory->NewTableBuilder(
TableBuilderOptions(

@ -323,7 +323,8 @@ class TableConstructor: public Constructor {
const stl_wrappers::KVMap& kv_map) override {
Reset();
soptions.use_mmap_reads = ioptions.allow_mmap_reads;
file_writer_.reset(test::GetWritableFileWriter(new test::StringSink()));
file_writer_.reset(test::GetWritableFileWriter(new test::StringSink(),
"" /* don't care */));
unique_ptr<TableBuilder> builder;
std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
int_tbl_prop_collector_factories;
@ -2550,7 +2551,7 @@ TEST_F(PlainTableTest, BasicPlainTableProperties) {
PlainTableFactory factory(plain_table_options);
test::StringSink sink;
unique_ptr<WritableFileWriter> file_writer(
test::GetWritableFileWriter(new test::StringSink()));
test::GetWritableFileWriter(new test::StringSink(), "" /* don't care */));
Options options;
const ImmutableCFOptions ioptions(options);
const MutableCFOptions moptions(options);
@ -3150,7 +3151,8 @@ TEST_F(PrefixTest, PrefixAndWholeKeyTest) {
TEST_P(BlockBasedTableTest, DISABLED_TableWithGlobalSeqno) {
BlockBasedTableOptions bbto = GetBlockBasedTableOptions();
test::StringSink* sink = new test::StringSink();
unique_ptr<WritableFileWriter> file_writer(test::GetWritableFileWriter(sink));
unique_ptr<WritableFileWriter> file_writer(
test::GetWritableFileWriter(sink, "" /* don't care */));
Options options;
options.table_factory.reset(NewBlockBasedTableFactory(bbto));
const ImmutableCFOptions ioptions(options);
@ -3331,7 +3333,8 @@ TEST_P(BlockBasedTableTest, BlockAlignTest) {
BlockBasedTableOptions bbto = GetBlockBasedTableOptions();
bbto.block_align = true;
test::StringSink* sink = new test::StringSink();
unique_ptr<WritableFileWriter> file_writer(test::GetWritableFileWriter(sink));
unique_ptr<WritableFileWriter> file_writer(
test::GetWritableFileWriter(sink, "" /* don't care */));
Options options;
options.compression = kNoCompression;
options.table_factory.reset(NewBlockBasedTableFactory(bbto));
@ -3420,7 +3423,8 @@ TEST_P(BlockBasedTableTest, PropertiesBlockRestartPointTest) {
BlockBasedTableOptions bbto = GetBlockBasedTableOptions();
bbto.block_align = true;
test::StringSink* sink = new test::StringSink();
unique_ptr<WritableFileWriter> file_writer(test::GetWritableFileWriter(sink));
unique_ptr<WritableFileWriter> file_writer(
test::GetWritableFileWriter(sink, "" /* don't care */));
Options options;
options.compression = kNoCompression;

@ -591,7 +591,7 @@ enum RepFactory StringToRepFactory(const char* ctype) {
#ifdef _MSC_VER
#pragma warning(push)
// truncation of constant value on static_cast
#pragma warning(disable: 4309)
#pragma warning(disable : 4309)
#endif
bool GetNextPrefix(const rocksdb::Slice& src, std::string* v) {
std::string ret = src.ToString();
@ -1997,14 +1997,14 @@ class StressTest {
} else if (prefixBound <= prob_op && prob_op < writeBound) {
// OPERATION write
TestPut(thread, write_opts, read_opts, rand_column_families, rand_keys,
value, lock);
value, lock);
} else if (writeBound <= prob_op && prob_op < delBound) {
// OPERATION delete
TestDelete(thread, write_opts, rand_column_families, rand_keys, lock);
} else if (delBound <= prob_op && prob_op < delRangeBound) {
// OPERATION delete range
TestDeleteRange(thread, write_opts, rand_column_families, rand_keys,
lock);
lock);
} else {
// OPERATION iterate
TestIterate(thread, read_opts, rand_column_families, rand_keys);
@ -2022,8 +2022,7 @@ class StressTest {
virtual bool ShouldAcquireMutexOnKey() const { return false; }
virtual std::vector<int> GenerateColumnFamilies(
const int /* num_column_families */,
int rand_column_family) const {
const int /* num_column_families */, int rand_column_family) const {
return {rand_column_family};
}

@ -59,7 +59,7 @@ void createSST(const std::string& file_name,
std::vector<std::unique_ptr<IntTblPropCollectorFactory> >
int_tbl_prop_collector_factories;
std::unique_ptr<WritableFileWriter> file_writer(
new WritableFileWriter(std::move(file), EnvOptions()));
new WritableFileWriter(std::move(file), file_name, EnvOptions()));
std::string column_family_name;
int unknown_level = -1;
tb.reset(opts.table_factory->NewTableBuilder(
@ -92,12 +92,11 @@ void cleanup(const std::string& file_name) {
// Test for sst dump tool "raw" mode
class SSTDumpToolTest : public testing::Test {
std::string testDir_;
public:
public:
BlockBasedTableOptions table_options_;
SSTDumpToolTest() {
testDir_ = test::TmpDir();
}
SSTDumpToolTest() { testDir_ = test::TmpDir(); }
~SSTDumpToolTest() {}
@ -107,9 +106,9 @@ public:
return path;
}
template<std::size_t N>
template <std::size_t N>
void PopulateCommandArgs(const std::string& file_path, const char* command,
char* (&usage)[N]) const {
char* (&usage)[N]) const {
for (int i = 0; i < static_cast<int>(N); ++i) {
usage[i] = new char[optLength];
}

@ -163,7 +163,8 @@ uint64_t SstFileReader::CalculateCompressedTableSize(
unique_ptr<Env> env(NewMemEnv(Env::Default()));
env->NewWritableFile(testFileName, &out_file, soptions_);
unique_ptr<WritableFileWriter> dest_writer;
dest_writer.reset(new WritableFileWriter(std::move(out_file), soptions_));
dest_writer.reset(
new WritableFileWriter(std::move(out_file), testFileName, soptions_));
BlockBasedTableOptions table_options;
table_options.block_size = block_size;
BlockBasedTableFactory block_based_tf(table_options);

@ -37,7 +37,7 @@ namespace rocksdb {
namespace {
static const int kMaxArgCount = 100;
static const size_t kArgBufferSize = 100000;
}
} // namespace
// The helper functions for the test
class TraceAnalyzerTest : public testing::Test {
@ -77,7 +77,7 @@ class TraceAnalyzerTest : public testing::Test {
ASSERT_OK(db_->Write(wo, &batch));
ASSERT_OK(db_->Get(ro, "a", &value));
std::this_thread::sleep_for (std::chrono::seconds(1));
std::this_thread::sleep_for(std::chrono::seconds(1));
db_->Get(ro, "g", &value);
ASSERT_OK(db_->EndTrace());

@ -740,7 +740,8 @@ Status TraceAnalyzer::MakeStatisticCorrelation(TraceStats& stats,
// Process the statistics of QPS
Status TraceAnalyzer::MakeStatisticQPS() {
uint32_t duration = static_cast<uint32_t>((end_time_ - begin_time_) / 1000000);
uint32_t duration =
static_cast<uint32_t>((end_time_ - begin_time_) / 1000000);
int ret;
Status s;
std::vector<std::vector<uint32_t>> type_qps(
@ -1794,5 +1795,5 @@ int trace_analyzer_tool(int argc, char** argv) {
}
} // namespace rocksdb
#endif // Endif of Gflag
#endif // RocksDB LITE
#endif // Endif of Gflag
#endif // RocksDB LITE

@ -64,7 +64,8 @@ extern Slice GetLengthPrefixedSlice(const char* data);
extern Slice GetSliceUntil(Slice* slice, char delimiter);
// Borrowed from https://github.com/facebook/fbthrift/blob/449a5f77f9f9bae72c9eb5e78093247eef185c04/thrift/lib/cpp/util/VarintUtils-inl.h#L202-L208
// Borrowed from
// https://github.com/facebook/fbthrift/blob/449a5f77f9f9bae72c9eb5e78093247eef185c04/thrift/lib/cpp/util/VarintUtils-inl.h#L202-L208
constexpr inline uint64_t i64ToZigzag(const int64_t l) {
return (static_cast<uint64_t>(l) << 1) ^ static_cast<uint64_t>(l >> 63);
}

@ -125,6 +125,7 @@ class RandomAccessFileReader {
class WritableFileWriter {
private:
std::unique_ptr<WritableFile> writable_file_;
std::string file_name_;
AlignedBuffer buf_;
size_t max_buffer_size_;
// Actually written data size can be used for truncate
@ -144,8 +145,10 @@ class WritableFileWriter {
public:
WritableFileWriter(std::unique_ptr<WritableFile>&& file,
const EnvOptions& options, Statistics* stats = nullptr)
const std::string& _file_name, const EnvOptions& options,
Statistics* stats = nullptr)
: writable_file_(std::move(file)),
file_name_(_file_name),
buf_(),
max_buffer_size_(options.writable_file_max_buffer_size),
filesize_(0),
@ -169,6 +172,8 @@ class WritableFileWriter {
~WritableFileWriter() { Close(); }
std::string file_name() const { return file_name_; }
Status Append(const Slice& data);
Status Pad(const size_t pad_bytes);

@ -73,7 +73,7 @@ TEST_F(WritableFileWriterTest, RangeSync) {
env_options.bytes_per_sync = kMb;
unique_ptr<FakeWF> wf(new FakeWF);
unique_ptr<WritableFileWriter> writer(
new WritableFileWriter(std::move(wf), env_options));
new WritableFileWriter(std::move(wf), "" /* don't care */, env_options));
Random r(301);
std::unique_ptr<char[]> large_buf(new char[10 * kMb]);
for (int i = 0; i < 1000; i++) {
@ -154,8 +154,8 @@ TEST_F(WritableFileWriterTest, IncrementalBuffer) {
false,
#endif
no_flush));
unique_ptr<WritableFileWriter> writer(
new WritableFileWriter(std::move(wf), env_options));
unique_ptr<WritableFileWriter> writer(new WritableFileWriter(
std::move(wf), "" /* don't care */, env_options));
std::string target;
for (int i = 0; i < 20; i++) {
@ -209,7 +209,7 @@ TEST_F(WritableFileWriterTest, AppendStatusReturn) {
unique_ptr<FakeWF> wf(new FakeWF());
wf->Setuse_direct_io(true);
unique_ptr<WritableFileWriter> writer(
new WritableFileWriter(std::move(wf), EnvOptions()));
new WritableFileWriter(std::move(wf), "" /* don't care */, EnvOptions()));
ASSERT_OK(writer->Append(std::string(2 * kMb, 'a')));
@ -238,8 +238,9 @@ class ReadaheadRandomAccessFileTest
return std::string(result.data(), result.size());
}
void ResetSourceStr(const std::string& str = "") {
auto write_holder = std::unique_ptr<WritableFileWriter>(
test::GetWritableFileWriter(new test::StringSink(&control_contents_)));
auto write_holder =
std::unique_ptr<WritableFileWriter>(test::GetWritableFileWriter(
new test::StringSink(&control_contents_), "" /* don't care */));
write_holder->Append(Slice(str));
write_holder->Flush();
auto read_holder = std::unique_ptr<RandomAccessFile>(

@ -42,7 +42,8 @@ Status CopyFile(Env* env, const std::string& source,
}
}
src_reader.reset(new SequentialFileReader(std::move(srcfile), source));
dest_writer.reset(new WritableFileWriter(std::move(destfile), soptions));
dest_writer.reset(
new WritableFileWriter(std::move(destfile), destination, soptions));
}
char buffer[4096];
@ -77,7 +78,8 @@ Status CreateFile(Env* env, const std::string& destination,
if (!s.ok()) {
return s;
}
dest_writer.reset(new WritableFileWriter(std::move(destfile), soptions));
dest_writer.reset(
new WritableFileWriter(std::move(destfile), destination, soptions));
return dest_writer->Append(Slice(contents));
}

@ -124,9 +124,10 @@ const Comparator* Uint64Comparator() {
return &uint64comp;
}
WritableFileWriter* GetWritableFileWriter(WritableFile* wf) {
WritableFileWriter* GetWritableFileWriter(WritableFile* wf,
const std::string& fname) {
unique_ptr<WritableFile> file(wf);
return new WritableFileWriter(std::move(file), EnvOptions());
return new WritableFileWriter(std::move(file), fname, EnvOptions());
}
RandomAccessFileReader* GetRandomAccessFileReader(RandomAccessFile* raf) {

@ -183,7 +183,8 @@ class VectorIterator : public InternalIterator {
std::vector<std::string> values_;
size_t current_;
};
extern WritableFileWriter* GetWritableFileWriter(WritableFile* wf);
extern WritableFileWriter* GetWritableFileWriter(WritableFile* wf,
const std::string& fname);
extern RandomAccessFileReader* GetRandomAccessFileReader(RandomAccessFile* raf);

@ -1214,7 +1214,7 @@ Status BackupEngineImpl::CopyOrCreateFile(
}
unique_ptr<WritableFileWriter> dest_writer(
new WritableFileWriter(std::move(dst_file), env_options));
new WritableFileWriter(std::move(dst_file), dst, env_options));
unique_ptr<SequentialFileReader> src_reader;
unique_ptr<char[]> buf;
if (!src.empty()) {

@ -338,7 +338,7 @@ Status BlobDBImpl::CreateWriterLocked(const std::shared_ptr<BlobFile>& bfile) {
}
std::unique_ptr<WritableFileWriter> fwriter;
fwriter.reset(new WritableFileWriter(std::move(wfile), env_options_));
fwriter.reset(new WritableFileWriter(std::move(wfile), fpath, env_options_));
uint64_t boffset = bfile->GetFileSize();
if (debug_level_ >= 2 && boffset) {
@ -562,8 +562,8 @@ class BlobDBImpl::BlobInserter : public WriteBatch::Handler {
return Status::NotSupported(
"Blob DB doesn't support non-default column family.");
}
Status s = blob_db_impl_->PutBlobValue(options_, key, value,
kNoExpiration, &batch_);
Status s = blob_db_impl_->PutBlobValue(options_, key, value, kNoExpiration,
&batch_);
return s;
}

@ -64,7 +64,8 @@ class RandomAccessFileMirror : public RandomAccessFile {
std::string fname;
explicit RandomAccessFileMirror(std::string f) : fname(f) {}
Status Read(uint64_t offset, size_t n, Slice* result, char* scratch) const override {
Status Read(uint64_t offset, size_t n, Slice* result,
char* scratch) const override {
Status as = a_->Read(offset, n, result, scratch);
if (as == Status::OK()) {
char* bscratch = new char[n];

@ -46,7 +46,8 @@ class CacheActivityLogger {
if (!status.ok()) {
return status;
}
file_writer_.reset(new WritableFileWriter(std::move(log_file), env_opts));
file_writer_.reset(new WritableFileWriter(std::move(log_file),
activity_log_file, env_opts));
max_logging_size_ = max_logging_size;
activity_logging_enabled_.store(true);

@ -109,7 +109,8 @@ Status NewFileTraceWriter(Env* env, const EnvOptions& env_options,
}
unique_ptr<WritableFileWriter> file_writer;
file_writer.reset(new WritableFileWriter(std::move(trace_file), env_options));
file_writer.reset(new WritableFileWriter(std::move(trace_file),
trace_filename, env_options));
trace_writer->reset(new FileTraceWriter(std::move(file_writer)));
return s;
}

Loading…
Cancel
Save