diff --git a/Makefile b/Makefile index ca51442d2..d06d6f2a1 100644 --- a/Makefile +++ b/Makefile @@ -116,6 +116,7 @@ TESTS = \ log_test \ manual_compaction_test \ memenv_test \ + mock_env_test \ merge_test \ merger_test \ redis_test \ @@ -511,6 +512,9 @@ $(MEMENVLIBRARY) : $(MEMENVOBJECTS) memenv_test : helpers/memenv/memenv_test.o $(MEMENVOBJECTS) $(LIBOBJECTS) $(TESTHARNESS) $(CXX) helpers/memenv/memenv_test.o $(MEMENVOBJECTS) $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) +mock_env_test : util/mock_env_test.o $(LIBOBJECTS) $(TESTHARNESS) + $(CXX) util/mock_env_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) + manual_compaction_test: util/manual_compaction_test.o $(LIBOBJECTS) $(TESTHARNESS) $(CXX) util/manual_compaction_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) diff --git a/db/db_test.cc b/db/db_test.cc index 62c5e483b..4807ef121 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -45,6 +45,7 @@ #include "util/scoped_arena_iterator.h" #include "util/sync_point.h" #include "util/testutil.h" +#include "util/mock_env.h" namespace rocksdb { @@ -238,6 +239,9 @@ class SpecialEnv : public EnvWrapper { return base_->Sync(); } } + uint64_t GetFileSize() { + return base_->GetFileSize(); + } }; class LogFile : public WritableFile { private: @@ -381,6 +385,7 @@ class DBTest { public: std::string dbname_; + MockEnv* mem_env_; SpecialEnv* env_; DB* db_; std::vector handles_; @@ -404,10 +409,11 @@ class DBTest { DBTest() : option_config_(kDefault), - env_(new SpecialEnv(Env::Default())) { - dbname_ = test::TmpDir() + "/db_test"; - Options options; - options.create_if_missing = true; + mem_env_(!getenv("MEM_ENV") ? nullptr : + new MockEnv(Env::Default())), + env_(new SpecialEnv(mem_env_ ? mem_env_ : Env::Default())) { + dbname_ = test::TmpDir(env_) + "/db_test"; + auto options = CurrentOptions(); ASSERT_OK(DestroyDB(dbname_, options)); db_ = nullptr; Reopen(options); @@ -561,10 +567,10 @@ class DBTest { options.num_levels = 3; break; case kDBLogDir: - options.db_log_dir = test::TmpDir(); + options.db_log_dir = test::TmpDir(env_); break; case kWalDirAndMmapReads: - options.wal_dir = test::TmpDir() + "/wal"; + options.wal_dir = test::TmpDir(env_) + "/wal"; // mmap reads should be orthogonal to WalDir setting, so we piggyback to // this option config to test mmap reads as well options.allow_mmap_reads = true; @@ -633,6 +639,8 @@ class DBTest { if (set_block_based_table_factory) { options.table_factory.reset(NewBlockBasedTableFactory(table_options)); } + options.env = env_; + options.create_if_missing = true; return options; } @@ -712,8 +720,8 @@ class DBTest { ASSERT_OK(DestroyDB(dbname_, options)); } - Status ReadOnlyReopen(Options* options) { - return DB::OpenForReadOnly(*options, dbname_, &db_); + Status ReadOnlyReopen(const Options& options) { + return DB::OpenForReadOnly(options, dbname_, &db_); } Status TryReopen(const Options& options) { @@ -1266,8 +1274,9 @@ TEST(DBTest, ReadOnlyDB) { ASSERT_OK(Put("foo", "v3")); Close(); - Options options; - ASSERT_OK(ReadOnlyReopen(&options)); + auto options = CurrentOptions(); + assert(options.env = env_); + ASSERT_OK(ReadOnlyReopen(options)); ASSERT_EQ("v3", Get("foo")); ASSERT_EQ("v2", Get("bar")); Iterator* iter = db_->NewIterator(ReadOptions()); @@ -1285,7 +1294,7 @@ TEST(DBTest, ReadOnlyDB) { Flush(); Close(); // Now check keys in read only mode. - ASSERT_OK(ReadOnlyReopen(&options)); + ASSERT_OK(ReadOnlyReopen(options)); ASSERT_EQ("v3", Get("foo")); ASSERT_EQ("v2", Get("bar")); } @@ -1299,19 +1308,20 @@ TEST(DBTest, CompactedDB) { options.target_file_size_base = kFileSize; options.max_bytes_for_level_base = 1 << 30; options.compression = kNoCompression; + options = CurrentOptions(options); Reopen(options); // 1 L0 file, use CompactedDB if max_open_files = -1 ASSERT_OK(Put("aaa", DummyString(kFileSize / 2, '1'))); Flush(); Close(); - ASSERT_OK(ReadOnlyReopen(&options)); + ASSERT_OK(ReadOnlyReopen(options)); Status s = Put("new", "value"); ASSERT_EQ(s.ToString(), "Not implemented: Not supported operation in read only mode."); ASSERT_EQ(DummyString(kFileSize / 2, '1'), Get("aaa")); Close(); options.max_open_files = -1; - ASSERT_OK(ReadOnlyReopen(&options)); + ASSERT_OK(ReadOnlyReopen(options)); s = Put("new", "value"); ASSERT_EQ(s.ToString(), "Not implemented: Not supported in compacted db mode."); @@ -1327,7 +1337,7 @@ TEST(DBTest, CompactedDB) { Flush(); Close(); - ASSERT_OK(ReadOnlyReopen(&options)); + ASSERT_OK(ReadOnlyReopen(options)); // Fallback to read-only DB s = Put("new", "value"); ASSERT_EQ(s.ToString(), @@ -1347,7 +1357,7 @@ TEST(DBTest, CompactedDB) { Close(); // CompactedDB - ASSERT_OK(ReadOnlyReopen(&options)); + ASSERT_OK(ReadOnlyReopen(options)); s = Put("new", "value"); ASSERT_EQ(s.ToString(), "Not implemented: Not supported in compacted db mode."); @@ -1493,36 +1503,6 @@ TEST(DBTest, LevelLimitReopen) { ASSERT_OK(TryReopenWithColumnFamilies({"default", "pikachu"}, options)); } -TEST(DBTest, Preallocation) { - const std::string src = dbname_ + "/alloc_test"; - unique_ptr srcfile; - const EnvOptions soptions; - ASSERT_OK(env_->NewWritableFile(src, &srcfile, soptions)); - srcfile->SetPreallocationBlockSize(1024 * 1024); - - // No writes should mean no preallocation - size_t block_size, last_allocated_block; - srcfile->GetPreallocationStatus(&block_size, &last_allocated_block); - ASSERT_EQ(last_allocated_block, 0UL); - - // Small write should preallocate one block - srcfile->Append("test"); - 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->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->Append(buf); - srcfile->GetPreallocationStatus(&block_size, &last_allocated_block); - ASSERT_EQ(last_allocated_block, 7UL); -} - TEST(DBTest, PutDeleteGet) { do { CreateAndReopenWithCF({"pikachu"}, CurrentOptions()); @@ -3146,8 +3126,7 @@ Options DeletionTriggerOptions() { } // anonymous namespace TEST(DBTest, CompactionDeletionTrigger) { - Options options = DeletionTriggerOptions(); - options.create_if_missing = true; + Options options = CurrentOptions(DeletionTriggerOptions()); for (int tid = 0; tid < 2; ++tid) { uint64_t db_size[2]; @@ -3184,8 +3163,7 @@ TEST(DBTest, CompactionDeletionTrigger) { TEST(DBTest, CompactionDeletionTriggerReopen) { for (int tid = 0; tid < 2; ++tid) { uint64_t db_size[3]; - Options options = DeletionTriggerOptions(); - options.create_if_missing = true; + Options options = CurrentOptions(DeletionTriggerOptions()); DestroyAndReopen(options); Random rnd(301); @@ -3474,6 +3452,7 @@ TEST(DBTest, UniversalCompactionSizeAmplification) { options.compaction_style = kCompactionStyleUniversal; options.write_buffer_size = 100<<10; //100KB options.level0_file_num_compaction_trigger = 3; + options = CurrentOptions(options); CreateAndReopenWithCF({"pikachu"}, options); // Trigger compaction if size amplification exceeds 110% @@ -3638,6 +3617,7 @@ TEST(DBTest, CompressedCache) { Options options; options.write_buffer_size = 64*1024; // small write buffer options.statistics = rocksdb::CreateDBStatistics(); + options = CurrentOptions(options); BlockBasedTableOptions table_options; switch (iter) { @@ -3675,6 +3655,7 @@ TEST(DBTest, CompressedCache) { // default column family doesn't have block cache Options no_block_cache_opts; no_block_cache_opts.statistics = options.statistics; + no_block_cache_opts = CurrentOptions(no_block_cache_opts); BlockBasedTableOptions table_options_no_bc; table_options_no_bc.no_block_cache = true; no_block_cache_opts.table_factory.reset( @@ -4587,6 +4568,7 @@ TEST(DBTest, CompactionFilterDeletesAll) { options.compaction_filter_factory = std::make_shared(); options.disable_auto_compactions = true; options.create_if_missing = true; + options = CurrentOptions(options); DestroyAndReopen(options); // put some data @@ -5685,20 +5667,20 @@ TEST(DBTest, ManualCompactionOutputPathId) { } TEST(DBTest, DBOpen_Options) { - std::string dbname = test::TmpDir() + "/db_options_test"; - ASSERT_OK(DestroyDB(dbname, Options())); + Options options = CurrentOptions(); + std::string dbname = test::TmpDir(env_) + "/db_options_test"; + ASSERT_OK(DestroyDB(dbname, options)); // Does not exist, and create_if_missing == false: error DB* db = nullptr; - Options opts; - opts.create_if_missing = false; - Status s = DB::Open(opts, dbname, &db); + options.create_if_missing = false; + Status s = DB::Open(options, dbname, &db); ASSERT_TRUE(strstr(s.ToString().c_str(), "does not exist") != nullptr); ASSERT_TRUE(db == nullptr); // Does not exist, and create_if_missing == true: OK - opts.create_if_missing = true; - s = DB::Open(opts, dbname, &db); + options.create_if_missing = true; + s = DB::Open(options, dbname, &db); ASSERT_OK(s); ASSERT_TRUE(db != nullptr); @@ -5706,16 +5688,16 @@ TEST(DBTest, DBOpen_Options) { db = nullptr; // Does exist, and error_if_exists == true: error - opts.create_if_missing = false; - opts.error_if_exists = true; - s = DB::Open(opts, dbname, &db); + options.create_if_missing = false; + options.error_if_exists = true; + s = DB::Open(options, dbname, &db); ASSERT_TRUE(strstr(s.ToString().c_str(), "exists") != nullptr); ASSERT_TRUE(db == nullptr); // Does exist, and error_if_exists == false: OK - opts.create_if_missing = true; - opts.error_if_exists = false; - s = DB::Open(opts, dbname, &db); + options.create_if_missing = true; + options.error_if_exists = false; + s = DB::Open(options, dbname, &db); ASSERT_OK(s); ASSERT_TRUE(db != nullptr); @@ -5724,57 +5706,56 @@ TEST(DBTest, DBOpen_Options) { } TEST(DBTest, DBOpen_Change_NumLevels) { - Options opts; - opts.create_if_missing = true; - opts.max_background_flushes = 0; - DestroyAndReopen(opts); + Options options = CurrentOptions(); + options.create_if_missing = true; + options.max_background_flushes = 0; + DestroyAndReopen(options); ASSERT_TRUE(db_ != nullptr); - CreateAndReopenWithCF({"pikachu"}, opts); + CreateAndReopenWithCF({"pikachu"}, options); ASSERT_OK(Put(1, "a", "123")); ASSERT_OK(Put(1, "b", "234")); db_->CompactRange(handles_[1], nullptr, nullptr); Close(); - opts.create_if_missing = false; - opts.num_levels = 2; - Status s = TryReopenWithColumnFamilies({"default", "pikachu"}, opts); + options.create_if_missing = false; + options.num_levels = 2; + Status s = TryReopenWithColumnFamilies({"default", "pikachu"}, options); ASSERT_TRUE(strstr(s.ToString().c_str(), "Invalid argument") != nullptr); ASSERT_TRUE(db_ == nullptr); } TEST(DBTest, DestroyDBMetaDatabase) { - std::string dbname = test::TmpDir() + "/db_meta"; + std::string dbname = test::TmpDir(env_) + "/db_meta"; std::string metadbname = MetaDatabaseName(dbname, 0); std::string metametadbname = MetaDatabaseName(metadbname, 0); // Destroy previous versions if they exist. Using the long way. - ASSERT_OK(DestroyDB(metametadbname, Options())); - ASSERT_OK(DestroyDB(metadbname, Options())); - ASSERT_OK(DestroyDB(dbname, Options())); + Options options = CurrentOptions(); + ASSERT_OK(DestroyDB(metametadbname, options)); + ASSERT_OK(DestroyDB(metadbname, options)); + ASSERT_OK(DestroyDB(dbname, options)); // Setup databases - Options opts; - opts.create_if_missing = true; DB* db = nullptr; - ASSERT_OK(DB::Open(opts, dbname, &db)); + ASSERT_OK(DB::Open(options, dbname, &db)); delete db; db = nullptr; - ASSERT_OK(DB::Open(opts, metadbname, &db)); + ASSERT_OK(DB::Open(options, metadbname, &db)); delete db; db = nullptr; - ASSERT_OK(DB::Open(opts, metametadbname, &db)); + ASSERT_OK(DB::Open(options, metametadbname, &db)); delete db; db = nullptr; // Delete databases - ASSERT_OK(DestroyDB(dbname, Options())); + ASSERT_OK(DestroyDB(dbname, options)); // Check if deletion worked. - opts.create_if_missing = false; - ASSERT_TRUE(!(DB::Open(opts, dbname, &db)).ok()); - ASSERT_TRUE(!(DB::Open(opts, metadbname, &db)).ok()); - ASSERT_TRUE(!(DB::Open(opts, metametadbname, &db)).ok()); + options.create_if_missing = false; + ASSERT_TRUE(!(DB::Open(options, dbname, &db)).ok()); + ASSERT_TRUE(!(DB::Open(options, metadbname, &db)).ok()); + ASSERT_TRUE(!(DB::Open(options, metametadbname, &db)).ok()); } // Check that number of files does not grow when writes are dropped @@ -6076,7 +6057,7 @@ TEST(DBTest, BloomFilterRate) { } TEST(DBTest, BloomFilterCompatibility) { - Options options; + Options options = CurrentOptions(); options.statistics = rocksdb::CreateDBStatistics(); BlockBasedTableOptions table_options; table_options.filter_policy.reset(NewBloomFilterPolicy(10, true)); @@ -6105,7 +6086,7 @@ TEST(DBTest, BloomFilterCompatibility) { } TEST(DBTest, BloomFilterReverseCompatibility) { - Options options; + Options options = CurrentOptions(); options.statistics = rocksdb::CreateDBStatistics(); BlockBasedTableOptions table_options; table_options.filter_policy.reset(NewBloomFilterPolicy(10, false)); @@ -6173,7 +6154,7 @@ class WrappedBloom : public FilterPolicy { } // namespace TEST(DBTest, BloomFilterWrapper) { - Options options; + Options options = CurrentOptions(); options.statistics = rocksdb::CreateDBStatistics(); BlockBasedTableOptions table_options; @@ -6241,8 +6222,7 @@ TEST(DBTest, SnapshotFiles) { // copy these files to a new snapshot directory std::string snapdir = dbname_ + ".snapdir/"; - std::string mkdir = "mkdir -p " + snapdir; - ASSERT_EQ(system(mkdir.c_str()), 0); + ASSERT_OK(env_->CreateDirIfMissing(snapdir)); for (unsigned int i = 0; i < files.size(); i++) { // our clients require that GetLiveFiles returns @@ -6270,7 +6250,6 @@ TEST(DBTest, SnapshotFiles) { // release file snapshot dbfull()->DisableFileDeletions(); - // overwrite one key, this key should not appear in the snapshot std::vector extras; for (unsigned int i = 0; i < 1; i++) { @@ -6285,6 +6264,7 @@ TEST(DBTest, SnapshotFiles) { std::vector cf_handles; DB* snapdb; DBOptions opts; + opts.env = env_; opts.create_if_missing = false; Status stat = DB::Open(opts, snapdir, column_families, &cf_handles, &snapdb); @@ -6446,7 +6426,7 @@ std::vector ListTableFiles(Env* env, const std::string& path) { } // namespace TEST(DBTest, FlushOneColumnFamily) { - Options options; + Options options = CurrentOptions(); CreateAndReopenWithCF({"pikachu", "ilya", "muromec", "dobrynia", "nikitich", "alyosha", "popovich"}, options); @@ -6473,7 +6453,7 @@ TEST(DBTest, FlushOneColumnFamily) { // we try to create the smallest number of table files by merging // updates from multiple logs TEST(DBTest, RecoverCheckFileAmountWithSmallWriteBuffer) { - Options options; + Options options = CurrentOptions(); options.write_buffer_size = 5000000; CreateAndReopenWithCF({"pikachu", "dobrynia", "nikitich"}, options); @@ -6528,7 +6508,7 @@ TEST(DBTest, RecoverCheckFileAmountWithSmallWriteBuffer) { // we try to create the smallest number of table files by merging // updates from multiple logs TEST(DBTest, RecoverCheckFileAmount) { - Options options; + Options options = CurrentOptions(); options.write_buffer_size = 100000; CreateAndReopenWithCF({"pikachu", "dobrynia", "nikitich"}, options); @@ -6790,10 +6770,14 @@ TEST(DBTest, TransactionLogIteratorCorruptedLog) { // Corrupt this log to create a gap rocksdb::VectorLogPtr wal_files; ASSERT_OK(dbfull()->GetSortedWalFiles(wal_files)); - const auto logfilePath = dbname_ + "/" + wal_files.front()->PathName(); - ASSERT_EQ( - 0, - truncate(logfilePath.c_str(), wal_files.front()->SizeFileBytes() / 2)); + const auto logfile_path = dbname_ + "/" + wal_files.front()->PathName(); + if (mem_env_) { + mem_env_->Truncate(logfile_path, wal_files.front()->SizeFileBytes() / 2); + } else { + ASSERT_EQ(0, truncate(logfile_path.c_str(), + wal_files.front()->SizeFileBytes() / 2)); + } + // Insert a new entry to a new log file Put("key1025", DummyString(10)); // Try to read from the beginning. Should stop before the gap and read less @@ -7939,6 +7923,7 @@ TEST(DBTest, FIFOCompactionTest) { if (iter == 1) { options.disable_auto_compactions = true; } + options = CurrentOptions(options); DestroyAndReopen(options); Random rnd(301); @@ -8200,7 +8185,7 @@ TEST(DBTest, TableOptionsSanitizeTest) { // block-based table BlockBasedTableOptions to; to.index_type = BlockBasedTableOptions::kHashSearch; - options = Options(); + options = CurrentOptions(); options.create_if_missing = true; options.table_factory.reset(NewBlockBasedTableFactory(to)); ASSERT_TRUE(TryReopen(options).IsInvalidArgument()); @@ -8209,7 +8194,7 @@ TEST(DBTest, TableOptionsSanitizeTest) { } TEST(DBTest, DBIteratorBoundTest) { - Options options; + Options options = CurrentOptions(); options.env = env_; options.create_if_missing = true; diff --git a/util/env_test.cc b/util/env_test.cc index f9c2336db..3bb4fb68c 100644 --- a/util/env_test.cc +++ b/util/env_test.cc @@ -805,6 +805,36 @@ TEST(EnvPosixTest, LogBufferMaxSizeTest) { } } +TEST(EnvPosixTest, Preallocation) { + const std::string src = test::TmpDir() + "/" + "testfile"; + unique_ptr srcfile; + const EnvOptions soptions; + ASSERT_OK(env_->NewWritableFile(src, &srcfile, soptions)); + srcfile->SetPreallocationBlockSize(1024 * 1024); + + // No writes should mean no preallocation + size_t block_size, last_allocated_block; + srcfile->GetPreallocationStatus(&block_size, &last_allocated_block); + ASSERT_EQ(last_allocated_block, 0UL); + + // Small write should preallocate one block + srcfile->Append("test"); + 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->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->Append(buf); + srcfile->GetPreallocationStatus(&block_size, &last_allocated_block); + ASSERT_EQ(last_allocated_block, 7UL); +} + } // namespace rocksdb int main(int argc, char** argv) { diff --git a/util/mock_env.cc b/util/mock_env.cc new file mode 100644 index 000000000..32c202beb --- /dev/null +++ b/util/mock_env.cc @@ -0,0 +1,607 @@ +// Copyright (c) 2014, 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/mock_env.h" +#include +#include +#include +#include "util/rate_limiter.h" + +namespace rocksdb { + +class MemFile { + public: + explicit MemFile(const std::string& fn) : + fn_(fn), refs_(0), size_(0), modified_time_(Now()) {} + + void Ref() { + MutexLock lock(&mutex_); + ++refs_; + } + + void Unref() { + bool do_delete = false; + { + MutexLock lock(&mutex_); + --refs_; + assert(refs_ >= 0); + if (refs_ <= 0) { + do_delete = true; + } + } + + if (do_delete) { + delete this; + } + } + + uint64_t Size() const { + return size_; + } + + void Truncate(size_t size) { + MutexLock lock(&mutex_); + if (size < size_) { + data_.resize(size); + size_ = size; + } + } + + Status Read(uint64_t offset, size_t n, Slice* result, char* scratch) const { + MutexLock lock(&mutex_); + if (offset > Size()) { + return Status::IOError("Offset greater than file size."); + } + const uint64_t available = Size() - offset; + if (n > available) { + n = available; + } + if (n == 0) { + *result = Slice(); + return Status::OK(); + } + if (scratch) { + memcpy(scratch, &(data_[offset]), n); + *result = Slice(scratch, n); + } else { + *result = Slice(&(data_[offset]), n); + } + return Status::OK(); + } + + Status Append(const Slice& data) { + MutexLock lock(&mutex_); + data_.append(data.data(), data.size()); + size_ = data_.size(); + modified_time_ = Now(); + return Status::OK(); + } + + Status Fsync() { + return Status::OK(); + } + + uint64_t ModifiedTime() const { + return modified_time_; + } + + private: + uint64_t Now() { + return std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()).count(); + } + + // Private since only Unref() should be used to delete it. + ~MemFile() { + assert(refs_ == 0); + } + + // No copying allowed. + MemFile(const MemFile&); + void operator=(const MemFile&); + + const std::string fn_; + mutable port::Mutex mutex_; + int refs_; + + std::string data_; + std::atomic size_; + std::atomic modified_time_; +}; + +namespace { + +class SequentialFileImpl : public SequentialFile { + public: + explicit SequentialFileImpl(MemFile* file) : file_(file), pos_(0) { + file_->Ref(); + } + + ~SequentialFileImpl() { + file_->Unref(); + } + + virtual Status Read(size_t n, Slice* result, char* scratch) { + Status s = file_->Read(pos_, n, result, scratch); + if (s.ok()) { + pos_ += result->size(); + } + return s; + } + + virtual Status Skip(uint64_t n) { + if (pos_ > file_->Size()) { + return Status::IOError("pos_ > file_->Size()"); + } + const size_t available = file_->Size() - pos_; + if (n > available) { + n = available; + } + pos_ += n; + return Status::OK(); + } + + private: + MemFile* file_; + size_t pos_; +}; + +class RandomAccessFileImpl : public RandomAccessFile { + public: + explicit RandomAccessFileImpl(MemFile* file) : file_(file) { + file_->Ref(); + } + + ~RandomAccessFileImpl() { + file_->Unref(); + } + + virtual Status Read(uint64_t offset, size_t n, Slice* result, + char* scratch) const { + return file_->Read(offset, n, result, scratch); + } + + private: + MemFile* file_; +}; + +class WritableFileImpl : public WritableFile { + public: + WritableFileImpl(MemFile* file, RateLimiter* rate_limiter) + : file_(file), + rate_limiter_(rate_limiter) { + file_->Ref(); + } + + ~WritableFileImpl() { + file_->Unref(); + } + + virtual Status Append(const Slice& data) { + uint64_t bytes_written = 0; + while (bytes_written < data.size()) { + auto bytes = RequestToken(data.size() - bytes_written); + Status s = file_->Append(Slice(data.data() + bytes_written, bytes)); + if (!s.ok()) { + return s; + } + bytes_written += bytes; + } + return Status::OK(); + } + + virtual Status Close() { + return Status::OK(); + } + + virtual Status Flush() { + return Status::OK(); + } + + virtual Status Sync() { + return file_->Fsync(); + } + + virtual uint64_t GetFileSize() { + return file_->Size(); + } + + private: + inline size_t RequestToken(size_t bytes) { + if (rate_limiter_ && io_priority_ < Env::IO_TOTAL) { + bytes = std::min(bytes, + static_cast(rate_limiter_->GetSingleBurstBytes())); + rate_limiter_->Request(bytes, io_priority_); + } + return bytes; + } + + MemFile* file_; + RateLimiter* rate_limiter_; +}; + +class MockEnvDirectory : public Directory { + public: + virtual Status Fsync() { return Status::OK(); } +}; + +class MockEnvFileLock : public FileLock { + public: + explicit MockEnvFileLock(const std::string& fname) + : fname_(fname) {} + + std::string FileName() const { + return fname_; + } + + private: + const std::string fname_; +}; + +class TestMemLogger : public Logger { + private: + std::unique_ptr file_; + std::atomic_size_t log_size_; + static const uint64_t flush_every_seconds_ = 5; + std::atomic_uint_fast64_t last_flush_micros_; + Env* env_; + bool flush_pending_; + + public: + TestMemLogger(std::unique_ptr f, Env* env, + const InfoLogLevel log_level = InfoLogLevel::ERROR_LEVEL) + : Logger(log_level), + file_(std::move(f)), + log_size_(0), + last_flush_micros_(0), + env_(env), + flush_pending_(false) {} + virtual ~TestMemLogger() { + } + + virtual void Flush() { + if (flush_pending_) { + flush_pending_ = false; + } + last_flush_micros_ = env_->NowMicros(); + } + virtual void Logv(const char* format, va_list ap) { + // We try twice: the first time with a fixed-size stack allocated buffer, + // and the second time with a much larger dynamically allocated buffer. + char buffer[500]; + for (int iter = 0; iter < 2; iter++) { + char* base; + int bufsize; + if (iter == 0) { + bufsize = sizeof(buffer); + base = buffer; + } else { + bufsize = 30000; + base = new char[bufsize]; + } + char* p = base; + char* limit = base + bufsize; + + struct timeval now_tv; + gettimeofday(&now_tv, nullptr); + const time_t seconds = now_tv.tv_sec; + struct tm t; + localtime_r(&seconds, &t); + p += snprintf(p, limit - p, + "%04d/%02d/%02d-%02d:%02d:%02d.%06d ", + t.tm_year + 1900, + t.tm_mon + 1, + t.tm_mday, + t.tm_hour, + t.tm_min, + t.tm_sec, + static_cast(now_tv.tv_usec)); + + // Print the message + if (p < limit) { + va_list backup_ap; + va_copy(backup_ap, ap); + p += vsnprintf(p, limit - p, format, backup_ap); + va_end(backup_ap); + } + + // Truncate to available space if necessary + if (p >= limit) { + if (iter == 0) { + continue; // Try again with larger buffer + } else { + p = limit - 1; + } + } + + // Add newline if necessary + if (p == base || p[-1] != '\n') { + *p++ = '\n'; + } + + assert(p <= limit); + const size_t write_size = p - base; + + file_->Append(Slice(base, write_size)); + flush_pending_ = true; + log_size_ += write_size; + uint64_t now_micros = static_cast(now_tv.tv_sec) * 1000000 + + now_tv.tv_usec; + if (now_micros - last_flush_micros_ >= flush_every_seconds_ * 1000000) { + flush_pending_ = false; + last_flush_micros_ = now_micros; + } + if (base != buffer) { + delete[] base; + } + break; + } + } + size_t GetLogFileSize() const { + return log_size_; + } +}; + +} // Anonymous namespace + +MockEnv::MockEnv(Env* base_env) + : EnvWrapper(base_env) {} + +MockEnv::~MockEnv() { + for (FileSystem::iterator i = file_map_.begin(); i != file_map_.end(); ++i) { + i->second->Unref(); + } +} + + // Partial implementation of the Env interface. +Status MockEnv::NewSequentialFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& soptions) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + if (file_map_.find(fn) == file_map_.end()) { + *result = NULL; + return Status::IOError(fn, "File not found"); + } + auto* f = file_map_[fn]; + result->reset(new SequentialFileImpl(f)); + return Status::OK(); +} + +Status MockEnv::NewRandomAccessFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& soptions) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + if (file_map_.find(fn) == file_map_.end()) { + *result = NULL; + return Status::IOError(fn, "File not found"); + } + auto* f = file_map_[fn]; + result->reset(new RandomAccessFileImpl(f)); + return Status::OK(); +} + +Status MockEnv::NewWritableFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& env_options) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + if (file_map_.find(fn) != file_map_.end()) { + DeleteFileInternal(fn); + } + MemFile* file = new MemFile(fn); + file->Ref(); + file_map_[fn] = file; + + result->reset(new WritableFileImpl(file, env_options.rate_limiter)); + return Status::OK(); +} + +Status MockEnv::NewRandomRWFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& options) { + return Status::OK(); +} + +Status MockEnv::NewDirectory(const std::string& name, + unique_ptr* result) { + result->reset(new MockEnvDirectory()); + return Status::OK(); +} + +bool MockEnv::FileExists(const std::string& fname) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + if (file_map_.find(fn) != file_map_.end()) { + // File exists + return true; + } + // Now also check if fn exists as a dir + for (const auto& iter : file_map_) { + const std::string& filename = iter.first; + if (filename.size() >= fn.size() + 1 && + filename[fn.size()] == '/' && + Slice(filename).starts_with(Slice(fn))) { + return true; + } + } + return false; +} + +Status MockEnv::GetChildren(const std::string& dir, + std::vector* result) { + auto d = NormalizePath(dir); + { + MutexLock lock(&mutex_); + result->clear(); + for (const auto& iter : file_map_) { + const std::string& filename = iter.first; + + if (filename.size() >= d.size() + 1 && filename[d.size()] == '/' && + Slice(filename).starts_with(Slice(d))) { + size_t next_slash = filename.find('/', d.size() + 1); + if (next_slash != std::string::npos) { + result->push_back(filename.substr( + d.size() + 1, next_slash - d.size() - 1)); + } else { + result->push_back(filename.substr(d.size() + 1)); + } + } + } + } + result->erase(std::unique(result->begin(), result->end()), result->end()); + return Status::OK(); +} + +void MockEnv::DeleteFileInternal(const std::string& fname) { + assert(fname == NormalizePath(fname)); + if (file_map_.find(fname) == file_map_.end()) { + return; + } + + file_map_[fname]->Unref(); + file_map_.erase(fname); +} + +Status MockEnv::DeleteFile(const std::string& fname) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + if (file_map_.find(fn) == file_map_.end()) { + return Status::IOError(fn, "File not found"); + } + + DeleteFileInternal(fn); + return Status::OK(); +} + +Status MockEnv::CreateDir(const std::string& dirname) { + return Status::OK(); +} + +Status MockEnv::CreateDirIfMissing(const std::string& dirname) { + return Status::OK(); +} + +Status MockEnv::DeleteDir(const std::string& dirname) { + return Status::OK(); +} + +Status MockEnv::GetFileSize(const std::string& fname, uint64_t* file_size) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + auto iter = file_map_.find(fn); + if (iter == file_map_.end()) { + return Status::IOError(fn, "File not found"); + } + + *file_size = iter->second->Size(); + return Status::OK(); +} + +Status MockEnv::GetFileModificationTime(const std::string& fname, + uint64_t* time) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + auto iter = file_map_.find(fn); + if (iter == file_map_.end()) { + return Status::IOError(fn, "File not found"); + } + *time = iter->second->ModifiedTime(); + return Status::OK(); +} + +Status MockEnv::RenameFile(const std::string& src, + const std::string& target) { + auto s = NormalizePath(src); + auto t = NormalizePath(target); + MutexLock lock(&mutex_); + if (file_map_.find(s) == file_map_.end()) { + return Status::IOError(s, "File not found"); + } + + DeleteFileInternal(t); + file_map_[t] = file_map_[s]; + file_map_.erase(s); + return Status::OK(); +} + +Status MockEnv::NewLogger(const std::string& fname, + shared_ptr* result) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + auto iter = file_map_.find(fn); + MemFile* file = nullptr; + if (iter == file_map_.end()) { + file = new MemFile(fn); + file->Ref(); + file_map_[fn] = file; + } else { + file = iter->second; + } + std::unique_ptr f(new WritableFileImpl(file, nullptr)); + result->reset(new TestMemLogger(std::move(f), this)); + return Status::OK(); +} + +Status MockEnv::LockFile(const std::string& fname, FileLock** flock) { + auto fn = NormalizePath(fname); + { + MutexLock lock(&mutex_); + if (file_map_.find(fn) != file_map_.end()) { + return Status::IOError(fn, "Lock file exists"); + } + file_map_[fn] = nullptr; + } + *flock = new MockEnvFileLock(fn); + return Status::OK(); +} + +Status MockEnv::UnlockFile(FileLock* flock) { + std::string fn = dynamic_cast(flock)->FileName(); + { + MutexLock lock(&mutex_); + auto iter = file_map_.find(fn); + if (iter != file_map_.end()) { + file_map_.erase(fn); + } + } + delete flock; + return Status::OK(); +} + +Status MockEnv::GetTestDirectory(std::string* path) { + *path = "/test"; + return Status::OK(); +} + + // Non-virtual functions, specific to MockEnv +Status MockEnv::Truncate(const std::string& fname, size_t size) { + auto fn = NormalizePath(fname); + MutexLock lock(&mutex_); + auto iter = file_map_.find(fn); + if (iter == file_map_.end()) { + return Status::IOError(fn, "File not found"); + } + iter->second->Truncate(size); + return Status::OK(); +} + +std::string MockEnv::NormalizePath(const std::string path) { + std::string dst; + for (auto c : path) { + if (!dst.empty() && c == '/' && dst.back() == '/') { + continue; + } + dst.push_back(c); + } + return dst; +} + +} // namespace rocksdb diff --git a/util/mock_env.h b/util/mock_env.h new file mode 100644 index 000000000..d128c75b0 --- /dev/null +++ b/util/mock_env.h @@ -0,0 +1,93 @@ +// Copyright (c) 2014, 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 +#include +#include +#include +#include "rocksdb/env.h" +#include "rocksdb/status.h" +#include "port/port.h" +#include "util/mutexlock.h" + +namespace rocksdb { + +class MemFile; +class MockEnv : public EnvWrapper { + public: + explicit MockEnv(Env* base_env); + + virtual ~MockEnv(); + + // Partial implementation of the Env interface. + virtual Status NewSequentialFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& soptions); + + virtual Status NewRandomAccessFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& soptions); + + virtual Status NewWritableFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& env_options); + + virtual Status NewRandomRWFile(const std::string& fname, + unique_ptr* result, + const EnvOptions& options); + + virtual Status NewDirectory(const std::string& name, + unique_ptr* result); + + virtual bool FileExists(const std::string& fname); + + virtual Status GetChildren(const std::string& dir, + std::vector* result); + + void DeleteFileInternal(const std::string& fname); + + virtual Status DeleteFile(const std::string& fname); + + virtual Status CreateDir(const std::string& dirname); + + virtual Status CreateDirIfMissing(const std::string& dirname); + + virtual Status DeleteDir(const std::string& dirname); + + virtual Status GetFileSize(const std::string& fname, uint64_t* file_size); + + virtual Status GetFileModificationTime(const std::string& fname, + uint64_t* time); + + virtual Status RenameFile(const std::string& src, + const std::string& target); + + virtual Status NewLogger(const std::string& fname, + shared_ptr* result); + + virtual Status LockFile(const std::string& fname, FileLock** flock); + + virtual Status UnlockFile(FileLock* flock); + + virtual Status GetTestDirectory(std::string* path); + + // Non-virtual functions, specific to MockEnv + Status Truncate(const std::string& fname, size_t size); + + private: + std::string NormalizePath(const std::string path); + + // Map from filenames to MemFile objects, representing a simple file system. + typedef std::map FileSystem; + port::Mutex mutex_; + FileSystem file_map_; // Protected by mutex_. +}; + +} // namespace rocksdb diff --git a/util/mock_env_test.cc b/util/mock_env_test.cc new file mode 100644 index 000000000..51ae8e296 --- /dev/null +++ b/util/mock_env_test.cc @@ -0,0 +1,232 @@ +// 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 +#include +#include +#include "util/mock_env.h" +#include "db/db_impl.h" +#include "rocksdb/db.h" +#include "rocksdb/env.h" +#include "util/testharness.h" + +namespace rocksdb { + +class MockEnvTest { + public: + Env* env_; + const EnvOptions soptions_; + + MockEnvTest() + : env_(new MockEnv(Env::Default())) { + } + ~MockEnvTest() { + delete env_; + } +}; + +TEST(MockEnvTest, Basics) { + uint64_t file_size; + unique_ptr writable_file; + std::vector children; + + ASSERT_OK(env_->CreateDir("/dir")); + + // Check that the directory is empty. + ASSERT_TRUE(!env_->FileExists("/dir/non_existent")); + ASSERT_TRUE(!env_->GetFileSize("/dir/non_existent", &file_size).ok()); + ASSERT_OK(env_->GetChildren("/dir", &children)); + ASSERT_EQ(0U, children.size()); + + // Create a file. + ASSERT_OK(env_->NewWritableFile("/dir/f", &writable_file, soptions_)); + writable_file.reset(); + + // Check that the file exists. + ASSERT_TRUE(env_->FileExists("/dir/f")); + ASSERT_OK(env_->GetFileSize("/dir/f", &file_size)); + ASSERT_EQ(0U, file_size); + ASSERT_OK(env_->GetChildren("/dir", &children)); + ASSERT_EQ(1U, children.size()); + ASSERT_EQ("f", children[0]); + + // Write to the file. + ASSERT_OK(env_->NewWritableFile("/dir/f", &writable_file, soptions_)); + ASSERT_OK(writable_file->Append("abc")); + writable_file.reset(); + + // Check for expected size. + ASSERT_OK(env_->GetFileSize("/dir/f", &file_size)); + ASSERT_EQ(3U, file_size); + + // Check that renaming works. + ASSERT_TRUE(!env_->RenameFile("/dir/non_existent", "/dir/g").ok()); + ASSERT_OK(env_->RenameFile("/dir/f", "/dir/g")); + ASSERT_TRUE(!env_->FileExists("/dir/f")); + ASSERT_TRUE(env_->FileExists("/dir/g")); + ASSERT_OK(env_->GetFileSize("/dir/g", &file_size)); + ASSERT_EQ(3U, file_size); + + // Check that opening non-existent file fails. + unique_ptr seq_file; + unique_ptr rand_file; + ASSERT_TRUE(!env_->NewSequentialFile("/dir/non_existent", &seq_file, + soptions_).ok()); + ASSERT_TRUE(!seq_file); + ASSERT_TRUE(!env_->NewRandomAccessFile("/dir/non_existent", &rand_file, + soptions_).ok()); + ASSERT_TRUE(!rand_file); + + // Check that deleting works. + ASSERT_TRUE(!env_->DeleteFile("/dir/non_existent").ok()); + ASSERT_OK(env_->DeleteFile("/dir/g")); + ASSERT_TRUE(!env_->FileExists("/dir/g")); + ASSERT_OK(env_->GetChildren("/dir", &children)); + ASSERT_EQ(0U, children.size()); + ASSERT_OK(env_->DeleteDir("/dir")); +} + +TEST(MockEnvTest, ReadWrite) { + unique_ptr writable_file; + unique_ptr seq_file; + unique_ptr rand_file; + Slice result; + char scratch[100]; + + ASSERT_OK(env_->CreateDir("/dir")); + + ASSERT_OK(env_->NewWritableFile("/dir/f", &writable_file, soptions_)); + ASSERT_OK(writable_file->Append("hello ")); + ASSERT_OK(writable_file->Append("world")); + writable_file.reset(); + + // Read sequentially. + ASSERT_OK(env_->NewSequentialFile("/dir/f", &seq_file, soptions_)); + ASSERT_OK(seq_file->Read(5, &result, scratch)); // Read "hello". + ASSERT_EQ(0, result.compare("hello")); + ASSERT_OK(seq_file->Skip(1)); + ASSERT_OK(seq_file->Read(1000, &result, scratch)); // Read "world". + ASSERT_EQ(0, result.compare("world")); + ASSERT_OK(seq_file->Read(1000, &result, scratch)); // Try reading past EOF. + ASSERT_EQ(0U, result.size()); + ASSERT_OK(seq_file->Skip(100)); // Try to skip past end of file. + ASSERT_OK(seq_file->Read(1000, &result, scratch)); + ASSERT_EQ(0U, result.size()); + + // Random reads. + ASSERT_OK(env_->NewRandomAccessFile("/dir/f", &rand_file, soptions_)); + ASSERT_OK(rand_file->Read(6, 5, &result, scratch)); // Read "world". + ASSERT_EQ(0, result.compare("world")); + ASSERT_OK(rand_file->Read(0, 5, &result, scratch)); // Read "hello". + ASSERT_EQ(0, result.compare("hello")); + ASSERT_OK(rand_file->Read(10, 100, &result, scratch)); // Read "d". + ASSERT_EQ(0, result.compare("d")); + + // Too high offset. + ASSERT_TRUE(!rand_file->Read(1000, 5, &result, scratch).ok()); +} + +TEST(MockEnvTest, Locks) { + FileLock* lock; + + // These are no-ops, but we test they return success. + ASSERT_OK(env_->LockFile("some file", &lock)); + ASSERT_OK(env_->UnlockFile(lock)); +} + +TEST(MockEnvTest, Misc) { + std::string test_dir; + ASSERT_OK(env_->GetTestDirectory(&test_dir)); + ASSERT_TRUE(!test_dir.empty()); + + unique_ptr writable_file; + ASSERT_OK(env_->NewWritableFile("/a/b", &writable_file, soptions_)); + + // These are no-ops, but we test they return success. + ASSERT_OK(writable_file->Sync()); + ASSERT_OK(writable_file->Flush()); + ASSERT_OK(writable_file->Close()); + writable_file.reset(); +} + +TEST(MockEnvTest, LargeWrite) { + const size_t kWriteSize = 300 * 1024; + char* scratch = new char[kWriteSize * 2]; + + std::string write_data; + for (size_t i = 0; i < kWriteSize; ++i) { + write_data.append(1, static_cast(i)); + } + + unique_ptr writable_file; + ASSERT_OK(env_->NewWritableFile("/dir/f", &writable_file, soptions_)); + ASSERT_OK(writable_file->Append("foo")); + ASSERT_OK(writable_file->Append(write_data)); + writable_file.reset(); + + unique_ptr seq_file; + Slice result; + ASSERT_OK(env_->NewSequentialFile("/dir/f", &seq_file, soptions_)); + ASSERT_OK(seq_file->Read(3, &result, scratch)); // Read "foo". + ASSERT_EQ(0, result.compare("foo")); + + size_t read = 0; + std::string read_data; + while (read < kWriteSize) { + ASSERT_OK(seq_file->Read(kWriteSize - read, &result, scratch)); + read_data.append(result.data(), result.size()); + read += result.size(); + } + ASSERT_TRUE(write_data == read_data); + delete [] scratch; +} + +TEST(MockEnvTest, DBTest) { + Options options; + options.create_if_missing = true; + options.env = env_; + DB* db; + + const Slice keys[] = {Slice("aaa"), Slice("bbb"), Slice("ccc")}; + const Slice vals[] = {Slice("foo"), Slice("bar"), Slice("baz")}; + + ASSERT_OK(DB::Open(options, "/dir/db", &db)); + for (size_t i = 0; i < 3; ++i) { + ASSERT_OK(db->Put(WriteOptions(), keys[i], vals[i])); + } + + for (size_t i = 0; i < 3; ++i) { + std::string res; + ASSERT_OK(db->Get(ReadOptions(), keys[i], &res)); + ASSERT_TRUE(res == vals[i]); + } + + Iterator* iterator = db->NewIterator(ReadOptions()); + iterator->SeekToFirst(); + for (size_t i = 0; i < 3; ++i) { + ASSERT_TRUE(iterator->Valid()); + ASSERT_TRUE(keys[i] == iterator->key()); + ASSERT_TRUE(vals[i] == iterator->value()); + iterator->Next(); + } + ASSERT_TRUE(!iterator->Valid()); + delete iterator; + + DBImpl* dbi = reinterpret_cast(db); + ASSERT_OK(dbi->TEST_FlushMemTable()); + + for (size_t i = 0; i < 3; ++i) { + std::string res; + ASSERT_OK(db->Get(ReadOptions(), keys[i], &res)); + ASSERT_TRUE(res == vals[i]); + } + + delete db; +} + +} // namespace rocksdb + +int main(int argc, char** argv) { + return rocksdb::test::RunAllTests(); +} diff --git a/util/testharness.cc b/util/testharness.cc index 16773f69f..967a8f20a 100644 --- a/util/testharness.cc +++ b/util/testharness.cc @@ -75,9 +75,9 @@ int RunAllTests() { return 0; } -std::string TmpDir() { +std::string TmpDir(Env* env) { std::string dir; - Status s = Env::Default()->GetTestDirectory(&dir); + Status s = env->GetTestDirectory(&dir); ASSERT_TRUE(s.ok()) << s.ToString(); return dir; } diff --git a/util/testharness.h b/util/testharness.h index af4b2858c..6115d68f7 100644 --- a/util/testharness.h +++ b/util/testharness.h @@ -12,6 +12,7 @@ #include #include #include +#include #include "port/stack_trace.h" #include "rocksdb/env.h" #include "rocksdb/slice.h" @@ -37,7 +38,7 @@ namespace test { extern int RunAllTests(); // Return the directory to use for temporary storage. -extern std::string TmpDir(); +extern std::string TmpDir(Env* env = Env::Default()); // Return a randomization seed for this run. Typically returns the // same number on repeated invocations of this binary, but automated