diff --git a/CMakeLists.txt b/CMakeLists.txt index 881384264..017fe8675 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -559,6 +559,7 @@ set(SOURCES env/env_chroot.cc env/env_encryption.cc env/env_hdfs.cc + env/file_system.cc env/mock_env.cc file/delete_scheduler.cc file/file_prefetch_buffer.cc @@ -766,6 +767,7 @@ else() list(APPEND SOURCES port/port_posix.cc env/env_posix.cc + env/fs_posix.cc env/io_posix.cc) endif() diff --git a/HISTORY.md b/HISTORY.md index 87fa2f84a..161c2d6e9 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -1,5 +1,9 @@ # Rocksdb Change Log ## Unreleased +### Public API Change +* Added a rocksdb::FileSystem class in include/rocksdb/file_system.h to encapsulate file creation/read/write operations, and an option DBOptions::file_system to allow a user to pass in an instance of rocksdb::FileSystem. If its a non-null value, this will take precendence over DBOptions::env for file operations. A new API rocksdb::FileSystem::Default() returns a platform default object. The DBOptions::env option and Env::Default() API will continue to be used for threading and other OS related functions, and where DBOptions::file_system is not specified, for file operations. For storage developers who are accustomed to rocksdb::Env, the interface in rocksdb::FileSystem is new and will probably undergo some changes as more storage systems are ported to it from rocksdb::Env. As of now, no env other than Posix has been ported to the new interface. +* A new rocksdb::NewSstFileManager() API that allows the caller to pass in separate Env and FileSystem objects. + ### Bug Fixes * Fix a bug that can cause unnecessary bg thread to be scheduled(#6104). * Fix a bug in which a snapshot read could be affected by a DeleteRange after the snapshot (#6062). diff --git a/TARGETS b/TARGETS index 2cd920449..4aed52b23 100644 --- a/TARGETS +++ b/TARGETS @@ -181,6 +181,8 @@ cpp_library( "env/env_encryption.cc", "env/env_hdfs.cc", "env/env_posix.cc", + "env/file_system.cc", + "env/fs_posix.cc", "env/io_posix.cc", "env/mock_env.cc", "file/delete_scheduler.cc", diff --git a/db/builder.cc b/db/builder.cc index e5d08a072..cde5c827a 100644 --- a/db/builder.cc +++ b/db/builder.cc @@ -66,8 +66,9 @@ TableBuilder* NewTableBuilder( } Status BuildTable( - const std::string& dbname, Env* env, const ImmutableCFOptions& ioptions, - const MutableCFOptions& mutable_cf_options, const EnvOptions& env_options, + const std::string& dbname, Env* env, FileSystem* fs, + const ImmutableCFOptions& ioptions, + const MutableCFOptions& mutable_cf_options, const FileOptions& file_options, TableCache* table_cache, InternalIterator* iter, std::vector> range_del_iters, @@ -115,12 +116,12 @@ Status BuildTable( compression_opts_for_flush.max_dict_bytes = 0; compression_opts_for_flush.zstd_max_train_bytes = 0; { - std::unique_ptr file; + std::unique_ptr file; #ifndef NDEBUG - bool use_direct_writes = env_options.use_direct_writes; + bool use_direct_writes = file_options.use_direct_writes; TEST_SYNC_POINT_CALLBACK("BuildTable:create_file", &use_direct_writes); #endif // !NDEBUG - s = NewWritableFile(env, fname, &file, env_options); + s = NewWritableFile(fs, fname, &file, file_options); if (!s.ok()) { EventHelpers::LogAndNotifyTableFileCreationFinished( event_logger, ioptions.listeners, dbname, column_family_name, fname, @@ -131,7 +132,7 @@ Status BuildTable( file->SetWriteLifeTimeHint(write_hint); file_writer.reset( - new WritableFileWriter(std::move(file), fname, env_options, env, + new WritableFileWriter(std::move(file), fname, file_options, env, ioptions.statistics, ioptions.listeners)); builder = NewTableBuilder( ioptions, mutable_cf_options, internal_comparator, @@ -218,7 +219,7 @@ Status BuildTable( // we will regrad this verification as user reads since the goal is // to cache it here for further user reads std::unique_ptr it(table_cache->NewIterator( - ReadOptions(), env_options, internal_comparator, *meta, + ReadOptions(), file_options, internal_comparator, *meta, nullptr /* range_del_agg */, mutable_cf_options.prefix_extractor.get(), nullptr, (internal_stats == nullptr) ? nullptr @@ -241,7 +242,7 @@ Status BuildTable( } if (!s.ok() || meta->fd.GetFileSize() == 0) { - env->DeleteFile(fname); + fs->DeleteFile(fname, IOOptions(), nullptr); } if (meta->fd.GetFileSize() == 0) { diff --git a/db/builder.h b/db/builder.h index 4fa56f50e..74012b337 100644 --- a/db/builder.h +++ b/db/builder.h @@ -62,8 +62,9 @@ TableBuilder* NewTableBuilder( // @param column_family_name Name of the column family that is also identified // by column_family_id, or empty string if unknown. extern Status BuildTable( - const std::string& dbname, Env* env, const ImmutableCFOptions& options, - const MutableCFOptions& mutable_cf_options, const EnvOptions& env_options, + const std::string& dbname, Env* env, FileSystem* fs, + const ImmutableCFOptions& options, + const MutableCFOptions& mutable_cf_options, const FileOptions& file_options, TableCache* table_cache, InternalIterator* iter, std::vector> range_del_iters, diff --git a/db/column_family.cc b/db/column_family.cc index 7dd02913a..ec4411eb2 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -475,7 +475,7 @@ ColumnFamilyData::ColumnFamilyData( uint32_t id, const std::string& name, Version* _dummy_versions, Cache* _table_cache, WriteBufferManager* write_buffer_manager, const ColumnFamilyOptions& cf_options, const ImmutableDBOptions& db_options, - const EnvOptions& env_options, ColumnFamilySet* column_family_set, + const FileOptions& file_options, ColumnFamilySet* column_family_set, BlockCacheTracer* const block_cache_tracer) : id_(id), name_(name), @@ -517,7 +517,7 @@ ColumnFamilyData::ColumnFamilyData( if (_dummy_versions != nullptr) { internal_stats_.reset( new InternalStats(ioptions_.num_levels, db_options.env, this)); - table_cache_.reset(new TableCache(ioptions_, env_options, _table_cache, + table_cache_.reset(new TableCache(ioptions_, file_options, _table_cache, block_cache_tracer)); if (ioptions_.compaction_style == kCompactionStyleLevel) { compaction_picker_.reset( @@ -967,8 +967,8 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions( return write_stall_condition; } -const EnvOptions* ColumnFamilyData::soptions() const { - return &(column_family_set_->env_options_); +const FileOptions* ColumnFamilyData::soptions() const { + return &(column_family_set_->file_options_); } void ColumnFamilyData::SetCurrent(Version* current_version) { @@ -1335,7 +1335,7 @@ Directory* ColumnFamilyData::GetDataDir(size_t path_id) const { ColumnFamilySet::ColumnFamilySet(const std::string& dbname, const ImmutableDBOptions* db_options, - const EnvOptions& env_options, + const FileOptions& file_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller, @@ -1343,11 +1343,11 @@ ColumnFamilySet::ColumnFamilySet(const std::string& dbname, : max_column_family_(0), dummy_cfd_(new ColumnFamilyData( 0, "", nullptr, nullptr, nullptr, ColumnFamilyOptions(), *db_options, - env_options, nullptr, block_cache_tracer)), + file_options, nullptr, block_cache_tracer)), default_cfd_cache_(nullptr), db_name_(dbname), db_options_(db_options), - env_options_(env_options), + file_options_(file_options), table_cache_(table_cache), write_buffer_manager_(write_buffer_manager), write_controller_(write_controller), @@ -1417,7 +1417,7 @@ ColumnFamilyData* ColumnFamilySet::CreateColumnFamily( assert(column_families_.find(name) == column_families_.end()); ColumnFamilyData* new_cfd = new ColumnFamilyData( id, name, dummy_versions, table_cache_, write_buffer_manager_, options, - *db_options_, env_options_, this, block_cache_tracer_); + *db_options_, file_options_, this, block_cache_tracer_); column_families_.insert({name, id}); column_family_data_.insert({id, new_cfd}); max_column_family_ = std::max(max_column_family_, id); diff --git a/db/column_family.h b/db/column_family.h index 88ca364d6..8529ff230 100644 --- a/db/column_family.h +++ b/db/column_family.h @@ -324,7 +324,7 @@ class ColumnFamilyData { } FlushReason GetFlushReason() const { return flush_reason_; } // thread-safe - const EnvOptions* soptions() const; + const FileOptions* soptions() const; const ImmutableCFOptions* ioptions() const { return &ioptions_; } // REQUIRES: DB mutex held // This returns the MutableCFOptions used by current SuperVersion @@ -510,7 +510,7 @@ class ColumnFamilyData { WriteBufferManager* write_buffer_manager, const ColumnFamilyOptions& options, const ImmutableDBOptions& db_options, - const EnvOptions& env_options, + const FileOptions& file_options, ColumnFamilySet* column_family_set, BlockCacheTracer* const block_cache_tracer); @@ -638,7 +638,7 @@ class ColumnFamilySet { ColumnFamilySet(const std::string& dbname, const ImmutableDBOptions* db_options, - const EnvOptions& env_options, Cache* table_cache, + const FileOptions& file_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller, BlockCacheTracer* const block_cache_tracer); @@ -696,7 +696,7 @@ class ColumnFamilySet { const std::string db_name_; const ImmutableDBOptions* const db_options_; - const EnvOptions env_options_; + const FileOptions file_options_; Cache* table_cache_; WriteBufferManager* write_buffer_manager_; WriteController* write_controller_; diff --git a/db/compaction/compaction_job.cc b/db/compaction/compaction_job.cc index 22c504fde..da29b1c53 100644 --- a/db/compaction/compaction_job.cc +++ b/db/compaction/compaction_job.cc @@ -301,7 +301,7 @@ void CompactionJob::AggregateStatistics() { CompactionJob::CompactionJob( int job_id, Compaction* compaction, const ImmutableDBOptions& db_options, - const EnvOptions env_options, VersionSet* versions, + const FileOptions& file_options, VersionSet* versions, const std::atomic* shutting_down, const SequenceNumber preserve_deletes_seqnum, LogBuffer* log_buffer, Directory* db_directory, Directory* output_directory, Statistics* stats, @@ -318,10 +318,11 @@ CompactionJob::CompactionJob( compaction_stats_(compaction->compaction_reason(), 1), dbname_(dbname), db_options_(db_options), - env_options_(env_options), + file_options_(file_options), env_(db_options.env), - env_options_for_read_( - env_->OptimizeForCompactionTableRead(env_options, db_options_)), + fs_(db_options.fs.get()), + file_options_for_read_( + fs_->OptimizeForCompactionTableRead(file_options, db_options_)), versions_(versions), shutting_down_(shutting_down), manual_compaction_paused_(manual_compaction_paused), @@ -647,7 +648,7 @@ Status CompactionJob::Run() { // we will regard this verification as user reads since the goal is // to cache it here for further user reads InternalIterator* iter = cfd->table_cache()->NewIterator( - ReadOptions(), env_options_, cfd->internal_comparator(), + ReadOptions(), file_options_, cfd->internal_comparator(), *files_meta[file_idx], /*range_del_agg=*/nullptr, prefix_extractor, /*table_reader_ptr=*/nullptr, cfd->internal_stats()->GetFileReadHist( @@ -836,7 +837,7 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) { // Although the v2 aggregator is what the level iterator(s) know about, // the AddTombstones calls will be propagated down to the v1 aggregator. std::unique_ptr input(versions_->MakeInputIterator( - sub_compact->compaction, &range_del_agg, env_options_for_read_)); + sub_compact->compaction, &range_del_agg, file_options_for_read_)); AutoThreadOperationStageUpdater stage_updater( ThreadStatus::STAGE_COMPACTION_PROCESS_KV); @@ -1457,13 +1458,13 @@ Status CompactionJob::OpenCompactionOutputFile( TableFileCreationReason::kCompaction); #endif // !ROCKSDB_LITE // Make the output file - std::unique_ptr writable_file; + std::unique_ptr writable_file; #ifndef NDEBUG - bool syncpoint_arg = env_options_.use_direct_writes; + bool syncpoint_arg = file_options_.use_direct_writes; TEST_SYNC_POINT_CALLBACK("CompactionJob::OpenCompactionOutputFile", &syncpoint_arg); #endif - Status s = NewWritableFile(env_, fname, &writable_file, env_options_); + Status s = NewWritableFile(fs_, fname, &writable_file, file_options_); if (!s.ok()) { ROCKS_LOG_ERROR( db_options_.info_log, @@ -1506,14 +1507,14 @@ Status CompactionJob::OpenCompactionOutputFile( sub_compact->outputs.push_back(out); } - writable_file->SetIOPriority(Env::IO_LOW); + writable_file->SetIOPriority(Env::IOPriority::IO_LOW); writable_file->SetWriteLifeTimeHint(write_hint_); writable_file->SetPreallocationBlockSize(static_cast( sub_compact->compaction->OutputFilePreallocationSize())); const auto& listeners = sub_compact->compaction->immutable_cf_options()->listeners; sub_compact->outfile.reset( - new WritableFileWriter(std::move(writable_file), fname, env_options_, + new WritableFileWriter(std::move(writable_file), fname, file_options_, env_, db_options_.statistics.get(), listeners)); // If the Column family flag is to only optimize filters for hits, diff --git a/db/compaction/compaction_job.h b/db/compaction/compaction_job.h index bca4c9942..e7da29f53 100644 --- a/db/compaction/compaction_job.h +++ b/db/compaction/compaction_job.h @@ -64,7 +64,7 @@ class CompactionJob { public: CompactionJob(int job_id, Compaction* compaction, const ImmutableDBOptions& db_options, - const EnvOptions env_options, VersionSet* versions, + const FileOptions& file_options, VersionSet* versions, const std::atomic* shutting_down, const SequenceNumber preserve_deletes_seqnum, LogBuffer* log_buffer, Directory* db_directory, @@ -150,11 +150,12 @@ class CompactionJob { // DBImpl state const std::string& dbname_; const ImmutableDBOptions& db_options_; - const EnvOptions env_options_; + const FileOptions file_options_; Env* env_; + FileSystem* fs_; // env_option optimized for compaction table reads - EnvOptions env_options_for_read_; + FileOptions file_options_for_read_; VersionSet* versions_; const std::atomic* shutting_down_; const std::atomic* manual_compaction_paused_; diff --git a/db/compaction/compaction_job_test.cc b/db/compaction/compaction_job_test.cc index 9fb3f0df5..f84b6c814 100644 --- a/db/compaction/compaction_job_test.cc +++ b/db/compaction/compaction_job_test.cc @@ -72,6 +72,7 @@ class CompactionJobTest : public testing::Test { public: CompactionJobTest() : env_(Env::Default()), + fs_(std::make_shared(env_)), dbname_(test::PerThreadDBPath("compaction_job_test")), db_options_(), mutable_cf_options_(cf_options_), @@ -86,6 +87,8 @@ class CompactionJobTest : public testing::Test { mock_table_factory_(new mock::MockTableFactory()), error_handler_(nullptr, db_options_, &mutex_) { EXPECT_OK(env_->CreateDirIfMissing(dbname_)); + db_options_.env = env_; + db_options_.fs = fs_; db_options_.db_paths.emplace_back(dbname_, std::numeric_limits::max()); } @@ -267,8 +270,8 @@ class CompactionJobTest : public testing::Test { Status s = env_->NewWritableFile( manifest, &file, env_->OptimizeForManifestWrite(env_options_)); ASSERT_OK(s); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(file), manifest, env_options_)); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(file)), manifest, env_options_)); { log::Writer log(std::move(file_writer), 0, false); std::string record; @@ -360,6 +363,7 @@ class CompactionJobTest : public testing::Test { } Env* env_; + std::shared_ptr fs_; std::string dbname_; EnvOptions env_options_; ImmutableDBOptions db_options_; diff --git a/db/convenience.cc b/db/convenience.cc index 320d5c6e1..78396cf4a 100644 --- a/db/convenience.cc +++ b/db/convenience.cc @@ -41,14 +41,16 @@ Status VerifySstFileChecksum(const Options& options, const EnvOptions& env_options, const ReadOptions& read_options, const std::string& file_path) { - std::unique_ptr file; + std::unique_ptr file; uint64_t file_size; InternalKeyComparator internal_comparator(options.comparator); ImmutableCFOptions ioptions(options); - Status s = ioptions.env->NewRandomAccessFile(file_path, &file, env_options); + Status s = ioptions.fs->NewRandomAccessFile(file_path, + FileOptions(env_options), + &file, nullptr); if (s.ok()) { - s = ioptions.env->GetFileSize(file_path, &file_size); + s = ioptions.fs->GetFileSize(file_path, IOOptions(), &file_size, nullptr); } else { return s; } diff --git a/db/corruption_test.cc b/db/corruption_test.cc index 4add9b26a..3735ceb1a 100644 --- a/db/corruption_test.cc +++ b/db/corruption_test.cc @@ -20,6 +20,7 @@ #include "db/db_test_util.h" #include "db/log_format.h" #include "db/version_set.h" +#include "env/composite_env_wrapper.h" #include "file/filename.h" #include "rocksdb/cache.h" #include "rocksdb/convenience.h" @@ -189,6 +190,7 @@ class CorruptionTest : public testing::Test { ASSERT_TRUE(s.ok()) << s.ToString(); Options options; EnvOptions env_options; + options.file_system.reset(new LegacyFileSystemWrapper(options.env)); ASSERT_NOK(VerifySstFileChecksum(options, env_options, fname)); } @@ -539,7 +541,8 @@ TEST_F(CorruptionTest, RangeDeletionCorrupted) { std::unique_ptr file; ASSERT_OK(options_.env->NewRandomAccessFile(filename, &file, EnvOptions())); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(file), filename)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(file), + filename)); uint64_t file_size; ASSERT_OK(options_.env->GetFileSize(filename, &file_size)); diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index 3a3bb886b..8bc09e8cf 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -52,6 +52,7 @@ #include "db/version_set.h" #include "db/write_batch_internal.h" #include "db/write_callback.h" +#include "env/composite_env_wrapper.h" #include "file/file_util.h" #include "file/filename.h" #include "file/random_access_file_reader.h" @@ -102,6 +103,7 @@ #include "util/string_util.h" namespace rocksdb { + const std::string kDefaultColumnFamilyName("default"); const std::string kPersistentStatsColumnFamilyName( "___rocksdb_stats_history___"); @@ -145,10 +147,11 @@ void DumpSupportInfo(Logger* logger) { DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, const bool seq_per_batch, const bool batch_per_txn) - : env_(options.env), - dbname_(dbname), + : dbname_(dbname), own_info_log_(options.info_log == nullptr), initial_db_options_(SanitizeOptions(dbname, options)), + env_(initial_db_options_.env), + fs_(initial_db_options_.file_system), immutable_db_options_(initial_db_options_), mutable_db_options_(initial_db_options_), stats_(immutable_db_options_.statistics.get()), @@ -156,9 +159,9 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, immutable_db_options_.use_adaptive_mutex), default_cf_handle_(nullptr), max_total_in_memory_state_(0), - env_options_(BuildDBOptions(immutable_db_options_, mutable_db_options_)), - env_options_for_compaction_(env_->OptimizeForCompactionTableWrite( - env_options_, immutable_db_options_)), + file_options_(BuildDBOptions(immutable_db_options_, mutable_db_options_)), + file_options_for_compaction_(fs_->OptimizeForCompactionTableWrite( + file_options_, immutable_db_options_)), seq_per_batch_(seq_per_batch), batch_per_txn_(batch_per_txn), db_lock_(nullptr), @@ -194,7 +197,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, unable_to_release_oldest_log_(false), num_running_ingest_file_(0), #ifndef ROCKSDB_LITE - wal_manager_(immutable_db_options_, env_options_, seq_per_batch), + wal_manager_(immutable_db_options_, file_options_, seq_per_batch), #endif // ROCKSDB_LITE event_logger_(immutable_db_options_.info_log.get()), bg_work_paused_(0), @@ -240,7 +243,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, co.metadata_charge_policy = kDontChargeCacheMetadata; table_cache_ = NewLRUCache(co); - versions_.reset(new VersionSet(dbname_, &immutable_db_options_, env_options_, + versions_.reset(new VersionSet(dbname_, &immutable_db_options_, file_options_, table_cache_.get(), write_buffer_manager_, &write_controller_, &block_cache_tracer_)); column_family_memtables_.reset( @@ -587,6 +590,7 @@ Status DBImpl::CloseHelper() { ret = s; } } + if (ret.IsAborted()) { // Reserve IsAborted() error for those where users didn't release // certain resource and they can release them and come back and @@ -1048,14 +1052,14 @@ Status DBImpl::SetDBOptions( wal_changed = mutable_db_options_.wal_bytes_per_sync != new_options.wal_bytes_per_sync; mutable_db_options_ = new_options; - env_options_for_compaction_ = EnvOptions(new_db_options); - env_options_for_compaction_ = env_->OptimizeForCompactionTableWrite( - env_options_for_compaction_, immutable_db_options_); - versions_->ChangeEnvOptions(mutable_db_options_); + file_options_for_compaction_ = FileOptions(new_db_options); + file_options_for_compaction_ = fs_->OptimizeForCompactionTableWrite( + file_options_for_compaction_, immutable_db_options_); + versions_->ChangeFileOptions(mutable_db_options_); //TODO(xiez): clarify why apply optimize for read to write options - env_options_for_compaction_ = env_->OptimizeForCompactionTableRead( - env_options_for_compaction_, immutable_db_options_); - env_options_for_compaction_.compaction_readahead_size = + file_options_for_compaction_ = fs_->OptimizeForCompactionTableRead( + file_options_for_compaction_, immutable_db_options_); + file_options_for_compaction_.compaction_readahead_size = mutable_db_options_.compaction_readahead_size; WriteThread::Writer w; write_thread_.EnterUnbatched(&w, &mutex_); @@ -1422,7 +1426,7 @@ InternalIterator* DBImpl::NewInternalIterator(const ReadOptions& read_options, if (s.ok()) { // Collect iterators for files in L0 - Ln if (read_options.read_tier != kMemtableTier) { - super_version->current->AddIterators(read_options, env_options_, + super_version->current->AddIterators(read_options, file_options_, &merge_iter_builder, range_del_agg); } internal_iter = merge_iter_builder.Finish(); @@ -2717,6 +2721,15 @@ const std::string& DBImpl::GetName() const { return dbname_; } Env* DBImpl::GetEnv() const { return env_; } +FileSystem* DB::GetFileSystem() const { + static LegacyFileSystemWrapper fs_wrap(GetEnv()); + return &fs_wrap; +} + +FileSystem* DBImpl::GetFileSystem() const { + return immutable_db_options_.fs.get(); +} + Options DBImpl::GetOptions(ColumnFamilyHandle* column_family) const { InstrumentedMutexLock l(&mutex_); auto cfh = reinterpret_cast(column_family); @@ -3289,12 +3302,12 @@ Status DBImpl::GetDbIdentity(std::string& identity) const { Status DBImpl::GetDbIdentityFromIdentityFile(std::string* identity) const { std::string idfilename = IdentityFileName(dbname_); - const EnvOptions soptions; + const FileOptions soptions; std::unique_ptr id_file_reader; Status s; { - std::unique_ptr idfile; - s = env_->NewSequentialFile(idfilename, &idfile, soptions); + std::unique_ptr idfile; + s = fs_->NewSequentialFile(idfilename, soptions, &idfile, nullptr); if (!s.ok()) { return s; } @@ -3303,7 +3316,7 @@ Status DBImpl::GetDbIdentityFromIdentityFile(std::string* identity) const { } uint64_t file_size; - s = env_->GetFileSize(idfilename, &file_size); + s = fs_->GetFileSize(idfilename, IOOptions(), &file_size, nullptr); if (!s.ok()) { return s; } @@ -3377,7 +3390,12 @@ Status DBImpl::Close() { Status DB::ListColumnFamilies(const DBOptions& db_options, const std::string& name, std::vector* column_families) { - return VersionSet::ListColumnFamilies(column_families, name, db_options.env); + FileSystem* fs = db_options.file_system.get(); + LegacyFileSystemWrapper legacy_fs(db_options.env); + if (!fs) { + fs = &legacy_fs; + } + return VersionSet::ListColumnFamilies(column_families, name, fs); } Snapshot::~Snapshot() {} @@ -3542,8 +3560,8 @@ Status DBImpl::WriteOptionsFile(bool need_mutex_lock, std::string file_name = TempOptionsFileName(GetName(), versions_->NewFileNumber()); - Status s = - PersistRocksDBOptions(db_options, cf_names, cf_opts, file_name, GetEnv()); + Status s = PersistRocksDBOptions(db_options, cf_names, cf_opts, file_name, + GetFileSystem()); if (s.ok()) { s = RenameTempFileToOptionsFile(file_name); @@ -3887,7 +3905,7 @@ Status DBImpl::IngestExternalFiles( for (const auto& arg : args) { auto* cfd = static_cast(arg.column_family)->cfd(); ingestion_jobs.emplace_back( - env_, versions_.get(), cfd, immutable_db_options_, env_options_, + env_, versions_.get(), cfd, immutable_db_options_, file_options_, &snapshots_, arg.options, &directories_, &event_logger_); } std::vector> exec_results; @@ -4151,7 +4169,7 @@ Status DBImpl::CreateColumnFamilyWithImport( auto cfh = reinterpret_cast(*handle); auto cfd = cfh->cfd(); ImportColumnFamilyJob import_job(env_, versions_.get(), cfd, - immutable_db_options_, env_options_, + immutable_db_options_, file_options_, import_options, metadata.files); SuperVersionContext dummy_sv_ctx(/* create_superversion */ true); @@ -4281,7 +4299,7 @@ Status DBImpl::VerifyChecksum(const ReadOptions& read_options) { const auto& fd = vstorage->LevelFilesBrief(i).files[j].fd; std::string fname = TableFileName(cfd->ioptions()->cf_paths, fd.GetNumber(), fd.GetPathId()); - s = rocksdb::VerifySstFileChecksum(opts, env_options_, read_options, + s = rocksdb::VerifySstFileChecksum(opts, file_options_, read_options, fname); } } diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index d56e0e14d..7d38ee4dd 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -306,6 +306,7 @@ class DBImpl : public DB { ColumnFamilyHandle* column_family) override; virtual const std::string& GetName() const override; virtual Env* GetEnv() const override; + virtual FileSystem* GetFileSystem() const override; using DB::GetOptions; virtual Options GetOptions(ColumnFamilyHandle* column_family) const override; using DB::GetDBOptions; @@ -941,13 +942,14 @@ class DBImpl : public DB { #endif // NDEBUG protected: - Env* const env_; const std::string dbname_; std::string db_id_; std::unique_ptr versions_; // Flag to check whether we allocated and own the info log file bool own_info_log_; const DBOptions initial_db_options_; + Env* const env_; + std::shared_ptr fs_; const ImmutableDBOptions immutable_db_options_; MutableDBOptions mutable_db_options_; Statistics* stats_; @@ -975,10 +977,10 @@ class DBImpl : public DB { bool single_column_family_mode_; // The options to access storage files - const EnvOptions env_options_; + const FileOptions file_options_; // Additonal options for compaction and flush - EnvOptions env_options_for_compaction_; + FileOptions file_options_for_compaction_; std::unique_ptr column_family_memtables_; diff --git a/db/db_impl/db_impl_compaction_flush.cc b/db/db_impl/db_impl_compaction_flush.cc index 99c67ea49..3c436eb74 100644 --- a/db/db_impl/db_impl_compaction_flush.cc +++ b/db/db_impl/db_impl_compaction_flush.cc @@ -148,7 +148,7 @@ Status DBImpl::FlushMemTableToOutputFile( FlushJob flush_job( dbname_, cfd, immutable_db_options_, mutable_cf_options, - nullptr /* memtable_id */, env_options_for_compaction_, versions_.get(), + nullptr /* memtable_id */, file_options_for_compaction_, versions_.get(), &mutex_, &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, job_context, log_buffer, directories_.GetDbDir(), GetDataDir(cfd, 0U), @@ -332,7 +332,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles( const uint64_t* max_memtable_id = &(bg_flush_args[i].max_memtable_id_); jobs.emplace_back(new FlushJob( dbname_, cfd, immutable_db_options_, mutable_cf_options, - max_memtable_id, env_options_for_compaction_, versions_.get(), &mutex_, + max_memtable_id, file_options_for_compaction_, versions_.get(), &mutex_, &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, job_context, log_buffer, directories_.GetDbDir(), data_dir, GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), @@ -968,7 +968,7 @@ Status DBImpl::CompactFilesImpl( CompactionJobStats compaction_job_stats; CompactionJob compaction_job( job_context->job_id, c.get(), immutable_db_options_, - env_options_for_compaction_, versions_.get(), &shutting_down_, + file_options_for_compaction_, versions_.get(), &shutting_down_, preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(), GetDataDir(c->column_family_data(), c->output_path_id()), stats_, &mutex_, &error_handler_, snapshot_seqs, earliest_write_conflict_snapshot, @@ -2747,7 +2747,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, assert(is_snapshot_supported_ || snapshots_.empty()); CompactionJob compaction_job( job_context->job_id, c.get(), immutable_db_options_, - env_options_for_compaction_, versions_.get(), &shutting_down_, + file_options_for_compaction_, versions_.get(), &shutting_down_, preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(), GetDataDir(c->column_family_data(), c->output_path_id()), stats_, &mutex_, &error_handler_, snapshot_seqs, diff --git a/db/db_impl/db_impl_open.cc b/db/db_impl/db_impl_open.cc index 05a46af83..b41a001a6 100644 --- a/db/db_impl/db_impl_open.cc +++ b/db/db_impl/db_impl_open.cc @@ -12,6 +12,7 @@ #include "db/builder.h" #include "db/error_handler.h" +#include "env/composite_env_wrapper.h" #include "file/read_write_util.h" #include "file/sst_file_manager_impl.h" #include "file/writable_file_writer.h" @@ -34,6 +35,18 @@ Options SanitizeOptions(const std::string& dbname, const Options& src) { DBOptions SanitizeOptions(const std::string& dbname, const DBOptions& src) { DBOptions result(src); + if (result.file_system == nullptr) { + if (result.env == Env::Default()) { + result.file_system = FileSystem::Default(); + } else { + result.file_system.reset(new LegacyFileSystemWrapper(result.env)); + } + } else { + if (result.env == nullptr) { + result.env = Env::Default(); + } + } + // result.max_open_files means an "infinite" open files. if (result.max_open_files != -1) { int max_max_open_files = port::GetMaxOpenFiles(); @@ -254,16 +267,16 @@ Status DBImpl::NewDB() { ROCKS_LOG_INFO(immutable_db_options_.info_log, "Creating manifest 1 \n"); const std::string manifest = DescriptorFileName(dbname_, 1); { - std::unique_ptr file; - EnvOptions env_options = env_->OptimizeForManifestWrite(env_options_); - s = NewWritableFile(env_, manifest, &file, env_options); + std::unique_ptr file; + FileOptions file_options = fs_->OptimizeForManifestWrite(file_options_); + s = NewWritableFile(fs_.get(), manifest, &file, file_options); if (!s.ok()) { return s; } file->SetPreallocationBlockSize( immutable_db_options_.manifest_preallocation_size); std::unique_ptr file_writer(new WritableFileWriter( - std::move(file), manifest, env_options, env_, nullptr /* stats */, + std::move(file), manifest, file_options, env_, nullptr /* stats */, immutable_db_options_.listeners)); log::Writer log(std::move(file_writer), 0, false); std::string record; @@ -277,7 +290,7 @@ Status DBImpl::NewDB() { // Make "CURRENT" file that points to the new manifest file. s = SetCurrentFile(env_, dbname_, 1, directories_.GetDbDir()); } else { - env_->DeleteFile(manifest); + fs_->DeleteFile(manifest, IOOptions(), nullptr); } return s; } @@ -375,20 +388,20 @@ Status DBImpl::Recover( assert(s.IsIOError()); return s; } - // Verify compatibility of env_options_ and filesystem + // Verify compatibility of file_options_ and filesystem { - std::unique_ptr idfile; - EnvOptions customized_env(env_options_); - customized_env.use_direct_reads |= + std::unique_ptr idfile; + FileOptions customized_fs(file_options_); + customized_fs.use_direct_reads |= immutable_db_options_.use_direct_io_for_flush_and_compaction; - s = env_->NewRandomAccessFile(CurrentFileName(dbname_), &idfile, - customized_env); + s = fs_->NewRandomAccessFile(CurrentFileName(dbname_), customized_fs, + &idfile, nullptr); if (!s.ok()) { std::string error_str = s.ToString(); // Check if unsupported Direct I/O is the root cause - customized_env.use_direct_reads = false; - s = env_->NewRandomAccessFile(CurrentFileName(dbname_), &idfile, - customized_env); + customized_fs.use_direct_reads = false; + s = fs_->NewRandomAccessFile(CurrentFileName(dbname_), customized_fs, + &idfile, nullptr); if (s.ok()) { return Status::InvalidArgument( "Direct I/O is not supported by the specified DB."); @@ -408,7 +421,7 @@ Status DBImpl::Recover( // the very first time. if (db_id_.empty()) { // Check for the IDENTITY file and create it if not there. - s = env_->FileExists(IdentityFileName(dbname_)); + s = fs_->FileExists(IdentityFileName(dbname_), IOOptions(), nullptr); // Typically Identity file is created in NewDB() and for some reason if // it is no longer available then at this point DB ID is not in Identity // file or Manifest. @@ -748,9 +761,10 @@ Status DBImpl::RecoverLogFiles(const std::vector& log_numbers, std::unique_ptr file_reader; { - std::unique_ptr file; - status = env_->NewSequentialFile(fname, &file, - env_->OptimizeForLogRead(env_options_)); + std::unique_ptr file; + status = fs_->NewSequentialFile(fname, + fs_->OptimizeForLogRead(file_options_), + &file, nullptr); if (!status.ok()) { MaybeIgnoreError(&status); if (!status.ok()) { @@ -1119,17 +1133,18 @@ Status DBImpl::RestoreAliveLogFiles(const std::vector& log_numbers) { // preallocated space are not needed anymore. It is likely only the last // log has such preallocated space, so we only truncate for the last log. if (log_number == log_numbers.back()) { - std::unique_ptr last_log; - Status truncate_status = env_->ReopenWritableFile( - fname, &last_log, - env_->OptimizeForLogWrite( - env_options_, - BuildDBOptions(immutable_db_options_, mutable_db_options_))); + std::unique_ptr last_log; + Status truncate_status = fs_->ReopenWritableFile( + fname, + fs_->OptimizeForLogWrite( + file_options_, + BuildDBOptions(immutable_db_options_, mutable_db_options_)), + &last_log, nullptr); if (truncate_status.ok()) { - truncate_status = last_log->Truncate(log.size); + truncate_status = last_log->Truncate(log.size, IOOptions(), nullptr); } if (truncate_status.ok()) { - truncate_status = last_log->Close(); + truncate_status = last_log->Close(IOOptions(), nullptr); } // Not a critical error if fail to truncate. if (!truncate_status.ok()) { @@ -1196,8 +1211,8 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, range_del_iters.emplace_back(range_del_iter); } s = BuildTable( - dbname_, env_, *cfd->ioptions(), mutable_cf_options, - env_options_for_compaction_, cfd->table_cache(), iter.get(), + dbname_, env_, fs_.get(), *cfd->ioptions(), mutable_cf_options, + file_options_for_compaction_, cfd->table_cache(), iter.get(), std::move(range_del_iters), &meta, cfd->internal_comparator(), cfd->int_tbl_prop_collector_factories(), cfd->GetID(), cfd->GetName(), snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, @@ -1280,12 +1295,12 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, Status DBImpl::CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number, size_t preallocate_block_size, log::Writer** new_log) { Status s; - std::unique_ptr lfile; + std::unique_ptr lfile; DBOptions db_options = BuildDBOptions(immutable_db_options_, mutable_db_options_); - EnvOptions opt_env_options = - env_->OptimizeForLogWrite(env_options_, db_options); + FileOptions opt_file_options = + fs_->OptimizeForLogWrite(file_options_, db_options); std::string log_fname = LogFileName(immutable_db_options_.wal_dir, log_file_num); @@ -1295,10 +1310,10 @@ Status DBImpl::CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number, recycle_log_number); 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_options); + s = fs_->ReuseWritableFile(log_fname, old_log_fname, opt_file_options, + &lfile, /*dbg=*/nullptr); } else { - s = NewWritableFile(env_, log_fname, &lfile, opt_env_options); + s = NewWritableFile(fs_.get(), log_fname, &lfile, opt_file_options); } if (s.ok()) { @@ -1307,7 +1322,7 @@ Status DBImpl::CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number, const auto& listeners = immutable_db_options_.listeners; std::unique_ptr file_writer( - new WritableFileWriter(std::move(lfile), log_fname, opt_env_options, + new WritableFileWriter(std::move(lfile), log_fname, opt_file_options, env_, nullptr /* stats */, listeners)); *new_log = new log::Writer(std::move(file_writer), log_file_num, immutable_db_options_.recycle_log_file_num > 0, diff --git a/db/db_impl/db_impl_secondary.cc b/db/db_impl/db_impl_secondary.cc index a113019a9..c33fdb59f 100644 --- a/db/db_impl/db_impl_secondary.cc +++ b/db/db_impl/db_impl_secondary.cc @@ -144,9 +144,10 @@ Status DBImplSecondary::MaybeInitLogReader( std::unique_ptr file_reader; { - std::unique_ptr file; - Status status = env_->NewSequentialFile( - fname, &file, env_->OptimizeForLogRead(env_options_)); + std::unique_ptr file; + Status status = fs_->NewSequentialFile( + fname, fs_->OptimizeForLogRead(file_options_), &file, + nullptr); if (!status.ok()) { *log_reader = nullptr; return status; @@ -600,7 +601,7 @@ Status DB::OpenAsSecondary( handles->clear(); DBImplSecondary* impl = new DBImplSecondary(tmp_opts, dbname); impl->versions_.reset(new ReactiveVersionSet( - dbname, &impl->immutable_db_options_, impl->env_options_, + dbname, &impl->immutable_db_options_, impl->file_options_, impl->table_cache_.get(), impl->write_buffer_manager_, &impl->write_controller_)); impl->column_family_memtables_.reset( diff --git a/db/db_wal_test.cc b/db/db_wal_test.cc index 4e0b08c9a..e9167bcbe 100644 --- a/db/db_wal_test.cc +++ b/db/db_wal_test.cc @@ -8,6 +8,7 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. #include "db/db_test_util.h" +#include "env/composite_env_wrapper.h" #include "options/options_helper.h" #include "port/port.h" #include "port/stack_trace.h" @@ -902,8 +903,8 @@ class RecoveryTestHelper { std::string fname = LogFileName(test->dbname_, current_log_number); std::unique_ptr file; ASSERT_OK(db_options.env->NewWritableFile(fname, &file, env_options)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(file), fname, env_options)); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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)); diff --git a/db/external_sst_file_ingestion_job.cc b/db/external_sst_file_ingestion_job.cc index fd79ff1d0..c6410af2f 100644 --- a/db/external_sst_file_ingestion_job.cc +++ b/db/external_sst_file_ingestion_job.cc @@ -101,14 +101,15 @@ Status ExternalSstFileIngestionJob::Prepare( TableFileName(cfd_->ioptions()->cf_paths, f.fd.GetNumber(), f.fd.GetPathId()); if (ingestion_options_.move_files) { - status = env_->LinkFile(path_outside_db, path_inside_db); + status = + fs_->LinkFile(path_outside_db, path_inside_db, IOOptions(), nullptr); if (status.ok()) { // It is unsafe to assume application had sync the file and file // directory before ingest the file. For integrity of RocksDB we need // to sync the file. - std::unique_ptr file_to_sync; - status = env_->ReopenWritableFile(path_inside_db, &file_to_sync, - env_options_); + std::unique_ptr file_to_sync; + status = fs_->ReopenWritableFile(path_inside_db, env_options_, + &file_to_sync, nullptr); if (status.ok()) { TEST_SYNC_POINT( "ExternalSstFileIngestionJob::BeforeSyncIngestedFile"); @@ -133,7 +134,7 @@ Status ExternalSstFileIngestionJob::Prepare( TEST_SYNC_POINT_CALLBACK("ExternalSstFileIngestionJob::Prepare:CopyFile", nullptr); // CopyFile also sync the new file. - status = CopyFile(env_, path_outside_db, path_inside_db, 0, + status = CopyFile(fs_, path_outside_db, path_inside_db, 0, db_options_.use_fsync); } TEST_SYNC_POINT("ExternalSstFileIngestionJob::Prepare:FileAdded"); @@ -360,17 +361,19 @@ Status ExternalSstFileIngestionJob::GetIngestedFileInfo( file_to_ingest->external_file_path = external_file; // Get external file size - Status status = env_->GetFileSize(external_file, &file_to_ingest->file_size); + Status status = fs_->GetFileSize(external_file, IOOptions(), + &file_to_ingest->file_size, nullptr); if (!status.ok()) { return status; } // Create TableReader for external file std::unique_ptr table_reader; - std::unique_ptr sst_file; + std::unique_ptr sst_file; std::unique_ptr sst_file_reader; - status = env_->NewRandomAccessFile(external_file, &sst_file, env_options_); + status = fs_->NewRandomAccessFile(external_file, env_options_, + &sst_file, nullptr); if (!status.ok()) { return status; } @@ -651,13 +654,15 @@ Status ExternalSstFileIngestionJob::AssignGlobalSeqnoForIngestedFile( // Determine if we can write global_seqno to a given offset of file. // If the file system does not support random write, then we should not. // Otherwise we should. - std::unique_ptr rwfile; - Status status = env_->NewRandomRWFile(file_to_ingest->internal_file_path, - &rwfile, env_options_); + std::unique_ptr rwfile; + Status status = + fs_->NewRandomRWFile(file_to_ingest->internal_file_path, env_options_, + &rwfile, nullptr); if (status.ok()) { std::string seqno_val; PutFixed64(&seqno_val, seqno); - status = rwfile->Write(file_to_ingest->global_seqno_offset, seqno_val); + status = rwfile->Write(file_to_ingest->global_seqno_offset, seqno_val, + IOOptions(), nullptr); if (status.ok()) { TEST_SYNC_POINT("ExternalSstFileIngestionJob::BeforeSyncGlobalSeqno"); status = SyncIngestedFile(rwfile.get()); @@ -715,9 +720,9 @@ template Status ExternalSstFileIngestionJob::SyncIngestedFile(TWritableFile* file) { assert(file != nullptr); if (db_options_.use_fsync) { - return file->Fsync(); + return file->Fsync(IOOptions(), nullptr); } else { - return file->Sync(); + return file->Sync(IOOptions(), nullptr); } } diff --git a/db/external_sst_file_ingestion_job.h b/db/external_sst_file_ingestion_job.h index 90b8326bb..f6733218d 100644 --- a/db/external_sst_file_ingestion_job.h +++ b/db/external_sst_file_ingestion_job.h @@ -74,6 +74,7 @@ class ExternalSstFileIngestionJob { const IngestExternalFileOptions& ingestion_options, Directories* directories, EventLogger* event_logger) : env_(env), + fs_(db_options.fs.get()), versions_(versions), cfd_(cfd), db_options_(db_options), @@ -158,6 +159,7 @@ class ExternalSstFileIngestionJob { Status SyncIngestedFile(TWritableFile* file); Env* env_; + FileSystem* fs_; VersionSet* versions_; ColumnFamilyData* cfd_; const ImmutableDBOptions& db_options_; diff --git a/db/flush_job.cc b/db/flush_job.cc index bdb4c179b..2190c45c2 100644 --- a/db/flush_job.cc +++ b/db/flush_job.cc @@ -86,7 +86,7 @@ FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd, const ImmutableDBOptions& db_options, const MutableCFOptions& mutable_cf_options, const uint64_t* max_memtable_id, - const EnvOptions& env_options, VersionSet* versions, + const FileOptions& file_options, VersionSet* versions, InstrumentedMutex* db_mutex, std::atomic* shutting_down, std::vector existing_snapshots, @@ -103,7 +103,7 @@ FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd, db_options_(db_options), mutable_cf_options_(mutable_cf_options), max_memtable_id_(max_memtable_id), - env_options_(env_options), + file_options_(file_options), versions_(versions), db_mutex_(db_mutex), shutting_down_(shutting_down), @@ -371,8 +371,8 @@ Status FlushJob::WriteLevel0Table() { meta_.file_creation_time = current_time; s = BuildTable( - dbname_, db_options_.env, *cfd_->ioptions(), mutable_cf_options_, - env_options_, cfd_->table_cache(), iter.get(), + dbname_, db_options_.env, db_options_.fs.get(), *cfd_->ioptions(), + mutable_cf_options_, file_options_, cfd_->table_cache(), iter.get(), std::move(range_del_iters), &meta_, cfd_->internal_comparator(), cfd_->int_tbl_prop_collector_factories(), cfd_->GetID(), cfd_->GetName(), existing_snapshots_, diff --git a/db/flush_job.h b/db/flush_job.h index b25aca352..52161d261 100644 --- a/db/flush_job.h +++ b/db/flush_job.h @@ -61,7 +61,7 @@ class FlushJob { FlushJob(const std::string& dbname, ColumnFamilyData* cfd, const ImmutableDBOptions& db_options, const MutableCFOptions& mutable_cf_options, - const uint64_t* max_memtable_id, const EnvOptions& env_options, + const uint64_t* max_memtable_id, const FileOptions& file_options, VersionSet* versions, InstrumentedMutex* db_mutex, std::atomic* shutting_down, std::vector existing_snapshots, @@ -108,7 +108,7 @@ class FlushJob { // equal to *max_memtable_id_ will be selected for flush. If null, then all // memtables in the column family will be selected. const uint64_t* max_memtable_id_; - const EnvOptions env_options_; + const FileOptions file_options_; VersionSet* versions_; InstrumentedMutex* db_mutex_; std::atomic* shutting_down_; diff --git a/db/flush_job_test.cc b/db/flush_job_test.cc index fec737942..6c53d2135 100644 --- a/db/flush_job_test.cc +++ b/db/flush_job_test.cc @@ -30,16 +30,13 @@ class FlushJobTest : public testing::Test { public: FlushJobTest() : env_(Env::Default()), + fs_(std::make_shared(env_)), dbname_(test::PerThreadDBPath("flush_job_test")), options_(), db_options_(options_), column_family_names_({kDefaultColumnFamilyName, "foo", "bar"}), table_cache_(NewLRUCache(50000, 16)), write_buffer_manager_(db_options_.db_write_buffer_size), - versions_(new VersionSet(dbname_, &db_options_, env_options_, - table_cache_.get(), &write_buffer_manager_, - &write_controller_, - /*block_cache_tracer=*/nullptr)), shutting_down_(false), mock_table_factory_(new mock::MockTableFactory()) { EXPECT_OK(env_->CreateDirIfMissing(dbname_)); @@ -54,6 +51,12 @@ class FlushJobTest : public testing::Test { column_families.emplace_back(cf_name, cf_options_); } + db_options_.env = env_; + db_options_.fs = fs_; + versions_.reset(new VersionSet(dbname_, &db_options_, env_options_, + table_cache_.get(), &write_buffer_manager_, + &write_controller_, + /*block_cache_tracer=*/nullptr)); EXPECT_OK(versions_->Recover(column_families, false)); } @@ -88,8 +91,8 @@ class FlushJobTest : public testing::Test { Status s = env_->NewWritableFile( manifest, &file, env_->OptimizeForManifestWrite(env_options_)); ASSERT_OK(s); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(file), manifest, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(file)), manifest, EnvOptions())); { log::Writer log(std::move(file_writer), 0, false); std::string record; @@ -109,6 +112,7 @@ class FlushJobTest : public testing::Test { } Env* env_; + std::shared_ptr fs_; std::string dbname_; EnvOptions env_options_; Options options_; diff --git a/db/import_column_family_job.cc b/db/import_column_family_job.cc index f52418a07..847935738 100644 --- a/db/import_column_family_job.cc +++ b/db/import_column_family_job.cc @@ -92,14 +92,15 @@ Status ImportColumnFamilyJob::Prepare(uint64_t next_file_number, cfd_->ioptions()->cf_paths, f.fd.GetNumber(), f.fd.GetPathId()); if (hardlink_files) { - status = env_->LinkFile(path_outside_db, path_inside_db); + status = + fs_->LinkFile(path_outside_db, path_inside_db, IOOptions(), nullptr); if (status.IsNotSupported()) { // Original file is on a different FS, use copy instead of hard linking hardlink_files = false; } } if (!hardlink_files) { - status = CopyFile(env_, path_outside_db, path_inside_db, 0, + status = CopyFile(fs_, path_outside_db, path_inside_db, 0, db_options_.use_fsync); } if (!status.ok()) { @@ -115,7 +116,8 @@ Status ImportColumnFamilyJob::Prepare(uint64_t next_file_number, if (f.internal_file_path.empty()) { break; } - const auto s = env_->DeleteFile(f.internal_file_path); + const auto s = + fs_->DeleteFile(f.internal_file_path, IOOptions(), nullptr); if (!s.ok()) { ROCKS_LOG_WARN(db_options_.info_log, "AddFile() clean up for file %s failed : %s", @@ -168,7 +170,8 @@ void ImportColumnFamilyJob::Cleanup(const Status& status) { if (!status.ok()) { // We failed to add files to the database remove all the files we copied. for (const auto& f : files_to_import_) { - const auto s = env_->DeleteFile(f.internal_file_path); + const auto s = + fs_->DeleteFile(f.internal_file_path, IOOptions(), nullptr); if (!s.ok()) { ROCKS_LOG_WARN(db_options_.info_log, "AddFile() clean up for file %s failed : %s", @@ -178,7 +181,8 @@ void ImportColumnFamilyJob::Cleanup(const Status& status) { } else if (status.ok() && import_options_.move_files) { // The files were moved and added successfully, remove original file links for (IngestedFileInfo& f : files_to_import_) { - const auto s = env_->DeleteFile(f.external_file_path); + const auto s = + fs_->DeleteFile(f.external_file_path, IOOptions(), nullptr); if (!s.ok()) { ROCKS_LOG_WARN( db_options_.info_log, @@ -196,17 +200,19 @@ Status ImportColumnFamilyJob::GetIngestedFileInfo( file_to_import->external_file_path = external_file; // Get external file size - auto status = env_->GetFileSize(external_file, &file_to_import->file_size); + Status status = fs_->GetFileSize(external_file, IOOptions(), + &file_to_import->file_size, nullptr); if (!status.ok()) { return status; } // Create TableReader for external file std::unique_ptr table_reader; - std::unique_ptr sst_file; + std::unique_ptr sst_file; std::unique_ptr sst_file_reader; - status = env_->NewRandomAccessFile(external_file, &sst_file, env_options_); + status = fs_->NewRandomAccessFile(external_file, env_options_, + &sst_file, nullptr); if (!status.ok()) { return status; } diff --git a/db/import_column_family_job.h b/db/import_column_family_job.h index 05796590b..1bdb4fb2b 100644 --- a/db/import_column_family_job.h +++ b/db/import_column_family_job.h @@ -29,6 +29,7 @@ class ImportColumnFamilyJob { versions_(versions), cfd_(cfd), db_options_(db_options), + fs_(db_options_.fs.get()), env_options_(env_options), import_options_(import_options), metadata_(metadata) {} @@ -60,6 +61,7 @@ class ImportColumnFamilyJob { VersionSet* versions_; ColumnFamilyData* cfd_; const ImmutableDBOptions& db_options_; + FileSystem* fs_; const EnvOptions& env_options_; autovector files_to_import_; VersionEdit edit_; diff --git a/db/log_test.cc b/db/log_test.cc index ecfae3e2d..b76e22118 100644 --- a/db/log_test.cc +++ b/db/log_test.cc @@ -9,6 +9,7 @@ #include "db/log_reader.h" #include "db/log_writer.h" +#include "env/composite_env_wrapper.h" #include "file/sequence_file_reader.h" #include "file/writable_file_writer.h" #include "rocksdb/env.h" @@ -120,6 +121,13 @@ class LogTest : public ::testing::TestWithParam> { } }; + inline StringSource* GetStringSourceFromLegacyReader( + SequentialFileReader* reader) { + LegacySequentialFileWrapper* file = + static_cast(reader->file()); + return static_cast(file->target()); + } + class ReportCollector : public Reader::Reporter { public: size_t dropped_bytes_; @@ -133,21 +141,19 @@ class LogTest : public ::testing::TestWithParam> { }; std::string& dest_contents() { - auto dest = - dynamic_cast(writer_.file()->writable_file()); + auto dest = test::GetStringSinkFromLegacyWriter(writer_.file()); assert(dest); return dest->contents_; } const std::string& dest_contents() const { - auto dest = - dynamic_cast(writer_.file()->writable_file()); + auto dest = test::GetStringSinkFromLegacyWriter(writer_.file()); assert(dest); return dest->contents_; } void reset_source_contents() { - auto src = dynamic_cast(reader_->file()->file()); + auto src = GetStringSourceFromLegacyReader(reader_->file()); assert(src); src->contents_ = dest_contents(); } @@ -214,8 +220,7 @@ class LogTest : public ::testing::TestWithParam> { } void ShrinkSize(int bytes) { - auto dest = - dynamic_cast(writer_.file()->writable_file()); + auto dest = test::GetStringSinkFromLegacyWriter(writer_.file()); assert(dest); dest->Drop(bytes); } @@ -230,7 +235,7 @@ class LogTest : public ::testing::TestWithParam> { } void ForceError(size_t position = 0) { - auto src = dynamic_cast(reader_->file()->file()); + auto src = GetStringSourceFromLegacyReader(reader_->file()); src->force_error_ = true; src->force_error_position_ = position; } @@ -244,13 +249,13 @@ class LogTest : public ::testing::TestWithParam> { } void ForceEOF(size_t position = 0) { - auto src = dynamic_cast(reader_->file()->file()); + auto src = GetStringSourceFromLegacyReader(reader_->file()); src->force_eof_ = true; src->force_eof_position_ = position; } void UnmarkEOF() { - auto src = dynamic_cast(reader_->file()->file()); + auto src = GetStringSourceFromLegacyReader(reader_->file()); src->returned_partial_ = false; reader_->UnmarkEOF(); } @@ -752,8 +757,9 @@ class RetriableLogTest : public ::testing::TestWithParam { s = env_->NewWritableFile(log_file_, &writable_file, env_options_); } if (s.ok()) { - writer_.reset(new WritableFileWriter(std::move(writable_file), log_file_, - env_options_)); + writer_.reset(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(writable_file)), log_file_, + env_options_)); assert(writer_ != nullptr); } std::unique_ptr seq_file; @@ -761,7 +767,8 @@ class RetriableLogTest : public ::testing::TestWithParam { s = env_->NewSequentialFile(log_file_, &seq_file, env_options_); } if (s.ok()) { - reader_.reset(new SequentialFileReader(std::move(seq_file), log_file_)); + reader_.reset(new SequentialFileReader( + NewLegacySequentialFileWrapper(seq_file), log_file_)); assert(reader_ != nullptr); log_reader_.reset(new FragmentBufferedReader( nullptr, std::move(reader_), &report_, true /* checksum */, @@ -772,8 +779,7 @@ class RetriableLogTest : public ::testing::TestWithParam { } std::string contents() { - auto file = - dynamic_cast(log_writer_->file()->writable_file()); + auto file = test::GetStringSinkFromLegacyWriter(log_writer_->file()); assert(file != nullptr); return file->contents_; } diff --git a/db/memtable_list_test.cc b/db/memtable_list_test.cc index 32a227f4b..7df20674a 100644 --- a/db/memtable_list_test.cc +++ b/db/memtable_list_test.cc @@ -92,6 +92,7 @@ class MemTableListTest : public testing::Test { CreateDB(); // Create a mock VersionSet DBOptions db_options; + db_options.file_system = FileSystem::Default(); ImmutableDBOptions immutable_db_options(db_options); EnvOptions env_options; std::shared_ptr table_cache(NewLRUCache(50000, 16)); @@ -138,6 +139,8 @@ class MemTableListTest : public testing::Test { CreateDB(); // Create a mock VersionSet DBOptions db_options; + db_options.file_system.reset(new LegacyFileSystemWrapper(db_options.env)); + ImmutableDBOptions immutable_db_options(db_options); EnvOptions env_options; std::shared_ptr table_cache(NewLRUCache(50000, 16)); diff --git a/db/repair.cc b/db/repair.cc index b71f725a2..b31a723d4 100644 --- a/db/repair.cc +++ b/db/repair.cc @@ -71,6 +71,7 @@ #include "db/table_cache.h" #include "db/version_edit.h" #include "db/write_batch_internal.h" +#include "env/composite_env_wrapper.h" #include "file/filename.h" #include "file/writable_file_writer.h" #include "options/cf_options.h" @@ -348,8 +349,8 @@ class Repairer { if (!status.ok()) { return status; } - std::unique_ptr lfile_reader( - new SequentialFileReader(std::move(lfile), logname)); + std::unique_ptr lfile_reader(new SequentialFileReader( + NewLegacySequentialFileWrapper(lfile), logname)); // Create the log reader. LogReporter reporter; @@ -421,17 +422,19 @@ class Repairer { if (range_del_iter != nullptr) { range_del_iters.emplace_back(range_del_iter); } + + LegacyFileSystemWrapper fs(env_); status = BuildTable( - dbname_, env_, *cfd->ioptions(), *cfd->GetLatestMutableCFOptions(), - env_options_, table_cache_, iter.get(), std::move(range_del_iters), - &meta, cfd->internal_comparator(), - cfd->int_tbl_prop_collector_factories(), cfd->GetID(), cfd->GetName(), - {}, kMaxSequenceNumber, snapshot_checker, kNoCompression, - 0 /* sample_for_compression */, CompressionOptions(), false, - nullptr /* internal_stats */, TableFileCreationReason::kRecovery, - nullptr /* event_logger */, 0 /* job_id */, Env::IO_HIGH, - nullptr /* table_properties */, -1 /* level */, current_time, - write_hint); + dbname_, env_, &fs, *cfd->ioptions(), + *cfd->GetLatestMutableCFOptions(), env_options_, table_cache_, + iter.get(), std::move(range_del_iters), &meta, + cfd->internal_comparator(), cfd->int_tbl_prop_collector_factories(), + cfd->GetID(), cfd->GetName(), {}, kMaxSequenceNumber, + snapshot_checker, kNoCompression, 0 /* sample_for_compression */, + CompressionOptions(), false, nullptr /* internal_stats */, + TableFileCreationReason::kRecovery, nullptr /* event_logger */, + 0 /* job_id */, Env::IO_HIGH, nullptr /* table_properties */, + -1 /* level */, current_time, write_hint); ROCKS_LOG_INFO(db_options_.info_log, "Log #%" PRIu64 ": %d ops saved to Table #%" PRIu64 " %s", log, counter, meta.fd.GetNumber(), @@ -667,8 +670,14 @@ Status RepairDB(const std::string& dbname, const DBOptions& db_options, } Status RepairDB(const std::string& dbname, const Options& options) { - DBOptions db_options(options); - ColumnFamilyOptions cf_options(options); + Options opts(options); + if (opts.file_system == nullptr) { + opts.file_system.reset(new LegacyFileSystemWrapper(opts.env)); + ; + } + + DBOptions db_options(opts); + ColumnFamilyOptions cf_options(opts); Repairer repairer(dbname, db_options, {}, cf_options /* default_cf_opts */, cf_options /* unknown_cf_opts */, diff --git a/db/repair_test.cc b/db/repair_test.cc index 21907e435..d9e0556c6 100644 --- a/db/repair_test.cc +++ b/db/repair_test.cc @@ -74,7 +74,9 @@ TEST_F(RepairTest, CorruptManifest) { Close(); ASSERT_OK(env_->FileExists(manifest_path)); - CreateFile(env_, manifest_path, "blah", false /* use_fsync */); + + LegacyFileSystemWrapper fs(env_); + CreateFile(&fs, manifest_path, "blah", false /* use_fsync */); ASSERT_OK(RepairDB(dbname_, CurrentOptions())); Reopen(CurrentOptions()); @@ -153,7 +155,9 @@ TEST_F(RepairTest, CorruptSst) { Flush(); auto sst_path = GetFirstSstPath(); ASSERT_FALSE(sst_path.empty()); - CreateFile(env_, sst_path, "blah", false /* use_fsync */); + + LegacyFileSystemWrapper fs(env_); + CreateFile(&fs, sst_path, "blah", false /* use_fsync */); Close(); ASSERT_OK(RepairDB(dbname_, CurrentOptions())); diff --git a/db/table_cache.cc b/db/table_cache.cc index 89acd3d84..8a8a80ba3 100644 --- a/db/table_cache.cc +++ b/db/table_cache.cc @@ -63,10 +63,10 @@ void AppendVarint64(IterKey* key, uint64_t v) { } // namespace TableCache::TableCache(const ImmutableCFOptions& ioptions, - const EnvOptions& env_options, Cache* const cache, + const FileOptions& file_options, Cache* const cache, BlockCacheTracer* const block_cache_tracer) : ioptions_(ioptions), - env_options_(env_options), + file_options_(file_options), cache_(cache), immortal_tables_(false), block_cache_tracer_(block_cache_tracer) { @@ -89,7 +89,7 @@ void TableCache::ReleaseHandle(Cache::Handle* handle) { } Status TableCache::GetTableReader( - const EnvOptions& env_options, + const FileOptions& file_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, bool sequential_mode, bool record_read_stats, HistogramImpl* file_read_hist, std::unique_ptr* table_reader, @@ -97,13 +97,14 @@ Status TableCache::GetTableReader( bool prefetch_index_and_filter_in_cache) { std::string fname = TableFileName(ioptions_.cf_paths, fd.GetNumber(), fd.GetPathId()); - std::unique_ptr file; - Status s = ioptions_.env->NewRandomAccessFile(fname, &file, env_options); + std::unique_ptr file; + Status s = ioptions_.fs->NewRandomAccessFile(fname, file_options, &file, + nullptr); RecordTick(ioptions_.statistics, NO_FILE_OPENS); if (s.ok()) { if (!sequential_mode && ioptions_.advise_random_on_open) { - file->Hint(RandomAccessFile::RANDOM); + file->Hint(FSRandomAccessFile::kRandom); } StopWatch sw(ioptions_.env, ioptions_.statistics, TABLE_OPEN_IO_MICROS); std::unique_ptr file_reader( @@ -112,7 +113,7 @@ Status TableCache::GetTableReader( record_read_stats ? ioptions_.statistics : nullptr, SST_READ_MICROS, file_read_hist, ioptions_.rate_limiter, ioptions_.listeners)); s = ioptions_.table_factory->NewTableReader( - TableReaderOptions(ioptions_, prefix_extractor, env_options, + TableReaderOptions(ioptions_, prefix_extractor, file_options, internal_comparator, skip_filters, immortal_tables_, level, fd.largest_seqno, block_cache_tracer_), std::move(file_reader), fd.GetFileSize(), table_reader, @@ -129,7 +130,7 @@ void TableCache::EraseHandle(const FileDescriptor& fd, Cache::Handle* handle) { cache_->Erase(key); } -Status TableCache::FindTable(const EnvOptions& env_options, +Status TableCache::FindTable(const FileOptions& file_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, Cache::Handle** handle, const SliceTransform* prefix_extractor, @@ -150,7 +151,7 @@ Status TableCache::FindTable(const EnvOptions& env_options, return Status::Incomplete("Table not found in table_cache, no_io is set"); } std::unique_ptr table_reader; - s = GetTableReader(env_options, internal_comparator, fd, + s = GetTableReader(file_options, internal_comparator, fd, false /* sequential mode */, record_read_stats, file_read_hist, &table_reader, prefix_extractor, skip_filters, level, prefetch_index_and_filter_in_cache); @@ -172,7 +173,7 @@ Status TableCache::FindTable(const EnvOptions& env_options, } InternalIterator* TableCache::NewIterator( - const ReadOptions& options, const EnvOptions& env_options, + const ReadOptions& options, const FileOptions& file_options, const InternalKeyComparator& icomparator, const FileMetaData& file_meta, RangeDelAggregator* range_del_agg, const SliceTransform* prefix_extractor, TableReader** table_reader_ptr, HistogramImpl* file_read_hist, @@ -191,7 +192,7 @@ InternalIterator* TableCache::NewIterator( auto& fd = file_meta.fd; table_reader = fd.table_reader; if (table_reader == nullptr) { - s = FindTable(env_options, icomparator, fd, &handle, prefix_extractor, + s = FindTable(file_options, icomparator, fd, &handle, prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */, !for_compaction /* record_read_stats */, file_read_hist, skip_filters, level); @@ -206,8 +207,8 @@ InternalIterator* TableCache::NewIterator( result = NewEmptyInternalIterator(arena); } else { result = table_reader->NewIterator(options, prefix_extractor, arena, - skip_filters, caller, - env_options.compaction_readahead_size); + skip_filters, caller, + file_options.compaction_readahead_size); } if (handle != nullptr) { result->RegisterCleanup(&UnrefEntry, cache_, handle); @@ -264,7 +265,7 @@ Status TableCache::GetRangeTombstoneIterator( TableReader* t = fd.table_reader; Cache::Handle* handle = nullptr; if (t == nullptr) { - s = FindTable(env_options_, internal_comparator, fd, &handle); + s = FindTable(file_options_, internal_comparator, fd, &handle); if (s.ok()) { t = GetTableReaderFromHandle(handle); } @@ -381,7 +382,7 @@ Status TableCache::Get(const ReadOptions& options, if (!done && s.ok()) { if (t == nullptr) { s = FindTable( - env_options_, internal_comparator, fd, &handle, prefix_extractor, + file_options_, internal_comparator, fd, &handle, prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */, true /* record_read_stats */, file_read_hist, skip_filters, level); if (s.ok()) { @@ -481,7 +482,7 @@ Status TableCache::MultiGet(const ReadOptions& options, if (s.ok() && !table_range.empty()) { if (t == nullptr) { s = FindTable( - env_options_, internal_comparator, fd, &handle, prefix_extractor, + file_options_, internal_comparator, fd, &handle, prefix_extractor, options.read_tier == kBlockCacheTier /* no_io */, true /* record_read_stats */, file_read_hist, skip_filters, level); if (s.ok()) { @@ -551,7 +552,7 @@ Status TableCache::MultiGet(const ReadOptions& options, } Status TableCache::GetTableProperties( - const EnvOptions& env_options, + const FileOptions& file_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, std::shared_ptr* properties, const SliceTransform* prefix_extractor, bool no_io) { @@ -565,7 +566,7 @@ Status TableCache::GetTableProperties( } Cache::Handle* table_handle = nullptr; - s = FindTable(env_options, internal_comparator, fd, &table_handle, + s = FindTable(file_options, internal_comparator, fd, &table_handle, prefix_extractor, no_io); if (!s.ok()) { return s; @@ -578,7 +579,7 @@ Status TableCache::GetTableProperties( } size_t TableCache::GetMemoryUsageByTableReader( - const EnvOptions& env_options, + const FileOptions& file_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, const SliceTransform* prefix_extractor) { Status s; @@ -589,7 +590,7 @@ size_t TableCache::GetMemoryUsageByTableReader( } Cache::Handle* table_handle = nullptr; - s = FindTable(env_options, internal_comparator, fd, &table_handle, + s = FindTable(file_options, internal_comparator, fd, &table_handle, prefix_extractor, true); if (!s.ok()) { return 0; @@ -614,7 +615,7 @@ uint64_t TableCache::ApproximateOffsetOf( Cache::Handle* table_handle = nullptr; if (table_reader == nullptr) { const bool for_compaction = (caller == TableReaderCaller::kCompaction); - Status s = FindTable(env_options_, internal_comparator, fd, &table_handle, + Status s = FindTable(file_options_, internal_comparator, fd, &table_handle, prefix_extractor, false /* no_io */, !for_compaction /* record_read_stats */); if (s.ok()) { @@ -641,7 +642,7 @@ uint64_t TableCache::ApproximateSize( Cache::Handle* table_handle = nullptr; if (table_reader == nullptr) { const bool for_compaction = (caller == TableReaderCaller::kCompaction); - Status s = FindTable(env_options_, internal_comparator, fd, &table_handle, + Status s = FindTable(file_options_, internal_comparator, fd, &table_handle, prefix_extractor, false /* no_io */, !for_compaction /* record_read_stats */); if (s.ok()) { diff --git a/db/table_cache.h b/db/table_cache.h index 088040672..8319901b2 100644 --- a/db/table_cache.h +++ b/db/table_cache.h @@ -49,7 +49,7 @@ class HistogramImpl; class TableCache { public: TableCache(const ImmutableCFOptions& ioptions, - const EnvOptions& storage_options, Cache* cache, + const FileOptions& storage_options, Cache* cache, BlockCacheTracer* const block_cache_tracer); ~TableCache(); @@ -67,7 +67,7 @@ class TableCache { // @param skip_filters Disables loading/accessing the filter block // @param level The level this table is at, -1 for "not set / don't know" InternalIterator* NewIterator( - const ReadOptions& options, const EnvOptions& toptions, + const ReadOptions& options, const FileOptions& toptions, const InternalKeyComparator& internal_comparator, const FileMetaData& file_meta, RangeDelAggregator* range_del_agg, const SliceTransform* prefix_extractor, TableReader** table_reader_ptr, @@ -128,7 +128,7 @@ class TableCache { // Find table reader // @param skip_filters Disables loading/accessing the filter block // @param level == -1 means not specified - Status FindTable(const EnvOptions& toptions, + Status FindTable(const FileOptions& toptions, const InternalKeyComparator& internal_comparator, const FileDescriptor& file_fd, Cache::Handle**, const SliceTransform* prefix_extractor = nullptr, @@ -146,7 +146,7 @@ class TableCache { // @returns: `properties` will be reset on success. Please note that we will // return Status::Incomplete() if table is not present in cache and // we set `no_io` to be true. - Status GetTableProperties(const EnvOptions& toptions, + Status GetTableProperties(const FileOptions& toptions, const InternalKeyComparator& internal_comparator, const FileDescriptor& file_meta, std::shared_ptr* properties, @@ -156,7 +156,7 @@ class TableCache { // Return total memory usage of the table reader of the file. // 0 if table reader of the file is not loaded. size_t GetMemoryUsageByTableReader( - const EnvOptions& toptions, + const FileOptions& toptions, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, const SliceTransform* prefix_extractor = nullptr); @@ -193,7 +193,7 @@ class TableCache { private: // Build a table reader - Status GetTableReader(const EnvOptions& env_options, + Status GetTableReader(const FileOptions& file_options, const InternalKeyComparator& internal_comparator, const FileDescriptor& fd, bool sequential_mode, bool record_read_stats, HistogramImpl* file_read_hist, @@ -216,7 +216,7 @@ class TableCache { size_t prefix_size, GetContext* get_context); const ImmutableCFOptions& ioptions_; - const EnvOptions& env_options_; + const FileOptions& file_options_; Cache* const cache_; std::string row_cache_id_; bool immortal_tables_; diff --git a/db/table_properties_collector_test.cc b/db/table_properties_collector_test.cc index e479fa008..9c8129570 100644 --- a/db/table_properties_collector_test.cc +++ b/db/table_properties_collector_test.cc @@ -12,6 +12,7 @@ #include "db/db_impl/db_impl.h" #include "db/dbformat.h" #include "db/table_properties_collector.h" +#include "env/composite_env_wrapper.h" #include "file/sequence_file_reader.h" #include "file/writable_file_writer.h" #include "options/cf_options.h" @@ -48,7 +49,8 @@ void MakeBuilder(const Options& options, const ImmutableCFOptions& ioptions, std::unique_ptr* builder) { std::unique_ptr wf(new test::StringSink); writable->reset( - new WritableFileWriter(std::move(wf), "" /* don't care */, EnvOptions())); + new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(wf)), + "" /* don't care */, EnvOptions())); int unknown_level = -1; builder->reset(NewTableBuilder( ioptions, moptions, internal_comparator, int_tbl_prop_collector_factories, @@ -282,8 +284,9 @@ void TestCustomizedTablePropertiesCollector( writer->Flush(); // -- Step 2: Read properties - test::StringSink* fwf = - static_cast(writer->writable_file()); + LegacyWritableFileWrapper* file = + static_cast(writer->writable_file()); + test::StringSink* fwf = static_cast(file->target()); std::unique_ptr fake_file_reader( test::GetRandomAccessFileReader( new test::StringSource(fwf->contents()))); @@ -422,8 +425,9 @@ void TestInternalKeyPropertiesCollector( ASSERT_OK(builder->Finish()); writable->Flush(); - test::StringSink* fwf = - static_cast(writable->writable_file()); + LegacyWritableFileWrapper* file = + static_cast(writable->writable_file()); + test::StringSink* fwf = static_cast(file->target()); std::unique_ptr reader( test::GetRandomAccessFileReader( new test::StringSource(fwf->contents()))); diff --git a/db/transaction_log_impl.cc b/db/transaction_log_impl.cc index 42c724c03..6c8f6dac8 100644 --- a/db/transaction_log_impl.cc +++ b/db/transaction_log_impl.cc @@ -42,22 +42,23 @@ TransactionLogIteratorImpl::TransactionLogIteratorImpl( Status TransactionLogIteratorImpl::OpenLogFile( const LogFile* log_file, std::unique_ptr* file_reader) { - Env* env = options_->env; - std::unique_ptr file; + FileSystem* fs = options_->fs.get(); + std::unique_ptr file; std::string fname; Status s; - EnvOptions optimized_env_options = env->OptimizeForLogRead(soptions_); + EnvOptions optimized_env_options = fs->OptimizeForLogRead(soptions_); if (log_file->Type() == kArchivedLogFile) { fname = ArchivedLogFileName(dir_, log_file->LogNumber()); - s = env->NewSequentialFile(fname, &file, optimized_env_options); + s = fs->NewSequentialFile(fname, optimized_env_options, &file, nullptr); } else { fname = LogFileName(dir_, log_file->LogNumber()); - s = env->NewSequentialFile(fname, &file, optimized_env_options); + s = fs->NewSequentialFile(fname, optimized_env_options, &file, nullptr); 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_, log_file->LogNumber()); - s = env->NewSequentialFile(fname, &file, optimized_env_options); + s = fs->NewSequentialFile(fname, optimized_env_options, + &file, nullptr); } } if (s.ok()) { diff --git a/db/version_builder.cc b/db/version_builder.cc index 53e25a446..6cd3de186 100644 --- a/db/version_builder.cc +++ b/db/version_builder.cc @@ -83,7 +83,7 @@ class VersionBuilder::Rep { std::unordered_map added_files; }; - const EnvOptions& env_options_; + const FileOptions& file_options_; Logger* info_log_; TableCache* table_cache_; VersionStorageInfo* base_vstorage_; @@ -101,9 +101,10 @@ class VersionBuilder::Rep { FileComparator level_nonzero_cmp_; public: - Rep(const EnvOptions& env_options, Logger* info_log, TableCache* table_cache, + Rep(const FileOptions& file_options, Logger* info_log, + TableCache* table_cache, VersionStorageInfo* base_vstorage) - : env_options_(env_options), + : file_options_(file_options), info_log_(info_log), table_cache_(table_cache), base_vstorage_(base_vstorage), @@ -461,7 +462,7 @@ class VersionBuilder::Rep { auto* file_meta = files_meta[file_idx].first; int level = files_meta[file_idx].second; statuses[file_idx] = table_cache_->FindTable( - env_options_, *(base_vstorage_->InternalComparator()), + file_options_, *(base_vstorage_->InternalComparator()), file_meta->fd, &file_meta->table_reader_handle, prefix_extractor, false /*no_io */, true /* record_read_stats */, internal_stats->GetFileReadHist(level), false, level, @@ -500,11 +501,11 @@ class VersionBuilder::Rep { } }; -VersionBuilder::VersionBuilder(const EnvOptions& env_options, +VersionBuilder::VersionBuilder(const FileOptions& file_options, TableCache* table_cache, VersionStorageInfo* base_vstorage, Logger* info_log) - : rep_(new Rep(env_options, info_log, table_cache, base_vstorage)) {} + : rep_(new Rep(file_options, info_log, table_cache, base_vstorage)) {} VersionBuilder::~VersionBuilder() { delete rep_; } diff --git a/db/version_builder.h b/db/version_builder.h index f5fd12189..ba38a2f2a 100644 --- a/db/version_builder.h +++ b/db/version_builder.h @@ -8,7 +8,7 @@ // found in the LICENSE file. See the AUTHORS file for names of contributors. // #pragma once -#include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/slice_transform.h" namespace rocksdb { @@ -24,7 +24,7 @@ class InternalStats; // Versions that contain full copies of the intermediate state. class VersionBuilder { public: - VersionBuilder(const EnvOptions& env_options, TableCache* table_cache, + VersionBuilder(const FileOptions& file_options, TableCache* table_cache, VersionStorageInfo* base_vstorage, Logger* info_log = nullptr); ~VersionBuilder(); Status CheckConsistency(VersionStorageInfo* vstorage); diff --git a/db/version_set.cc b/db/version_set.cc index f9627f81d..06d4abd7f 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -853,7 +853,7 @@ namespace { class LevelIterator final : public InternalIterator { public: LevelIterator(TableCache* table_cache, const ReadOptions& read_options, - const EnvOptions& env_options, + const FileOptions& file_options, const InternalKeyComparator& icomparator, const LevelFilesBrief* flevel, const SliceTransform* prefix_extractor, bool should_sample, @@ -863,7 +863,7 @@ class LevelIterator final : public InternalIterator { compaction_boundaries = nullptr) : table_cache_(table_cache), read_options_(read_options), - env_options_(env_options), + file_options_(file_options), icomparator_(icomparator), user_comparator_(icomparator.user_comparator()), flevel_(flevel), @@ -975,7 +975,7 @@ class LevelIterator final : public InternalIterator { } CheckMayBeOutOfLowerBound(); return table_cache_->NewIterator( - read_options_, env_options_, icomparator_, *file_meta.file_metadata, + read_options_, file_options_, icomparator_, *file_meta.file_metadata, range_del_agg_, prefix_extractor_, nullptr /* don't need reference to table */, file_read_hist_, caller_, /*arena=*/nullptr, skip_filters_, level_, smallest_compaction_key, @@ -998,7 +998,7 @@ class LevelIterator final : public InternalIterator { TableCache* table_cache_; const ReadOptions read_options_; - const EnvOptions& env_options_; + const FileOptions& file_options_; const InternalKeyComparator& icomparator_; const UserComparatorWrapper user_comparator_; const LevelFilesBrief* flevel_; @@ -1206,7 +1206,7 @@ class BaseReferencedVersionBuilder { public: explicit BaseReferencedVersionBuilder(ColumnFamilyData* cfd) : version_builder_(new VersionBuilder( - cfd->current()->version_set()->env_options(), cfd->table_cache(), + cfd->current()->version_set()->file_options(), cfd->table_cache(), cfd->current()->storage_info(), cfd->ioptions()->info_log)), version_(cfd->current()) { version_->Ref(); @@ -1227,7 +1227,7 @@ Status Version::GetTableProperties(std::shared_ptr* tp, auto table_cache = cfd_->table_cache(); auto ioptions = cfd_->ioptions(); Status s = table_cache->GetTableProperties( - env_options_, cfd_->internal_comparator(), file_meta->fd, tp, + file_options_, cfd_->internal_comparator(), file_meta->fd, tp, mutable_cf_options_.prefix_extractor.get(), true /* no io */); if (s.ok()) { return s; @@ -1241,7 +1241,7 @@ Status Version::GetTableProperties(std::shared_ptr* tp, // 2. Table is not present in table cache, we'll read the table properties // directly from the properties block in the file. - std::unique_ptr file; + std::unique_ptr file; std::string file_name; if (fname != nullptr) { file_name = *fname; @@ -1250,7 +1250,8 @@ Status Version::GetTableProperties(std::shared_ptr* tp, TableFileName(ioptions->cf_paths, file_meta->fd.GetNumber(), file_meta->fd.GetPathId()); } - s = ioptions->env->NewRandomAccessFile(file_name, &file, env_options_); + s = ioptions->fs->NewRandomAccessFile(file_name, file_options_, &file, + nullptr); if (!s.ok()) { return s; } @@ -1420,7 +1421,7 @@ size_t Version::GetMemoryUsageByTableReaders() { for (auto& file_level : storage_info_.level_files_brief_) { for (size_t i = 0; i < file_level.num_files; i++) { total_usage += cfd_->table_cache()->GetMemoryUsageByTableReader( - env_options_, cfd_->internal_comparator(), file_level.files[i].fd, + file_options_, cfd_->internal_comparator(), file_level.files[i].fd, mutable_cf_options_.prefix_extractor.get()); } } @@ -1547,7 +1548,7 @@ double VersionStorageInfo::GetEstimatedCompressionRatioAtLevel( } void Version::AddIterators(const ReadOptions& read_options, - const EnvOptions& soptions, + const FileOptions& soptions, MergeIteratorBuilder* merge_iter_builder, RangeDelAggregator* range_del_agg) { assert(storage_info_.finalized_); @@ -1559,7 +1560,7 @@ void Version::AddIterators(const ReadOptions& read_options, } void Version::AddIteratorsForLevel(const ReadOptions& read_options, - const EnvOptions& soptions, + const FileOptions& soptions, MergeIteratorBuilder* merge_iter_builder, int level, RangeDelAggregator* range_del_agg) { @@ -1614,7 +1615,7 @@ void Version::AddIteratorsForLevel(const ReadOptions& read_options, } Status Version::OverlapWithLevelIterator(const ReadOptions& read_options, - const EnvOptions& env_options, + const FileOptions& file_options, const Slice& smallest_user_key, const Slice& largest_user_key, int level, bool* overlap) { @@ -1638,7 +1639,7 @@ Status Version::OverlapWithLevelIterator(const ReadOptions& read_options, continue; } ScopedArenaIterator iter(cfd_->table_cache()->NewIterator( - read_options, env_options, cfd_->internal_comparator(), + read_options, file_options, cfd_->internal_comparator(), *file->file_metadata, &range_del_agg, mutable_cf_options_.prefix_extractor.get(), nullptr, cfd_->internal_stats()->GetFileReadHist(0), @@ -1655,7 +1656,7 @@ Status Version::OverlapWithLevelIterator(const ReadOptions& read_options, } else if (storage_info_.LevelFilesBrief(level).num_files > 0) { auto mem = arena.AllocateAligned(sizeof(LevelIterator)); ScopedArenaIterator iter(new (mem) LevelIterator( - cfd_->table_cache(), read_options, env_options, + cfd_->table_cache(), read_options, file_options, cfd_->internal_comparator(), &storage_info_.LevelFilesBrief(level), mutable_cf_options_.prefix_extractor.get(), should_sample_file_read(), cfd_->internal_stats()->GetFileReadHist(level), @@ -1719,7 +1720,7 @@ VersionStorageInfo::VersionStorageInfo( } Version::Version(ColumnFamilyData* column_family_data, VersionSet* vset, - const EnvOptions& env_opt, + const FileOptions& file_opt, const MutableCFOptions mutable_cf_options, uint64_t version_number) : env_(vset->env_), @@ -1744,7 +1745,7 @@ Version::Version(ColumnFamilyData* column_family_data, VersionSet* vset, next_(this), prev_(this), refs_(0), - env_options_(env_opt), + file_options_(file_opt), mutable_cf_options_(mutable_cf_options), version_number_(version_number) {} @@ -3509,7 +3510,7 @@ void AtomicGroupReadBuffer::Clear() { VersionSet::VersionSet(const std::string& dbname, const ImmutableDBOptions* _db_options, - const EnvOptions& storage_options, Cache* table_cache, + const FileOptions& storage_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller, BlockCacheTracer* const block_cache_tracer) @@ -3517,6 +3518,7 @@ VersionSet::VersionSet(const std::string& dbname, dbname, _db_options, storage_options, table_cache, write_buffer_manager, write_controller, block_cache_tracer)), env_(_db_options->env), + fs_(_db_options->fs.get()), dbname_(dbname), db_options_(_db_options), next_file_number_(2), @@ -3529,7 +3531,7 @@ VersionSet::VersionSet(const std::string& dbname, prev_log_number_(0), current_version_number_(0), manifest_file_size_(0), - env_options_(storage_options), + file_options_(storage_options), block_cache_tracer_(block_cache_tracer) {} VersionSet::~VersionSet() { @@ -3655,7 +3657,7 @@ Status VersionSet::ProcessManifestWrites( } } if (version == nullptr) { - version = new Version(last_writer->cfd, this, env_options_, + version = new Version(last_writer->cfd, this, file_options_, last_writer->mutable_cf_options, current_version_number_++); versions.push_back(version); @@ -3761,7 +3763,7 @@ Status VersionSet::ProcessManifestWrites( } { - EnvOptions opt_env_opts = env_->OptimizeForManifestWrite(env_options_); + FileOptions opt_file_opts = fs_->OptimizeForManifestWrite(file_options_); mu->Unlock(); TEST_SYNC_POINT("VersionSet::LogAndApply:WriteManifest"); @@ -3788,15 +3790,15 @@ Status VersionSet::ProcessManifestWrites( pending_manifest_file_number_); std::string descriptor_fname = DescriptorFileName(dbname_, pending_manifest_file_number_); - std::unique_ptr descriptor_file; - s = NewWritableFile(env_, descriptor_fname, &descriptor_file, - opt_env_opts); + std::unique_ptr descriptor_file; + s = NewWritableFile(fs_, descriptor_fname, &descriptor_file, + opt_file_opts); if (s.ok()) { descriptor_file->SetPreallocationBlockSize( db_options_->manifest_preallocation_size); std::unique_ptr file_writer(new WritableFileWriter( - std::move(descriptor_file), descriptor_fname, opt_env_opts, env_, + std::move(descriptor_file), descriptor_fname, opt_file_opts, env_, nullptr, db_options_->listeners)); descriptor_log_.reset( new log::Writer(std::move(file_writer), 0, false)); @@ -4251,15 +4253,16 @@ Status VersionSet::ExtractInfoFromVersionEdit( return Status::OK(); } -Status VersionSet::GetCurrentManifestPath(const std::string& dbname, Env* env, +Status VersionSet::GetCurrentManifestPath(const std::string& dbname, + FileSystem* fs, std::string* manifest_path, uint64_t* manifest_file_number) { - assert(env != nullptr); + assert(fs != nullptr); assert(manifest_path != nullptr); assert(manifest_file_number != nullptr); std::string fname; - Status s = ReadFileToString(env, CurrentFileName(dbname), &fname); + Status s = ReadFileToString(fs, CurrentFileName(dbname), &fname); if (!s.ok()) { return s; } @@ -4361,7 +4364,7 @@ Status VersionSet::Recover( // Read "CURRENT" file, which contains a pointer to the current manifest file std::string manifest_path; - Status s = GetCurrentManifestPath(dbname_, env_, &manifest_path, + Status s = GetCurrentManifestPath(dbname_, fs_, &manifest_path, &manifest_file_number_); if (!s.ok()) { return s; @@ -4372,9 +4375,10 @@ Status VersionSet::Recover( std::unique_ptr manifest_file_reader; { - std::unique_ptr manifest_file; - s = env_->NewSequentialFile(manifest_path, &manifest_file, - env_->OptimizeForManifestRead(env_options_)); + std::unique_ptr manifest_file; + s = fs_->NewSequentialFile(manifest_path, + fs_->OptimizeForManifestRead(file_options_), + &manifest_file, nullptr); if (!s.ok()) { return s; } @@ -4383,7 +4387,8 @@ Status VersionSet::Recover( db_options_->log_readahead_size)); } uint64_t current_manifest_file_size; - s = env_->GetFileSize(manifest_path, ¤t_manifest_file_size); + s = fs_->GetFileSize(manifest_path, IOOptions(), ¤t_manifest_file_size, + nullptr); if (!s.ok()) { return s; } @@ -4490,7 +4495,7 @@ Status VersionSet::Recover( true /* is_initial_load */, cfd->GetLatestMutableCFOptions()->prefix_extractor.get()); - Version* v = new Version(cfd, this, env_options_, + Version* v = new Version(cfd, this, file_options_, *cfd->GetLatestMutableCFOptions(), current_version_number_++); builder->SaveTo(v->storage_info()); @@ -4535,23 +4540,24 @@ Status VersionSet::Recover( } Status VersionSet::ListColumnFamilies(std::vector* column_families, - const std::string& dbname, Env* env) { + const std::string& dbname, + FileSystem* fs) { // these are just for performance reasons, not correcntes, // so we're fine using the defaults - EnvOptions soptions; + FileOptions soptions; // Read "CURRENT" file, which contains a pointer to the current manifest file std::string manifest_path; uint64_t manifest_file_number; - Status s = GetCurrentManifestPath(dbname, env, &manifest_path, - &manifest_file_number); + Status s = + GetCurrentManifestPath(dbname, fs, &manifest_path, &manifest_file_number); if (!s.ok()) { return s; } std::unique_ptr file_reader; { - std::unique_ptr file; - s = env->NewSequentialFile(manifest_path, &file, soptions); + std::unique_ptr file; + s = fs->NewSequentialFile(manifest_path, soptions, &file, nullptr); if (!s.ok()) { return s; } @@ -4605,7 +4611,7 @@ Status VersionSet::ListColumnFamilies(std::vector* column_families, #ifndef ROCKSDB_LITE Status VersionSet::ReduceNumberOfLevels(const std::string& dbname, const Options* options, - const EnvOptions& env_options, + const FileOptions& file_options, int new_levels) { if (new_levels <= 1) { return Status::InvalidArgument( @@ -4618,7 +4624,7 @@ Status VersionSet::ReduceNumberOfLevels(const std::string& dbname, options->table_cache_numshardbits)); WriteController wc(options->delayed_write_rate); WriteBufferManager wb(options->db_write_buffer_size); - VersionSet versions(dbname, &db_options, env_options, tc.get(), &wb, &wc, + VersionSet versions(dbname, &db_options, file_options, tc.get(), &wb, &wc, /*block_cache_tracer=*/nullptr); Status status; @@ -4694,9 +4700,11 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname, std::unique_ptr file_reader; Status s; { - std::unique_ptr file; - s = options.env->NewSequentialFile( - dscname, &file, env_->OptimizeForManifestRead(env_options_)); + std::unique_ptr file; + s = options.file_system->NewSequentialFile( + dscname, + options.file_system->OptimizeForManifestRead(file_options_), &file, + nullptr); if (!s.ok()) { return s; } @@ -4856,7 +4864,7 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname, assert(builders_iter != builders.end()); auto builder = builders_iter->second->version_builder(); - Version* v = new Version(cfd, this, env_options_, + Version* v = new Version(cfd, this, file_options_, *cfd->GetLatestMutableCFOptions(), current_version_number_++); builder->SaveTo(v->storage_info()); @@ -5233,7 +5241,7 @@ void VersionSet::AddLiveFiles(std::vector* live_list) { InternalIterator* VersionSet::MakeInputIterator( const Compaction* c, RangeDelAggregator* range_del_agg, - const EnvOptions& env_options_compactions) { + const FileOptions& file_options_compactions) { auto cfd = c->column_family_data(); ReadOptions read_options; read_options.verify_checksums = true; @@ -5258,7 +5266,8 @@ InternalIterator* VersionSet::MakeInputIterator( const LevelFilesBrief* flevel = c->input_levels(which); for (size_t i = 0; i < flevel->num_files; i++) { list[num++] = cfd->table_cache()->NewIterator( - read_options, env_options_compactions, cfd->internal_comparator(), + read_options, file_options_compactions, + cfd->internal_comparator(), *flevel->files[i].file_metadata, range_del_agg, c->mutable_cf_options()->prefix_extractor.get(), /*table_reader_ptr=*/nullptr, @@ -5271,7 +5280,7 @@ InternalIterator* VersionSet::MakeInputIterator( } else { // Create concatenating iterator for the files from this level list[num++] = new LevelIterator( - cfd->table_cache(), read_options, env_options_compactions, + cfd->table_cache(), read_options, file_options_compactions, cfd->internal_comparator(), c->input_levels(which), c->mutable_cf_options()->prefix_extractor.get(), /*should_sample=*/false, @@ -5424,7 +5433,7 @@ ColumnFamilyData* VersionSet::CreateColumnFamily( MutableCFOptions dummy_cf_options; Version* dummy_versions = - new Version(nullptr, this, env_options_, dummy_cf_options); + new Version(nullptr, this, file_options_, dummy_cf_options); // Ref() dummy version once so that later we can call Unref() to delete it // by avoiding calling "delete" explicitly (~Version is private) dummy_versions->Ref(); @@ -5432,7 +5441,7 @@ ColumnFamilyData* VersionSet::CreateColumnFamily( edit->column_family_name_, edit->column_family_, dummy_versions, cf_options); - Version* v = new Version(new_cfd, this, env_options_, + Version* v = new Version(new_cfd, this, file_options_, *new_cfd->GetLatestMutableCFOptions(), current_version_number_++); @@ -5476,11 +5485,11 @@ uint64_t VersionSet::GetTotalSstFilesSize(Version* dummy_versions) { ReactiveVersionSet::ReactiveVersionSet(const std::string& dbname, const ImmutableDBOptions* _db_options, - const EnvOptions& _env_options, + const FileOptions& _file_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller) - : VersionSet(dbname, _db_options, _env_options, table_cache, + : VersionSet(dbname, _db_options, _file_options, table_cache, write_buffer_manager, write_controller, /*block_cache_tracer=*/nullptr), number_of_edits_to_skip_(0) {} @@ -5611,7 +5620,7 @@ Status ReactiveVersionSet::Recover( assert(builders_iter != builders.end()); auto* builder = builders_iter->second->version_builder(); - Version* v = new Version(cfd, this, env_options_, + Version* v = new Version(cfd, this, file_options_, *cfd->GetLatestMutableCFOptions(), current_version_number_++); builder->SaveTo(v->storage_info()); @@ -5831,7 +5840,7 @@ Status ReactiveVersionSet::ApplyOneVersionEditToBuilder( &s); if (s.ok()) { - auto version = new Version(cfd, this, env_options_, + auto version = new Version(cfd, this, file_options_, *cfd->GetLatestMutableCFOptions(), current_version_number_++); builder->SaveTo(version->storage_info()); @@ -5874,9 +5883,9 @@ Status ReactiveVersionSet::MaybeSwitchManifest( Status s; do { std::string manifest_path; - s = GetCurrentManifestPath(dbname_, env_, &manifest_path, + s = GetCurrentManifestPath(dbname_, fs_, &manifest_path, &manifest_file_number_); - std::unique_ptr manifest_file; + std::unique_ptr manifest_file; if (s.ok()) { if (nullptr == manifest_reader->get() || manifest_reader->get()->file()->file_name() != manifest_path) { @@ -5886,9 +5895,9 @@ Status ReactiveVersionSet::MaybeSwitchManifest( TEST_SYNC_POINT( "ReactiveVersionSet::MaybeSwitchManifest:" "AfterGetCurrentManifestPath:1"); - s = env_->NewSequentialFile( - manifest_path, &manifest_file, - env_->OptimizeForManifestRead(env_options_)); + s = fs_->NewSequentialFile(manifest_path, + env_->OptimizeForManifestRead(file_options_), + &manifest_file, nullptr); } else { // No need to switch manifest. break; diff --git a/db/version_set.h b/db/version_set.h index 758bd5e5d..aa18cabb5 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -559,15 +559,15 @@ class Version { // Append to *iters a sequence of iterators that will // yield the contents of this Version when merged together. // REQUIRES: This version has been saved (see VersionSet::SaveTo) - void AddIterators(const ReadOptions&, const EnvOptions& soptions, + void AddIterators(const ReadOptions&, const FileOptions& soptions, MergeIteratorBuilder* merger_iter_builder, RangeDelAggregator* range_del_agg); - void AddIteratorsForLevel(const ReadOptions&, const EnvOptions& soptions, + void AddIteratorsForLevel(const ReadOptions&, const FileOptions& soptions, MergeIteratorBuilder* merger_iter_builder, int level, RangeDelAggregator* range_del_agg); - Status OverlapWithLevelIterator(const ReadOptions&, const EnvOptions&, + Status OverlapWithLevelIterator(const ReadOptions&, const FileOptions&, const Slice& smallest_user_key, const Slice& largest_user_key, int level, bool* overlap); @@ -684,6 +684,7 @@ class Version { private: Env* env_; + FileSystem* fs_; friend class ReactiveVersionSet; friend class VersionSet; @@ -729,14 +730,14 @@ class Version { Version* next_; // Next version in linked list Version* prev_; // Previous version in linked list int refs_; // Number of live refs to this version - const EnvOptions env_options_; + const FileOptions file_options_; const MutableCFOptions mutable_cf_options_; // A version number that uniquely represents this version. This is // used for debugging and logging purposes only. uint64_t version_number_; - Version(ColumnFamilyData* cfd, VersionSet* vset, const EnvOptions& env_opt, + Version(ColumnFamilyData* cfd, VersionSet* vset, const FileOptions& file_opt, MutableCFOptions mutable_cf_options, uint64_t version_number = 0); ~Version(); @@ -801,7 +802,7 @@ class AtomicGroupReadBuffer { class VersionSet { public: VersionSet(const std::string& dbname, const ImmutableDBOptions* db_options, - const EnvOptions& env_options, Cache* table_cache, + const FileOptions& file_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller, BlockCacheTracer* const block_cache_tracer); @@ -863,7 +864,8 @@ class VersionSet { bool new_descriptor_log = false, const ColumnFamilyOptions* new_cf_options = nullptr); - static Status GetCurrentManifestPath(const std::string& dbname, Env* env, + static Status GetCurrentManifestPath(const std::string& dbname, + FileSystem* fs, std::string* manifest_filename, uint64_t* manifest_file_number); @@ -876,7 +878,7 @@ class VersionSet { // Reads a manifest file and returns a list of column families in // column_families. static Status ListColumnFamilies(std::vector* column_families, - const std::string& dbname, Env* env); + const std::string& dbname, FileSystem* fs); #ifndef ROCKSDB_LITE // Try to reduce the number of levels. This call is valid when @@ -890,7 +892,7 @@ class VersionSet { // among [4-6] contains files. static Status ReduceNumberOfLevels(const std::string& dbname, const Options* options, - const EnvOptions& env_options, + const FileOptions& file_options, int new_levels); // printf contents (for debugging) @@ -1004,7 +1006,7 @@ class VersionSet { // The caller should delete the iterator when no longer needed. InternalIterator* MakeInputIterator( const Compaction* c, RangeDelAggregator* range_del_agg, - const EnvOptions& env_options_compactions); + const FileOptions& file_options_compactions); // Add all files listed in any live version to *live. void AddLiveFiles(std::vector* live_list); @@ -1037,9 +1039,9 @@ class VersionSet { uint64_t min_pending_output); ColumnFamilySet* GetColumnFamilySet() { return column_family_set_.get(); } - const EnvOptions& env_options() { return env_options_; } - void ChangeEnvOptions(const MutableDBOptions& new_options) { - env_options_.writable_file_max_buffer_size = + const FileOptions& file_options() { return file_options_; } + void ChangeFileOptions(const MutableDBOptions& new_options) { + file_options_.writable_file_max_buffer_size = new_options.writable_file_max_buffer_size; } @@ -1106,6 +1108,7 @@ class VersionSet { std::unique_ptr column_family_set_; Env* const env_; + FileSystem* const fs_; const std::string dbname_; std::string db_id_; const ImmutableDBOptions* const db_options_; @@ -1150,7 +1153,7 @@ class VersionSet { std::vector obsolete_manifests_; // env options for all reads and writes except compactions - EnvOptions env_options_; + FileOptions file_options_; BlockCacheTracer* const block_cache_tracer_; @@ -1174,7 +1177,7 @@ class ReactiveVersionSet : public VersionSet { public: ReactiveVersionSet(const std::string& dbname, const ImmutableDBOptions* _db_options, - const EnvOptions& _env_options, Cache* table_cache, + const FileOptions& _file_options, Cache* table_cache, WriteBufferManager* write_buffer_manager, WriteController* write_controller); diff --git a/db/version_set_test.cc b/db/version_set_test.cc index 66ad930f5..6e9a8dd12 100644 --- a/db/version_set_test.cc +++ b/db/version_set_test.cc @@ -610,21 +610,25 @@ class VersionSetTestBase { VersionSetTestBase() : env_(Env::Default()), + fs_(std::make_shared(env_)), dbname_(test::PerThreadDBPath("version_set_test")), db_options_(), mutable_cf_options_(cf_options_), table_cache_(NewLRUCache(50000, 16)), write_buffer_manager_(db_options_.db_write_buffer_size), - versions_(new VersionSet(dbname_, &db_options_, env_options_, - table_cache_.get(), &write_buffer_manager_, - &write_controller_, - /*block_cache_tracer=*/nullptr)), - reactive_versions_(std::make_shared( - dbname_, &db_options_, env_options_, table_cache_.get(), - &write_buffer_manager_, &write_controller_)), shutting_down_(false), mock_table_factory_(std::make_shared()) { EXPECT_OK(env_->CreateDirIfMissing(dbname_)); + + db_options_.env = env_; + db_options_.fs = fs_; + versions_.reset(new VersionSet(dbname_, &db_options_, env_options_, + table_cache_.get(), &write_buffer_manager_, + &write_controller_, + /*block_cache_tracer=*/nullptr)), + reactive_versions_ = std::make_shared( + dbname_, &db_options_, env_options_, table_cache_.get(), + &write_buffer_manager_, &write_controller_); db_options_.db_paths.emplace_back(dbname_, std::numeric_limits::max()); } @@ -669,8 +673,8 @@ class VersionSetTestBase { Status s = env_->NewWritableFile( manifest, &file, env_->OptimizeForManifestWrite(env_options_)); ASSERT_OK(s); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(file), manifest, env_options_)); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(file)), manifest, env_options_)); { log_writer->reset(new log::Writer(std::move(file_writer), 0, false)); std::string record; @@ -709,6 +713,7 @@ class VersionSetTestBase { } Env* env_; + std::shared_ptr fs_; const std::string dbname_; EnvOptions env_options_; ImmutableDBOptions db_options_; diff --git a/db/wal_manager.cc b/db/wal_manager.cc index d3f59a10e..bb74ad03d 100644 --- a/db/wal_manager.cc +++ b/db/wal_manager.cc @@ -120,7 +120,7 @@ Status WalManager::GetUpdatesSince( return s; } iter->reset(new TransactionLogIteratorImpl( - db_options_.wal_dir, &db_options_, read_options, env_options_, seq, + db_options_.wal_dir, &db_options_, read_options, file_options_, seq, std::move(wal_files), version_set, seq_per_batch_)); return (*iter)->status(); } @@ -464,9 +464,10 @@ Status WalManager::ReadFirstLine(const std::string& fname, } }; - std::unique_ptr file; - Status status = env_->NewSequentialFile( - fname, &file, env_->OptimizeForLogRead(env_options_)); + std::unique_ptr file; + Status status = fs_->NewSequentialFile(fname, + fs_->OptimizeForLogRead(file_options_), + &file, nullptr); std::unique_ptr file_reader( new SequentialFileReader(std::move(file), fname)); diff --git a/db/wal_manager.h b/db/wal_manager.h index 97211f000..6c387bf3a 100644 --- a/db/wal_manager.h +++ b/db/wal_manager.h @@ -36,10 +36,11 @@ namespace rocksdb { class WalManager { public: WalManager(const ImmutableDBOptions& db_options, - const EnvOptions& env_options, const bool seq_per_batch = false) + const FileOptions& file_options, const bool seq_per_batch = false) : db_options_(db_options), - env_options_(env_options), + file_options_(file_options), env_(db_options.env), + fs_(db_options.fs.get()), purge_wal_files_last_run_(0), seq_per_batch_(seq_per_batch), wal_in_db_path_(IsWalDirSameAsDBPath(&db_options)) {} @@ -88,8 +89,9 @@ class WalManager { // ------- state from DBImpl ------ const ImmutableDBOptions& db_options_; - const EnvOptions& env_options_; + const FileOptions file_options_; Env* env_; + FileSystem* fs_; // ------- WalManager state ------- // cache for ReadFirstRecord() calls diff --git a/db/wal_manager_test.cc b/db/wal_manager_test.cc index 4f15a064d..4763a6ba2 100644 --- a/db/wal_manager_test.cc +++ b/db/wal_manager_test.cc @@ -47,6 +47,8 @@ class WalManagerTest : public testing::Test { std::numeric_limits::max()); db_options_.wal_dir = dbname_; db_options_.env = env_.get(); + fs_.reset(new LegacyFileSystemWrapper(env_.get())); + db_options_.fs = fs_; versions_.reset(new VersionSet(dbname_, &db_options_, env_options_, table_cache_.get(), &write_buffer_manager_, @@ -79,8 +81,8 @@ class WalManagerTest : public testing::Test { std::string fname = ArchivedLogFileName(dbname_, current_log_number_); std::unique_ptr file; ASSERT_OK(env_->NewWritableFile(fname, &file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(file), fname, env_options_)); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(file)), fname, env_options_)); current_log_writer_.reset(new log::Writer(std::move(file_writer), 0, false)); } @@ -111,6 +113,7 @@ class WalManagerTest : public testing::Test { WriteBufferManager write_buffer_manager_; std::unique_ptr versions_; std::unique_ptr wal_manager_; + std::shared_ptr fs_; std::unique_ptr current_log_writer_; uint64_t current_log_number_; @@ -130,8 +133,8 @@ TEST_F(WalManagerTest, ReadFirstRecordCache) { wal_manager_->TEST_ReadFirstRecord(kAliveLogFile, 1 /* number */, &s)); ASSERT_EQ(s, 0U); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(file), path, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(file)), path, EnvOptions())); log::Writer writer(std::move(file_writer), 1, db_options_.recycle_log_file_num > 0); WriteBatch batch; diff --git a/env/composite_env_wrapper.h b/env/composite_env_wrapper.h new file mode 100644 index 000000000..51f892f34 --- /dev/null +++ b/env/composite_env_wrapper.h @@ -0,0 +1,1116 @@ +// Copyright (c) 2019-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include "rocksdb/env.h" +#include "rocksdb/file_system.h" + +namespace rocksdb { + +// The CompositeEnvWrapper class provides an interface that is compatible +// with the old monolithic Env API, and an implementation that wraps around +// the new Env that provides threading and other OS related functionality, and +// the new FileSystem API that provides storage functionality. By +// providing the old Env interface, it allows the rest of RocksDB code to +// be agnostic of whether the underlying Env implementation is a monolithic +// Env or an Env + FileSystem. In the former case, the user will specify +// Options::env only, whereas in the latter case, the user will specify +// Options::env and Options::file_system. + +inline IOStatus status_to_io_status(Status&& status) { + if (status.ok()) { + // Fast path + return IOStatus::OK(); + } else { + const char* state = status.getState(); + if (state) { + return IOStatus(status.code(), status.subcode(), + Slice(state, strlen(status.getState()) + 1), + Slice()); + } else { + return IOStatus(status.code(), status.subcode()); + } + } +} + +class CompositeSequentialFileWrapper : public SequentialFile { + public: + explicit CompositeSequentialFileWrapper( + std::unique_ptr& target) + : target_(std::move(target)) {} + + Status Read(size_t n, Slice* result, char* scratch) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Read(n, io_opts, result, scratch, &dbg); + } + Status Skip(uint64_t n) override { return target_->Skip(n); } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + Status InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + Status PositionedRead(uint64_t offset, size_t n, Slice* result, + char* scratch) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->PositionedRead(offset, n, io_opts, result, scratch, &dbg); + } + + private: + std::unique_ptr target_; +}; + +class CompositeRandomAccessFileWrapper : public RandomAccessFile { + public: + explicit CompositeRandomAccessFileWrapper( + std::unique_ptr& target) + : target_(std::move(target)) {} + + Status Read(uint64_t offset, size_t n, Slice* result, + char* scratch) const override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Read(offset, n, io_opts, result, scratch, &dbg); + } + Status MultiRead(ReadRequest* reqs, size_t num_reqs) override { + IOOptions io_opts; + IODebugContext dbg; + std::vector fs_reqs; + Status status; + + fs_reqs.resize(num_reqs); + for (size_t i = 0; i < num_reqs; ++i) { + fs_reqs[i].offset = reqs[i].offset; + fs_reqs[i].len = reqs[i].len; + fs_reqs[i].scratch = reqs[i].scratch; + fs_reqs[i].status = IOStatus::OK(); + } + status = target_->MultiRead(fs_reqs.data(), num_reqs, io_opts, &dbg); + for (size_t i = 0; i < num_reqs; ++i) { + reqs[i].result = fs_reqs[i].result; + reqs[i].status = fs_reqs[i].status; + } + return status; + } + Status Prefetch(uint64_t offset, size_t n) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Prefetch(offset, n, io_opts, &dbg); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + }; + void Hint(AccessPattern pattern) override { + target_->Hint((FSRandomAccessFile::AccessPattern)pattern); + } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + Status InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + + private: + std::unique_ptr target_; +}; + +class CompositeWritableFileWrapper : public WritableFile { + public: + explicit CompositeWritableFileWrapper(std::unique_ptr& t) + : target_(std::move(t)) {} + + Status Append(const Slice& data) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Append(data, io_opts, &dbg); + } + Status PositionedAppend(const Slice& data, uint64_t offset) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->PositionedAppend(data, offset, io_opts, &dbg); + } + Status Truncate(uint64_t size) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Truncate(size, io_opts, &dbg); + } + Status Close() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Close(io_opts, &dbg); + } + Status Flush() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Flush(io_opts, &dbg); + } + Status Sync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Sync(io_opts, &dbg); + } + Status Fsync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Fsync(io_opts, &dbg); + } + bool IsSyncThreadSafe() const override { return target_->IsSyncThreadSafe(); } + + bool use_direct_io() const override { return target_->use_direct_io(); } + + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + + void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) override { + target_->SetWriteLifeTimeHint(hint); + } + + Env::WriteLifeTimeHint GetWriteLifeTimeHint() override { + return target_->GetWriteLifeTimeHint(); + } + + uint64_t GetFileSize() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->GetFileSize(io_opts, &dbg); + } + + void SetPreallocationBlockSize(size_t size) override { + target_->SetPreallocationBlockSize(size); + } + + void GetPreallocationStatus(size_t* block_size, + size_t* last_allocated_block) override { + target_->GetPreallocationStatus(block_size, last_allocated_block); + } + + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + Status InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + + Status RangeSync(uint64_t offset, uint64_t nbytes) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->RangeSync(offset, nbytes, io_opts, &dbg); + } + + void PrepareWrite(size_t offset, size_t len) override { + IOOptions io_opts; + IODebugContext dbg; + target_->PrepareWrite(offset, len, io_opts, &dbg); + } + + Status Allocate(uint64_t offset, uint64_t len) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Allocate(offset, len, io_opts, &dbg); + } + + std::unique_ptr* target() { return &target_; } + + private: + std::unique_ptr target_; +}; + +class CompositeRandomRWFileWrapper : public RandomRWFile { + public: + explicit CompositeRandomRWFileWrapper(std::unique_ptr& target) + : target_(std::move(target)) {} + + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + Status Write(uint64_t offset, const Slice& data) override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Write(offset, data, io_opts, &dbg); + } + Status Read(uint64_t offset, size_t n, Slice* result, + char* scratch) const override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Read(offset, n, io_opts, result, scratch, &dbg); + } + Status Flush() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Flush(io_opts, &dbg); + } + Status Sync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Sync(io_opts, &dbg); + } + Status Fsync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Fsync(io_opts, &dbg); + } + Status Close() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Close(io_opts, &dbg); + } + + private: + std::unique_ptr target_; +}; + +class CompositeDirectoryWrapper : public Directory { + public: + explicit CompositeDirectoryWrapper(std::unique_ptr& target) + : target_(std::move(target)) {} + + Status Fsync() override { + IOOptions io_opts; + IODebugContext dbg; + return target_->Fsync(io_opts, &dbg); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + private: + std::unique_ptr target_; +}; + +class CompositeEnvWrapper : public Env { + public: + // Initialize a CompositeEnvWrapper that delegates all thread/time related + // calls to env, and all file operations to fs + explicit CompositeEnvWrapper(Env* env, FileSystem* fs) + : env_target_(env), fs_env_target_(fs) {} + ~CompositeEnvWrapper() {} + + // Return the target to which this Env forwards all calls + Env* env_target() const { return env_target_; } + + FileSystem* fs_env_target() const { return fs_env_target_; } + + // The following text is boilerplate that forwards all methods to target() + Status NewSequentialFile(const std::string& f, + std::unique_ptr* r, + const EnvOptions& options) override { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = + fs_env_target_->NewSequentialFile(f, FileOptions(options), &file, &dbg); + if (status.ok()) { + r->reset(new CompositeSequentialFileWrapper(file)); + } + return status; + } + Status NewRandomAccessFile(const std::string& f, + std::unique_ptr* r, + const EnvOptions& options) override { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = fs_env_target_->NewRandomAccessFile(f, FileOptions(options), &file, + &dbg); + if (status.ok()) { + r->reset(new CompositeRandomAccessFileWrapper(file)); + } + return status; + } + Status NewWritableFile(const std::string& f, std::unique_ptr* r, + const EnvOptions& options) override { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = + fs_env_target_->NewWritableFile(f, FileOptions(options), &file, &dbg); + if (status.ok()) { + r->reset(new CompositeWritableFileWrapper(file)); + } + return status; + } + Status ReopenWritableFile(const std::string& fname, + std::unique_ptr* result, + const EnvOptions& options) override { + IODebugContext dbg; + Status status; + std::unique_ptr file; + status = fs_env_target_->ReopenWritableFile(fname, FileOptions(options), + &file, &dbg); + if (status.ok()) { + result->reset(new CompositeWritableFileWrapper(file)); + } + return status; + } + Status ReuseWritableFile(const std::string& fname, + const std::string& old_fname, + std::unique_ptr* r, + const EnvOptions& options) override { + IODebugContext dbg; + Status status; + std::unique_ptr file; + status = fs_env_target_->ReuseWritableFile( + fname, old_fname, FileOptions(options), &file, &dbg); + if (status.ok()) { + r->reset(new CompositeWritableFileWrapper(file)); + } + return status; + } + Status NewRandomRWFile(const std::string& fname, + std::unique_ptr* result, + const EnvOptions& options) override { + IODebugContext dbg; + std::unique_ptr file; + Status status; + status = fs_env_target_->NewRandomRWFile(fname, FileOptions(options), &file, + &dbg); + if (status.ok()) { + result->reset(new CompositeRandomRWFileWrapper(file)); + } + return status; + } + Status NewMemoryMappedFileBuffer( + const std::string& fname, + std::unique_ptr* result) override { + return fs_env_target_->NewMemoryMappedFileBuffer(fname, result); + } + Status NewDirectory(const std::string& name, + std::unique_ptr* result) override { + IOOptions io_opts; + IODebugContext dbg; + std::unique_ptr dir; + Status status; + status = fs_env_target_->NewDirectory(name, io_opts, &dir, &dbg); + if (status.ok()) { + result->reset(new CompositeDirectoryWrapper(dir)); + } + return status; + } + Status FileExists(const std::string& f) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->FileExists(f, io_opts, &dbg); + } + Status GetChildren(const std::string& dir, + std::vector* r) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->GetChildren(dir, io_opts, r, &dbg); + } + Status GetChildrenFileAttributes( + const std::string& dir, std::vector* result) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->GetChildrenFileAttributes(dir, io_opts, result, + &dbg); + } + Status DeleteFile(const std::string& f) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->DeleteFile(f, io_opts, &dbg); + } + Status Truncate(const std::string& fname, size_t size) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->Truncate(fname, size, io_opts, &dbg); + } + Status CreateDir(const std::string& d) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->CreateDir(d, io_opts, &dbg); + } + Status CreateDirIfMissing(const std::string& d) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->CreateDirIfMissing(d, io_opts, &dbg); + } + Status DeleteDir(const std::string& d) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->DeleteDir(d, io_opts, &dbg); + } + Status GetFileSize(const std::string& f, uint64_t* s) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->GetFileSize(f, io_opts, s, &dbg); + } + + Status GetFileModificationTime(const std::string& fname, + uint64_t* file_mtime) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->GetFileModificationTime(fname, io_opts, file_mtime, + &dbg); + } + + Status RenameFile(const std::string& s, const std::string& t) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->RenameFile(s, t, io_opts, &dbg); + } + + Status LinkFile(const std::string& s, const std::string& t) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->LinkFile(s, t, io_opts, &dbg); + } + + Status NumFileLinks(const std::string& fname, uint64_t* count) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->NumFileLinks(fname, io_opts, count, &dbg); + } + + Status AreFilesSame(const std::string& first, const std::string& second, + bool* res) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->AreFilesSame(first, second, io_opts, res, &dbg); + } + + Status LockFile(const std::string& f, FileLock** l) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->LockFile(f, io_opts, l, &dbg); + } + + Status UnlockFile(FileLock* l) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->UnlockFile(l, io_opts, &dbg); + } + + Status GetAbsolutePath(const std::string& db_path, + std::string* output_path) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->GetAbsolutePath(db_path, io_opts, output_path, &dbg); + } + +#if !defined(OS_WIN) && !defined(ROCKSDB_NO_DYNAMIC_EXTENSION) + Status LoadLibrary(const std::string& lib_name, + const std::string& search_path, + std::shared_ptr* result) override { + return env_target_->LoadLibrary(lib_name, search_path, result); + } +#endif + + void Schedule(void (*f)(void* arg), void* a, Priority pri, + void* tag = nullptr, void (*u)(void* arg) = nullptr) override { + return env_target_->Schedule(f, a, pri, tag, u); + } + + int UnSchedule(void* tag, Priority pri) override { + return env_target_->UnSchedule(tag, pri); + } + + void StartThread(void (*f)(void*), void* a) override { + return env_target_->StartThread(f, a); + } + void WaitForJoin() override { return env_target_->WaitForJoin(); } + unsigned int GetThreadPoolQueueLen(Priority pri = LOW) const override { + return env_target_->GetThreadPoolQueueLen(pri); + } + Status GetTestDirectory(std::string* path) override { + return env_target_->GetTestDirectory(path); + } + Status NewLogger(const std::string& fname, + std::shared_ptr* result) override { + return env_target_->NewLogger(fname, result); + } + uint64_t NowMicros() override { return env_target_->NowMicros(); } + uint64_t NowNanos() override { return env_target_->NowNanos(); } + uint64_t NowCPUNanos() override { return env_target_->NowCPUNanos(); } + + void SleepForMicroseconds(int micros) override { + env_target_->SleepForMicroseconds(micros); + } + Status GetHostName(char* name, uint64_t len) override { + return env_target_->GetHostName(name, len); + } + Status GetCurrentTime(int64_t* unix_time) override { + return env_target_->GetCurrentTime(unix_time); + } + void SetBackgroundThreads(int num, Priority pri) override { + return env_target_->SetBackgroundThreads(num, pri); + } + int GetBackgroundThreads(Priority pri) override { + return env_target_->GetBackgroundThreads(pri); + } + + Status SetAllowNonOwnerAccess(bool allow_non_owner_access) override { + return env_target_->SetAllowNonOwnerAccess(allow_non_owner_access); + } + + void IncBackgroundThreadsIfNeeded(int num, Priority pri) override { + return env_target_->IncBackgroundThreadsIfNeeded(num, pri); + } + + void LowerThreadPoolIOPriority(Priority pool = LOW) override { + env_target_->LowerThreadPoolIOPriority(pool); + } + + void LowerThreadPoolCPUPriority(Priority pool = LOW) override { + env_target_->LowerThreadPoolCPUPriority(pool); + } + + std::string TimeToString(uint64_t time) override { + return env_target_->TimeToString(time); + } + + Status GetThreadList(std::vector* thread_list) override { + return env_target_->GetThreadList(thread_list); + } + + ThreadStatusUpdater* GetThreadStatusUpdater() const override { + return env_target_->GetThreadStatusUpdater(); + } + + uint64_t GetThreadID() const override { return env_target_->GetThreadID(); } + + std::string GenerateUniqueId() override { + return env_target_->GenerateUniqueId(); + } + + EnvOptions OptimizeForLogRead(const EnvOptions& env_options) const override { + return fs_env_target_->OptimizeForLogRead(FileOptions(env_options)); + } + EnvOptions OptimizeForManifestRead( + const EnvOptions& env_options) const override { + return fs_env_target_->OptimizeForManifestRead( + FileOptions(env_options)); + } + EnvOptions OptimizeForLogWrite(const EnvOptions& env_options, + const DBOptions& db_options) const override { + return fs_env_target_->OptimizeForLogWrite(FileOptions(env_options), + db_options); + } + EnvOptions OptimizeForManifestWrite( + const EnvOptions& env_options) const override { + return fs_env_target_->OptimizeForManifestWrite( + FileOptions(env_options)); + } + EnvOptions OptimizeForCompactionTableWrite( + const EnvOptions& env_options, + const ImmutableDBOptions& immutable_ops) const override { + return fs_env_target_->OptimizeForCompactionTableWrite( + FileOptions(env_options), + immutable_ops); + } + EnvOptions OptimizeForCompactionTableRead( + const EnvOptions& env_options, + const ImmutableDBOptions& db_options) const override { + return fs_env_target_->OptimizeForCompactionTableRead( + FileOptions(env_options), + db_options); + } + Status GetFreeSpace(const std::string& path, uint64_t* diskfree) override { + IOOptions io_opts; + IODebugContext dbg; + return fs_env_target_->GetFreeSpace(path, io_opts, diskfree, &dbg); + } + + private: + Env* env_target_; + FileSystem* fs_env_target_; +}; + +class LegacySequentialFileWrapper : public FSSequentialFile { + public: + explicit LegacySequentialFileWrapper(std::unique_ptr&& target) + : target_(std::move(target)) {} + + IOStatus Read(size_t n, const IOOptions& /*options*/, Slice* result, + char* scratch, IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Read(n, result, scratch)); + } + IOStatus Skip(uint64_t n) override { + return status_to_io_status(target_->Skip(n)); + } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + IOStatus InvalidateCache(size_t offset, size_t length) override { + return status_to_io_status(target_->InvalidateCache(offset, length)); + } + IOStatus PositionedRead(uint64_t offset, size_t n, + const IOOptions& /*options*/, Slice* result, + char* scratch, IODebugContext* /*dbg*/) override { + return status_to_io_status( + target_->PositionedRead(offset, n, result, scratch)); + } + SequentialFile* target() { return target_.get(); } + + private: + std::unique_ptr target_; +}; + +class LegacyRandomAccessFileWrapper : public FSRandomAccessFile { + public: + explicit LegacyRandomAccessFileWrapper( + std::unique_ptr&& target) + : target_(std::move(target)) {} + + IOStatus Read(uint64_t offset, size_t n, const IOOptions& /*options*/, + Slice* result, char* scratch, + IODebugContext* /*dbg*/) const override { + return status_to_io_status(target_->Read(offset, n, result, scratch)); + } + IOStatus MultiRead(FSReadRequest* fs_reqs, size_t num_reqs, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + std::vector reqs; + Status status; + + reqs.reserve(num_reqs); + for (size_t i = 0; i < num_reqs; ++i) { + ReadRequest req; + + req.offset = fs_reqs[i].offset; + req.len = fs_reqs[i].len; + req.scratch = fs_reqs[i].scratch; + req.status = Status::OK(); + + reqs.emplace_back(req); + } + status = target_->MultiRead(reqs.data(), num_reqs); + for (size_t i = 0; i < num_reqs; ++i) { + fs_reqs[i].result = reqs[i].result; + fs_reqs[i].status = status_to_io_status(std::move(reqs[i].status)); + } + return status_to_io_status(std::move(status)); + ; + } + IOStatus Prefetch(uint64_t offset, size_t n, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Prefetch(offset, n)); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + }; + void Hint(AccessPattern pattern) override { + target_->Hint((RandomAccessFile::AccessPattern)pattern); + } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + IOStatus InvalidateCache(size_t offset, size_t length) override { + return status_to_io_status(target_->InvalidateCache(offset, length)); + } + + private: + std::unique_ptr target_; +}; + +class LegacyWritableFileWrapper : public FSWritableFile { + public: + explicit LegacyWritableFileWrapper(std::unique_ptr&& target) + : target_(std::move(target)) {} + + IOStatus Append(const Slice& data, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Append(data)); + } + IOStatus PositionedAppend(const Slice& data, uint64_t offset, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->PositionedAppend(data, offset)); + } + IOStatus Truncate(uint64_t size, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Truncate(size)); + } + IOStatus Close(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Close()); + } + IOStatus Flush(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Flush()); + } + IOStatus Sync(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Sync()); + } + IOStatus Fsync(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Fsync()); + } + bool IsSyncThreadSafe() const override { return target_->IsSyncThreadSafe(); } + + bool use_direct_io() const override { return target_->use_direct_io(); } + + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + + void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) override { + target_->SetWriteLifeTimeHint(hint); + } + + Env::WriteLifeTimeHint GetWriteLifeTimeHint() override { + return target_->GetWriteLifeTimeHint(); + } + + uint64_t GetFileSize(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return target_->GetFileSize(); + } + + void SetPreallocationBlockSize(size_t size) override { + target_->SetPreallocationBlockSize(size); + } + + void GetPreallocationStatus(size_t* block_size, + size_t* last_allocated_block) override { + target_->GetPreallocationStatus(block_size, last_allocated_block); + } + + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + IOStatus InvalidateCache(size_t offset, size_t length) override { + return status_to_io_status(target_->InvalidateCache(offset, length)); + } + + IOStatus RangeSync(uint64_t offset, uint64_t nbytes, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->RangeSync(offset, nbytes)); + } + + void PrepareWrite(size_t offset, size_t len, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + target_->PrepareWrite(offset, len); + } + + IOStatus Allocate(uint64_t offset, uint64_t len, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Allocate(offset, len)); + } + + WritableFile* target() { return target_.get(); } + + private: + std::unique_ptr target_; +}; + +class LegacyRandomRWFileWrapper : public FSRandomRWFile { + public: + explicit LegacyRandomRWFileWrapper(std::unique_ptr&& target) + : target_(std::move(target)) {} + + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + IOStatus Write(uint64_t offset, const Slice& data, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Write(offset, data)); + } + IOStatus Read(uint64_t offset, size_t n, const IOOptions& /*options*/, + Slice* result, char* scratch, + IODebugContext* /*dbg*/) const override { + return status_to_io_status(target_->Read(offset, n, result, scratch)); + } + IOStatus Flush(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Flush()); + } + IOStatus Sync(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Sync()); + } + IOStatus Fsync(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Fsync()); + } + IOStatus Close(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Close()); + } + + private: + std::unique_ptr target_; +}; + +class LegacyDirectoryWrapper : public FSDirectory { + public: + explicit LegacyDirectoryWrapper(std::unique_ptr&& target) + : target_(std::move(target)) {} + + IOStatus Fsync(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Fsync()); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + private: + std::unique_ptr target_; +}; + +class LegacyFileSystemWrapper : public FileSystem { + public: + // Initialize an EnvWrapper that delegates all calls to *t + explicit LegacyFileSystemWrapper(Env* t) : target_(t) {} + ~LegacyFileSystemWrapper() override {} + + const char* Name() const override { return "Legacy File System"; } + + // Return the target to which this Env forwards all calls + Env* target() const { return target_; } + + // The following text is boilerplate that forwards all methods to target() + IOStatus NewSequentialFile(const std::string& f, + const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* /*dbg*/) override { + std::unique_ptr file; + Status s = target_->NewSequentialFile(f, &file, file_opts); + if (s.ok()) { + r->reset(new LegacySequentialFileWrapper(std::move(file))); + } + return status_to_io_status(std::move(s)); + } + IOStatus NewRandomAccessFile(const std::string& f, + const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* /*dbg*/) override { + std::unique_ptr file; + Status s = target_->NewRandomAccessFile(f, &file, file_opts); + if (s.ok()) { + r->reset(new LegacyRandomAccessFileWrapper(std::move(file))); + } + return status_to_io_status(std::move(s)); + } + IOStatus NewWritableFile(const std::string& f, const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* /*dbg*/) override { + std::unique_ptr file; + Status s = target_->NewWritableFile(f, &file, file_opts); + if (s.ok()) { + r->reset(new LegacyWritableFileWrapper(std::move(file))); + } + return status_to_io_status(std::move(s)); + } + IOStatus ReopenWritableFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + std::unique_ptr file; + Status s = target_->ReopenWritableFile(fname, &file, file_opts); + if (s.ok()) { + result->reset(new LegacyWritableFileWrapper(std::move(file))); + } + return status_to_io_status(std::move(s)); + } + IOStatus ReuseWritableFile(const std::string& fname, + const std::string& old_fname, + const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* /*dbg*/) override { + std::unique_ptr file; + Status s = target_->ReuseWritableFile(fname, old_fname, &file, file_opts); + if (s.ok()) { + r->reset(new LegacyWritableFileWrapper(std::move(file))); + } + return status_to_io_status(std::move(s)); + } + IOStatus NewRandomRWFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + std::unique_ptr file; + Status s = target_->NewRandomRWFile(fname, &file, file_opts); + if (s.ok()) { + result->reset(new LegacyRandomRWFileWrapper(std::move(file))); + } + return status_to_io_status(std::move(s)); + } + IOStatus NewMemoryMappedFileBuffer( + const std::string& fname, + std::unique_ptr* result) override { + return status_to_io_status( + target_->NewMemoryMappedFileBuffer(fname, result)); + } + IOStatus NewDirectory(const std::string& name, const IOOptions& /*io_opts*/, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + std::unique_ptr dir; + Status s = target_->NewDirectory(name, &dir); + if (s.ok()) { + result->reset(new LegacyDirectoryWrapper(std::move(dir))); + } + return status_to_io_status(std::move(s)); + } + IOStatus FileExists(const std::string& f, const IOOptions& /*io_opts*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->FileExists(f)); + } + IOStatus GetChildren(const std::string& dir, const IOOptions& /*io_opts*/, + std::vector* r, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->GetChildren(dir, r)); + } + IOStatus GetChildrenFileAttributes(const std::string& dir, + const IOOptions& /*options*/, + std::vector* result, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->GetChildrenFileAttributes(dir, result)); + } + IOStatus DeleteFile(const std::string& f, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->DeleteFile(f)); + } + IOStatus Truncate(const std::string& fname, size_t size, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->Truncate(fname, size)); + } + IOStatus CreateDir(const std::string& d, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->CreateDir(d)); + } + IOStatus CreateDirIfMissing(const std::string& d, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->CreateDirIfMissing(d)); + } + IOStatus DeleteDir(const std::string& d, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->DeleteDir(d)); + } + IOStatus GetFileSize(const std::string& f, const IOOptions& /*options*/, + uint64_t* s, IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->GetFileSize(f, s)); + } + + IOStatus GetFileModificationTime(const std::string& fname, + const IOOptions& /*options*/, + uint64_t* file_mtime, + IODebugContext* /*dbg*/) override { + return status_to_io_status( + target_->GetFileModificationTime(fname, file_mtime)); + } + + IOStatus GetAbsolutePath(const std::string& db_path, + const IOOptions& /*options*/, + std::string* output_path, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->GetAbsolutePath(db_path, output_path)); + } + + IOStatus RenameFile(const std::string& s, const std::string& t, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->RenameFile(s, t)); + } + + IOStatus LinkFile(const std::string& s, const std::string& t, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->LinkFile(s, t)); + } + + IOStatus NumFileLinks(const std::string& fname, const IOOptions& /*options*/, + uint64_t* count, IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->NumFileLinks(fname, count)); + } + + IOStatus AreFilesSame(const std::string& first, const std::string& second, + const IOOptions& /*options*/, bool* res, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->AreFilesSame(first, second, res)); + } + + IOStatus LockFile(const std::string& f, const IOOptions& /*options*/, + FileLock** l, IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->LockFile(f, l)); + } + + IOStatus UnlockFile(FileLock* l, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->UnlockFile(l)); + } + + IOStatus GetTestDirectory(const IOOptions& /*options*/, std::string* path, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->GetTestDirectory(path)); + } + IOStatus NewLogger(const std::string& fname, const IOOptions& /*options*/, + std::shared_ptr* result, + IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->NewLogger(fname, result)); + } + + FileOptions OptimizeForLogRead( + const FileOptions& file_options) const override { + return target_->OptimizeForLogRead(file_options); + } + FileOptions OptimizeForManifestRead( + const FileOptions& file_options) const override { + return target_->OptimizeForManifestRead(file_options); + } + FileOptions OptimizeForLogWrite(const FileOptions& file_options, + const DBOptions& db_options) const override { + return target_->OptimizeForLogWrite(file_options, db_options); + } + FileOptions OptimizeForManifestWrite( + const FileOptions& file_options) const override { + return target_->OptimizeForManifestWrite(file_options); + } + FileOptions OptimizeForCompactionTableWrite( + const FileOptions& file_options, + const ImmutableDBOptions& immutable_ops) const override { + return target_->OptimizeForCompactionTableWrite(file_options, + immutable_ops); + } + FileOptions OptimizeForCompactionTableRead( + const FileOptions& file_options, + const ImmutableDBOptions& db_options) const override { + return target_->OptimizeForCompactionTableRead(file_options, db_options); + } + IOStatus GetFreeSpace(const std::string& path, const IOOptions& /*options*/, + uint64_t* diskfree, IODebugContext* /*dbg*/) override { + return status_to_io_status(target_->GetFreeSpace(path, diskfree)); + } + + private: + Env* target_; +}; + +inline std::unique_ptr NewLegacySequentialFileWrapper( + std::unique_ptr& file) { + return std::unique_ptr( + new LegacySequentialFileWrapper(std::move(file))); +} + +inline std::unique_ptr NewLegacyRandomAccessFileWrapper( + std::unique_ptr& file) { + return std::unique_ptr( + new LegacyRandomAccessFileWrapper(std::move(file))); +} + +inline std::unique_ptr NewLegacyWritableFileWrapper( + std::unique_ptr&& file) { + return std::unique_ptr( + new LegacyWritableFileWrapper(std::move(file))); +} + +} // namespace rocksdb diff --git a/env/env.cc b/env/env.cc index 6aad4a53e..514f78fe3 100644 --- a/env/env.cc +++ b/env/env.cc @@ -10,6 +10,7 @@ #include "rocksdb/env.h" #include +#include "env/composite_env_wrapper.h" #include "logging/env_logger.h" #include "memory/arena.h" #include "options/db_options.h" @@ -485,8 +486,9 @@ Status NewEnvLogger(const std::string& fname, Env* env, return status; } - *result = std::make_shared(std::move(writable_file), fname, - options, env); + *result = std::make_shared( + NewLegacyWritableFileWrapper(std::move(writable_file)), fname, options, + env); return Status::OK(); } diff --git a/env/env_posix.cc b/env/env_posix.cc index 0723e1832..c9faf1859 100644 --- a/env/env_posix.cc +++ b/env/env_posix.cc @@ -53,6 +53,7 @@ #include #include +#include "env/composite_env_wrapper.h" #include "env/io_posix.h" #include "logging/logging.h" #include "logging/posix_logger.h" @@ -98,45 +99,16 @@ ThreadStatusUpdater* CreateThreadStatusUpdater() { return new ThreadStatusUpdater(); } -inline mode_t GetDBFileMode(bool allow_non_owner_access) { - return allow_non_owner_access ? 0644 : 0600; -} - // list of pathnames that are locked static std::set lockedFiles; static port::Mutex mutex_lockedFiles; -static int LockOrUnlock(int fd, bool lock) { - errno = 0; - struct flock f; - memset(&f, 0, sizeof(f)); - f.l_type = (lock ? F_WRLCK : F_UNLCK); - f.l_whence = SEEK_SET; - f.l_start = 0; - f.l_len = 0; // Lock/unlock entire file - int value = fcntl(fd, F_SETLK, &f); - - return value; -} - class PosixFileLock : public FileLock { public: int fd_; std::string filename; }; -int cloexec_flags(int flags, const EnvOptions* options) { - // If the system supports opening the file with cloexec enabled, - // do so, as this avoids a race condition if a db is opened around - // the same time that a child process is forked -#ifdef O_CLOEXEC - if (options == nullptr || options->set_fd_cloexec) { - flags |= O_CLOEXEC; - } -#endif - return flags; -} - #ifndef ROCKSDB_NO_DYNAMIC_EXTENSION class PosixDynamicLibrary : public DynamicLibrary { public: @@ -164,7 +136,7 @@ class PosixDynamicLibrary : public DynamicLibrary { }; #endif // !ROCKSDB_NO_DYNAMIC_EXTENSION -class PosixEnv : public Env { +class PosixEnv : public CompositeEnvWrapper { public: PosixEnv(); @@ -190,599 +162,6 @@ class PosixEnv : public Env { } } - Status NewSequentialFile(const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) override { - result->reset(); - int fd = -1; - int flags = cloexec_flags(O_RDONLY, &options); - FILE* file = nullptr; - - if (options.use_direct_reads && !options.use_mmap_reads) { -#ifdef ROCKSDB_LITE - return Status::IOError(fname, "Direct I/O not supported in RocksDB lite"); -#endif // !ROCKSDB_LITE -#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) - flags |= O_DIRECT; -#endif - } - - do { - IOSTATS_TIMER_GUARD(open_nanos); - fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); - } while (fd < 0 && errno == EINTR); - if (fd < 0) { - return IOError("While opening a file for sequentially reading", fname, - errno); - } - - SetFD_CLOEXEC(fd, &options); - - if (options.use_direct_reads && !options.use_mmap_reads) { -#ifdef OS_MACOSX - if (fcntl(fd, F_NOCACHE, 1) == -1) { - close(fd); - return IOError("While fcntl NoCache", fname, errno); - } -#endif - } else { - do { - IOSTATS_TIMER_GUARD(open_nanos); - file = fdopen(fd, "r"); - } while (file == nullptr && errno == EINTR); - if (file == nullptr) { - close(fd); - return IOError("While opening file for sequentially read", fname, - errno); - } - } - result->reset(new PosixSequentialFile(fname, file, fd, options)); - return Status::OK(); - } - - Status NewRandomAccessFile(const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) override { - result->reset(); - Status s; - int fd; - int flags = cloexec_flags(O_RDONLY, &options); - - if (options.use_direct_reads && !options.use_mmap_reads) { -#ifdef ROCKSDB_LITE - return Status::IOError(fname, "Direct I/O not supported in RocksDB lite"); -#endif // !ROCKSDB_LITE -#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) - flags |= O_DIRECT; - TEST_SYNC_POINT_CALLBACK("NewRandomAccessFile:O_DIRECT", &flags); -#endif - } - - do { - IOSTATS_TIMER_GUARD(open_nanos); - fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); - } while (fd < 0 && errno == EINTR); - if (fd < 0) { - return IOError("While open a file for random read", fname, errno); - } - SetFD_CLOEXEC(fd, &options); - - if (options.use_mmap_reads && sizeof(void*) >= 8) { - // Use of mmap for random reads has been removed because it - // kills performance when storage is fast. - // Use mmap when virtual address-space is plentiful. - uint64_t size; - s = GetFileSize(fname, &size); - if (s.ok()) { - void* base = mmap(nullptr, size, PROT_READ, MAP_SHARED, fd, 0); - if (base != MAP_FAILED) { - result->reset(new PosixMmapReadableFile(fd, fname, base, - size, options)); - } else { - s = IOError("while mmap file for read", fname, errno); - close(fd); - } - } - } else { - if (options.use_direct_reads && !options.use_mmap_reads) { -#ifdef OS_MACOSX - if (fcntl(fd, F_NOCACHE, 1) == -1) { - close(fd); - return IOError("while fcntl NoCache", fname, errno); - } -#endif - } - result->reset(new PosixRandomAccessFile(fname, fd, options -#if defined(ROCKSDB_IOURING_PRESENT) - , - thread_local_io_urings_.get() -#endif - )); - } - return s; - } - - virtual Status OpenWritableFile(const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options, - bool reopen = false) { - result->reset(); - Status s; - int fd = -1; - int flags = (reopen) ? (O_CREAT | O_APPEND) : (O_CREAT | O_TRUNC); - // Direct IO mode with O_DIRECT flag or F_NOCAHCE (MAC OSX) - if (options.use_direct_writes && !options.use_mmap_writes) { - // Note: we should avoid O_APPEND here due to ta the following bug: - // POSIX requires that opening a file with the O_APPEND flag should - // have no affect on the location at which pwrite() writes data. - // However, on Linux, if a file is opened with O_APPEND, pwrite() - // appends data to the end of the file, regardless of the value of - // offset. - // More info here: https://linux.die.net/man/2/pwrite -#ifdef ROCKSDB_LITE - return Status::IOError(fname, "Direct I/O not supported in RocksDB lite"); -#endif // ROCKSDB_LITE - flags |= O_WRONLY; -#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) - flags |= O_DIRECT; -#endif - TEST_SYNC_POINT_CALLBACK("NewWritableFile:O_DIRECT", &flags); - } else if (options.use_mmap_writes) { - // non-direct I/O - flags |= O_RDWR; - } else { - flags |= O_WRONLY; - } - - flags = cloexec_flags(flags, &options); - - do { - IOSTATS_TIMER_GUARD(open_nanos); - fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); - } while (fd < 0 && errno == EINTR); - - if (fd < 0) { - s = IOError("While open a file for appending", fname, errno); - return s; - } - SetFD_CLOEXEC(fd, &options); - - if (options.use_mmap_writes) { - if (!checkedDiskForMmap_) { - // this will be executed once in the program's lifetime. - // do not use mmapWrite on non ext-3/xfs/tmpfs systems. - if (!SupportsFastAllocate(fname)) { - forceMmapOff_ = true; - } - checkedDiskForMmap_ = true; - } - } - if (options.use_mmap_writes && !forceMmapOff_) { - result->reset(new PosixMmapFile(fname, fd, page_size_, options)); - } else if (options.use_direct_writes && !options.use_mmap_writes) { -#ifdef OS_MACOSX - if (fcntl(fd, F_NOCACHE, 1) == -1) { - close(fd); - s = IOError("While fcntl NoCache an opened file for appending", fname, - errno); - return s; - } -#elif defined(OS_SOLARIS) - if (directio(fd, DIRECTIO_ON) == -1) { - if (errno != ENOTTY) { // ZFS filesystems don't support DIRECTIO_ON - close(fd); - s = IOError("While calling directio()", fname, errno); - return s; - } - } -#endif - result->reset(new PosixWritableFile(fname, fd, options)); - } else { - // disable mmap writes - EnvOptions no_mmap_writes_options = options; - no_mmap_writes_options.use_mmap_writes = false; - result->reset(new PosixWritableFile(fname, fd, no_mmap_writes_options)); - } - return s; - } - - Status NewWritableFile(const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) override { - return OpenWritableFile(fname, result, options, false); - } - - Status ReopenWritableFile(const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) override { - return OpenWritableFile(fname, result, options, true); - } - - Status ReuseWritableFile(const std::string& fname, - const std::string& old_fname, - std::unique_ptr* result, - const EnvOptions& options) override { - result->reset(); - Status s; - int fd = -1; - - int flags = 0; - // Direct IO mode with O_DIRECT flag or F_NOCAHCE (MAC OSX) - if (options.use_direct_writes && !options.use_mmap_writes) { -#ifdef ROCKSDB_LITE - return Status::IOError(fname, "Direct I/O not supported in RocksDB lite"); -#endif // !ROCKSDB_LITE - flags |= O_WRONLY; -#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) - flags |= O_DIRECT; -#endif - TEST_SYNC_POINT_CALLBACK("NewWritableFile:O_DIRECT", &flags); - } else if (options.use_mmap_writes) { - // mmap needs O_RDWR mode - flags |= O_RDWR; - } else { - flags |= O_WRONLY; - } - - flags = cloexec_flags(flags, &options); - - do { - IOSTATS_TIMER_GUARD(open_nanos); - fd = open(old_fname.c_str(), flags, - GetDBFileMode(allow_non_owner_access_)); - } while (fd < 0 && errno == EINTR); - if (fd < 0) { - s = IOError("while reopen file for write", fname, errno); - return s; - } - - SetFD_CLOEXEC(fd, &options); - // rename into place - if (rename(old_fname.c_str(), fname.c_str()) != 0) { - s = IOError("while rename file to " + fname, old_fname, errno); - close(fd); - return s; - } - - if (options.use_mmap_writes) { - if (!checkedDiskForMmap_) { - // this will be executed once in the program's lifetime. - // do not use mmapWrite on non ext-3/xfs/tmpfs systems. - if (!SupportsFastAllocate(fname)) { - forceMmapOff_ = true; - } - checkedDiskForMmap_ = true; - } - } - if (options.use_mmap_writes && !forceMmapOff_) { - result->reset(new PosixMmapFile(fname, fd, page_size_, options)); - } else if (options.use_direct_writes && !options.use_mmap_writes) { -#ifdef OS_MACOSX - if (fcntl(fd, F_NOCACHE, 1) == -1) { - close(fd); - s = IOError("while fcntl NoCache for reopened file for append", fname, - errno); - return s; - } -#elif defined(OS_SOLARIS) - if (directio(fd, DIRECTIO_ON) == -1) { - if (errno != ENOTTY) { // ZFS filesystems don't support DIRECTIO_ON - close(fd); - s = IOError("while calling directio()", fname, errno); - return s; - } - } -#endif - result->reset(new PosixWritableFile(fname, fd, options)); - } else { - // disable mmap writes - EnvOptions no_mmap_writes_options = options; - no_mmap_writes_options.use_mmap_writes = false; - result->reset(new PosixWritableFile(fname, fd, no_mmap_writes_options)); - } - return s; - } - - Status NewRandomRWFile(const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) override { - int fd = -1; - int flags = cloexec_flags(O_RDWR, &options); - - while (fd < 0) { - IOSTATS_TIMER_GUARD(open_nanos); - - fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); - if (fd < 0) { - // Error while opening the file - if (errno == EINTR) { - continue; - } - return IOError("While open file for random read/write", fname, errno); - } - } - - SetFD_CLOEXEC(fd, &options); - result->reset(new PosixRandomRWFile(fname, fd, options)); - return Status::OK(); - } - - Status NewMemoryMappedFileBuffer( - const std::string& fname, - std::unique_ptr* result) override { - int fd = -1; - Status status; - int flags = cloexec_flags(O_RDWR, nullptr); - - while (fd < 0) { - IOSTATS_TIMER_GUARD(open_nanos); - fd = open(fname.c_str(), flags, 0644); - if (fd < 0) { - // Error while opening the file - if (errno == EINTR) { - continue; - } - status = - IOError("While open file for raw mmap buffer access", fname, errno); - break; - } - } - uint64_t size; - if (status.ok()) { - status = GetFileSize(fname, &size); - } - void* base = nullptr; - if (status.ok()) { - base = mmap(nullptr, static_cast(size), PROT_READ | PROT_WRITE, - MAP_SHARED, fd, 0); - if (base == MAP_FAILED) { - status = IOError("while mmap file for read", fname, errno); - } - } - if (status.ok()) { - result->reset( - new PosixMemoryMappedFileBuffer(base, static_cast(size))); - } - if (fd >= 0) { - // don't need to keep it open after mmap has been called - close(fd); - } - return status; - } - - Status NewDirectory(const std::string& name, - std::unique_ptr* result) override { - result->reset(); - int fd; - int flags = cloexec_flags(0, nullptr); - { - IOSTATS_TIMER_GUARD(open_nanos); - fd = open(name.c_str(), flags); - } - if (fd < 0) { - return IOError("While open directory", name, errno); - } else { - result->reset(new PosixDirectory(fd)); - } - return Status::OK(); - } - - Status FileExists(const std::string& fname) override { - int result = access(fname.c_str(), F_OK); - - if (result == 0) { - return Status::OK(); - } - - int err = errno; - switch (err) { - case EACCES: - case ELOOP: - case ENAMETOOLONG: - case ENOENT: - case ENOTDIR: - return Status::NotFound(); - default: - assert(err == EIO || err == ENOMEM); - return Status::IOError("Unexpected error(" + ToString(err) + - ") accessing file `" + fname + "' "); - } - } - - Status GetChildren(const std::string& dir, - std::vector* result) override { - result->clear(); - DIR* d = opendir(dir.c_str()); - if (d == nullptr) { - switch (errno) { - case EACCES: - case ENOENT: - case ENOTDIR: - return Status::NotFound(); - default: - return IOError("While opendir", dir, errno); - } - } - struct dirent* entry; - while ((entry = readdir(d)) != nullptr) { - result->push_back(entry->d_name); - } - closedir(d); - return Status::OK(); - } - - Status DeleteFile(const std::string& fname) override { - Status result; - if (unlink(fname.c_str()) != 0) { - result = IOError("while unlink() file", fname, errno); - } - return result; - }; - - Status CreateDir(const std::string& name) override { - Status result; - if (mkdir(name.c_str(), 0755) != 0) { - result = IOError("While mkdir", name, errno); - } - return result; - }; - - Status CreateDirIfMissing(const std::string& name) override { - Status result; - if (mkdir(name.c_str(), 0755) != 0) { - if (errno != EEXIST) { - result = IOError("While mkdir if missing", name, errno); - } else if (!DirExists(name)) { // Check that name is actually a - // directory. - // Message is taken from mkdir - result = Status::IOError("`"+name+"' exists but is not a directory"); - } - } - return result; - }; - - Status DeleteDir(const std::string& name) override { - Status result; - if (rmdir(name.c_str()) != 0) { - result = IOError("file rmdir", name, errno); - } - return result; - }; - - Status GetFileSize(const std::string& fname, uint64_t* size) override { - Status s; - struct stat sbuf; - if (stat(fname.c_str(), &sbuf) != 0) { - *size = 0; - s = IOError("while stat a file for size", fname, errno); - } else { - *size = sbuf.st_size; - } - return s; - } - - Status GetFileModificationTime(const std::string& fname, - uint64_t* file_mtime) override { - struct stat s; - if (stat(fname.c_str(), &s) !=0) { - return IOError("while stat a file for modification time", fname, errno); - } - *file_mtime = static_cast(s.st_mtime); - return Status::OK(); - } - Status RenameFile(const std::string& src, - const std::string& target) override { - Status result; - if (rename(src.c_str(), target.c_str()) != 0) { - result = IOError("While renaming a file to " + target, src, errno); - } - return result; - } - - Status LinkFile(const std::string& src, const std::string& target) override { - Status result; - if (link(src.c_str(), target.c_str()) != 0) { - if (errno == EXDEV) { - return Status::NotSupported("No cross FS links allowed"); - } - result = IOError("while link file to " + target, src, errno); - } - return result; - } - - Status NumFileLinks(const std::string& fname, uint64_t* count) override { - struct stat s; - if (stat(fname.c_str(), &s) != 0) { - return IOError("while stat a file for num file links", fname, errno); - } - *count = static_cast(s.st_nlink); - return Status::OK(); - } - - Status AreFilesSame(const std::string& first, const std::string& second, - bool* res) override { - struct stat statbuf[2]; - if (stat(first.c_str(), &statbuf[0]) != 0) { - return IOError("stat file", first, errno); - } - if (stat(second.c_str(), &statbuf[1]) != 0) { - return IOError("stat file", second, errno); - } - - if (major(statbuf[0].st_dev) != major(statbuf[1].st_dev) || - minor(statbuf[0].st_dev) != minor(statbuf[1].st_dev) || - statbuf[0].st_ino != statbuf[1].st_ino) { - *res = false; - } else { - *res = true; - } - return Status::OK(); - } - - Status LockFile(const std::string& fname, FileLock** lock) override { - *lock = nullptr; - Status result; - - mutex_lockedFiles.Lock(); - // If it already exists in the lockedFiles set, then it is already locked, - // and fail this lock attempt. Otherwise, insert it into lockedFiles. - // This check is needed because fcntl() does not detect lock conflict - // if the fcntl is issued by the same thread that earlier acquired - // this lock. - // We must do this check *before* opening the file: - // Otherwise, we will open a new file descriptor. Locks are associated with - // a process, not a file descriptor and when *any* file descriptor is closed, - // all locks the process holds for that *file* are released - if (lockedFiles.insert(fname).second == false) { - mutex_lockedFiles.Unlock(); - errno = ENOLCK; - return IOError("lock ", fname, errno); - } - - int fd; - int flags = cloexec_flags(O_RDWR | O_CREAT, nullptr); - - { - IOSTATS_TIMER_GUARD(open_nanos); - fd = open(fname.c_str(), flags, 0644); - } - if (fd < 0) { - result = IOError("while open a file for lock", fname, errno); - } else if (LockOrUnlock(fd, true) == -1) { - // if there is an error in locking, then remove the pathname from lockedfiles - lockedFiles.erase(fname); - result = IOError("While lock file", fname, errno); - close(fd); - } else { - SetFD_CLOEXEC(fd, nullptr); - PosixFileLock* my_lock = new PosixFileLock; - my_lock->fd_ = fd; - my_lock->filename = fname; - *lock = my_lock; - } - - mutex_lockedFiles.Unlock(); - return result; - } - - Status UnlockFile(FileLock* lock) override { - PosixFileLock* my_lock = reinterpret_cast(lock); - Status result; - mutex_lockedFiles.Lock(); - // If we are unlocking, then verify that we had locked it earlier, - // it should already exist in lockedFiles. Remove it from lockedFiles. - if (lockedFiles.erase(my_lock->filename) != 1) { - errno = ENOLCK; - result = IOError("unlock", my_lock->filename, errno); - } else if (LockOrUnlock(my_lock->fd_, false) == -1) { - result = IOError("unlock", my_lock->filename, errno); - } - close(my_lock->fd_); - delete my_lock; - mutex_lockedFiles.Unlock(); - return result; - } - #ifndef ROCKSDB_NO_DYNAMIC_EXTENSION // Loads the named library into the result. // If the input name is empty, the current executable is loaded @@ -883,17 +262,6 @@ class PosixEnv : public Env { uint64_t GetThreadID() const override { return gettid(pthread_self()); } - Status GetFreeSpace(const std::string& fname, uint64_t* free_space) override { - struct statvfs sbuf; - - if (statvfs(fname.c_str(), &sbuf) < 0) { - return IOError("While doing statvfs", fname, errno); - } - - *free_space = ((uint64_t)sbuf.f_bsize * sbuf.f_bfree); - return Status::OK(); - } - Status NewLogger(const std::string& fname, std::shared_ptr* result) override { FILE* f; @@ -963,10 +331,11 @@ class PosixEnv : public Env { Status GetHostName(char* name, uint64_t len) override { int ret = gethostname(name, static_cast(len)); if (ret < 0) { - if (errno == EFAULT || errno == EINVAL) + if (errno == EFAULT || errno == EINVAL) { return Status::InvalidArgument(strerror(errno)); - else + } else { return IOError("GetHostName", name, errno); + } } return Status::OK(); } @@ -980,23 +349,12 @@ class PosixEnv : public Env { return Status::OK(); } - Status GetAbsolutePath(const std::string& db_path, - std::string* output_path) override { - if (!db_path.empty() && db_path[0] == '/') { - *output_path = db_path; - return Status::OK(); - } - - char the_path[256]; - char* ret = getcwd(the_path, 256); - if (ret == nullptr) { - return Status::IOError(strerror(errno)); - } - - *output_path = ret; - return Status::OK(); + ThreadStatusUpdater* GetThreadStatusUpdater() const override { + return Env::GetThreadStatusUpdater(); } + std::string GenerateUniqueId() override { return Env::GenerateUniqueId(); } + // Allow increasing the number of worker threads. void SetBackgroundThreads(int num, Priority pri) override { assert(pri >= Priority::BOTTOM && pri <= Priority::HIGH); @@ -1057,72 +415,7 @@ class PosixEnv : public Env { return dummy; } - EnvOptions OptimizeForLogWrite(const EnvOptions& env_options, - const DBOptions& db_options) const override { - EnvOptions optimized = env_options; - optimized.use_mmap_writes = false; - optimized.use_direct_writes = false; - optimized.bytes_per_sync = db_options.wal_bytes_per_sync; - // TODO(icanadi) it's faster if fallocate_with_keep_size is false, but it - // breaks TransactionLogIteratorStallAtLastRecord unit test. Fix the unit - // test and make this false - optimized.fallocate_with_keep_size = true; - optimized.writable_file_max_buffer_size = - db_options.writable_file_max_buffer_size; - return optimized; - } - - EnvOptions OptimizeForManifestWrite( - const EnvOptions& env_options) const override { - EnvOptions optimized = env_options; - optimized.use_mmap_writes = false; - optimized.use_direct_writes = false; - optimized.fallocate_with_keep_size = true; - return optimized; - } - private: - bool checkedDiskForMmap_; - bool forceMmapOff_; // do we override Env options? - - // Returns true iff the named directory exists and is a directory. - virtual bool DirExists(const std::string& dname) { - struct stat statbuf; - if (stat(dname.c_str(), &statbuf) == 0) { - return S_ISDIR(statbuf.st_mode); - } - return false; // stat() failed return false - } - - bool SupportsFastAllocate(const std::string& path) { -#ifdef ROCKSDB_FALLOCATE_PRESENT - struct statfs s; - if (statfs(path.c_str(), &s)){ - return false; - } - switch (s.f_type) { - case EXT4_SUPER_MAGIC: - return true; - case XFS_SUPER_MAGIC: - return true; - case TMPFS_MAGIC: - return true; - default: - return false; - } -#else - (void)path; - return false; -#endif - } - -#if defined(ROCKSDB_IOURING_PRESENT) - // io_uring instance - std::unique_ptr thread_local_io_urings_; -#endif - - size_t page_size_; - std::vector thread_pools_; pthread_mutex_t mu_; std::vector threads_to_join_; @@ -1132,9 +425,7 @@ class PosixEnv : public Env { }; PosixEnv::PosixEnv() - : checkedDiskForMmap_(false), - forceMmapOff_(false), - page_size_(getpagesize()), + : CompositeEnvWrapper(this, FileSystem::Default().get()), thread_pools_(Priority::TOTAL), allow_non_owner_access_(true) { ThreadPoolImpl::PthreadCall("mutex_init", pthread_mutex_init(&mu_, nullptr)); @@ -1145,17 +436,6 @@ PosixEnv::PosixEnv() thread_pools_[pool_id].SetHostEnv(this); } thread_status_updater_ = CreateThreadStatusUpdater(); - -#if defined(ROCKSDB_IOURING_PRESENT) - // Test whether IOUring is supported, and if it does, create a managing - // object for thread local point so that in the future thread-local - // io_uring can be created. - struct io_uring* new_io_uring = CreateIOUring(); - if (new_io_uring != nullptr) { - thread_local_io_urings_.reset(new ThreadLocalPtr(DeleteIOUring)); - delete new_io_uring; - } -#endif } void PosixEnv::Schedule(void (*function)(void* arg1), void* arg, Priority pri, @@ -1249,7 +529,9 @@ Env* Env::Default() { CompressionContextCache::InitSingleton(); INIT_SYNC_POINT_SINGLETONS(); static PosixEnv default_env; - return &default_env; + static CompositeEnvWrapper composite_env(&default_env, + FileSystem::Default().get()); + return &composite_env; } } // namespace rocksdb diff --git a/env/file_system.cc b/env/file_system.cc new file mode 100644 index 000000000..4b6954872 --- /dev/null +++ b/env/file_system.cc @@ -0,0 +1,110 @@ +// Copyright (c) 2019-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +#include "env/composite_env_wrapper.h" +#include "rocksdb/file_system.h" +#include "options/db_options.h" +#include "rocksdb/utilities/object_registry.h" + +namespace rocksdb { + +FileSystem::FileSystem() {} + +FileSystem::~FileSystem() {} + +Status FileSystem::Load(const std::string& value, + std::shared_ptr* result) { + Status s; +#ifndef ROCKSDB_LITE + s = ObjectRegistry::NewInstance()->NewSharedObject(value, result); +#else + (void)result; + s = Status::NotSupported("Cannot load FileSystem in LITE mode: ", value); +#endif + return s; +} + +FileOptions FileSystem::OptimizeForLogRead( + const FileOptions& file_options) const { + FileOptions optimized_file_options(file_options); + optimized_file_options.use_direct_reads = false; + return optimized_file_options; +} + +FileOptions FileSystem::OptimizeForManifestRead( + const FileOptions& file_options) const { + FileOptions optimized_file_options(file_options); + optimized_file_options.use_direct_reads = false; + return optimized_file_options; +} + +FileOptions FileSystem::OptimizeForLogWrite(const FileOptions& file_options, + const DBOptions& db_options) const { + FileOptions optimized_file_options(file_options); + optimized_file_options.bytes_per_sync = db_options.wal_bytes_per_sync; + optimized_file_options.writable_file_max_buffer_size = + db_options.writable_file_max_buffer_size; + return optimized_file_options; +} + +FileOptions FileSystem::OptimizeForManifestWrite( + const FileOptions& file_options) const { + return file_options; +} + +FileOptions FileSystem::OptimizeForCompactionTableWrite( + const FileOptions& file_options, + const ImmutableDBOptions& db_options) const { + FileOptions optimized_file_options(file_options); + optimized_file_options.use_direct_writes = + db_options.use_direct_io_for_flush_and_compaction; + return optimized_file_options; +} + +FileOptions FileSystem::OptimizeForCompactionTableRead( + const FileOptions& file_options, + const ImmutableDBOptions& db_options) const { + FileOptions optimized_file_options(file_options); + optimized_file_options.use_direct_reads = db_options.use_direct_reads; + return optimized_file_options; +} + +Status ReadFileToString(FileSystem* fs, const std::string& fname, + std::string* data) { + FileOptions soptions; + data->clear(); + std::unique_ptr file; + Status s = fs->NewSequentialFile(fname, soptions, &file, nullptr); + if (!s.ok()) { + return s; + } + static const int kBufferSize = 8192; + char* space = new char[kBufferSize]; + while (true) { + Slice fragment; + s = file->Read(kBufferSize, IOOptions(), &fragment, space, + nullptr); + if (!s.ok()) { + break; + } + data->append(fragment.data(), fragment.size()); + if (fragment.empty()) { + break; + } + } + delete[] space; + return s; +} + +#ifdef OS_WIN +std::shared_ptr FileSystem::Default() { + static LegacyFileSystemWrapper default_fs(Env::Default()); + static std::shared_ptr default_fs_ptr( + &default_fs, [](LegacyFileSystemWrapper*) {}); + return default_fs_ptr; +} +#endif + +} // namespace rocksdb diff --git a/env/fs_posix.cc b/env/fs_posix.cc new file mode 100644 index 000000000..2b398f131 --- /dev/null +++ b/env/fs_posix.cc @@ -0,0 +1,913 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors +#include +#ifndef ROCKSDB_NO_DYNAMIC_EXTENSION +#include +#endif +#include +#include + +#if defined(OS_LINUX) +#include +#endif +#include +#include +#include +#include +#include +#include +#include +#include +#if defined(OS_LINUX) || defined(OS_SOLARIS) || defined(OS_ANDROID) +#include +#include +#include +#endif +#include +#include +#include +#include +#include +// Get nano time includes +#if defined(OS_LINUX) || defined(OS_FREEBSD) +#elif defined(__MACH__) +#include +#include +#include +#else +#include +#endif +#include +#include +#include + +#include "env/io_posix.h" +#include "logging/logging.h" +#include "logging/posix_logger.h" +#include "monitoring/iostats_context_imp.h" +#include "monitoring/thread_status_updater.h" +#include "port/port.h" +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "test_util/sync_point.h" +#include "util/coding.h" +#include "util/compression_context_cache.h" +#include "util/random.h" +#include "util/string_util.h" +#include "util/thread_local.h" +#include "util/threadpool_imp.h" + +#if !defined(TMPFS_MAGIC) +#define TMPFS_MAGIC 0x01021994 +#endif +#if !defined(XFS_SUPER_MAGIC) +#define XFS_SUPER_MAGIC 0x58465342 +#endif +#if !defined(EXT4_SUPER_MAGIC) +#define EXT4_SUPER_MAGIC 0xEF53 +#endif + +namespace rocksdb { + +namespace { + +inline mode_t GetDBFileMode(bool allow_non_owner_access) { + return allow_non_owner_access ? 0644 : 0600; +} + +// list of pathnames that are locked +static std::set lockedFiles; +static port::Mutex mutex_lockedFiles; + +static int LockOrUnlock(int fd, bool lock) { + errno = 0; + struct flock f; + memset(&f, 0, sizeof(f)); + f.l_type = (lock ? F_WRLCK : F_UNLCK); + f.l_whence = SEEK_SET; + f.l_start = 0; + f.l_len = 0; // Lock/unlock entire file + int value = fcntl(fd, F_SETLK, &f); + + return value; +} + +class PosixFileLock : public FileLock { + public: + int fd_; + std::string filename; +}; + +int cloexec_flags(int flags, const EnvOptions* options) { + // If the system supports opening the file with cloexec enabled, + // do so, as this avoids a race condition if a db is opened around + // the same time that a child process is forked +#ifdef O_CLOEXEC + if (options == nullptr || options->set_fd_cloexec) { + flags |= O_CLOEXEC; + } +#endif + return flags; +} + +class PosixFileSystem : public FileSystem { + public: + PosixFileSystem(); + + const char* Name() const override { return "Posix File System"; } + + ~PosixFileSystem() override {} + + void SetFD_CLOEXEC(int fd, const EnvOptions* options) { + if ((options == nullptr || options->set_fd_cloexec) && fd > 0) { + fcntl(fd, F_SETFD, fcntl(fd, F_GETFD) | FD_CLOEXEC); + } + } + + IOStatus NewSequentialFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + result->reset(); + int fd = -1; + int flags = cloexec_flags(O_RDONLY, &options); + FILE* file = nullptr; + + if (options.use_direct_reads && !options.use_mmap_reads) { +#ifdef ROCKSDB_LITE + return IOStatus::IOError(fname, + "Direct I/O not supported in RocksDB lite"); +#endif // !ROCKSDB_LITE +#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) + flags |= O_DIRECT; +#endif + } + + do { + IOSTATS_TIMER_GUARD(open_nanos); + fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); + } while (fd < 0 && errno == EINTR); + if (fd < 0) { + return IOError("While opening a file for sequentially reading", fname, + errno); + } + + SetFD_CLOEXEC(fd, &options); + + if (options.use_direct_reads && !options.use_mmap_reads) { +#ifdef OS_MACOSX + if (fcntl(fd, F_NOCACHE, 1) == -1) { + close(fd); + return IOError("While fcntl NoCache", fname, errno); + } +#endif + } else { + do { + IOSTATS_TIMER_GUARD(open_nanos); + file = fdopen(fd, "r"); + } while (file == nullptr && errno == EINTR); + if (file == nullptr) { + close(fd); + return IOError("While opening file for sequentially read", fname, + errno); + } + } + result->reset(new PosixSequentialFile(fname, file, fd, options)); + return IOStatus::OK(); + } + + IOStatus NewRandomAccessFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + result->reset(); + IOStatus s; + int fd; + int flags = cloexec_flags(O_RDONLY, &options); + + if (options.use_direct_reads && !options.use_mmap_reads) { +#ifdef ROCKSDB_LITE + return IOStatus::IOError(fname, + "Direct I/O not supported in RocksDB lite"); +#endif // !ROCKSDB_LITE +#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) + flags |= O_DIRECT; + TEST_SYNC_POINT_CALLBACK("NewRandomAccessFile:O_DIRECT", &flags); +#endif + } + + do { + IOSTATS_TIMER_GUARD(open_nanos); + fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); + } while (fd < 0 && errno == EINTR); + if (fd < 0) { + return IOError("While open a file for random read", fname, errno); + } + SetFD_CLOEXEC(fd, &options); + + if (options.use_mmap_reads && sizeof(void*) >= 8) { + // Use of mmap for random reads has been removed because it + // kills performance when storage is fast. + // Use mmap when virtual address-space is plentiful. + uint64_t size; + IOOptions opts; + s = GetFileSize(fname, opts, &size, nullptr); + if (s.ok()) { + void* base = mmap(nullptr, size, PROT_READ, MAP_SHARED, fd, 0); + if (base != MAP_FAILED) { + result->reset( + new PosixMmapReadableFile(fd, fname, base, size, options)); + } else { + s = IOError("while mmap file for read", fname, errno); + close(fd); + } + } + } else { + if (options.use_direct_reads && !options.use_mmap_reads) { +#ifdef OS_MACOSX + if (fcntl(fd, F_NOCACHE, 1) == -1) { + close(fd); + return IOError("while fcntl NoCache", fname, errno); + } +#endif + } + result->reset(new PosixRandomAccessFile(fname, fd, options +#if defined(ROCKSDB_IOURING_PRESENT) + , + thread_local_io_urings_.get() +#endif + )); + } + return s; + } + + virtual IOStatus OpenWritableFile(const std::string& fname, + const FileOptions& options, + bool reopen, + std::unique_ptr* result, + IODebugContext* /*dbg*/) { + result->reset(); + IOStatus s; + int fd = -1; + int flags = (reopen) ? (O_CREAT | O_APPEND) : (O_CREAT | O_TRUNC); + // Direct IO mode with O_DIRECT flag or F_NOCAHCE (MAC OSX) + if (options.use_direct_writes && !options.use_mmap_writes) { + // Note: we should avoid O_APPEND here due to ta the following bug: + // POSIX requires that opening a file with the O_APPEND flag should + // have no affect on the location at which pwrite() writes data. + // However, on Linux, if a file is opened with O_APPEND, pwrite() + // appends data to the end of the file, regardless of the value of + // offset. + // More info here: https://linux.die.net/man/2/pwrite +#ifdef ROCKSDB_LITE + return IOStatus::IOError(fname, + "Direct I/O not supported in RocksDB lite"); +#endif // ROCKSDB_LITE + flags |= O_WRONLY; +#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) + flags |= O_DIRECT; +#endif + TEST_SYNC_POINT_CALLBACK("NewWritableFile:O_DIRECT", &flags); + } else if (options.use_mmap_writes) { + // non-direct I/O + flags |= O_RDWR; + } else { + flags |= O_WRONLY; + } + + flags = cloexec_flags(flags, &options); + + do { + IOSTATS_TIMER_GUARD(open_nanos); + fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); + } while (fd < 0 && errno == EINTR); + + if (fd < 0) { + s = IOError("While open a file for appending", fname, errno); + return s; + } + SetFD_CLOEXEC(fd, &options); + + if (options.use_mmap_writes) { + if (!checkedDiskForMmap_) { + // this will be executed once in the program's lifetime. + // do not use mmapWrite on non ext-3/xfs/tmpfs systems. + if (!SupportsFastAllocate(fname)) { + forceMmapOff_ = true; + } + checkedDiskForMmap_ = true; + } + } + if (options.use_mmap_writes && !forceMmapOff_) { + result->reset(new PosixMmapFile(fname, fd, page_size_, options)); + } else if (options.use_direct_writes && !options.use_mmap_writes) { +#ifdef OS_MACOSX + if (fcntl(fd, F_NOCACHE, 1) == -1) { + close(fd); + s = IOError("While fcntl NoCache an opened file for appending", fname, + errno); + return s; + } +#elif defined(OS_SOLARIS) + if (directio(fd, DIRECTIO_ON) == -1) { + if (errno != ENOTTY) { // ZFS filesystems don't support DIRECTIO_ON + close(fd); + s = IOError("While calling directio()", fname, errno); + return s; + } + } +#endif + result->reset(new PosixWritableFile(fname, fd, options)); + } else { + // disable mmap writes + EnvOptions no_mmap_writes_options = options; + no_mmap_writes_options.use_mmap_writes = false; + result->reset(new PosixWritableFile(fname, fd, no_mmap_writes_options)); + } + return s; + } + + IOStatus NewWritableFile(const std::string& fname, const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) override { + return OpenWritableFile(fname, options, false, result, dbg); + } + + IOStatus ReopenWritableFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) override { + return OpenWritableFile(fname, options, true, result, dbg); + } + + IOStatus ReuseWritableFile(const std::string& fname, + const std::string& old_fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + result->reset(); + IOStatus s; + int fd = -1; + + int flags = 0; + // Direct IO mode with O_DIRECT flag or F_NOCAHCE (MAC OSX) + if (options.use_direct_writes && !options.use_mmap_writes) { +#ifdef ROCKSDB_LITE + return IOStatus::IOError(fname, + "Direct I/O not supported in RocksDB lite"); +#endif // !ROCKSDB_LITE + flags |= O_WRONLY; +#if !defined(OS_MACOSX) && !defined(OS_OPENBSD) && !defined(OS_SOLARIS) + flags |= O_DIRECT; +#endif + TEST_SYNC_POINT_CALLBACK("NewWritableFile:O_DIRECT", &flags); + } else if (options.use_mmap_writes) { + // mmap needs O_RDWR mode + flags |= O_RDWR; + } else { + flags |= O_WRONLY; + } + + flags = cloexec_flags(flags, &options); + + do { + IOSTATS_TIMER_GUARD(open_nanos); + fd = open(old_fname.c_str(), flags, + GetDBFileMode(allow_non_owner_access_)); + } while (fd < 0 && errno == EINTR); + if (fd < 0) { + s = IOError("while reopen file for write", fname, errno); + return s; + } + + SetFD_CLOEXEC(fd, &options); + // rename into place + if (rename(old_fname.c_str(), fname.c_str()) != 0) { + s = IOError("while rename file to " + fname, old_fname, errno); + close(fd); + return s; + } + + if (options.use_mmap_writes) { + if (!checkedDiskForMmap_) { + // this will be executed once in the program's lifetime. + // do not use mmapWrite on non ext-3/xfs/tmpfs systems. + if (!SupportsFastAllocate(fname)) { + forceMmapOff_ = true; + } + checkedDiskForMmap_ = true; + } + } + if (options.use_mmap_writes && !forceMmapOff_) { + result->reset(new PosixMmapFile(fname, fd, page_size_, options)); + } else if (options.use_direct_writes && !options.use_mmap_writes) { +#ifdef OS_MACOSX + if (fcntl(fd, F_NOCACHE, 1) == -1) { + close(fd); + s = IOError("while fcntl NoCache for reopened file for append", fname, + errno); + return s; + } +#elif defined(OS_SOLARIS) + if (directio(fd, DIRECTIO_ON) == -1) { + if (errno != ENOTTY) { // ZFS filesystems don't support DIRECTIO_ON + close(fd); + s = IOError("while calling directio()", fname, errno); + return s; + } + } +#endif + result->reset(new PosixWritableFile(fname, fd, options)); + } else { + // disable mmap writes + FileOptions no_mmap_writes_options = options; + no_mmap_writes_options.use_mmap_writes = false; + result->reset(new PosixWritableFile(fname, fd, no_mmap_writes_options)); + } + return s; + } + + IOStatus NewRandomRWFile(const std::string& fname, const FileOptions& options, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + int fd = -1; + int flags = cloexec_flags(O_RDWR, &options); + + while (fd < 0) { + IOSTATS_TIMER_GUARD(open_nanos); + + fd = open(fname.c_str(), flags, GetDBFileMode(allow_non_owner_access_)); + if (fd < 0) { + // Error while opening the file + if (errno == EINTR) { + continue; + } + return IOError("While open file for random read/write", fname, errno); + } + } + + SetFD_CLOEXEC(fd, &options); + result->reset(new PosixRandomRWFile(fname, fd, options)); + return IOStatus::OK(); + } + + IOStatus NewMemoryMappedFileBuffer( + const std::string& fname, + std::unique_ptr* result) override { + int fd = -1; + IOStatus status; + int flags = cloexec_flags(O_RDWR, nullptr); + + while (fd < 0) { + IOSTATS_TIMER_GUARD(open_nanos); + fd = open(fname.c_str(), flags, 0644); + if (fd < 0) { + // Error while opening the file + if (errno == EINTR) { + continue; + } + status = + IOError("While open file for raw mmap buffer access", fname, errno); + break; + } + } + uint64_t size; + if (status.ok()) { + IOOptions opts; + status = GetFileSize(fname, opts, &size, nullptr); + } + void* base = nullptr; + if (status.ok()) { + base = mmap(nullptr, static_cast(size), PROT_READ | PROT_WRITE, + MAP_SHARED, fd, 0); + if (base == MAP_FAILED) { + status = IOError("while mmap file for read", fname, errno); + } + } + if (status.ok()) { + result->reset( + new PosixMemoryMappedFileBuffer(base, static_cast(size))); + } + if (fd >= 0) { + // don't need to keep it open after mmap has been called + close(fd); + } + return status; + } + + IOStatus NewDirectory(const std::string& name, const IOOptions& /*opts*/, + std::unique_ptr* result, + IODebugContext* /*dbg*/) override { + result->reset(); + int fd; + int flags = cloexec_flags(0, nullptr); + { + IOSTATS_TIMER_GUARD(open_nanos); + fd = open(name.c_str(), flags); + } + if (fd < 0) { + return IOError("While open directory", name, errno); + } else { + result->reset(new PosixDirectory(fd)); + } + return IOStatus::OK(); + } + + IOStatus NewLogger(const std::string& /*fname*/, const IOOptions& /*opts*/, + std::shared_ptr* /*ptr*/, + IODebugContext* /*dbg*/) override { + return IOStatus::NotSupported(); + } + + IOStatus FileExists(const std::string& fname, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + int result = access(fname.c_str(), F_OK); + + if (result == 0) { + return IOStatus::OK(); + } + + int err = errno; + switch (err) { + case EACCES: + case ELOOP: + case ENAMETOOLONG: + case ENOENT: + case ENOTDIR: + return IOStatus::NotFound(); + default: + assert(err == EIO || err == ENOMEM); + return IOStatus::IOError("Unexpected error(" + ToString(err) + + ") accessing file `" + fname + "' "); + } + } + + IOStatus GetChildren(const std::string& dir, const IOOptions& /*opts*/, + std::vector* result, + IODebugContext* /*dbg*/) override { + result->clear(); + DIR* d = opendir(dir.c_str()); + if (d == nullptr) { + switch (errno) { + case EACCES: + case ENOENT: + case ENOTDIR: + return IOStatus::NotFound(); + default: + return IOError("While opendir", dir, errno); + } + } + struct dirent* entry; + while ((entry = readdir(d)) != nullptr) { + result->push_back(entry->d_name); + } + closedir(d); + return IOStatus::OK(); + } + + IOStatus DeleteFile(const std::string& fname, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + IOStatus result; + if (unlink(fname.c_str()) != 0) { + result = IOError("while unlink() file", fname, errno); + } + return result; + } + + IOStatus CreateDir(const std::string& name, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + IOStatus result; + if (mkdir(name.c_str(), 0755) != 0) { + result = IOError("While mkdir", name, errno); + } + return result; + } + + IOStatus CreateDirIfMissing(const std::string& name, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + IOStatus result; + if (mkdir(name.c_str(), 0755) != 0) { + if (errno != EEXIST) { + result = IOError("While mkdir if missing", name, errno); + } else if (!DirExists(name)) { // Check that name is actually a + // directory. + // Message is taken from mkdir + result = + IOStatus::IOError("`" + name + "' exists but is not a directory"); + } + } + return result; + } + + IOStatus DeleteDir(const std::string& name, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + IOStatus result; + if (rmdir(name.c_str()) != 0) { + result = IOError("file rmdir", name, errno); + } + return result; + } + + IOStatus GetFileSize(const std::string& fname, const IOOptions& /*opts*/, + uint64_t* size, IODebugContext* /*dbg*/) override { + IOStatus s; + struct stat sbuf; + if (stat(fname.c_str(), &sbuf) != 0) { + *size = 0; + s = IOError("while stat a file for size", fname, errno); + } else { + *size = sbuf.st_size; + } + return s; + } + + IOStatus GetFileModificationTime(const std::string& fname, + const IOOptions& /*opts*/, + uint64_t* file_mtime, + IODebugContext* /*dbg*/) override { + struct stat s; + if (stat(fname.c_str(), &s) != 0) { + return IOError("while stat a file for modification time", fname, errno); + } + *file_mtime = static_cast(s.st_mtime); + return IOStatus::OK(); + } + + IOStatus RenameFile(const std::string& src, const std::string& target, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + IOStatus result; + if (rename(src.c_str(), target.c_str()) != 0) { + result = IOError("While renaming a file to " + target, src, errno); + } + return result; + } + + IOStatus LinkFile(const std::string& src, const std::string& target, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + IOStatus result; + if (link(src.c_str(), target.c_str()) != 0) { + if (errno == EXDEV) { + return IOStatus::NotSupported("No cross FS links allowed"); + } + result = IOError("while link file to " + target, src, errno); + } + return result; + } + + IOStatus NumFileLinks(const std::string& fname, const IOOptions& /*opts*/, + uint64_t* count, IODebugContext* /*dbg*/) override { + struct stat s; + if (stat(fname.c_str(), &s) != 0) { + return IOError("while stat a file for num file links", fname, errno); + } + *count = static_cast(s.st_nlink); + return IOStatus::OK(); + } + + IOStatus AreFilesSame(const std::string& first, const std::string& second, + const IOOptions& /*opts*/, bool* res, + IODebugContext* /*dbg*/) override { + struct stat statbuf[2]; + if (stat(first.c_str(), &statbuf[0]) != 0) { + return IOError("stat file", first, errno); + } + if (stat(second.c_str(), &statbuf[1]) != 0) { + return IOError("stat file", second, errno); + } + + if (major(statbuf[0].st_dev) != major(statbuf[1].st_dev) || + minor(statbuf[0].st_dev) != minor(statbuf[1].st_dev) || + statbuf[0].st_ino != statbuf[1].st_ino) { + *res = false; + } else { + *res = true; + } + return IOStatus::OK(); + } + + IOStatus LockFile(const std::string& fname, const IOOptions& /*opts*/, + FileLock** lock, IODebugContext* /*dbg*/) override { + *lock = nullptr; + IOStatus result; + + mutex_lockedFiles.Lock(); + // If it already exists in the lockedFiles set, then it is already locked, + // and fail this lock attempt. Otherwise, insert it into lockedFiles. + // This check is needed because fcntl() does not detect lock conflict + // if the fcntl is issued by the same thread that earlier acquired + // this lock. + // We must do this check *before* opening the file: + // Otherwise, we will open a new file descriptor. Locks are associated with + // a process, not a file descriptor and when *any* file descriptor is + // closed, all locks the process holds for that *file* are released + if (lockedFiles.insert(fname).second == false) { + mutex_lockedFiles.Unlock(); + errno = ENOLCK; + return IOError("lock ", fname, errno); + } + + int fd; + int flags = cloexec_flags(O_RDWR | O_CREAT, nullptr); + + { + IOSTATS_TIMER_GUARD(open_nanos); + fd = open(fname.c_str(), flags, 0644); + } + if (fd < 0) { + result = IOError("while open a file for lock", fname, errno); + } else if (LockOrUnlock(fd, true) == -1) { + // if there is an error in locking, then remove the pathname from + // lockedfiles + lockedFiles.erase(fname); + result = IOError("While lock file", fname, errno); + close(fd); + } else { + SetFD_CLOEXEC(fd, nullptr); + PosixFileLock* my_lock = new PosixFileLock; + my_lock->fd_ = fd; + my_lock->filename = fname; + *lock = my_lock; + } + + mutex_lockedFiles.Unlock(); + return result; + } + + IOStatus UnlockFile(FileLock* lock, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + PosixFileLock* my_lock = reinterpret_cast(lock); + IOStatus result; + mutex_lockedFiles.Lock(); + // If we are unlocking, then verify that we had locked it earlier, + // it should already exist in lockedFiles. Remove it from lockedFiles. + if (lockedFiles.erase(my_lock->filename) != 1) { + errno = ENOLCK; + result = IOError("unlock", my_lock->filename, errno); + } else if (LockOrUnlock(my_lock->fd_, false) == -1) { + result = IOError("unlock", my_lock->filename, errno); + } + close(my_lock->fd_); + delete my_lock; + mutex_lockedFiles.Unlock(); + return result; + } + + IOStatus GetAbsolutePath(const std::string& db_path, + const IOOptions& /*opts*/, std::string* output_path, + IODebugContext* /*dbg*/) override { + if (!db_path.empty() && db_path[0] == '/') { + *output_path = db_path; + return IOStatus::OK(); + } + + char the_path[256]; + char* ret = getcwd(the_path, 256); + if (ret == nullptr) { + return IOStatus::IOError(strerror(errno)); + } + + *output_path = ret; + return IOStatus::OK(); + } + + IOStatus GetTestDirectory(const IOOptions& /*opts*/, std::string* result, + IODebugContext* /*dbg*/) override { + const char* env = getenv("TEST_TMPDIR"); + if (env && env[0] != '\0') { + *result = env; + } else { + char buf[100]; + snprintf(buf, sizeof(buf), "/tmp/rocksdbtest-%d", int(geteuid())); + *result = buf; + } + // Directory may already exist + { + IOOptions opts; + CreateDir(*result, opts, nullptr); + } + return IOStatus::OK(); + } + + IOStatus GetFreeSpace(const std::string& fname, const IOOptions& /*opts*/, + uint64_t* free_space, + IODebugContext* /*dbg*/) override { + struct statvfs sbuf; + + if (statvfs(fname.c_str(), &sbuf) < 0) { + return IOError("While doing statvfs", fname, errno); + } + + *free_space = ((uint64_t)sbuf.f_bsize * sbuf.f_bfree); + return IOStatus::OK(); + } + + FileOptions OptimizeForLogWrite(const FileOptions& file_options, + const DBOptions& db_options) const override { + FileOptions optimized = file_options; + optimized.use_mmap_writes = false; + optimized.use_direct_writes = false; + optimized.bytes_per_sync = db_options.wal_bytes_per_sync; + // TODO(icanadi) it's faster if fallocate_with_keep_size is false, but it + // breaks TransactionLogIteratorStallAtLastRecord unit test. Fix the unit + // test and make this false + optimized.fallocate_with_keep_size = true; + optimized.writable_file_max_buffer_size = + db_options.writable_file_max_buffer_size; + return optimized; + } + + FileOptions OptimizeForManifestWrite( + const FileOptions& file_options) const override { + FileOptions optimized = file_options; + optimized.use_mmap_writes = false; + optimized.use_direct_writes = false; + optimized.fallocate_with_keep_size = true; + return optimized; + } + + private: + bool checkedDiskForMmap_; + bool forceMmapOff_; // do we override Env options? + + // Returns true iff the named directory exists and is a directory. + virtual bool DirExists(const std::string& dname) { + struct stat statbuf; + if (stat(dname.c_str(), &statbuf) == 0) { + return S_ISDIR(statbuf.st_mode); + } + return false; // stat() failed return false + } + + bool SupportsFastAllocate(const std::string& path) { +#ifdef ROCKSDB_FALLOCATE_PRESENT + struct statfs s; + if (statfs(path.c_str(), &s)) { + return false; + } + switch (s.f_type) { + case EXT4_SUPER_MAGIC: + return true; + case XFS_SUPER_MAGIC: + return true; + case TMPFS_MAGIC: + return true; + default: + return false; + } +#else + (void)path; + return false; +#endif + } + +#if defined(ROCKSDB_IOURING_PRESENT) + // io_uring instance + std::unique_ptr thread_local_io_urings_; +#endif + + size_t page_size_; + + // If true, allow non owner read access for db files. Otherwise, non-owner + // has no access to db files. + bool allow_non_owner_access_; +}; + +PosixFileSystem::PosixFileSystem() + : checkedDiskForMmap_(false), + forceMmapOff_(false), + page_size_(getpagesize()), + allow_non_owner_access_(true) { +#if defined(ROCKSDB_IOURING_PRESENT) + // Test whether IOUring is supported, and if it does, create a managing + // object for thread local point so that in the future thread-local + // io_uring can be created. + struct io_uring* new_io_uring = CreateIOUring(); + if (new_io_uring != nullptr) { + thread_local_io_urings_.reset(new ThreadLocalPtr(DeleteIOUring)); + delete new_io_uring; + } +#endif +} + +} // namespace + +// +// Default Posix FileSystem +// +std::shared_ptr FileSystem::Default() { + static PosixFileSystem default_fs; + static std::shared_ptr default_fs_ptr( + &default_fs, [](PosixFileSystem*) {}); + return default_fs_ptr; +} + +} // namespace rocksdb diff --git a/env/io_posix.cc b/env/io_posix.cc index e8a3b1044..d550ae8c2 100644 --- a/env/io_posix.cc +++ b/env/io_posix.cc @@ -266,9 +266,11 @@ PosixSequentialFile::~PosixSequentialFile() { } } -Status PosixSequentialFile::Read(size_t n, Slice* result, char* scratch) { +IOStatus PosixSequentialFile::Read(size_t n, const IOOptions& /*opts*/, + Slice* result, char* scratch, + IODebugContext* /*dbg*/) { assert(result != nullptr && !use_direct_io()); - Status s; + IOStatus s; size_t r = 0; do { r = fread_unlocked(scratch, 1, n, file_); @@ -288,14 +290,16 @@ Status PosixSequentialFile::Read(size_t n, Slice* result, char* scratch) { return s; } -Status PosixSequentialFile::PositionedRead(uint64_t offset, size_t n, - Slice* result, char* scratch) { +IOStatus PosixSequentialFile::PositionedRead(uint64_t offset, size_t n, + const IOOptions& /*opts*/, + Slice* result, char* scratch, + IODebugContext* /*dbg*/) { assert(use_direct_io()); assert(IsSectorAligned(offset, GetRequiredBufferAlignment())); assert(IsSectorAligned(n, GetRequiredBufferAlignment())); assert(IsSectorAligned(scratch, GetRequiredBufferAlignment())); - Status s; + IOStatus s; ssize_t r = -1; size_t left = n; char* ptr = scratch; @@ -326,19 +330,19 @@ Status PosixSequentialFile::PositionedRead(uint64_t offset, size_t n, return s; } -Status PosixSequentialFile::Skip(uint64_t n) { +IOStatus PosixSequentialFile::Skip(uint64_t n) { if (fseek(file_, static_cast(n), SEEK_CUR)) { return IOError("While fseek to skip " + ToString(n) + " bytes", filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } -Status PosixSequentialFile::InvalidateCache(size_t offset, size_t length) { +IOStatus PosixSequentialFile::InvalidateCache(size_t offset, size_t length) { #ifndef OS_LINUX (void)offset; (void)length; - return Status::OK(); + return IOStatus::OK(); #else if (!use_direct_io()) { // free OS pages @@ -349,7 +353,7 @@ Status PosixSequentialFile::InvalidateCache(size_t offset, size_t length) { filename_, errno); } } - return Status::OK(); + return IOStatus::OK(); #endif } @@ -432,14 +436,16 @@ PosixRandomAccessFile::PosixRandomAccessFile( PosixRandomAccessFile::~PosixRandomAccessFile() { close(fd_); } -Status PosixRandomAccessFile::Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const { +IOStatus PosixRandomAccessFile::Read(uint64_t offset, size_t n, + const IOOptions& /*opts*/, Slice* result, + char* scratch, + IODebugContext* /*dbg*/) const { if (use_direct_io()) { assert(IsSectorAligned(offset, GetRequiredBufferAlignment())); assert(IsSectorAligned(n, GetRequiredBufferAlignment())); assert(IsSectorAligned(scratch, GetRequiredBufferAlignment())); } - Status s; + IOStatus s; ssize_t r = -1; size_t left = n; char* ptr = scratch; @@ -471,7 +477,10 @@ Status PosixRandomAccessFile::Read(uint64_t offset, size_t n, Slice* result, return s; } -Status PosixRandomAccessFile::MultiRead(ReadRequest* reqs, size_t num_reqs) { +IOStatus PosixRandomAccessFile::MultiRead(FSReadRequest* reqs, + size_t num_reqs, + const IOOptions& options, + IODebugContext* dbg) { #if defined(ROCKSDB_IOURING_PRESENT) size_t reqs_off; ssize_t ret __attribute__((__unused__)); @@ -490,13 +499,13 @@ Status PosixRandomAccessFile::MultiRead(ReadRequest* reqs, size_t num_reqs) { // Init failed, platform doesn't support io_uring. Fall back to // serialized reads if (iu == nullptr) { - return RandomAccessFile::MultiRead(reqs, num_reqs); + return FSRandomAccessFile::MultiRead(reqs, num_reqs, options, dbg); } struct WrappedReadRequest { - ReadRequest* req; + FSReadRequest* req; struct iovec iov; - explicit WrappedReadRequest(ReadRequest* r) : req(r) {} + explicit WrappedReadRequest(FSReadRequest* r) : req(r) {} }; autovector req_wraps; @@ -539,10 +548,10 @@ Status PosixRandomAccessFile::MultiRead(ReadRequest* reqs, size_t num_reqs) { ret = io_uring_wait_cqe(iu, &cqe); assert(!ret); req_wrap = static_cast(io_uring_cqe_get_data(cqe)); - ReadRequest* req = req_wrap->req; + FSReadRequest* req = req_wrap->req; if (static_cast(cqe->res) == req_wrap->iov.iov_len) { req->result = Slice(req->scratch, cqe->res); - req->status = Status::OK(); + req->status = IOStatus::OK(); } else if (cqe->res >= 0) { req->result = Slice(req->scratch, req_wrap->iov.iov_len - cqe->res); } else { @@ -554,14 +563,16 @@ Status PosixRandomAccessFile::MultiRead(ReadRequest* reqs, size_t num_reqs) { num_reqs -= this_reqs; reqs_off += this_reqs; } - return Status::OK(); + return IOStatus::OK(); #else - return RandomAccessFile::MultiRead(reqs, num_reqs); + return FSRandomAccessFile::MultiRead(reqs, num_reqs, options, dbg); #endif } -Status PosixRandomAccessFile::Prefetch(uint64_t offset, size_t n) { - Status s; +IOStatus PosixRandomAccessFile::Prefetch(uint64_t offset, size_t n, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + IOStatus s; if (!use_direct_io()) { ssize_t r = 0; #ifdef OS_LINUX @@ -593,19 +604,19 @@ void PosixRandomAccessFile::Hint(AccessPattern pattern) { return; } switch (pattern) { - case NORMAL: + case kNormal: Fadvise(fd_, 0, 0, POSIX_FADV_NORMAL); break; - case RANDOM: + case kRandom: Fadvise(fd_, 0, 0, POSIX_FADV_RANDOM); break; - case SEQUENTIAL: + case kSequential: Fadvise(fd_, 0, 0, POSIX_FADV_SEQUENTIAL); break; - case WILLNEED: + case kWillNeed: Fadvise(fd_, 0, 0, POSIX_FADV_WILLNEED); break; - case DONTNEED: + case kWontNeed: Fadvise(fd_, 0, 0, POSIX_FADV_DONTNEED); break; default: @@ -614,19 +625,19 @@ void PosixRandomAccessFile::Hint(AccessPattern pattern) { } } -Status PosixRandomAccessFile::InvalidateCache(size_t offset, size_t length) { +IOStatus PosixRandomAccessFile::InvalidateCache(size_t offset, size_t length) { if (use_direct_io()) { - return Status::OK(); + return IOStatus::OK(); } #ifndef OS_LINUX (void)offset; (void)length; - return Status::OK(); + return IOStatus::OK(); #else // free OS pages int ret = Fadvise(fd_, offset, length, POSIX_FADV_DONTNEED); if (ret == 0) { - return Status::OK(); + return IOStatus::OK(); } return IOError("While fadvise NotNeeded offset " + ToString(offset) + " len " + ToString(length), @@ -662,9 +673,11 @@ PosixMmapReadableFile::~PosixMmapReadableFile() { close(fd_); } -Status PosixMmapReadableFile::Read(uint64_t offset, size_t n, Slice* result, - char* /*scratch*/) const { - Status s; +IOStatus PosixMmapReadableFile::Read(uint64_t offset, size_t n, + const IOOptions& /*opts*/, Slice* result, + char* /*scratch*/, + IODebugContext* /*dbg*/) const { + IOStatus s; if (offset > length_) { *result = Slice(); return IOError("While mmap read offset " + ToString(offset) + @@ -677,16 +690,16 @@ Status PosixMmapReadableFile::Read(uint64_t offset, size_t n, Slice* result, return s; } -Status PosixMmapReadableFile::InvalidateCache(size_t offset, size_t length) { +IOStatus PosixMmapReadableFile::InvalidateCache(size_t offset, size_t length) { #ifndef OS_LINUX (void)offset; (void)length; - return Status::OK(); + return IOStatus::OK(); #else // free OS pages int ret = Fadvise(fd_, offset, length, POSIX_FADV_DONTNEED); if (ret == 0) { - return Status::OK(); + return IOStatus::OK(); } return IOError("While fadvise not needed. Offset " + ToString(offset) + " len" + ToString(length), @@ -702,7 +715,7 @@ Status PosixMmapReadableFile::InvalidateCache(size_t offset, size_t length) { * file before reading from it, or for log files, the reading code * knows enough to skip zero suffixes. */ -Status PosixMmapFile::UnmapCurrentRegion() { +IOStatus PosixMmapFile::UnmapCurrentRegion() { TEST_KILL_RANDOM("PosixMmapFile::UnmapCurrentRegion:0", rocksdb_kill_odds); if (base_ != nullptr) { int munmap_status = munmap(base_, limit_ - base_); @@ -720,10 +733,10 @@ Status PosixMmapFile::UnmapCurrentRegion() { map_size_ *= 2; } } - return Status::OK(); + return IOStatus::OK(); } -Status PosixMmapFile::MapNewRegion() { +IOStatus PosixMmapFile::MapNewRegion() { #ifdef ROCKSDB_FALLOCATE_PRESENT assert(base_ == nullptr); TEST_KILL_RANDOM("PosixMmapFile::UnmapCurrentRegion:0", rocksdb_kill_odds); @@ -736,8 +749,8 @@ Status PosixMmapFile::MapNewRegion() { alloc_status = posix_fallocate(fd_, file_offset_, map_size_); } if (alloc_status != 0) { - return Status::IOError("Error allocating space to file : " + filename_ + - "Error : " + strerror(alloc_status)); + return IOStatus::IOError("Error allocating space to file : " + filename_ + + "Error : " + strerror(alloc_status)); } } @@ -745,7 +758,7 @@ Status PosixMmapFile::MapNewRegion() { void* ptr = mmap(nullptr, map_size_, PROT_READ | PROT_WRITE, MAP_SHARED, fd_, file_offset_); if (ptr == MAP_FAILED) { - return Status::IOError("MMap failed on " + filename_); + return IOStatus::IOError("MMap failed on " + filename_); } TEST_KILL_RANDOM("PosixMmapFile::Append:2", rocksdb_kill_odds); @@ -753,15 +766,15 @@ Status PosixMmapFile::MapNewRegion() { limit_ = base_ + map_size_; dst_ = base_; last_sync_ = base_; - return Status::OK(); + return IOStatus::OK(); #else - return Status::NotSupported("This platform doesn't support fallocate()"); + return IOStatus::NotSupported("This platform doesn't support fallocate()"); #endif } -Status PosixMmapFile::Msync() { +IOStatus PosixMmapFile::Msync() { if (dst_ == last_sync_) { - return Status::OK(); + return IOStatus::OK(); } // Find the beginnings of the pages that contain the first and last // bytes to be synced. @@ -772,7 +785,7 @@ Status PosixMmapFile::Msync() { if (msync(base_ + p1, p2 - p1 + page_size_, MS_SYNC) < 0) { return IOError("While msync", filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } PosixMmapFile::PosixMmapFile(const std::string& fname, int fd, size_t page_size, @@ -799,11 +812,12 @@ PosixMmapFile::PosixMmapFile(const std::string& fname, int fd, size_t page_size, PosixMmapFile::~PosixMmapFile() { if (fd_ >= 0) { - PosixMmapFile::Close(); + PosixMmapFile::Close(IOOptions(), nullptr); } } -Status PosixMmapFile::Append(const Slice& data) { +IOStatus PosixMmapFile::Append(const Slice& data, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { const char* src = data.data(); size_t left = data.size(); while (left > 0) { @@ -811,7 +825,7 @@ Status PosixMmapFile::Append(const Slice& data) { assert(dst_ <= limit_); size_t avail = limit_ - dst_; if (avail == 0) { - Status s = UnmapCurrentRegion(); + IOStatus s = UnmapCurrentRegion(); if (!s.ok()) { return s; } @@ -829,11 +843,12 @@ Status PosixMmapFile::Append(const Slice& data) { src += n; left -= n; } - return Status::OK(); + return IOStatus::OK(); } -Status PosixMmapFile::Close() { - Status s; +IOStatus PosixMmapFile::Close(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + IOStatus s; size_t unused = limit_ - dst_; s = UnmapCurrentRegion(); @@ -858,9 +873,13 @@ Status PosixMmapFile::Close() { return s; } -Status PosixMmapFile::Flush() { return Status::OK(); } +IOStatus PosixMmapFile::Flush(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + return IOStatus::OK(); +} -Status PosixMmapFile::Sync() { +IOStatus PosixMmapFile::Sync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (fdatasync(fd_) < 0) { return IOError("While fdatasync mmapped file", filename_, errno); } @@ -871,7 +890,8 @@ Status PosixMmapFile::Sync() { /** * Flush data as well as metadata to stable storage. */ -Status PosixMmapFile::Fsync() { +IOStatus PosixMmapFile::Fsync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (fsync(fd_) < 0) { return IOError("While fsync mmaped file", filename_, errno); } @@ -884,28 +904,31 @@ Status PosixMmapFile::Fsync() { * size that is returned from the filesystem because we use mmap * to extend file by map_size every time. */ -uint64_t PosixMmapFile::GetFileSize() { +uint64_t PosixMmapFile::GetFileSize(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { size_t used = dst_ - base_; return file_offset_ + used; } -Status PosixMmapFile::InvalidateCache(size_t offset, size_t length) { +IOStatus PosixMmapFile::InvalidateCache(size_t offset, size_t length) { #ifndef OS_LINUX (void)offset; (void)length; - return Status::OK(); + return IOStatus::OK(); #else // free OS pages int ret = Fadvise(fd_, offset, length, POSIX_FADV_DONTNEED); if (ret == 0) { - return Status::OK(); + return IOStatus::OK(); } return IOError("While fadvise NotNeeded mmapped file", filename_, errno); #endif } #ifdef ROCKSDB_FALLOCATE_PRESENT -Status PosixMmapFile::Allocate(uint64_t offset, uint64_t len) { +IOStatus PosixMmapFile::Allocate(uint64_t offset, uint64_t len, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { assert(offset <= static_cast(std::numeric_limits::max())); assert(len <= static_cast(std::numeric_limits::max())); TEST_KILL_RANDOM("PosixMmapFile::Allocate:0", rocksdb_kill_odds); @@ -916,7 +939,7 @@ Status PosixMmapFile::Allocate(uint64_t offset, uint64_t len) { static_cast(offset), static_cast(len)); } if (alloc_status == 0) { - return Status::OK(); + return IOStatus::OK(); } else { return IOError( "While fallocate offset " + ToString(offset) + " len " + ToString(len), @@ -932,7 +955,7 @@ Status PosixMmapFile::Allocate(uint64_t offset, uint64_t len) { */ PosixWritableFile::PosixWritableFile(const std::string& fname, int fd, const EnvOptions& options) - : WritableFile(options), + : FSWritableFile(options), filename_(fname), use_direct_io_(options.use_direct_writes), fd_(fd), @@ -950,11 +973,12 @@ PosixWritableFile::PosixWritableFile(const std::string& fname, int fd, PosixWritableFile::~PosixWritableFile() { if (fd_ >= 0) { - PosixWritableFile::Close(); + PosixWritableFile::Close(IOOptions(), nullptr); } } -Status PosixWritableFile::Append(const Slice& data) { +IOStatus PosixWritableFile::Append(const Slice& data, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (use_direct_io()) { assert(IsSectorAligned(data.size(), GetRequiredBufferAlignment())); assert(IsSectorAligned(data.data(), GetRequiredBufferAlignment())); @@ -967,10 +991,12 @@ Status PosixWritableFile::Append(const Slice& data) { } filesize_ += nbytes; - return Status::OK(); + return IOStatus::OK(); } -Status PosixWritableFile::PositionedAppend(const Slice& data, uint64_t offset) { +IOStatus PosixWritableFile::PositionedAppend(const Slice& data, uint64_t offset, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (use_direct_io()) { assert(IsSectorAligned(offset, GetRequiredBufferAlignment())); assert(IsSectorAligned(data.size(), GetRequiredBufferAlignment())); @@ -984,11 +1010,12 @@ Status PosixWritableFile::PositionedAppend(const Slice& data, uint64_t offset) { filename_, errno); } filesize_ = offset + nbytes; - return Status::OK(); + return IOStatus::OK(); } -Status PosixWritableFile::Truncate(uint64_t size) { - Status s; +IOStatus PosixWritableFile::Truncate(uint64_t size, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + IOStatus s; int r = ftruncate(fd_, size); if (r < 0) { s = IOError("While ftruncate file to size " + ToString(size), filename_, @@ -999,8 +1026,9 @@ Status PosixWritableFile::Truncate(uint64_t size) { return s; } -Status PosixWritableFile::Close() { - Status s; +IOStatus PosixWritableFile::Close(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + IOStatus s; size_t block_size; size_t last_allocated_block; @@ -1054,25 +1082,33 @@ Status PosixWritableFile::Close() { } // write out the cached data to the OS cache -Status PosixWritableFile::Flush() { return Status::OK(); } +IOStatus PosixWritableFile::Flush(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + return IOStatus::OK(); +} -Status PosixWritableFile::Sync() { +IOStatus PosixWritableFile::Sync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (fdatasync(fd_) < 0) { return IOError("While fdatasync", filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } -Status PosixWritableFile::Fsync() { +IOStatus PosixWritableFile::Fsync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (fsync(fd_) < 0) { return IOError("While fsync", filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } bool PosixWritableFile::IsSyncThreadSafe() const { return true; } -uint64_t PosixWritableFile::GetFileSize() { return filesize_; } +uint64_t PosixWritableFile::GetFileSize(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + return filesize_; +} void PosixWritableFile::SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) { #ifdef OS_LINUX @@ -1092,26 +1128,28 @@ void PosixWritableFile::SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) { #endif // OS_LINUX } -Status PosixWritableFile::InvalidateCache(size_t offset, size_t length) { +IOStatus PosixWritableFile::InvalidateCache(size_t offset, size_t length) { if (use_direct_io()) { - return Status::OK(); + return IOStatus::OK(); } #ifndef OS_LINUX (void)offset; (void)length; - return Status::OK(); + return IOStatus::OK(); #else // free OS pages int ret = Fadvise(fd_, offset, length, POSIX_FADV_DONTNEED); if (ret == 0) { - return Status::OK(); + return IOStatus::OK(); } return IOError("While fadvise NotNeeded", filename_, errno); #endif } #ifdef ROCKSDB_FALLOCATE_PRESENT -Status PosixWritableFile::Allocate(uint64_t offset, uint64_t len) { +IOStatus PosixWritableFile::Allocate(uint64_t offset, uint64_t len, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { assert(offset <= static_cast(std::numeric_limits::max())); assert(len <= static_cast(std::numeric_limits::max())); TEST_KILL_RANDOM("PosixWritableFile::Allocate:0", rocksdb_kill_odds); @@ -1123,7 +1161,7 @@ Status PosixWritableFile::Allocate(uint64_t offset, uint64_t len) { static_cast(offset), static_cast(len)); } if (alloc_status == 0) { - return Status::OK(); + return IOStatus::OK(); } else { return IOError( "While fallocate offset " + ToString(offset) + " len " + ToString(len), @@ -1132,7 +1170,9 @@ Status PosixWritableFile::Allocate(uint64_t offset, uint64_t len) { } #endif -Status PosixWritableFile::RangeSync(uint64_t offset, uint64_t nbytes) { +IOStatus PosixWritableFile::RangeSync(uint64_t offset, uint64_t nbytes, + const IOOptions& opts, + IODebugContext* dbg) { #ifdef ROCKSDB_RANGESYNC_PRESENT assert(offset <= static_cast(std::numeric_limits::max())); assert(nbytes <= static_cast(std::numeric_limits::max())); @@ -1153,10 +1193,10 @@ Status PosixWritableFile::RangeSync(uint64_t offset, uint64_t nbytes) { return IOError("While sync_file_range returned " + ToString(ret), filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } #endif // ROCKSDB_RANGESYNC_PRESENT - return WritableFile::RangeSync(offset, nbytes); + return FSWritableFile::RangeSync(offset, nbytes, opts, dbg); } #ifdef OS_LINUX @@ -1175,11 +1215,13 @@ PosixRandomRWFile::PosixRandomRWFile(const std::string& fname, int fd, PosixRandomRWFile::~PosixRandomRWFile() { if (fd_ >= 0) { - Close(); + Close(IOOptions(), nullptr); } } -Status PosixRandomRWFile::Write(uint64_t offset, const Slice& data) { +IOStatus PosixRandomRWFile::Write(uint64_t offset, const Slice& data, + const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { const char* src = data.data(); size_t nbytes = data.size(); if (!PosixPositionedWrite(fd_, src, nbytes, static_cast(offset))) { @@ -1188,11 +1230,12 @@ Status PosixRandomRWFile::Write(uint64_t offset, const Slice& data) { filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } -Status PosixRandomRWFile::Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const { +IOStatus PosixRandomRWFile::Read(uint64_t offset, size_t n, + const IOOptions& /*opts*/, Slice* result, + char* scratch, IODebugContext* /*dbg*/) const { size_t left = n; char* ptr = scratch; while (left > 0) { @@ -1218,31 +1261,37 @@ Status PosixRandomRWFile::Read(uint64_t offset, size_t n, Slice* result, } *result = Slice(scratch, n - left); - return Status::OK(); + return IOStatus::OK(); } -Status PosixRandomRWFile::Flush() { return Status::OK(); } +IOStatus PosixRandomRWFile::Flush(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { + return IOStatus::OK(); +} -Status PosixRandomRWFile::Sync() { +IOStatus PosixRandomRWFile::Sync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (fdatasync(fd_) < 0) { return IOError("While fdatasync random read/write file", filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } -Status PosixRandomRWFile::Fsync() { +IOStatus PosixRandomRWFile::Fsync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (fsync(fd_) < 0) { return IOError("While fsync random read/write file", filename_, errno); } - return Status::OK(); + return IOStatus::OK(); } -Status PosixRandomRWFile::Close() { +IOStatus PosixRandomRWFile::Close(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { if (close(fd_) < 0) { return IOError("While close random read/write file", filename_, errno); } fd_ = -1; - return Status::OK(); + return IOStatus::OK(); } PosixMemoryMappedFileBuffer::~PosixMemoryMappedFileBuffer() { @@ -1256,13 +1305,14 @@ PosixMemoryMappedFileBuffer::~PosixMemoryMappedFileBuffer() { PosixDirectory::~PosixDirectory() { close(fd_); } -Status PosixDirectory::Fsync() { +IOStatus PosixDirectory::Fsync(const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) { #ifndef OS_AIX if (fsync(fd_) == -1) { return IOError("While fsync", "a directory", errno); } #endif - return Status::OK(); + return IOStatus::OK(); } } // namespace rocksdb #endif diff --git a/env/io_posix.h b/env/io_posix.h index 195902b51..67fdd95d8 100644 --- a/env/io_posix.h +++ b/env/io_posix.h @@ -17,6 +17,8 @@ #include #include "rocksdb/env.h" #include "util/thread_local.h" +#include "rocksdb/file_system.h" +#include "rocksdb/io_status.h" // For non linux platform, the following macros are used only as place // holder. @@ -38,20 +40,23 @@ static std::string IOErrorMsg(const std::string& context, } // file_name can be left empty if it is not unkown. -static Status IOError(const std::string& context, const std::string& file_name, - int err_number) { +static IOStatus IOError(const std::string& context, + const std::string& file_name, int err_number) { switch (err_number) { - case ENOSPC: - return Status::NoSpace(IOErrorMsg(context, file_name), - strerror(err_number)); + case ENOSPC: { + IOStatus s = IOStatus::NoSpace(IOErrorMsg(context, file_name), + strerror(err_number)); + s.SetRetryable(true); + return s; + } case ESTALE: - return Status::IOError(Status::kStaleFile); + return IOStatus::IOError(IOStatus::kStaleFile); case ENOENT: - return Status::PathNotFound(IOErrorMsg(context, file_name), - strerror(err_number)); + return IOStatus::PathNotFound(IOErrorMsg(context, file_name), + strerror(err_number)); default: - return Status::IOError(IOErrorMsg(context, file_name), - strerror(err_number)); + return IOStatus::IOError(IOErrorMsg(context, file_name), + strerror(err_number)); } } @@ -60,7 +65,7 @@ class PosixHelper { static size_t GetUniqueIdFromFile(int fd, char* id, size_t max_size); }; -class PosixSequentialFile : public SequentialFile { +class PosixSequentialFile : public FSSequentialFile { private: std::string filename_; FILE* file_; @@ -73,11 +78,13 @@ class PosixSequentialFile : public SequentialFile { const EnvOptions& options); virtual ~PosixSequentialFile(); - virtual Status Read(size_t n, Slice* result, char* scratch) override; - virtual Status PositionedRead(uint64_t offset, size_t n, Slice* result, - char* scratch) override; - virtual Status Skip(uint64_t n) override; - virtual Status InvalidateCache(size_t offset, size_t length) override; + virtual IOStatus Read(size_t n, const IOOptions& opts, Slice* result, + char* scratch, IODebugContext* dbg) override; + virtual IOStatus PositionedRead(uint64_t offset, size_t n, + const IOOptions& opts, Slice* result, + char* scratch, IODebugContext* dbg) override; + virtual IOStatus Skip(uint64_t n) override; + virtual IOStatus InvalidateCache(size_t offset, size_t length) override; virtual bool use_direct_io() const override { return use_direct_io_; } virtual size_t GetRequiredBufferAlignment() const override { return logical_sector_size_; @@ -104,7 +111,7 @@ inline struct io_uring* CreateIOUring() { } #endif // defined(ROCKSDB_IOURING_PRESENT) -class PosixRandomAccessFile : public RandomAccessFile { +class PosixRandomAccessFile : public FSRandomAccessFile { protected: std::string filename_; int fd_; @@ -124,25 +131,29 @@ class PosixRandomAccessFile : public RandomAccessFile { ); virtual ~PosixRandomAccessFile(); - virtual Status Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const override; + virtual IOStatus Read(uint64_t offset, size_t n, const IOOptions& opts, + Slice* result, char* scratch, + IODebugContext* dbg) const override; - virtual Status MultiRead(ReadRequest* reqs, size_t num_reqs) override; + virtual IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs, + const IOOptions& options, + IODebugContext* dbg) override; - virtual Status Prefetch(uint64_t offset, size_t n) override; + virtual IOStatus Prefetch(uint64_t offset, size_t n, const IOOptions& opts, + IODebugContext* dbg) override; #if defined(OS_LINUX) || defined(OS_MACOSX) || defined(OS_AIX) virtual size_t GetUniqueId(char* id, size_t max_size) const override; #endif virtual void Hint(AccessPattern pattern) override; - virtual Status InvalidateCache(size_t offset, size_t length) override; + virtual IOStatus InvalidateCache(size_t offset, size_t length) override; virtual bool use_direct_io() const override { return use_direct_io_; } virtual size_t GetRequiredBufferAlignment() const override { return logical_sector_size_; } }; -class PosixWritableFile : public WritableFile { +class PosixWritableFile : public FSWritableFile { protected: const std::string filename_; const bool use_direct_io_; @@ -166,32 +177,41 @@ class PosixWritableFile : public WritableFile { // Need to implement this so the file is truncated correctly // with direct I/O - virtual Status Truncate(uint64_t size) override; - virtual Status Close() override; - virtual Status Append(const Slice& data) override; - virtual Status PositionedAppend(const Slice& data, uint64_t offset) override; - virtual Status Flush() override; - virtual Status Sync() override; - virtual Status Fsync() override; + virtual IOStatus Truncate(uint64_t size, const IOOptions& opts, + IODebugContext* dbg) override; + virtual IOStatus Close(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Append(const Slice& data, const IOOptions& opts, + IODebugContext* dbg) override; + virtual IOStatus PositionedAppend(const Slice& data, uint64_t offset, + const IOOptions& opts, + IODebugContext* dbg) override; + virtual IOStatus Flush(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Sync(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Fsync(const IOOptions& opts, IODebugContext* dbg) override; virtual bool IsSyncThreadSafe() const override; virtual bool use_direct_io() const override { return use_direct_io_; } virtual void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) override; - virtual uint64_t GetFileSize() override; - virtual Status InvalidateCache(size_t offset, size_t length) override; + virtual uint64_t GetFileSize(const IOOptions& opts, + IODebugContext* dbg) override; + virtual IOStatus InvalidateCache(size_t offset, size_t length) override; virtual size_t GetRequiredBufferAlignment() const override { return logical_sector_size_; } #ifdef ROCKSDB_FALLOCATE_PRESENT - virtual Status Allocate(uint64_t offset, uint64_t len) override; + virtual IOStatus Allocate(uint64_t offset, uint64_t len, + const IOOptions& opts, + IODebugContext* dbg) override; #endif - virtual Status RangeSync(uint64_t offset, uint64_t nbytes) override; + virtual IOStatus RangeSync(uint64_t offset, uint64_t nbytes, + const IOOptions& opts, + IODebugContext* dbg) override; #ifdef OS_LINUX virtual size_t GetUniqueId(char* id, size_t max_size) const override; #endif }; // mmap() based random-access -class PosixMmapReadableFile : public RandomAccessFile { +class PosixMmapReadableFile : public FSRandomAccessFile { private: int fd_; std::string filename_; @@ -202,12 +222,13 @@ class PosixMmapReadableFile : public RandomAccessFile { PosixMmapReadableFile(const int fd, const std::string& fname, void* base, size_t length, const EnvOptions& options); virtual ~PosixMmapReadableFile(); - virtual Status Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const override; - virtual Status InvalidateCache(size_t offset, size_t length) override; + virtual IOStatus Read(uint64_t offset, size_t n, const IOOptions& opts, + Slice* result, char* scratch, + IODebugContext* dbg) const override; + virtual IOStatus InvalidateCache(size_t offset, size_t length) override; }; -class PosixMmapFile : public WritableFile { +class PosixMmapFile : public FSWritableFile { private: std::string filename_; int fd_; @@ -232,9 +253,9 @@ class PosixMmapFile : public WritableFile { return s; } - Status MapNewRegion(); - Status UnmapCurrentRegion(); - Status Msync(); + IOStatus MapNewRegion(); + IOStatus UnmapCurrentRegion(); + IOStatus Msync(); public: PosixMmapFile(const std::string& fname, int fd, size_t page_size, @@ -243,34 +264,43 @@ class PosixMmapFile : public WritableFile { // Means Close() will properly take care of truncate // and it does not need any additional information - virtual Status Truncate(uint64_t /*size*/) override { return Status::OK(); } - virtual Status Close() override; - virtual Status Append(const Slice& data) override; - virtual Status Flush() override; - virtual Status Sync() override; - virtual Status Fsync() override; - virtual uint64_t GetFileSize() override; - virtual Status InvalidateCache(size_t offset, size_t length) override; + virtual IOStatus Truncate(uint64_t /*size*/, const IOOptions& /*opts*/, + IODebugContext* /*dbg*/) override { + return IOStatus::OK(); + } + virtual IOStatus Close(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Append(const Slice& data, const IOOptions& opts, + IODebugContext* dbg) override; + virtual IOStatus Flush(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Sync(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Fsync(const IOOptions& opts, IODebugContext* dbg) override; + virtual uint64_t GetFileSize(const IOOptions& opts, + IODebugContext* dbg) override; + virtual IOStatus InvalidateCache(size_t offset, size_t length) override; #ifdef ROCKSDB_FALLOCATE_PRESENT - virtual Status Allocate(uint64_t offset, uint64_t len) override; + virtual IOStatus Allocate(uint64_t offset, uint64_t len, + const IOOptions& opts, + IODebugContext* dbg) override; #endif }; -class PosixRandomRWFile : public RandomRWFile { +class PosixRandomRWFile : public FSRandomRWFile { public: explicit PosixRandomRWFile(const std::string& fname, int fd, const EnvOptions& options); virtual ~PosixRandomRWFile(); - virtual Status Write(uint64_t offset, const Slice& data) override; + virtual IOStatus Write(uint64_t offset, const Slice& data, + const IOOptions& opts, IODebugContext* dbg) override; - virtual Status Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const override; + virtual IOStatus Read(uint64_t offset, size_t n, const IOOptions& opts, + Slice* result, char* scratch, + IODebugContext* dbg) const override; - virtual Status Flush() override; - virtual Status Sync() override; - virtual Status Fsync() override; - virtual Status Close() override; + virtual IOStatus Flush(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Sync(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Fsync(const IOOptions& opts, IODebugContext* dbg) override; + virtual IOStatus Close(const IOOptions& opts, IODebugContext* dbg) override; private: const std::string filename_; @@ -283,11 +313,11 @@ struct PosixMemoryMappedFileBuffer : public MemoryMappedFileBuffer { virtual ~PosixMemoryMappedFileBuffer(); }; -class PosixDirectory : public Directory { +class PosixDirectory : public FSDirectory { public: explicit PosixDirectory(int fd) : fd_(fd) {} ~PosixDirectory(); - virtual Status Fsync() override; + virtual IOStatus Fsync(const IOOptions& opts, IODebugContext* dbg) override; private: int fd_; diff --git a/file/delete_scheduler.cc b/file/delete_scheduler.cc index b66956ca0..d528d6713 100644 --- a/file/delete_scheduler.cc +++ b/file/delete_scheduler.cc @@ -19,12 +19,13 @@ namespace rocksdb { -DeleteScheduler::DeleteScheduler(Env* env, int64_t rate_bytes_per_sec, - Logger* info_log, +DeleteScheduler::DeleteScheduler(Env* env, FileSystem* fs, + int64_t rate_bytes_per_sec, Logger* info_log, SstFileManagerImpl* sst_file_manager, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk) : env_(env), + fs_(fs), total_trash_size_(0), rate_bytes_per_sec_(rate_bytes_per_sec), pending_files_(0), @@ -61,7 +62,7 @@ Status DeleteScheduler::DeleteFile(const std::string& file_path, // Rate limiting is disabled or trash size makes up more than // max_trash_db_ratio_ (default 25%) of the total DB size TEST_SYNC_POINT("DeleteScheduler::DeleteFile"); - s = env_->DeleteFile(file_path); + s = fs_->DeleteFile(file_path, IOOptions(), nullptr); if (s.ok()) { sst_file_manager_->OnDeleteFile(file_path); } @@ -74,7 +75,7 @@ Status DeleteScheduler::DeleteFile(const std::string& file_path, if (!s.ok()) { ROCKS_LOG_ERROR(info_log_, "Failed to mark %s as trash", file_path.c_str()); - s = env_->DeleteFile(file_path); + s = fs_->DeleteFile(file_path, IOOptions(), nullptr); if (s.ok()) { sst_file_manager_->OnDeleteFile(file_path); } @@ -83,7 +84,7 @@ Status DeleteScheduler::DeleteFile(const std::string& file_path, // Update the total trash size uint64_t trash_file_size = 0; - env_->GetFileSize(trash_file, &trash_file_size); + fs_->GetFileSize(trash_file, IOOptions(), &trash_file_size, nullptr); total_trash_size_.fetch_add(trash_file_size); // Add file to delete queue @@ -165,10 +166,10 @@ Status DeleteScheduler::MarkAsTrash(const std::string& file_path, int cnt = 0; InstrumentedMutexLock l(&file_move_mu_); while (true) { - s = env_->FileExists(*trash_file); + s = fs_->FileExists(*trash_file, IOOptions(), nullptr); if (s.IsNotFound()) { // We found a path for our file in trash - s = env_->RenameFile(file_path, *trash_file); + s = fs_->RenameFile(file_path, *trash_file, IOOptions(), nullptr); break; } else if (s.ok()) { // Name conflict, generate new random suffix @@ -262,7 +263,7 @@ Status DeleteScheduler::DeleteTrashFile(const std::string& path_in_trash, uint64_t* deleted_bytes, bool* is_complete) { uint64_t file_size; - Status s = env_->GetFileSize(path_in_trash, &file_size); + Status s = fs_->GetFileSize(path_in_trash, IOOptions(), &file_size, nullptr); *is_complete = true; TEST_SYNC_POINT("DeleteScheduler::DeleteTrashFile:DeleteFile"); if (s.ok()) { @@ -273,17 +274,19 @@ Status DeleteScheduler::DeleteTrashFile(const std::string& path_in_trash, // file after the number of file link check and ftruncte because // the file is now in trash and no hardlink is supposed to create // to trash files by RocksDB. - Status my_status = env_->NumFileLinks(path_in_trash, &num_hard_links); + Status my_status = fs_->NumFileLinks(path_in_trash, IOOptions(), + &num_hard_links, nullptr); if (my_status.ok()) { if (num_hard_links == 1) { - std::unique_ptr wf; - my_status = - env_->ReopenWritableFile(path_in_trash, &wf, EnvOptions()); + std::unique_ptr wf; + my_status = fs_->ReopenWritableFile(path_in_trash, FileOptions(), + &wf, nullptr); if (my_status.ok()) { - my_status = wf->Truncate(file_size - bytes_max_delete_chunk_); + my_status = wf->Truncate(file_size - bytes_max_delete_chunk_, + IOOptions(), nullptr); if (my_status.ok()) { TEST_SYNC_POINT("DeleteScheduler::DeleteTrashFile:Fsync"); - my_status = wf->Fsync(); + my_status = wf->Fsync(IOOptions(), nullptr); } } if (my_status.ok()) { @@ -312,14 +315,14 @@ Status DeleteScheduler::DeleteTrashFile(const std::string& path_in_trash, } if (need_full_delete) { - s = env_->DeleteFile(path_in_trash); + s = fs_->DeleteFile(path_in_trash, IOOptions(), nullptr); if (!dir_to_sync.empty()) { - std::unique_ptr dir_obj; + std::unique_ptr dir_obj; if (s.ok()) { - s = env_->NewDirectory(dir_to_sync, &dir_obj); + s = fs_->NewDirectory(dir_to_sync, IOOptions(), &dir_obj, nullptr); } if (s.ok()) { - s = dir_obj->Fsync(); + s = dir_obj->Fsync(IOOptions(), nullptr); TEST_SYNC_POINT_CALLBACK( "DeleteScheduler::DeleteTrashFile::AfterSyncDir", reinterpret_cast(const_cast(&dir_to_sync))); diff --git a/file/delete_scheduler.h b/file/delete_scheduler.h index 29b70517b..4e72296c8 100644 --- a/file/delete_scheduler.h +++ b/file/delete_scheduler.h @@ -15,6 +15,7 @@ #include "monitoring/instrumented_mutex.h" #include "port/port.h" +#include "rocksdb/file_system.h" #include "rocksdb/status.h" namespace rocksdb { @@ -32,8 +33,8 @@ class SstFileManagerImpl; // case DeleteScheduler will delete files immediately. class DeleteScheduler { public: - DeleteScheduler(Env* env, int64_t rate_bytes_per_sec, Logger* info_log, - SstFileManagerImpl* sst_file_manager, + DeleteScheduler(Env* env, FileSystem* fs, int64_t rate_bytes_per_sec, + Logger* info_log, SstFileManagerImpl* sst_file_manager, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk); ~DeleteScheduler(); @@ -91,6 +92,8 @@ class DeleteScheduler { void BackgroundEmptyTrash(); Env* env_; + FileSystem* fs_; + // total size of trash files std::atomic total_trash_size_; // Maximum number of bytes that should be deleted per second diff --git a/file/delete_scheduler_test.cc b/file/delete_scheduler_test.cc index b6d4b903c..39d7af145 100644 --- a/file/delete_scheduler_test.cc +++ b/file/delete_scheduler_test.cc @@ -93,8 +93,10 @@ class DeleteSchedulerTest : public testing::Test { // Tests in this file are for DeleteScheduler component and dont create any // DBs, so we need to set max_trash_db_ratio to 100% (instead of default // 25%) + std::shared_ptr + fs(std::make_shared(env_)); sst_file_mgr_.reset( - new SstFileManagerImpl(env_, nullptr, rate_bytes_per_sec_, + new SstFileManagerImpl(env_, fs, nullptr, rate_bytes_per_sec_, /* max_trash_db_ratio= */ 1.1, 128 * 1024)); delete_scheduler_ = sst_file_mgr_->delete_scheduler(); } diff --git a/file/file_util.cc b/file/file_util.cc index f1bf6596b..f34178705 100644 --- a/file/file_util.cc +++ b/file/file_util.cc @@ -17,28 +17,28 @@ namespace rocksdb { // Utility function to copy a file up to a specified length -Status CopyFile(Env* env, const std::string& source, +Status CopyFile(FileSystem* fs, const std::string& source, const std::string& destination, uint64_t size, bool use_fsync) { - const EnvOptions soptions; + const FileOptions soptions; Status s; std::unique_ptr src_reader; std::unique_ptr dest_writer; { - std::unique_ptr srcfile; - s = env->NewSequentialFile(source, &srcfile, soptions); + std::unique_ptr srcfile; + s = fs->NewSequentialFile(source, soptions, &srcfile, nullptr); if (!s.ok()) { return s; } - std::unique_ptr destfile; - s = env->NewWritableFile(destination, &destfile, soptions); + std::unique_ptr destfile; + s = fs->NewWritableFile(destination, soptions, &destfile, nullptr); if (!s.ok()) { return s; } if (size == 0) { // default argument means copy everything - s = env->GetFileSize(source, &size); + s = fs->GetFileSize(source, IOOptions(), &size, nullptr); if (!s.ok()) { return s; } @@ -69,14 +69,14 @@ Status CopyFile(Env* env, const std::string& source, } // Utility function to create a file with the provided contents -Status CreateFile(Env* env, const std::string& destination, +Status CreateFile(FileSystem* fs, const std::string& destination, const std::string& contents, bool use_fsync) { const EnvOptions soptions; Status s; std::unique_ptr dest_writer; - std::unique_ptr destfile; - s = env->NewWritableFile(destination, &destfile, soptions); + std::unique_ptr destfile; + s = fs->NewWritableFile(destination, soptions, &destfile, nullptr); if (!s.ok()) { return s; } diff --git a/file/file_util.h b/file/file_util.h index 75d6d7eb9..8547c2279 100644 --- a/file/file_util.h +++ b/file/file_util.h @@ -9,17 +9,18 @@ #include "file/filename.h" #include "options/db_options.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/status.h" #include "rocksdb/types.h" namespace rocksdb { // use_fsync maps to options.use_fsync, which determines the way that // the file is synced after copying. -extern Status CopyFile(Env* env, const std::string& source, +extern Status CopyFile(FileSystem* fs, const std::string& source, const std::string& destination, uint64_t size, bool use_fsync); -extern Status CreateFile(Env* env, const std::string& destination, +extern Status CreateFile(FileSystem* fs, const std::string& destination, const std::string& contents, bool use_fsync); extern Status DeleteDBFile(const ImmutableDBOptions* db_options, diff --git a/file/random_access_file_reader.cc b/file/random_access_file_reader.cc index 5b5a19ff8..6bd3a25d2 100644 --- a/file/random_access_file_reader.cc +++ b/file/random_access_file_reader.cc @@ -61,8 +61,8 @@ Status RandomAccessFileReader::Read(uint64_t offset, size_t n, Slice* result, } { IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, env_); - s = file_->Read(aligned_offset + buf.CurrentSize(), allowed, &tmp, - buf.Destination()); + s = file_->Read(aligned_offset + buf.CurrentSize(), allowed, + IOOptions(), &tmp, buf.Destination(), nullptr); } if (ShouldNotifyListeners()) { auto finish_ts = std::chrono::system_clock::now(); @@ -110,7 +110,8 @@ Status RandomAccessFileReader::Read(uint64_t offset, size_t n, Slice* result, #endif { IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, env_); - s = file_->Read(offset + pos, allowed, &tmp_result, scratch + pos); + s = file_->Read(offset + pos, allowed, IOOptions(), &tmp_result, + scratch + pos, nullptr); } #ifndef ROCKSDB_LITE if (ShouldNotifyListeners()) { @@ -145,7 +146,7 @@ Status RandomAccessFileReader::Read(uint64_t offset, size_t n, Slice* result, return s; } -Status RandomAccessFileReader::MultiRead(ReadRequest* read_reqs, +Status RandomAccessFileReader::MultiRead(FSReadRequest* read_reqs, size_t num_reqs) const { Status s; uint64_t elapsed = 0; @@ -165,7 +166,7 @@ Status RandomAccessFileReader::MultiRead(ReadRequest* read_reqs, #endif // ROCKSDB_LITE { IOSTATS_CPU_TIMER_GUARD(cpu_read_nanos, env_); - s = file_->MultiRead(read_reqs, num_reqs); + s = file_->MultiRead(read_reqs, num_reqs, IOOptions(), nullptr); } for (size_t i = 0; i < num_reqs; ++i) { #ifndef ROCKSDB_LITE diff --git a/file/random_access_file_reader.h b/file/random_access_file_reader.h index abbc71ff1..4f0e5307f 100644 --- a/file/random_access_file_reader.h +++ b/file/random_access_file_reader.h @@ -13,12 +13,12 @@ #include #include "port/port.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/listener.h" #include "rocksdb/rate_limiter.h" #include "util/aligned_buffer.h" namespace rocksdb { - class Statistics; class HistogramImpl; @@ -48,7 +48,7 @@ class RandomAccessFileReader { bool ShouldNotifyListeners() const { return !listeners_.empty(); } - std::unique_ptr file_; + std::unique_ptr file_; std::string file_name_; Env* env_; Statistics* stats_; @@ -59,7 +59,7 @@ class RandomAccessFileReader { public: explicit RandomAccessFileReader( - std::unique_ptr&& raf, std::string _file_name, + std::unique_ptr&& raf, std::string _file_name, Env* env = nullptr, Statistics* stats = nullptr, uint32_t hist_type = 0, HistogramImpl* file_read_hist = nullptr, RateLimiter* rate_limiter = nullptr, @@ -105,13 +105,13 @@ class RandomAccessFileReader { Status Read(uint64_t offset, size_t n, Slice* result, char* scratch, bool for_compaction = false) const; - Status MultiRead(ReadRequest* reqs, size_t num_reqs) const; + Status MultiRead(FSReadRequest* reqs, size_t num_reqs) const; Status Prefetch(uint64_t offset, size_t n) const { - return file_->Prefetch(offset, n); + return file_->Prefetch(offset, n, IOOptions(), nullptr); } - RandomAccessFile* file() { return file_.get(); } + FSRandomAccessFile* file() { return file_.get(); } std::string file_name() const { return file_name_; } diff --git a/file/read_write_util.cc b/file/read_write_util.cc index 892499b8c..de39008ba 100644 --- a/file/read_write_util.cc +++ b/file/read_write_util.cc @@ -13,15 +13,16 @@ #include "test_util/sync_point.h" namespace rocksdb { -Status NewWritableFile(Env* env, const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) { - Status s = env->NewWritableFile(fname, result, options); + +IOStatus NewWritableFile(FileSystem* fs, const std::string& fname, + std::unique_ptr* result, + const FileOptions& options) { + IOStatus s = fs->NewWritableFile(fname, options, result, nullptr); TEST_KILL_RANDOM("NewWritableFile:0", rocksdb_kill_odds * REDUCE_ODDS2); return s; } -bool ReadOneLine(std::istringstream* iss, SequentialFile* seq_file, +bool ReadOneLine(std::istringstream* iss, FSSequentialFile* seq_file, std::string* output, bool* has_data, Status* result) { const int kBufferSize = 8192; char buffer[kBufferSize + 1]; @@ -39,7 +40,8 @@ bool ReadOneLine(std::istringstream* iss, SequentialFile* seq_file, // if we're not sure whether we have a complete line, // further read from the file. if (*has_data) { - *result = seq_file->Read(kBufferSize, &input_slice, buffer); + *result = seq_file->Read(kBufferSize, IOOptions(), + &input_slice, buffer, nullptr); } if (input_slice.size() == 0) { // meaning we have read all the data diff --git a/file/read_write_util.h b/file/read_write_util.h index be975e854..3abb3388e 100644 --- a/file/read_write_util.h +++ b/file/read_write_util.h @@ -10,6 +10,7 @@ #pragma once #include #include "rocksdb/env.h" +#include "rocksdb/file_system.h" namespace rocksdb { // Returns a WritableFile. @@ -18,12 +19,12 @@ namespace rocksdb { // fname : the file name. // result : output arg. A WritableFile based on `fname` returned. // options : the Env Options. -extern Status NewWritableFile(Env* env, const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options); +extern IOStatus NewWritableFile(FileSystem* fs, const std::string& fname, + std::unique_ptr* result, + const FileOptions& options); // Read a single line from a file. -bool ReadOneLine(std::istringstream* iss, SequentialFile* seq_file, +bool ReadOneLine(std::istringstream* iss, FSSequentialFile* seq_file, std::string* output, bool* has_data, Status* result); #ifndef NDEBUG diff --git a/file/sequence_file_reader.cc b/file/sequence_file_reader.cc index be766a68b..4f3fa45b8 100644 --- a/file/sequence_file_reader.cc +++ b/file/sequence_file_reader.cc @@ -36,7 +36,8 @@ Status SequentialFileReader::Read(size_t n, Slice* result, char* scratch) { buf.Alignment(alignment); buf.AllocateNewBuffer(size); Slice tmp; - s = file_->PositionedRead(aligned_offset, size, &tmp, buf.BufferStart()); + s = file_->PositionedRead(aligned_offset, size, IOOptions(), &tmp, + buf.BufferStart(), nullptr); if (s.ok() && offset_advance < tmp.size()) { buf.Size(tmp.size()); r = buf.Read(scratch, offset_advance, @@ -45,7 +46,7 @@ Status SequentialFileReader::Read(size_t n, Slice* result, char* scratch) { *result = Slice(scratch, r); #endif // !ROCKSDB_LITE } else { - s = file_->Read(n, result, scratch); + s = file_->Read(n, IOOptions(), result, scratch, nullptr); } IOSTATS_ADD(bytes_read, result->size()); return s; @@ -66,9 +67,9 @@ namespace { // of being able to prefetch up to readahead_size bytes and then serve them // from memory, avoiding the entire round-trip if, for example, the data for the // file is actually remote. -class ReadaheadSequentialFile : public SequentialFile { +class ReadaheadSequentialFile : public FSSequentialFile { public: - ReadaheadSequentialFile(std::unique_ptr&& file, + ReadaheadSequentialFile(std::unique_ptr&& file, size_t readahead_size) : file_(std::move(file)), alignment_(file_->GetRequiredBufferAlignment()), @@ -84,7 +85,8 @@ class ReadaheadSequentialFile : public SequentialFile { ReadaheadSequentialFile& operator=(const ReadaheadSequentialFile&) = delete; - Status Read(size_t n, Slice* result, char* scratch) override { + IOStatus Read(size_t n, const IOOptions& opts, Slice* result, char* scratch, + IODebugContext* dbg) override { std::unique_lock lk(lock_); size_t cached_len = 0; @@ -96,14 +98,14 @@ class ReadaheadSequentialFile : public SequentialFile { (cached_len == n || buffer_.CurrentSize() < readahead_size_)) { // We read exactly what we needed, or we hit end of file - return. *result = Slice(scratch, cached_len); - return Status::OK(); + return IOStatus::OK(); } n -= cached_len; - Status s; + IOStatus s; // Read-ahead only make sense if we have some slack left after reading if (n + alignment_ >= readahead_size_) { - s = file_->Read(n, result, scratch + cached_len); + s = file_->Read(n, opts, result, scratch + cached_len, dbg); if (s.ok()) { read_offset_ += result->size(); *result = Slice(scratch, cached_len + result->size()); @@ -112,7 +114,7 @@ class ReadaheadSequentialFile : public SequentialFile { return s; } - s = ReadIntoBuffer(readahead_size_); + s = ReadIntoBuffer(readahead_size_, opts, dbg); if (s.ok()) { // The data we need is now in cache, so we can safely read it size_t remaining_len; @@ -122,9 +124,9 @@ class ReadaheadSequentialFile : public SequentialFile { return s; } - Status Skip(uint64_t n) override { + IOStatus Skip(uint64_t n) override { std::unique_lock lk(lock_); - Status s = Status::OK(); + IOStatus s = IOStatus::OK(); // First check if we need to skip already cached data if (buffer_.CurrentSize() > 0) { // Do we need to skip beyond cached data? @@ -149,12 +151,13 @@ class ReadaheadSequentialFile : public SequentialFile { return s; } - Status PositionedRead(uint64_t offset, size_t n, Slice* result, - char* scratch) override { - return file_->PositionedRead(offset, n, result, scratch); + IOStatus PositionedRead(uint64_t offset, size_t n, const IOOptions& opts, + Slice* result, char* scratch, + IODebugContext* dbg) override { + return file_->PositionedRead(offset, n, opts, result, scratch, dbg); } - Status InvalidateCache(size_t offset, size_t length) override { + IOStatus InvalidateCache(size_t offset, size_t length) override { std::unique_lock lk(lock_); buffer_.Clear(); return file_->InvalidateCache(offset, length); @@ -184,13 +187,14 @@ class ReadaheadSequentialFile : public SequentialFile { // Reads into buffer_ the next n bytes from file_. // Can actually read less if EOF was reached. // Returns the status of the read operastion on the file. - Status ReadIntoBuffer(size_t n) { + IOStatus ReadIntoBuffer(size_t n, const IOOptions& opts, + IODebugContext* dbg) { if (n > buffer_.Capacity()) { n = buffer_.Capacity(); } assert(IsFileSectorAligned(n, alignment_)); Slice result; - Status s = file_->Read(n, &result, buffer_.BufferStart()); + IOStatus s = file_->Read(n, opts, &result, buffer_.BufferStart(), dbg); if (s.ok()) { buffer_offset_ = read_offset_; buffer_.Size(result.size()); @@ -199,7 +203,7 @@ class ReadaheadSequentialFile : public SequentialFile { return s; } - const std::unique_ptr file_; + const std::unique_ptr file_; const size_t alignment_; const size_t readahead_size_; @@ -218,15 +222,15 @@ class ReadaheadSequentialFile : public SequentialFile { }; } // namespace -std::unique_ptr +std::unique_ptr SequentialFileReader::NewReadaheadSequentialFile( - std::unique_ptr&& file, size_t readahead_size) { + std::unique_ptr&& file, size_t readahead_size) { if (file->GetRequiredBufferAlignment() >= readahead_size) { // Short-circuit and return the original file if readahead_size is // too small and hence doesn't make sense to be used for prefetching. return std::move(file); } - std::unique_ptr result( + std::unique_ptr result( new ReadaheadSequentialFile(std::move(file), readahead_size)); return result; } diff --git a/file/sequence_file_reader.h b/file/sequence_file_reader.h index 6a6350e1d..5fb3f7b0c 100644 --- a/file/sequence_file_reader.h +++ b/file/sequence_file_reader.h @@ -12,6 +12,7 @@ #include #include "port/port.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" namespace rocksdb { @@ -20,16 +21,16 @@ namespace rocksdb { // cache disabled) reads appropriately, and also updates the IO stats. class SequentialFileReader { private: - std::unique_ptr file_; + std::unique_ptr file_; std::string file_name_; std::atomic offset_{0}; // read offset public: - explicit SequentialFileReader(std::unique_ptr&& _file, + explicit SequentialFileReader(std::unique_ptr&& _file, const std::string& _file_name) : file_(std::move(_file)), file_name_(_file_name) {} - explicit SequentialFileReader(std::unique_ptr&& _file, + explicit SequentialFileReader(std::unique_ptr&& _file, const std::string& _file_name, size_t _readahead_size) : file_(NewReadaheadSequentialFile(std::move(_file), _readahead_size)), @@ -51,7 +52,7 @@ class SequentialFileReader { Status Skip(uint64_t n); - SequentialFile* file() { return file_.get(); } + FSSequentialFile* file() { return file_.get(); } std::string file_name() { return file_name_; } @@ -60,7 +61,7 @@ class SequentialFileReader { private: // NewReadaheadSequentialFile provides a wrapper over SequentialFile to // always prefetch additional data with every read. - static std::unique_ptr NewReadaheadSequentialFile( - std::unique_ptr&& file, size_t readahead_size); + static std::unique_ptr NewReadaheadSequentialFile( + std::unique_ptr&& file, size_t readahead_size); }; } // namespace rocksdb diff --git a/file/sst_file_manager_impl.cc b/file/sst_file_manager_impl.cc index 08ea87325..02005846e 100644 --- a/file/sst_file_manager_impl.cc +++ b/file/sst_file_manager_impl.cc @@ -9,6 +9,7 @@ #include #include "db/db_impl/db_impl.h" +#include "env/composite_env_wrapper.h" #include "port/port.h" #include "rocksdb/env.h" #include "rocksdb/sst_file_manager.h" @@ -18,26 +19,27 @@ namespace rocksdb { #ifndef ROCKSDB_LITE -SstFileManagerImpl::SstFileManagerImpl(Env* env, std::shared_ptr logger, +SstFileManagerImpl::SstFileManagerImpl(Env* env, std::shared_ptr fs, + std::shared_ptr logger, int64_t rate_bytes_per_sec, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk) : env_(env), + fs_(fs), logger_(logger), total_files_size_(0), in_progress_files_size_(0), compaction_buffer_size_(0), cur_compactions_reserved_size_(0), max_allowed_space_(0), - delete_scheduler_(env, rate_bytes_per_sec, logger.get(), this, + delete_scheduler_(env, fs_.get(), rate_bytes_per_sec, logger.get(), this, max_trash_db_ratio, bytes_max_delete_chunk), cv_(&mu_), closing_(false), bg_thread_(nullptr), reserved_disk_buffer_(0), free_space_trigger_(0), - cur_instance_(nullptr) { -} + cur_instance_(nullptr) {} SstFileManagerImpl::~SstFileManagerImpl() { Close(); @@ -60,7 +62,7 @@ void SstFileManagerImpl::Close() { Status SstFileManagerImpl::OnAddFile(const std::string& file_path, bool compaction) { uint64_t file_size; - Status s = env_->GetFileSize(file_path, &file_size); + Status s = fs_->GetFileSize(file_path, IOOptions(), &file_size, nullptr); if (s.ok()) { MutexLock l(&mu_); OnAddFileImpl(file_path, file_size, compaction); @@ -178,7 +180,7 @@ bool SstFileManagerImpl::EnoughRoomForCompaction( TableFileName(cfd->ioptions()->cf_paths, inputs[0][0]->fd.GetNumber(), inputs[0][0]->fd.GetPathId()); uint64_t free_space = 0; - env_->GetFreeSpace(fn, &free_space); + fs_->GetFreeSpace(fn, IOOptions(), &free_space, nullptr); // needed_headroom is based on current size reserved by compactions, // minus any files created by running compactions as they would count // against the reserved size. If user didn't specify any compaction @@ -261,7 +263,7 @@ void SstFileManagerImpl::ClearError() { } uint64_t free_space = 0; - Status s = env_->GetFreeSpace(path_, &free_space); + Status s = fs_->GetFreeSpace(path_, IOOptions(), &free_space, nullptr); free_space = max_allowed_space_ > 0 ? std::min(max_allowed_space_, free_space) : free_space; @@ -471,8 +473,28 @@ SstFileManager* NewSstFileManager(Env* env, std::shared_ptr info_log, bool delete_existing_trash, Status* status, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk) { + std::shared_ptr fs; + + if (env == Env::Default()) { + fs = FileSystem::Default(); + } else { + fs.reset(new LegacyFileSystemWrapper(env)); + } + + return NewSstFileManager(env, fs, info_log, trash_dir, rate_bytes_per_sec, + delete_existing_trash, status, max_trash_db_ratio, + bytes_max_delete_chunk); +} + +SstFileManager* NewSstFileManager(Env* env, std::shared_ptr fs, + std::shared_ptr info_log, + const std::string& trash_dir, + int64_t rate_bytes_per_sec, + bool delete_existing_trash, Status* status, + double max_trash_db_ratio, + uint64_t bytes_max_delete_chunk) { SstFileManagerImpl* res = - new SstFileManagerImpl(env, info_log, rate_bytes_per_sec, + new SstFileManagerImpl(env, fs, info_log, rate_bytes_per_sec, max_trash_db_ratio, bytes_max_delete_chunk); // trash_dir is deprecated and not needed anymore, but if user passed it @@ -480,7 +502,7 @@ SstFileManager* NewSstFileManager(Env* env, std::shared_ptr info_log, Status s; if (delete_existing_trash && trash_dir != "") { std::vector files_in_trash; - s = env->GetChildren(trash_dir, &files_in_trash); + s = fs->GetChildren(trash_dir, IOOptions(), &files_in_trash, nullptr); if (s.ok()) { for (const std::string& trash_file : files_in_trash) { if (trash_file == "." || trash_file == "..") { diff --git a/file/sst_file_manager_impl.h b/file/sst_file_manager_impl.h index 893042278..396dfb862 100644 --- a/file/sst_file_manager_impl.h +++ b/file/sst_file_manager_impl.h @@ -14,6 +14,7 @@ #include "db/compaction/compaction.h" #include "db/error_handler.h" #include "file/delete_scheduler.h" +#include "rocksdb/file_system.h" #include "rocksdb/sst_file_manager.h" namespace rocksdb { @@ -26,7 +27,8 @@ class Logger; // All SstFileManager public functions are thread-safe. class SstFileManagerImpl : public SstFileManager { public: - explicit SstFileManagerImpl(Env* env, std::shared_ptr logger, + explicit SstFileManagerImpl(Env* env, std::shared_ptr fs, + std::shared_ptr logger, int64_t rate_bytes_per_sec, double max_trash_db_ratio, uint64_t bytes_max_delete_chunk); @@ -141,6 +143,7 @@ class SstFileManagerImpl : public SstFileManager { } Env* env_; + std::shared_ptr fs_; std::shared_ptr logger_; // Mutex to protect tracked_files_, total_files_size_ port::Mutex mu_; diff --git a/file/writable_file_writer.cc b/file/writable_file_writer.cc index 277e55500..43f60c833 100644 --- a/file/writable_file_writer.cc +++ b/file/writable_file_writer.cc @@ -32,7 +32,8 @@ Status WritableFileWriter::Append(const Slice& data) { { IOSTATS_TIMER_GUARD(prepare_write_nanos); TEST_SYNC_POINT("WritableFileWriter::Append:BeforePrepareWrite"); - writable_file_->PrepareWrite(static_cast(GetFileSize()), left); + writable_file_->PrepareWrite(static_cast(GetFileSize()), left, + IOOptions(), nullptr); } // See whether we need to enlarge the buffer to avoid the flush @@ -134,9 +135,9 @@ Status WritableFileWriter::Close() { // In direct I/O mode we write whole pages so // we need to let the file know where data ends. if (use_direct_io()) { - interim = writable_file_->Truncate(filesize_); + interim = writable_file_->Truncate(filesize_, IOOptions(), nullptr); if (interim.ok()) { - interim = writable_file_->Fsync(); + interim = writable_file_->Fsync(IOOptions(), nullptr); } if (!interim.ok() && s.ok()) { s = interim; @@ -144,7 +145,7 @@ Status WritableFileWriter::Close() { } TEST_KILL_RANDOM("WritableFileWriter::Close:0", rocksdb_kill_odds); - interim = writable_file_->Close(); + interim = writable_file_->Close(IOOptions(), nullptr); if (!interim.ok() && s.ok()) { s = interim; } @@ -177,7 +178,7 @@ Status WritableFileWriter::Flush() { } } - s = writable_file_->Flush(); + s = writable_file_->Flush(IOOptions(), nullptr); if (!s.ok()) { return s; @@ -249,9 +250,9 @@ Status WritableFileWriter::SyncInternal(bool use_fsync) { auto prev_perf_level = GetPerfLevel(); IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, env_); if (use_fsync) { - s = writable_file_->Fsync(); + s = writable_file_->Fsync(IOOptions(), nullptr); } else { - s = writable_file_->Sync(); + s = writable_file_->Sync(IOOptions(), nullptr); } SetPerfLevel(prev_perf_level); return s; @@ -260,7 +261,7 @@ Status WritableFileWriter::SyncInternal(bool use_fsync) { Status WritableFileWriter::RangeSync(uint64_t offset, uint64_t nbytes) { IOSTATS_TIMER_GUARD(range_sync_nanos); TEST_SYNC_POINT("WritableFileWriter::RangeSync:0"); - return writable_file_->RangeSync(offset, nbytes); + return writable_file_->RangeSync(offset, nbytes, IOOptions(), nullptr); } // This method writes to disk the specified data and makes use of the rate @@ -287,7 +288,7 @@ Status WritableFileWriter::WriteBuffered(const char* data, size_t size) { #ifndef ROCKSDB_LITE FileOperationInfo::TimePoint start_ts; - uint64_t old_size = writable_file_->GetFileSize(); + uint64_t old_size = writable_file_->GetFileSize(IOOptions(), nullptr); if (ShouldNotifyListeners()) { start_ts = std::chrono::system_clock::now(); old_size = next_write_offset_; @@ -296,7 +297,7 @@ Status WritableFileWriter::WriteBuffered(const char* data, size_t size) { { auto prev_perf_level = GetPerfLevel(); IOSTATS_CPU_TIMER_GUARD(cpu_write_nanos, env_); - s = writable_file_->Append(Slice(src, allowed)); + s = writable_file_->Append(Slice(src, allowed), IOOptions(), nullptr); SetPerfLevel(prev_perf_level); } #ifndef ROCKSDB_LITE @@ -370,7 +371,8 @@ Status WritableFileWriter::WriteDirect() { start_ts = std::chrono::system_clock::now(); } // direct writes must be positional - s = writable_file_->PositionedAppend(Slice(src, size), write_offset); + s = writable_file_->PositionedAppend(Slice(src, size), write_offset, + IOOptions(), nullptr); if (ShouldNotifyListeners()) { auto finish_ts = std::chrono::system_clock::now(); NotifyOnFileWriteFinish(write_offset, size, start_ts, finish_ts, s); diff --git a/file/writable_file_writer.h b/file/writable_file_writer.h index 09d612233..243ccd9bc 100644 --- a/file/writable_file_writer.h +++ b/file/writable_file_writer.h @@ -12,13 +12,13 @@ #include #include "port/port.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/listener.h" #include "rocksdb/rate_limiter.h" #include "test_util/sync_point.h" #include "util/aligned_buffer.h" namespace rocksdb { - class Statistics; // WritableFileWriter is a wrapper on top of Env::WritableFile. It provides @@ -48,7 +48,7 @@ class WritableFileWriter { bool ShouldNotifyListeners() const { return !listeners_.empty(); } - std::unique_ptr writable_file_; + std::unique_ptr writable_file_; std::string file_name_; Env* env_; AlignedBuffer buf_; @@ -71,8 +71,8 @@ class WritableFileWriter { public: WritableFileWriter( - std::unique_ptr&& file, const std::string& _file_name, - const EnvOptions& options, Env* env = nullptr, + std::unique_ptr&& file, const std::string& _file_name, + const FileOptions& options, Env* env = nullptr, Statistics* stats = nullptr, const std::vector>& listeners = {}) : writable_file_(std::move(file)), @@ -135,7 +135,7 @@ class WritableFileWriter { return writable_file_->InvalidateCache(offset, length); } - WritableFile* writable_file() const { return writable_file_.get(); } + FSWritableFile* writable_file() const { return writable_file_.get(); } bool use_direct_io() { return writable_file_->use_direct_io(); } diff --git a/include/rocksdb/db.h b/include/rocksdb/db.h index bc93aeda1..e4bbe7871 100644 --- a/include/rocksdb/db.h +++ b/include/rocksdb/db.h @@ -57,6 +57,7 @@ class TraceWriter; #ifdef ROCKSDB_LITE class CompactionJobInfo; #endif +class FileSystem; extern const std::string kDefaultColumnFamilyName; extern const std::string kPersistentStatsColumnFamilyName; @@ -1067,6 +1068,8 @@ class DB { // Get Env object from the DB virtual Env* GetEnv() const = 0; + virtual FileSystem* GetFileSystem() const; + // Get DB Options that we use. During the process of opening the // column family, the options provided when calling DB::Open() or // DB::CreateColumnFamily() will have been "sanitized" and transformed diff --git a/include/rocksdb/file_system.h b/include/rocksdb/file_system.h new file mode 100644 index 000000000..30e29742e --- /dev/null +++ b/include/rocksdb/file_system.h @@ -0,0 +1,1356 @@ +// Copyright (c) 2019-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// A FileSystem is an interface used by the rocksdb implementation to access +// storage functionality like the filesystem etc. Callers +// may wish to provide a custom FileSystem object when opening a database to +// get fine gain control; e.g., to rate limit file system operations. +// +// All FileSystem implementations are safe for concurrent access from +// multiple threads without any external synchronization. +// +// WARNING: Since this is a new interface, it is expected that there will be +// some changes as storage systems are ported over. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "rocksdb/env.h" +#include "rocksdb/io_status.h" +#include "rocksdb/options.h" +#include "rocksdb/thread_status.h" + +namespace rocksdb { + +class FileLock; +class FSDirectory; +class FSRandomAccessFile; +class FSRandomRWFile; +class FSSequentialFile; +class FSWritableFile; +class Logger; +class Slice; +struct ImmutableDBOptions; +struct MutableDBOptions; +class RateLimiter; + +using AccessPattern = RandomAccessFile::AccessPattern; +using FileAttributes = Env::FileAttributes; + +// Priority of an IO request. This is a hint and does not guarantee any +// particular QoS. +// IO_LOW - Typically background reads/writes such as compaction/flush +// IO_HIGH - Typically user reads/synchronous WAL writes +enum class IOPriority : uint8_t { + kIOLow, + kIOHigh, + kIOTotal, +}; + +// Type of the data begin read/written. It can be passed down as a flag +// for the FileSystem implementation to optionally handle different types in +// different ways +enum class IOType : uint8_t { + kData, + kFilter, + kIndex, + kMetadata, + kWAL, + kManifest, + kLog, + kUnknown, + kInvalid, +}; + +// Per-request options that can be passed down to the FileSystem +// implementation. These are hints and are not necessarily guaranteed to be +// honored. More hints can be added here in the future to indicate things like +// storage media (HDD/SSD) to be used, replication level etc. +struct IOOptions { + // Timeout for the operation in milliseconds + std::chrono::milliseconds timeout; + + // Priority - high or low + IOPriority prio; + + // Type of data being read/written + IOType type; +}; + +// File scope options that control how a file is opened/created and accessed +// while its open. We may add more options here in the future such as +// redundancy level, media to use etc. +struct FileOptions : EnvOptions { + // Embedded IOOptions to control the parameters for any IOs that need + // to be issued for the file open/creation + IOOptions io_options; + + FileOptions() : EnvOptions() {} + + FileOptions(const DBOptions& opts) + : EnvOptions(opts) {} + + FileOptions(const EnvOptions& opts) + : EnvOptions(opts) {} +}; + +// A structure to pass back some debugging information from the FileSystem +// implementation to RocksDB in case of an IO error +struct IODebugContext { + // file_path to be filled in by RocksDB in case of an error + std::string file_path; + + // A map of counter names to values - set by the FileSystem implementation + std::map counters; + + // To be set by the FileSystem implementation + std::string msg; + + IODebugContext() {} + + void AddCounter(std::string& name, uint64_t value) { + counters.emplace(name, value); + } + + std::string ToString() { + std::ostringstream ss; + ss << file_path << ", "; + for (auto counter : counters) { + ss << counter.first << " = " << counter.second << ","; + } + ss << msg; + return ss.str(); + } +}; + +// The FileSystem, FSSequentialFile, FSRandomAccessFile, FSWritableFile, +// FSRandomRWFileclass, and FSDIrectory classes define the interface between +// RocksDB and storage systems, such as Posix filesystems, +// remote filesystems etc. +// The interface allows for fine grained control of individual IO operations, +// such as setting a timeout, prioritization, hints on data placement, +// different handling based on type of IO etc. +// This is accomplished by passing an instance of IOOptions to every +// API call that can potentially perform IO. Additionally, each such API is +// passed a pointer to a IODebugContext structure that can be used by the +// storage system to include troubleshooting information. The return values +// of the APIs is of type IOStatus, which can indicate an error code/sub-code, +// as well as metadata about the error such as its scope and whether its +// retryable. +class FileSystem { + public: + FileSystem(); + + // No copying allowed + FileSystem(const FileSystem&) = delete; + + virtual ~FileSystem(); + + virtual const char* Name() const = 0; + + static const char* Type() { return "FileSystem"; } + + // Loads the FileSystem specified by the input value into the result + static Status Load(const std::string& value, + std::shared_ptr* result); + + // Return a default fie_system suitable for the current operating + // system. Sophisticated users may wish to provide their own Env + // implementation instead of relying on this default file_system + // + // The result of Default() belongs to rocksdb and must never be deleted. + static std::shared_ptr Default(); + + // Create a brand new sequentially-readable file with the specified name. + // On success, stores a pointer to the new file in *result and returns OK. + // On failure stores nullptr in *result and returns non-OK. If the file does + // not exist, returns a non-OK status. + // + // The returned file will only be accessed by one thread at a time. + virtual IOStatus NewSequentialFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) = 0; + + // Create a brand new random access read-only file with the + // specified name. On success, stores a pointer to the new file in + // *result and returns OK. On failure stores nullptr in *result and + // returns non-OK. If the file does not exist, returns a non-OK + // status. + // + // The returned file may be concurrently accessed by multiple threads. + virtual IOStatus NewRandomAccessFile( + const std::string& fname, const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) = 0; + // These values match Linux definition + // https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/include/uapi/linux/fcntl.h#n56 + enum WriteLifeTimeHint { + kWLTHNotSet = 0, // No hint information set + kWLTHNone, // No hints about write life time + kWLTHShort, // Data written has a short life time + kWLTHMedium, // Data written has a medium life time + kWLTHLong, // Data written has a long life time + kWLTHExtreme, // Data written has an extremely long life time + }; + + // Create an object that writes to a new file with the specified + // name. Deletes any existing file with the same name and creates a + // new file. On success, stores a pointer to the new file in + // *result and returns OK. On failure stores nullptr in *result and + // returns non-OK. + // + // The returned file will only be accessed by one thread at a time. + virtual IOStatus NewWritableFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) = 0; + + // Create an object that writes to a new file with the specified + // name. Deletes any existing file with the same name and creates a + // new file. On success, stores a pointer to the new file in + // *result and returns OK. On failure stores nullptr in *result and + // returns non-OK. + // + // The returned file will only be accessed by one thread at a time. + virtual IOStatus ReopenWritableFile( + const std::string& /*fname*/, const FileOptions& /*options*/, + std::unique_ptr* /*result*/, IODebugContext* /*dbg*/) { + return IOStatus::NotSupported(); + } + + // Reuse an existing file by renaming it and opening it as writable. + virtual IOStatus ReuseWritableFile(const std::string& fname, + const std::string& old_fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) = 0; + + // Open `fname` for random read and write, if file doesn't exist the file + // will be created. On success, stores a pointer to the new file in + // *result and returns OK. On failure returns non-OK. + // + // The returned file will only be accessed by one thread at a time. + virtual IOStatus NewRandomRWFile(const std::string& /*fname*/, + const FileOptions& /*options*/, + std::unique_ptr* /*result*/, + IODebugContext* /*dbg*/) { + return IOStatus::NotSupported( + "RandomRWFile is not implemented in this FileSystem"); + } + + // Opens `fname` as a memory-mapped file for read and write (in-place updates + // only, i.e., no appends). On success, stores a raw buffer covering the whole + // file in `*result`. The file must exist prior to this call. + virtual IOStatus NewMemoryMappedFileBuffer( + const std::string& /*fname*/, + std::unique_ptr* /*result*/) { + return IOStatus::NotSupported( + "MemoryMappedFileBuffer is not implemented in this FileSystem"); + } + + // Create an object that represents a directory. Will fail if directory + // doesn't exist. If the directory exists, it will open the directory + // and create a new Directory object. + // + // On success, stores a pointer to the new Directory in + // *result and returns OK. On failure stores nullptr in *result and + // returns non-OK. + virtual IOStatus NewDirectory(const std::string& name, + const IOOptions& io_opts, + std::unique_ptr* result, + IODebugContext* dbg) = 0; + + // Returns OK if the named file exists. + // NotFound if the named file does not exist, + // the calling process does not have permission to determine + // whether this file exists, or if the path is invalid. + // IOError if an IO Error was encountered + virtual IOStatus FileExists(const std::string& fname, + const IOOptions& options, + IODebugContext* dbg) = 0; + + // Store in *result the names of the children of the specified directory. + // The names are relative to "dir". + // Original contents of *results are dropped. + // Returns OK if "dir" exists and "*result" contains its children. + // NotFound if "dir" does not exist, the calling process does not have + // permission to access "dir", or if "dir" is invalid. + // IOError if an IO Error was encountered + virtual IOStatus GetChildren(const std::string& dir, const IOOptions& options, + std::vector* result, + IODebugContext* dbg) = 0; + + // Store in *result the attributes of the children of the specified directory. + // In case the implementation lists the directory prior to iterating the files + // and files are concurrently deleted, the deleted files will be omitted from + // result. + // The name attributes are relative to "dir". + // Original contents of *results are dropped. + // Returns OK if "dir" exists and "*result" contains its children. + // NotFound if "dir" does not exist, the calling process does not have + // permission to access "dir", or if "dir" is invalid. + // IOError if an IO Error was encountered + virtual IOStatus GetChildrenFileAttributes( + const std::string& dir, const IOOptions& options, + std::vector* result, IODebugContext* dbg) { + assert(result != nullptr); + std::vector child_fnames; + IOStatus s = GetChildren(dir, options, &child_fnames, dbg); + if (!s.ok()) { + return s; + } + result->resize(child_fnames.size()); + size_t result_size = 0; + for (size_t i = 0; i < child_fnames.size(); ++i) { + const std::string path = dir + "/" + child_fnames[i]; + if (!(s = GetFileSize(path, options, &(*result)[result_size].size_bytes, + dbg)) + .ok()) { + if (FileExists(path, options, dbg).IsNotFound()) { + // The file may have been deleted since we listed the directory + continue; + } + return s; + } + (*result)[result_size].name = std::move(child_fnames[i]); + result_size++; + } + result->resize(result_size); + return IOStatus::OK(); + } + + // Delete the named file. + virtual IOStatus DeleteFile(const std::string& fname, + const IOOptions& options, + IODebugContext* dbg) = 0; + + // Truncate the named file to the specified size. + virtual IOStatus Truncate(const std::string& /*fname*/, size_t /*size*/, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + return IOStatus::NotSupported("Truncate is not supported for this FileSystem"); + } + + // Create the specified directory. Returns error if directory exists. + virtual IOStatus CreateDir(const std::string& dirname, + const IOOptions& options, IODebugContext* dbg) = 0; + + // Creates directory if missing. Return Ok if it exists, or successful in + // Creating. + virtual IOStatus CreateDirIfMissing(const std::string& dirname, + const IOOptions& options, + IODebugContext* dbg) = 0; + + // Delete the specified directory. + virtual IOStatus DeleteDir(const std::string& dirname, + const IOOptions& options, IODebugContext* dbg) = 0; + + // Store the size of fname in *file_size. + virtual IOStatus GetFileSize(const std::string& fname, + const IOOptions& options, uint64_t* file_size, + IODebugContext* dbg) = 0; + + // Store the last modification time of fname in *file_mtime. + virtual IOStatus GetFileModificationTime(const std::string& fname, + const IOOptions& options, + uint64_t* file_mtime, + IODebugContext* dbg) = 0; + // Rename file src to target. + virtual IOStatus RenameFile(const std::string& src, const std::string& target, + const IOOptions& options, + IODebugContext* dbg) = 0; + + // Hard Link file src to target. + virtual IOStatus LinkFile(const std::string& /*src*/, + const std::string& /*target*/, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + return IOStatus::NotSupported("LinkFile is not supported for this FileSystem"); + } + + virtual IOStatus NumFileLinks(const std::string& /*fname*/, + const IOOptions& /*options*/, + uint64_t* /*count*/, IODebugContext* /*dbg*/) { + return IOStatus::NotSupported( + "Getting number of file links is not supported for this FileSystem"); + } + + virtual IOStatus AreFilesSame(const std::string& /*first*/, + const std::string& /*second*/, + const IOOptions& /*options*/, bool* /*res*/, + IODebugContext* /*dbg*/) { + return IOStatus::NotSupported("AreFilesSame is not supported for this FileSystem"); + } + + // Lock the specified file. Used to prevent concurrent access to + // the same db by multiple processes. On failure, stores nullptr in + // *lock and returns non-OK. + // + // On success, stores a pointer to the object that represents the + // acquired lock in *lock and returns OK. The caller should call + // UnlockFile(*lock) to release the lock. If the process exits, + // the lock will be automatically released. + // + // If somebody else already holds the lock, finishes immediately + // with a failure. I.e., this call does not wait for existing locks + // to go away. + // + // May create the named file if it does not already exist. + virtual IOStatus LockFile(const std::string& fname, const IOOptions& options, + FileLock** lock, IODebugContext* dbg) = 0; + + // Release the lock acquired by a previous successful call to LockFile. + // REQUIRES: lock was returned by a successful LockFile() call + // REQUIRES: lock has not already been unlocked. + virtual IOStatus UnlockFile(FileLock* lock, const IOOptions& options, + IODebugContext* dbg) = 0; + + // *path is set to a temporary directory that can be used for testing. It may + // or many not have just been created. The directory may or may not differ + // between runs of the same process, but subsequent calls will return the + // same directory. + virtual IOStatus GetTestDirectory(const IOOptions& options, std::string* path, + IODebugContext* dbg) = 0; + + // Create and returns a default logger (an instance of EnvLogger) for storing + // informational messages. Derived classes can overide to provide custom + // logger. + virtual IOStatus NewLogger(const std::string& fname, const IOOptions& io_opts, + std::shared_ptr* result, + IODebugContext* dbg) = 0; + + // Get full directory name for this db. + virtual IOStatus GetAbsolutePath(const std::string& db_path, + const IOOptions& options, + std::string* output_path, + IODebugContext* dbg) = 0; + + // OptimizeForLogRead will create a new FileOptions object that is a copy of + // the FileOptions in the parameters, but is optimized for reading log files. + virtual FileOptions OptimizeForLogRead(const FileOptions& file_options) const; + + // OptimizeForManifestRead will create a new FileOptions object that is a copy + // of the FileOptions in the parameters, but is optimized for reading manifest + // files. + virtual FileOptions OptimizeForManifestRead( + const FileOptions& file_options) const; + + // OptimizeForLogWrite will create a new FileOptions object that is a copy of + // the FileOptions in the parameters, but is optimized for writing log files. + // Default implementation returns the copy of the same object. + virtual FileOptions OptimizeForLogWrite(const FileOptions& file_options, + const DBOptions& db_options) const; + + // OptimizeForManifestWrite will create a new FileOptions object that is a + // copy of the FileOptions in the parameters, but is optimized for writing + // manifest files. Default implementation returns the copy of the same + // object. + virtual FileOptions OptimizeForManifestWrite( + const FileOptions& file_options) const; + + // OptimizeForCompactionTableWrite will create a new FileOptions object that + // is a copy of the FileOptions in the parameters, but is optimized for + // writing table files. + virtual FileOptions OptimizeForCompactionTableWrite( + const FileOptions& file_options, + const ImmutableDBOptions& immutable_ops) const; + + // OptimizeForCompactionTableRead will create a new FileOptions object that + // is a copy of the FileOptions in the parameters, but is optimized for + // reading table files. + virtual FileOptions OptimizeForCompactionTableRead( + const FileOptions& file_options, + const ImmutableDBOptions& db_options) const; + +// This seems to clash with a macro on Windows, so #undef it here +#ifdef GetFreeSpace +#undef GetFreeSpace +#endif + + // Get the amount of free disk space + virtual IOStatus GetFreeSpace(const std::string& /*path*/, + const IOOptions& /*options*/, + uint64_t* /*diskfree*/, + IODebugContext* /*dbg*/) { + return IOStatus::NotSupported(); + } + + // If you're adding methods here, remember to add them to EnvWrapper too. + + private: + void operator=(const FileSystem&); +}; + +// A file abstraction for reading sequentially through a file +class FSSequentialFile { + public: + FSSequentialFile() {} + + virtual ~FSSequentialFile() {} + + // Read up to "n" bytes from the file. "scratch[0..n-1]" may be + // written by this routine. Sets "*result" to the data that was + // read (including if fewer than "n" bytes were successfully read). + // May set "*result" to point at data in "scratch[0..n-1]", so + // "scratch[0..n-1]" must be live when "*result" is used. + // If an error was encountered, returns a non-OK status. + // + // REQUIRES: External synchronization + virtual IOStatus Read(size_t n, const IOOptions& options, Slice* result, + char* scratch, IODebugContext* dbg) = 0; + + // Skip "n" bytes from the file. This is guaranteed to be no + // slower that reading the same data, but may be faster. + // + // If end of file is reached, skipping will stop at the end of the + // file, and Skip will return OK. + // + // REQUIRES: External synchronization + virtual IOStatus Skip(uint64_t n) = 0; + + // Indicates the upper layers if the current SequentialFile implementation + // uses direct IO. + virtual bool use_direct_io() const { return false; } + + // Use the returned alignment value to allocate + // aligned buffer for Direct I/O + virtual size_t GetRequiredBufferAlignment() const { return kDefaultPageSize; } + + // Remove any kind of caching of data from the offset to offset+length + // of this file. If the length is 0, then it refers to the end of file. + // If the system is not caching the file contents, then this is a noop. + virtual IOStatus InvalidateCache(size_t /*offset*/, size_t /*length*/) { + return IOStatus::NotSupported("InvalidateCache not supported."); + } + + // Positioned Read for direct I/O + // If Direct I/O enabled, offset, n, and scratch should be properly aligned + virtual IOStatus PositionedRead(uint64_t /*offset*/, size_t /*n*/, + const IOOptions& /*options*/, + Slice* /*result*/, char* /*scratch*/, + IODebugContext* /*dbg*/) { + return IOStatus::NotSupported(); + } + + // If you're adding methods here, remember to add them to + // SequentialFileWrapper too. +}; + +// A read IO request structure for use in MultiRead +struct FSReadRequest { + // File offset in bytes + uint64_t offset; + + // Length to read in bytes + size_t len; + + // A buffer that MultiRead() can optionally place data in. It can + // ignore this and allocate its own buffer + char* scratch; + + // Output parameter set by MultiRead() to point to the data buffer, and + // the number of valid bytes + Slice result; + + // Status of read + IOStatus status; +}; + +// A file abstraction for randomly reading the contents of a file. +class FSRandomAccessFile { + public: + FSRandomAccessFile() {} + + virtual ~FSRandomAccessFile() {} + + // Read up to "n" bytes from the file starting at "offset". + // "scratch[0..n-1]" may be written by this routine. Sets "*result" + // to the data that was read (including if fewer than "n" bytes were + // successfully read). May set "*result" to point at data in + // "scratch[0..n-1]", so "scratch[0..n-1]" must be live when + // "*result" is used. If an error was encountered, returns a non-OK + // status. + // + // Safe for concurrent use by multiple threads. + // If Direct I/O enabled, offset, n, and scratch should be aligned properly. + virtual IOStatus Read(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const = 0; + + // Readahead the file starting from offset by n bytes for caching. + virtual IOStatus Prefetch(uint64_t /*offset*/, size_t /*n*/, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + return IOStatus::OK(); + } + + // Read a bunch of blocks as described by reqs. The blocks can + // optionally be read in parallel. This is a synchronous call, i.e it + // should return after all reads have completed. The reads will be + // non-overlapping. If the function return Status is not ok, status of + // individual requests will be ignored and return status will be assumed + // for all read requests. The function return status is only meant for any + // any errors that occur before even processing specific read requests + virtual IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs, + const IOOptions& options, IODebugContext* dbg) { + assert(reqs != nullptr); + for (size_t i = 0; i < num_reqs; ++i) { + FSReadRequest& req = reqs[i]; + req.status = + Read(req.offset, req.len, options, &req.result, req.scratch, dbg); + } + return IOStatus::OK(); + } + + // Tries to get an unique ID for this file that will be the same each time + // the file is opened (and will stay the same while the file is open). + // Furthermore, it tries to make this ID at most "max_size" bytes. If such an + // ID can be created this function returns the length of the ID and places it + // in "id"; otherwise, this function returns 0, in which case "id" + // may not have been modified. + // + // This function guarantees, for IDs from a given environment, two unique ids + // cannot be made equal to each other by adding arbitrary bytes to one of + // them. That is, no unique ID is the prefix of another. + // + // This function guarantees that the returned ID will not be interpretable as + // a single varint. + // + // Note: these IDs are only valid for the duration of the process. + virtual size_t GetUniqueId(char* /*id*/, size_t /*max_size*/) const { + return 0; // Default implementation to prevent issues with backwards + // compatibility. + }; + + enum AccessPattern { kNormal, kRandom, kSequential, kWillNeed, kWontNeed }; + + virtual void Hint(AccessPattern /*pattern*/) {} + + // Indicates the upper layers if the current RandomAccessFile implementation + // uses direct IO. + virtual bool use_direct_io() const { return false; } + + // Use the returned alignment value to allocate + // aligned buffer for Direct I/O + virtual size_t GetRequiredBufferAlignment() const { return kDefaultPageSize; } + + // Remove any kind of caching of data from the offset to offset+length + // of this file. If the length is 0, then it refers to the end of file. + // If the system is not caching the file contents, then this is a noop. + virtual IOStatus InvalidateCache(size_t /*offset*/, size_t /*length*/) { + return IOStatus::NotSupported("InvalidateCache not supported."); + } + + // If you're adding methods here, remember to add them to + // RandomAccessFileWrapper too. +}; + +// A file abstraction for sequential writing. The implementation +// must provide buffering since callers may append small fragments +// at a time to the file. +class FSWritableFile { + public: + FSWritableFile() + : last_preallocated_block_(0), + preallocation_block_size_(0), + io_priority_(Env::IO_TOTAL), + write_hint_(Env::WLTH_NOT_SET), + strict_bytes_per_sync_(false) {} + + explicit FSWritableFile(const FileOptions& options) + : last_preallocated_block_(0), + preallocation_block_size_(0), + io_priority_(Env::IO_TOTAL), + write_hint_(Env::WLTH_NOT_SET), + strict_bytes_per_sync_(options.strict_bytes_per_sync) {} + + virtual ~FSWritableFile() {} + + // Append data to the end of the file + // Note: A WriteabelFile object must support either Append or + // PositionedAppend, so the users cannot mix the two. + virtual IOStatus Append(const Slice& data, const IOOptions& options, + IODebugContext* dbg) = 0; + + // PositionedAppend data to the specified offset. The new EOF after append + // must be larger than the previous EOF. This is to be used when writes are + // not backed by OS buffers and hence has to always start from the start of + // the sector. The implementation thus needs to also rewrite the last + // partial sector. + // Note: PositionAppend does not guarantee moving the file offset after the + // write. A WritableFile object must support either Append or + // PositionedAppend, so the users cannot mix the two. + // + // PositionedAppend() can only happen on the page/sector boundaries. For that + // reason, if the last write was an incomplete sector we still need to rewind + // back to the nearest sector/page and rewrite the portion of it with whatever + // we need to add. We need to keep where we stop writing. + // + // PositionedAppend() can only write whole sectors. For that reason we have to + // pad with zeros for the last write and trim the file when closing according + // to the position we keep in the previous step. + // + // PositionedAppend() requires aligned buffer to be passed in. The alignment + // required is queried via GetRequiredBufferAlignment() + virtual IOStatus PositionedAppend(const Slice& /* data */, + uint64_t /* offset */, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + return IOStatus::NotSupported(); + } + + // Truncate is necessary to trim the file to the correct size + // before closing. It is not always possible to keep track of the file + // size due to whole pages writes. The behavior is undefined if called + // with other writes to follow. + virtual IOStatus Truncate(uint64_t /*size*/, const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + return IOStatus::OK(); + } + virtual IOStatus Close(const IOOptions& options, IODebugContext* dbg) = 0; + virtual IOStatus Flush(const IOOptions& options, IODebugContext* dbg) = 0; + virtual IOStatus Sync(const IOOptions& options, + IODebugContext* dbg) = 0; // sync data + + /* + * Sync data and/or metadata as well. + * By default, sync only data. + * Override this method for environments where we need to sync + * metadata as well. + */ + virtual IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) { + return Sync(options, dbg); + } + + // true if Sync() and Fsync() are safe to call concurrently with Append() + // and Flush(). + virtual bool IsSyncThreadSafe() const { return false; } + + // Indicates the upper layers if the current WritableFile implementation + // uses direct IO. + virtual bool use_direct_io() const { return false; } + + // Use the returned alignment value to allocate + // aligned buffer for Direct I/O + virtual size_t GetRequiredBufferAlignment() const { return kDefaultPageSize; } + + virtual void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) { + write_hint_ = hint; + } + + virtual void SetIOPriority(Env::IOPriority pri) { io_priority_ = pri; } + + virtual Env::IOPriority GetIOPriority() { return io_priority_; } + + virtual Env::WriteLifeTimeHint GetWriteLifeTimeHint() { return write_hint_; } + /* + * Get the size of valid data in the file. + */ + virtual uint64_t GetFileSize(const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + return 0; + } + + /* + * Get and set the default pre-allocation block size for writes to + * this file. If non-zero, then Allocate will be used to extend the + * underlying storage of a file (generally via fallocate) if the Env + * instance supports it. + */ + virtual void SetPreallocationBlockSize(size_t size) { + preallocation_block_size_ = size; + } + + virtual void GetPreallocationStatus(size_t* block_size, + size_t* last_allocated_block) { + *last_allocated_block = last_preallocated_block_; + *block_size = preallocation_block_size_; + } + + // For documentation, refer to RandomAccessFile::GetUniqueId() + virtual size_t GetUniqueId(char* /*id*/, size_t /*max_size*/) const { + return 0; // Default implementation to prevent issues with backwards + } + + // Remove any kind of caching of data from the offset to offset+length + // of this file. If the length is 0, then it refers to the end of file. + // If the system is not caching the file contents, then this is a noop. + // This call has no effect on dirty pages in the cache. + virtual IOStatus InvalidateCache(size_t /*offset*/, size_t /*length*/) { + return IOStatus::NotSupported("InvalidateCache not supported."); + } + + // 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 IOStatus RangeSync(uint64_t /*offset*/, uint64_t /*nbytes*/, + const IOOptions& options, IODebugContext* dbg) { + if (strict_bytes_per_sync_) { + return Sync(options, dbg); + } + return IOStatus::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 + // fragmentation and/or less waste from over-zealous filesystem + // pre-allocation. + virtual void PrepareWrite(size_t offset, size_t len, const IOOptions& options, + IODebugContext* dbg) { + if (preallocation_block_size_ == 0) { + return; + } + // If this write would cross one or more preallocation blocks, + // determine what the last preallocation block necessary to + // cover this write would be and Allocate to that point. + const auto block_size = preallocation_block_size_; + size_t new_last_preallocated_block = + (offset + len + block_size - 1) / block_size; + if (new_last_preallocated_block > last_preallocated_block_) { + size_t num_spanned_blocks = + new_last_preallocated_block - last_preallocated_block_; + Allocate(block_size * last_preallocated_block_, + block_size * num_spanned_blocks, options, dbg); + last_preallocated_block_ = new_last_preallocated_block; + } + } + + // Pre-allocates space for a file. + virtual IOStatus Allocate(uint64_t /*offset*/, uint64_t /*len*/, + const IOOptions& /*options*/, + IODebugContext* /*dbg*/) { + return IOStatus::OK(); + } + + // If you're adding methods here, remember to add them to + // WritableFileWrapper too. + + protected: + size_t preallocation_block_size() { return preallocation_block_size_; } + + private: + size_t last_preallocated_block_; + size_t preallocation_block_size_; + // No copying allowed + FSWritableFile(const FSWritableFile&); + void operator=(const FSWritableFile&); + + protected: + Env::IOPriority io_priority_; + Env::WriteLifeTimeHint write_hint_; + const bool strict_bytes_per_sync_; +}; + +// A file abstraction for random reading and writing. +class FSRandomRWFile { + public: + FSRandomRWFile() {} + + virtual ~FSRandomRWFile() {} + + // Indicates if the class makes use of direct I/O + // If false you must pass aligned buffer to Write() + virtual bool use_direct_io() const { return false; } + + // Use the returned alignment value to allocate + // aligned buffer for Direct I/O + virtual size_t GetRequiredBufferAlignment() const { return kDefaultPageSize; } + + // Write bytes in `data` at offset `offset`, Returns Status::OK() on success. + // Pass aligned buffer when use_direct_io() returns true. + virtual IOStatus Write(uint64_t offset, const Slice& data, + const IOOptions& options, IODebugContext* dbg) = 0; + + // Read up to `n` bytes starting from offset `offset` and store them in + // result, provided `scratch` size should be at least `n`. + // Returns Status::OK() on success. + virtual IOStatus Read(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const = 0; + + virtual IOStatus Flush(const IOOptions& options, IODebugContext* dbg) = 0; + + virtual IOStatus Sync(const IOOptions& options, IODebugContext* dbg) = 0; + + virtual IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) { + return Sync(options, dbg); + } + + virtual IOStatus Close(const IOOptions& options, IODebugContext* dbg) = 0; + + // If you're adding methods here, remember to add them to + // RandomRWFileWrapper too. + + // No copying allowed + FSRandomRWFile(const RandomRWFile&) = delete; + FSRandomRWFile& operator=(const RandomRWFile&) = delete; +}; + +// MemoryMappedFileBuffer object represents a memory-mapped file's raw buffer. +// Subclasses should release the mapping upon destruction. +class FSMemoryMappedFileBuffer { + public: + FSMemoryMappedFileBuffer(void* _base, size_t _length) + : base_(_base), length_(_length) {} + + virtual ~FSMemoryMappedFileBuffer() = 0; + + // We do not want to unmap this twice. We can make this class + // movable if desired, however, since + FSMemoryMappedFileBuffer(const FSMemoryMappedFileBuffer&) = delete; + FSMemoryMappedFileBuffer& operator=(const FSMemoryMappedFileBuffer&) = delete; + + void* GetBase() const { return base_; } + size_t GetLen() const { return length_; } + + protected: + void* base_; + const size_t length_; +}; + +// Directory object represents collection of files and implements +// filesystem operations that can be executed on directories. +class FSDirectory { + public: + virtual ~FSDirectory() {} + // Fsync directory. Can be called concurrently from multiple threads. + virtual IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) = 0; + + virtual size_t GetUniqueId(char* /*id*/, size_t /*max_size*/) const { + return 0; + } + + // If you're adding methods here, remember to add them to + // DirectoryWrapper too. +}; + +// Below are helpers for wrapping most of the classes in this file. +// They forward all calls to another instance of the class. +// Useful when wrapping the default implementations. +// Typical usage is to inherit your wrapper from *Wrapper, e.g.: +// +// class MySequentialFileWrapper : public rocksdb::FSSequentialFileWrapper { +// public: +// MySequentialFileWrapper(rocksdb::FSSequentialFile* target): +// rocksdb::FSSequentialFileWrapper(target) {} +// Status Read(size_t n, FileSystem::IOOptions& options, Slice* result, +// char* scratch, FileSystem::IODebugContext* dbg) override { +// cout << "Doing a read of size " << n << "!" << endl; +// return rocksdb::FSSequentialFileWrapper::Read(n, options, result, +// scratch, dbg); +// } +// // All other methods are forwarded to target_ automatically. +// }; +// +// This is often more convenient than inheriting the class directly because +// (a) Don't have to override and forward all methods - the Wrapper will +// forward everything you're not explicitly overriding. +// (b) Don't need to update the wrapper when more methods are added to the +// rocksdb class. Unless you actually want to override the behavior. +// (And unless rocksdb people forgot to update the *Wrapper class.) + +// An implementation of Env that forwards all calls to another Env. +// May be useful to clients who wish to override just part of the +// functionality of another Env. +class FileSystemWrapper : public FileSystem { + public: + // Initialize an EnvWrapper that delegates all calls to *t + explicit FileSystemWrapper(FileSystem* t) : target_(t) {} + ~FileSystemWrapper() override {} + + // Return the target to which this Env forwards all calls + FileSystem* target() const { return target_; } + + // The following text is boilerplate that forwards all methods to target() + IOStatus NewSequentialFile(const std::string& f, + const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* dbg) override { + return target_->NewSequentialFile(f, file_opts, r, dbg); + } + IOStatus NewRandomAccessFile(const std::string& f, + const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* dbg) override { + return target_->NewRandomAccessFile(f, file_opts, r, dbg); + } + IOStatus NewWritableFile(const std::string& f, const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* dbg) override { + return target_->NewWritableFile(f, file_opts, r, dbg); + } + IOStatus ReopenWritableFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) override { + return target_->ReopenWritableFile(fname, file_opts, result, dbg); + } + IOStatus ReuseWritableFile(const std::string& fname, + const std::string& old_fname, + const FileOptions& file_opts, + std::unique_ptr* r, + IODebugContext* dbg) override { + return target_->ReuseWritableFile(fname, old_fname, file_opts, r, + dbg); + } + IOStatus NewRandomRWFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) override { + return target_->NewRandomRWFile(fname, file_opts, result, dbg); + } + IOStatus NewMemoryMappedFileBuffer( + const std::string& fname, + std::unique_ptr* result) override { + return target_->NewMemoryMappedFileBuffer(fname, result); + } + IOStatus NewDirectory(const std::string& name, const IOOptions& io_opts, + std::unique_ptr* result, + IODebugContext* dbg) override { + return target_->NewDirectory(name, io_opts, result, dbg); + } + IOStatus FileExists(const std::string& f, const IOOptions& io_opts, + IODebugContext* dbg) override { + return target_->FileExists(f, io_opts, dbg); + } + IOStatus GetChildren(const std::string& dir, const IOOptions& io_opts, + std::vector* r, + IODebugContext* dbg) override { + return target_->GetChildren(dir, io_opts, r, dbg); + } + IOStatus GetChildrenFileAttributes(const std::string& dir, + const IOOptions& options, + std::vector* result, + IODebugContext* dbg) override { + return target_->GetChildrenFileAttributes(dir, options, result, dbg); + } + IOStatus DeleteFile(const std::string& f, const IOOptions& options, + IODebugContext* dbg) override { + return target_->DeleteFile(f, options, dbg); + } + IOStatus Truncate(const std::string& fname, size_t size, + const IOOptions& options, IODebugContext* dbg) override { + return target_->Truncate(fname, size, options, dbg); + } + IOStatus CreateDir(const std::string& d, const IOOptions& options, + IODebugContext* dbg) override { + return target_->CreateDir(d, options, dbg); + } + IOStatus CreateDirIfMissing(const std::string& d, const IOOptions& options, + IODebugContext* dbg) override { + return target_->CreateDirIfMissing(d, options, dbg); + } + IOStatus DeleteDir(const std::string& d, const IOOptions& options, + IODebugContext* dbg) override { + return target_->DeleteDir(d, options, dbg); + } + IOStatus GetFileSize(const std::string& f, const IOOptions& options, + uint64_t* s, IODebugContext* dbg) override { + return target_->GetFileSize(f, options, s, dbg); + } + + IOStatus GetFileModificationTime(const std::string& fname, + const IOOptions& options, + uint64_t* file_mtime, + IODebugContext* dbg) override { + return target_->GetFileModificationTime(fname, options, file_mtime, dbg); + } + + IOStatus GetAbsolutePath(const std::string& db_path, const IOOptions& options, + std::string* output_path, + IODebugContext* dbg) override { + return target_->GetAbsolutePath(db_path, options, output_path, dbg); + } + + IOStatus RenameFile(const std::string& s, const std::string& t, + const IOOptions& options, IODebugContext* dbg) override { + return target_->RenameFile(s, t, options, dbg); + } + + IOStatus LinkFile(const std::string& s, const std::string& t, + const IOOptions& options, IODebugContext* dbg) override { + return target_->LinkFile(s, t, options, dbg); + } + + IOStatus NumFileLinks(const std::string& fname, const IOOptions& options, + uint64_t* count, IODebugContext* dbg) override { + return target_->NumFileLinks(fname, options, count, dbg); + } + + IOStatus AreFilesSame(const std::string& first, const std::string& second, + const IOOptions& options, bool* res, + IODebugContext* dbg) override { + return target_->AreFilesSame(first, second, options, res, dbg); + } + + IOStatus LockFile(const std::string& f, const IOOptions& options, + FileLock** l, IODebugContext* dbg) override { + return target_->LockFile(f, options, l, dbg); + } + + IOStatus UnlockFile(FileLock* l, const IOOptions& options, + IODebugContext* dbg) override { + return target_->UnlockFile(l, options, dbg); + } + + IOStatus GetTestDirectory(const IOOptions& options, std::string* path, + IODebugContext* dbg) override { + return target_->GetTestDirectory(options, path, dbg); + } + IOStatus NewLogger(const std::string& fname, const IOOptions& options, + std::shared_ptr* result, + IODebugContext* dbg) override { + return target_->NewLogger(fname, options, result, dbg); + } + + FileOptions OptimizeForLogRead( + const FileOptions& file_options) const override { + return target_->OptimizeForLogRead(file_options); + } + FileOptions OptimizeForManifestRead( + const FileOptions& file_options) const override { + return target_->OptimizeForManifestRead(file_options); + } + FileOptions OptimizeForLogWrite(const FileOptions& file_options, + const DBOptions& db_options) const override { + return target_->OptimizeForLogWrite(file_options, db_options); + } + FileOptions OptimizeForManifestWrite( + const FileOptions& file_options) const override { + return target_->OptimizeForManifestWrite(file_options); + } + FileOptions OptimizeForCompactionTableWrite( + const FileOptions& file_options, + const ImmutableDBOptions& immutable_ops) const override { + return target_->OptimizeForCompactionTableWrite(file_options, + immutable_ops); + } + FileOptions OptimizeForCompactionTableRead( + const FileOptions& file_options, + const ImmutableDBOptions& db_options) const override { + return target_->OptimizeForCompactionTableRead(file_options, db_options); + } + IOStatus GetFreeSpace(const std::string& path, const IOOptions& options, + uint64_t* diskfree, IODebugContext* dbg) override { + return target_->GetFreeSpace(path, options, diskfree, dbg); + } + + private: + FileSystem* target_; +}; + +class FSSequentialFileWrapper : public FSSequentialFile { + public: + explicit FSSequentialFileWrapper(FSSequentialFile* target) + : target_(target) {} + + IOStatus Read(size_t n, const IOOptions& options, Slice* result, + char* scratch, IODebugContext* dbg) override { + return target_->Read(n, options, result, scratch, dbg); + } + IOStatus Skip(uint64_t n) override { return target_->Skip(n); } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + IOStatus InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + IOStatus PositionedRead(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) override { + return target_->PositionedRead(offset, n, options, result, scratch, dbg); + } + + private: + FSSequentialFile* target_; +}; + +class FSRandomAccessFileWrapper : public FSRandomAccessFile { + public: + explicit FSRandomAccessFileWrapper(FSRandomAccessFile* target) + : target_(target) {} + + IOStatus Read(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + return target_->Read(offset, n, options, result, scratch, dbg); + } + IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs, + const IOOptions& options, IODebugContext* dbg) override { + return target_->MultiRead(reqs, num_reqs, options, dbg); + } + IOStatus Prefetch(uint64_t offset, size_t n, const IOOptions& options, + IODebugContext* dbg) override { + return target_->Prefetch(offset, n, options, dbg); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + }; + void Hint(AccessPattern pattern) override { target_->Hint(pattern); } + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + IOStatus InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + + private: + FSRandomAccessFile* target_; +}; + +class FSWritableFileWrapper : public FSWritableFile { + public: + explicit FSWritableFileWrapper(FSWritableFile* t) : target_(t) {} + + IOStatus Append(const Slice& data, const IOOptions& options, + IODebugContext* dbg) override { + return target_->Append(data, options, dbg); + } + IOStatus PositionedAppend(const Slice& data, uint64_t offset, + const IOOptions& options, + IODebugContext* dbg) override { + return target_->PositionedAppend(data, offset, options, dbg); + } + IOStatus Truncate(uint64_t size, const IOOptions& options, + IODebugContext* dbg) override { + return target_->Truncate(size, options, dbg); + } + IOStatus Close(const IOOptions& options, IODebugContext* dbg) override { + return target_->Close(options, dbg); + } + IOStatus Flush(const IOOptions& options, IODebugContext* dbg) override { + return target_->Flush(options, dbg); + } + IOStatus Sync(const IOOptions& options, IODebugContext* dbg) override { + return target_->Sync(options, dbg); + } + IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) override { + return target_->Fsync(options, dbg); + } + bool IsSyncThreadSafe() const override { return target_->IsSyncThreadSafe(); } + + bool use_direct_io() const override { return target_->use_direct_io(); } + + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + + void SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) override { + target_->SetWriteLifeTimeHint(hint); + } + + Env::WriteLifeTimeHint GetWriteLifeTimeHint() override { + return target_->GetWriteLifeTimeHint(); + } + + uint64_t GetFileSize(const IOOptions& options, IODebugContext* dbg) override { + return target_->GetFileSize(options, dbg); + } + + void SetPreallocationBlockSize(size_t size) override { + target_->SetPreallocationBlockSize(size); + } + + void GetPreallocationStatus(size_t* block_size, + size_t* last_allocated_block) override { + target_->GetPreallocationStatus(block_size, last_allocated_block); + } + + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + IOStatus InvalidateCache(size_t offset, size_t length) override { + return target_->InvalidateCache(offset, length); + } + + IOStatus RangeSync(uint64_t offset, uint64_t nbytes, const IOOptions& options, + IODebugContext* dbg) override { + return target_->RangeSync(offset, nbytes, options, dbg); + } + + void PrepareWrite(size_t offset, size_t len, const IOOptions& options, + IODebugContext* dbg) override { + target_->PrepareWrite(offset, len, options, dbg); + } + + IOStatus Allocate(uint64_t offset, uint64_t len, const IOOptions& options, + IODebugContext* dbg) override { + return target_->Allocate(offset, len, options, dbg); + } + + private: + FSWritableFile* target_; +}; + +class FSRandomRWFileWrapper : public FSRandomRWFile { + public: + explicit FSRandomRWFileWrapper(FSRandomRWFile* target) : target_(target) {} + + bool use_direct_io() const override { return target_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return target_->GetRequiredBufferAlignment(); + } + IOStatus Write(uint64_t offset, const Slice& data, const IOOptions& options, + IODebugContext* dbg) override { + return target_->Write(offset, data, options, dbg); + } + IOStatus Read(uint64_t offset, size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) const override { + return target_->Read(offset, n, options, result, scratch, dbg); + } + IOStatus Flush(const IOOptions& options, IODebugContext* dbg) override { + return target_->Flush(options, dbg); + } + IOStatus Sync(const IOOptions& options, IODebugContext* dbg) override { + return target_->Sync(options, dbg); + } + IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) override { + return target_->Fsync(options, dbg); + } + IOStatus Close(const IOOptions& options, IODebugContext* dbg) override { + return target_->Close(options, dbg); + } + + private: + FSRandomRWFile* target_; +}; + +class FSDirectoryWrapper : public FSDirectory { + public: + explicit FSDirectoryWrapper(FSDirectory* target) : target_(target) {} + + IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) override { + return target_->Fsync(options, dbg); + } + size_t GetUniqueId(char* id, size_t max_size) const override { + return target_->GetUniqueId(id, max_size); + } + + private: + FSDirectory* target_; +}; + +// A utility routine: read contents of named file into *data +extern Status ReadFileToString(FileSystem* fs, const std::string& fname, + std::string* data); + +} // namespace rocksdb diff --git a/include/rocksdb/io_status.h b/include/rocksdb/io_status.h new file mode 100644 index 000000000..9f2d7e495 --- /dev/null +++ b/include/rocksdb/io_status.h @@ -0,0 +1,232 @@ +// Copyright (c) 2019-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// An IOStatus encapsulates the result of an operation. It may indicate +// success, or it may indicate an error with an associated error message. +// +// Multiple threads can invoke const methods on an IOStatus without +// external synchronization, but if any of the threads may call a +// non-const method, all threads accessing the same IOStatus must use +// external synchronization. + +#pragma once + +#include +#include "rocksdb/slice.h" +#ifdef OS_WIN +#include +#endif +#include +#include "status.h" + +namespace rocksdb { + +class IOStatus : public Status { + public: + using Code = Status::Code; + using SubCode = Status::SubCode; + + enum IOErrorScope { + kIOErrorScopeFileSystem, + kIOErrorScopeFile, + kIOErrorScopeRange, + kIOErrorScopeMax, + }; + + // Create a success status. + IOStatus() : IOStatus(kOk, kNone) {} + ~IOStatus() {} + + // Copy the specified status. + IOStatus(const IOStatus& s); + IOStatus& operator=(const IOStatus& s); + IOStatus(IOStatus&& s) +#if !(defined _MSC_VER) || ((defined _MSC_VER) && (_MSC_VER >= 1900)) + noexcept +#endif + ; + IOStatus& operator=(IOStatus&& s) +#if !(defined _MSC_VER) || ((defined _MSC_VER) && (_MSC_VER >= 1900)) + noexcept +#endif + ; + bool operator==(const IOStatus& rhs) const; + bool operator!=(const IOStatus& rhs) const; + + void SetRetryable(bool retryable) { retryable_ = retryable; } + void SetDataLoss(bool data_loss) { data_loss_ = data_loss; } + void SetScope(IOErrorScope scope) { scope_ = scope; } + + bool GetRetryable() const { return retryable_; } + bool GetDataLoss() const { return data_loss_; } + IOErrorScope GetScope() const { return scope_; } + + // Return a success status. + static IOStatus OK() { return IOStatus(); } + + static IOStatus NotSupported(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kNotSupported, msg, msg2); + } + static IOStatus NotSupported(SubCode msg = kNone) { + return IOStatus(kNotSupported, msg); + } + + // Return error status of an appropriate type. + static IOStatus NotFound(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kNotFound, msg, msg2); + } + // Fast path for not found without malloc; + static IOStatus NotFound(SubCode msg = kNone) { + return IOStatus(kNotFound, msg); + } + + static IOStatus Corruption(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kCorruption, msg, msg2); + } + static IOStatus Corruption(SubCode msg = kNone) { + return IOStatus(kCorruption, msg); + } + + static IOStatus InvalidArgument(const Slice& msg, + const Slice& msg2 = Slice()) { + return IOStatus(kInvalidArgument, msg, msg2); + } + static IOStatus InvalidArgument(SubCode msg = kNone) { + return IOStatus(kInvalidArgument, msg); + } + + static IOStatus IOError(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kIOError, msg, msg2); + } + static IOStatus IOError(SubCode msg = kNone) { + return IOStatus(kIOError, msg); + } + + static IOStatus Busy(SubCode msg = kNone) { return IOStatus(kBusy, msg); } + static IOStatus Busy(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kBusy, msg, msg2); + } + + static IOStatus TimedOut(SubCode msg = kNone) { + return IOStatus(kTimedOut, msg); + } + static IOStatus TimedOut(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kTimedOut, msg, msg2); + } + + static IOStatus NoSpace() { return IOStatus(kIOError, kNoSpace); } + static IOStatus NoSpace(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kIOError, kNoSpace, msg, msg2); + } + + static IOStatus PathNotFound() { return IOStatus(kIOError, kPathNotFound); } + static IOStatus PathNotFound(const Slice& msg, const Slice& msg2 = Slice()) { + return IOStatus(kIOError, kPathNotFound, msg, msg2); + } + + // Return a string representation of this status suitable for printing. + // Returns the string "OK" for success. + // std::string ToString() const; + + private: + friend IOStatus status_to_io_status(Status&&); + bool retryable_; + bool data_loss_; + IOErrorScope scope_; + + explicit IOStatus(Code _code, SubCode _subcode = kNone) + : Status(_code, _subcode), + retryable_(false), + data_loss_(false), + scope_(kIOErrorScopeFileSystem) {} + + IOStatus(Code _code, SubCode _subcode, const Slice& msg, const Slice& msg2); + IOStatus(Code _code, const Slice& msg, const Slice& msg2) + : IOStatus(_code, kNone, msg, msg2) {} +}; + +inline IOStatus::IOStatus(Code _code, SubCode _subcode, const Slice& msg, + const Slice& msg2) + : Status(_code, _subcode), + retryable_(false), + data_loss_(false), + scope_(kIOErrorScopeFileSystem) { + assert(code_ != kOk); + assert(subcode_ != kMaxSubCode); + const size_t len1 = msg.size(); + const size_t len2 = msg2.size(); + const size_t size = len1 + (len2 ? (2 + len2) : 0); + char* const result = new char[size + 1]; // +1 for null terminator + memcpy(result, msg.data(), len1); + if (len2) { + result[len1] = ':'; + result[len1 + 1] = ' '; + memcpy(result + len1 + 2, msg2.data(), len2); + } + result[size] = '\0'; // null terminator for C style string + state_ = result; +} + +inline IOStatus::IOStatus(const IOStatus& s) : Status(s.code_, s.subcode_) { + retryable_ = s.retryable_; + data_loss_ = s.data_loss_; + scope_ = s.scope_; + state_ = (s.state_ == nullptr) ? nullptr : CopyState(s.state_); +} +inline IOStatus& IOStatus::operator=(const IOStatus& s) { + // The following condition catches both aliasing (when this == &s), + // and the common case where both s and *this are ok. + if (this != &s) { + code_ = s.code_; + subcode_ = s.subcode_; + retryable_ = s.retryable_; + data_loss_ = s.data_loss_; + scope_ = s.scope_; + delete[] state_; + state_ = (s.state_ == nullptr) ? nullptr : CopyState(s.state_); + } + return *this; +} + +inline IOStatus::IOStatus(IOStatus&& s) +#if !(defined _MSC_VER) || ((defined _MSC_VER) && (_MSC_VER >= 1900)) + noexcept +#endif + : IOStatus() { + *this = std::move(s); +} + +inline IOStatus& IOStatus::operator=(IOStatus&& s) +#if !(defined _MSC_VER) || ((defined _MSC_VER) && (_MSC_VER >= 1900)) + noexcept +#endif +{ + if (this != &s) { + code_ = std::move(s.code_); + s.code_ = kOk; + subcode_ = std::move(s.subcode_); + s.subcode_ = kNone; + retryable_ = s.retryable_; + retryable_ = false; + data_loss_ = s.data_loss_; + data_loss_ = false; + scope_ = s.scope_; + scope_ = kIOErrorScopeFileSystem; + delete[] state_; + state_ = nullptr; + std::swap(state_, s.state_); + } + return *this; +} + +inline bool IOStatus::operator==(const IOStatus& rhs) const { + return (code_ == rhs.code_); +} + +inline bool IOStatus::operator!=(const IOStatus& rhs) const { + return !(*this == rhs); +} + +} // namespace rocksdb diff --git a/include/rocksdb/options.h b/include/rocksdb/options.h index d6d0b5289..9abefbcab 100644 --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -48,6 +48,7 @@ class Slice; class Statistics; class InternalKeyComparator; class WalFilter; +class FileSystem; // DB contents are stored in a set of blocks, each of which holds a // sequence of key,value pairs. Each block may be compressed before @@ -388,10 +389,17 @@ struct DBOptions { bool paranoid_checks = true; // Use the specified object to interact with the environment, - // e.g. to read/write files, schedule background work, etc. + // e.g. to read/write files, schedule background work, etc. In the near + // future, support for doing storage operations such as read/write files + // through env will be deprecated in favor of file_system (see below) // Default: Env::Default() Env* env = Env::Default(); + // Use the specified object to interact with the storage to + // read/write files. This is in addition to env. This option should be used + // if the desired storage subsystem provides a FileSystem implementation. + std::shared_ptr file_system = nullptr; + // Use to control write rate of flush and compaction. Flush has higher // priority than compaction. Rate limiting is disabled if nullptr. // If rate limiter is enabled, bytes_per_sync is set to 1MB by default. diff --git a/include/rocksdb/sst_file_manager.h b/include/rocksdb/sst_file_manager.h index 3e3ef859b..760da19d3 100644 --- a/include/rocksdb/sst_file_manager.h +++ b/include/rocksdb/sst_file_manager.h @@ -10,6 +10,7 @@ #include #include +#include "rocksdb/file_system.h" #include "rocksdb/status.h" namespace rocksdb { @@ -87,6 +88,7 @@ class SstFileManager { // there deletion rate. // // @param env: Pointer to Env object, please see "rocksdb/env.h". +// @param fs: Pointer to FileSystem object (rocksdb/file_system.h" // @param info_log: If not nullptr, info_log will be used to log errors. // // == Deletion rate limiting specific arguments == @@ -110,6 +112,16 @@ class SstFileManager { // `rate_bytes_per_sec` will be appreciated. NOTE that with this option, // files already renamed as a trash may be partial, so users should not // directly recover them without checking. +extern SstFileManager* NewSstFileManager( + Env* env, std::shared_ptr fs, + std::shared_ptr info_log = nullptr, + const std::string& trash_dir = "", int64_t rate_bytes_per_sec = 0, + bool delete_existing_trash = true, Status* status = nullptr, + double max_trash_db_ratio = 0.25, + uint64_t bytes_max_delete_chunk = 64 * 1024 * 1024); + +// Same as above, but takes a pointer to a legacy Env object, instead of +// Env and FileSystem objects extern SstFileManager* NewSstFileManager( Env* env, std::shared_ptr info_log = nullptr, std::string trash_dir = "", int64_t rate_bytes_per_sec = 0, diff --git a/include/rocksdb/status.h b/include/rocksdb/status.h index 507d04168..f015348c3 100644 --- a/include/rocksdb/status.h +++ b/include/rocksdb/status.h @@ -306,7 +306,7 @@ class Status { // Returns the string "OK" for success. std::string ToString() const; - private: + protected: // A nullptr state_ (which is always the case for OK) means the message // is empty. // of the following form: diff --git a/include/rocksdb/utilities/stackable_db.h b/include/rocksdb/utilities/stackable_db.h index c0cb7e316..e0b737255 100644 --- a/include/rocksdb/utilities/stackable_db.h +++ b/include/rocksdb/utilities/stackable_db.h @@ -299,6 +299,10 @@ class StackableDB : public DB { virtual Env* GetEnv() const override { return db_->GetEnv(); } + virtual FileSystem* GetFileSystem() const override { + return db_->GetFileSystem(); + } + using DB::GetOptions; virtual Options GetOptions(ColumnFamilyHandle* column_family) const override { return db_->GetOptions(column_family); diff --git a/logging/env_logger.h b/logging/env_logger.h index 7e8212dd2..5aeb1d07c 100644 --- a/logging/env_logger.h +++ b/logging/env_logger.h @@ -27,7 +27,7 @@ namespace rocksdb { class EnvLogger : public Logger { public: - EnvLogger(std::unique_ptr&& writable_file, + EnvLogger(std::unique_ptr&& writable_file, const std::string& fname, const EnvOptions& options, Env* env, InfoLogLevel log_level = InfoLogLevel::ERROR_LEVEL) : Logger(log_level), diff --git a/options/cf_options.cc b/options/cf_options.cc index ef06eaf15..23529b405 100644 --- a/options/cf_options.cc +++ b/options/cf_options.cc @@ -13,6 +13,7 @@ #include "port/port.h" #include "rocksdb/concurrent_task_limiter.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/options.h" namespace rocksdb { @@ -42,6 +43,7 @@ ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options, rate_limiter(db_options.rate_limiter.get()), info_log_level(db_options.info_log_level), env(db_options.env), + fs(db_options.fs.get()), allow_mmap_reads(db_options.allow_mmap_reads), allow_mmap_writes(db_options.allow_mmap_writes), db_paths(db_options.db_paths), diff --git a/options/cf_options.h b/options/cf_options.h index 3a6be6381..1a48d0027 100644 --- a/options/cf_options.h +++ b/options/cf_options.h @@ -61,6 +61,8 @@ struct ImmutableCFOptions { Env* env; + FileSystem* fs; + // Allow the OS to mmap file for reading sst tables. Default: false bool allow_mmap_reads; diff --git a/options/db_options.cc b/options/db_options.cc index ca2800d07..1c8776fa1 100644 --- a/options/db_options.cc +++ b/options/db_options.cc @@ -11,6 +11,7 @@ #include "port/port.h" #include "rocksdb/cache.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/sst_file_manager.h" #include "rocksdb/wal_filter.h" @@ -24,6 +25,7 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options) error_if_exists(options.error_if_exists), paranoid_checks(options.paranoid_checks), env(options.env), + fs(options.file_system), rate_limiter(options.rate_limiter), sst_file_manager(options.sst_file_manager), info_log(options.info_log), @@ -101,6 +103,8 @@ void ImmutableDBOptions::Dump(Logger* log) const { paranoid_checks); ROCKS_LOG_HEADER(log, " Options.env: %p", env); + ROCKS_LOG_HEADER(log, " Options.fs: %s", + fs->Name()); ROCKS_LOG_HEADER(log, " Options.info_log: %p", info_log.get()); ROCKS_LOG_HEADER(log, " Options.max_file_opening_threads: %d", diff --git a/options/db_options.h b/options/db_options.h index 7c71b12a0..9e8e40058 100644 --- a/options/db_options.h +++ b/options/db_options.h @@ -23,6 +23,7 @@ struct ImmutableDBOptions { bool error_if_exists; bool paranoid_checks; Env* env; + std::shared_ptr fs; std::shared_ptr rate_limiter; std::shared_ptr sst_file_manager; std::shared_ptr info_log; diff --git a/options/options_helper.cc b/options/options_helper.cc index d30264e82..446c9c707 100644 --- a/options/options_helper.cc +++ b/options/options_helper.cc @@ -38,6 +38,7 @@ DBOptions BuildDBOptions(const ImmutableDBOptions& immutable_db_options, options.error_if_exists = immutable_db_options.error_if_exists; options.paranoid_checks = immutable_db_options.paranoid_checks; options.env = immutable_db_options.env; + options.file_system = immutable_db_options.fs; options.rate_limiter = immutable_db_options.rate_limiter; options.sst_file_manager = immutable_db_options.sst_file_manager; options.info_log = immutable_db_options.info_log; diff --git a/options/options_parser.cc b/options/options_parser.cc index 6d38f0192..53bc56760 100644 --- a/options/options_parser.cc +++ b/options/options_parser.cc @@ -37,15 +37,16 @@ static const std::string option_file_header = Status PersistRocksDBOptions(const DBOptions& db_opt, const std::vector& cf_names, const std::vector& cf_opts, - const std::string& file_name, Env* env) { + const std::string& file_name, FileSystem* fs) { TEST_SYNC_POINT("PersistRocksDBOptions:start"); if (cf_names.size() != cf_opts.size()) { return Status::InvalidArgument( "cf_names.size() and cf_opts.size() must be the same"); } - std::unique_ptr wf; + std::unique_ptr wf; - Status s = env->NewWritableFile(file_name, &wf, EnvOptions()); + Status s = + fs->NewWritableFile(file_name, FileOptions(), &wf, nullptr); if (!s.ok()) { return s; } @@ -103,7 +104,7 @@ Status PersistRocksDBOptions(const DBOptions& db_opt, writable->Close(); return RocksDBOptionsParser::VerifyRocksDBOptionsFromFile( - db_opt, cf_names, cf_opts, file_name, env); + db_opt, cf_names, cf_opts, file_name, fs); } RocksDBOptionsParser::RocksDBOptionsParser() { Reset(); } @@ -201,12 +202,13 @@ Status RocksDBOptionsParser::ParseStatement(std::string* name, return Status::OK(); } -Status RocksDBOptionsParser::Parse(const std::string& file_name, Env* env, +Status RocksDBOptionsParser::Parse(const std::string& file_name, FileSystem* fs, bool ignore_unknown_options) { Reset(); - std::unique_ptr seq_file; - Status s = env->NewSequentialFile(file_name, &seq_file, EnvOptions()); + std::unique_ptr seq_file; + Status s = fs->NewSequentialFile(file_name, FileOptions(), &seq_file, + nullptr); if (!s.ok()) { return s; } @@ -652,11 +654,10 @@ bool AreEqualOptions( Status RocksDBOptionsParser::VerifyRocksDBOptionsFromFile( const DBOptions& db_opt, const std::vector& cf_names, const std::vector& cf_opts, - const std::string& file_name, Env* env, + const std::string& file_name, FileSystem* fs, OptionsSanityCheckLevel sanity_check_level, bool ignore_unknown_options) { RocksDBOptionsParser parser; - std::unique_ptr seq_file; - Status s = parser.Parse(file_name, env, ignore_unknown_options); + Status s = parser.Parse(file_name, fs, ignore_unknown_options); if (!s.ok()) { return s; } diff --git a/options/options_parser.h b/options/options_parser.h index b2a806f17..b675b28fe 100644 --- a/options/options_parser.h +++ b/options/options_parser.h @@ -35,7 +35,7 @@ static const std::string opt_section_titles[] = { Status PersistRocksDBOptions(const DBOptions& db_opt, const std::vector& cf_names, const std::vector& cf_opts, - const std::string& file_name, Env* env); + const std::string& file_name, FileSystem* fs); extern bool AreEqualOptions( const char* opt1, const char* opt2, const OptionTypeInfo& type_info, @@ -48,7 +48,7 @@ class RocksDBOptionsParser { ~RocksDBOptionsParser() {} void Reset(); - Status Parse(const std::string& file_name, Env* env, + Status Parse(const std::string& file_name, FileSystem* fs, bool ignore_unknown_options = false); static std::string TrimAndRemoveComment(const std::string& line, const bool trim_only = false); @@ -72,7 +72,7 @@ class RocksDBOptionsParser { static Status VerifyRocksDBOptionsFromFile( const DBOptions& db_opt, const std::vector& cf_names, const std::vector& cf_opts, - const std::string& file_name, Env* env, + const std::string& file_name, FileSystem* fs, OptionsSanityCheckLevel sanity_check_level = kSanityLevelExactMatch, bool ignore_unknown_options = false); diff --git a/options/options_settable_test.cc b/options/options_settable_test.cc index bc2e088a6..4d13585f7 100644 --- a/options/options_settable_test.cc +++ b/options/options_settable_test.cc @@ -181,6 +181,8 @@ TEST_F(OptionsSettableTest, BlockBasedTableOptionsAllFieldsSettable) { TEST_F(OptionsSettableTest, DBOptionsAllFieldsSettable) { const OffsetGap kDBOptionsBlacklist = { {offsetof(struct DBOptions, env), sizeof(Env*)}, + {offsetof(struct DBOptions, file_system), + sizeof(std::shared_ptr)}, {offsetof(struct DBOptions, rate_limiter), sizeof(std::shared_ptr)}, {offsetof(struct DBOptions, sst_file_manager), diff --git a/options/options_test.cc b/options/options_test.cc index d1c9db039..ceffc483a 100644 --- a/options/options_test.cc +++ b/options/options_test.cc @@ -1112,10 +1112,14 @@ TEST_F(OptionsTest, ConvertOptionsTest) { #ifndef ROCKSDB_LITE class OptionsParserTest : public testing::Test { public: - OptionsParserTest() { env_.reset(new test::StringEnv(Env::Default())); } + OptionsParserTest() { + env_.reset(new test::StringEnv(Env::Default())); + fs_.reset(new LegacyFileSystemWrapper(env_.get())); + } protected: std::unique_ptr env_; + std::unique_ptr fs_; }; TEST_F(OptionsParserTest, Comment) { @@ -1146,7 +1150,7 @@ TEST_F(OptionsParserTest, Comment) { const std::string kTestFileName = "test-rocksdb-options.ini"; env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_OK(parser.Parse(kTestFileName, env_.get())); + ASSERT_OK(parser.Parse(kTestFileName, fs_.get())); ASSERT_OK(RocksDBOptionsParser::VerifyDBOptions(*parser.db_opt(), db_opt)); ASSERT_EQ(parser.NumColumnFamilies(), 1U); @@ -1172,7 +1176,7 @@ TEST_F(OptionsParserTest, ExtraSpace) { const std::string kTestFileName = "test-rocksdb-options.ini"; env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_OK(parser.Parse(kTestFileName, env_.get())); + ASSERT_OK(parser.Parse(kTestFileName, fs_.get())); } TEST_F(OptionsParserTest, MissingDBOptions) { @@ -1189,7 +1193,7 @@ TEST_F(OptionsParserTest, MissingDBOptions) { const std::string kTestFileName = "test-rocksdb-options.ini"; env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_NOK(parser.Parse(kTestFileName, env_.get())); + ASSERT_NOK(parser.Parse(kTestFileName, fs_.get())); } TEST_F(OptionsParserTest, DoubleDBOptions) { @@ -1217,7 +1221,7 @@ TEST_F(OptionsParserTest, DoubleDBOptions) { const std::string kTestFileName = "test-rocksdb-options.ini"; env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_NOK(parser.Parse(kTestFileName, env_.get())); + ASSERT_NOK(parser.Parse(kTestFileName, fs_.get())); } TEST_F(OptionsParserTest, NoDefaultCFOptions) { @@ -1244,7 +1248,7 @@ TEST_F(OptionsParserTest, NoDefaultCFOptions) { const std::string kTestFileName = "test-rocksdb-options.ini"; env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_NOK(parser.Parse(kTestFileName, env_.get())); + ASSERT_NOK(parser.Parse(kTestFileName, fs_.get())); } TEST_F(OptionsParserTest, DefaultCFOptionsMustBeTheFirst) { @@ -1273,7 +1277,7 @@ TEST_F(OptionsParserTest, DefaultCFOptionsMustBeTheFirst) { const std::string kTestFileName = "test-rocksdb-options.ini"; env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_NOK(parser.Parse(kTestFileName, env_.get())); + ASSERT_NOK(parser.Parse(kTestFileName, fs_.get())); } TEST_F(OptionsParserTest, DuplicateCFOptions) { @@ -1301,7 +1305,7 @@ TEST_F(OptionsParserTest, DuplicateCFOptions) { const std::string kTestFileName = "test-rocksdb-options.ini"; env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_NOK(parser.Parse(kTestFileName, env_.get())); + ASSERT_NOK(parser.Parse(kTestFileName, fs_.get())); } TEST_F(OptionsParserTest, IgnoreUnknownOptions) { @@ -1369,12 +1373,12 @@ TEST_F(OptionsParserTest, IgnoreUnknownOptions) { env_->DeleteFile(kTestFileName); env_->WriteToNewFile(kTestFileName, options_file_content); RocksDBOptionsParser parser; - ASSERT_NOK(parser.Parse(kTestFileName, env_.get())); + ASSERT_NOK(parser.Parse(kTestFileName, fs_.get())); if (should_ignore) { - ASSERT_OK(parser.Parse(kTestFileName, env_.get(), + ASSERT_OK(parser.Parse(kTestFileName, fs_.get(), true /* ignore_unknown_options */)); } else { - ASSERT_NOK(parser.Parse(kTestFileName, env_.get(), + ASSERT_NOK(parser.Parse(kTestFileName, fs_.get(), true /* ignore_unknown_options */)); } } @@ -1413,7 +1417,7 @@ TEST_F(OptionsParserTest, ParseVersion) { parser.Reset(); env_->WriteToNewFile(iv, buffer); - ASSERT_NOK(parser.Parse(iv, env_.get())); + ASSERT_NOK(parser.Parse(iv, fs_.get())); } const std::vector valid_versions = { @@ -1422,7 +1426,7 @@ TEST_F(OptionsParserTest, ParseVersion) { snprintf(buffer, kLength - 1, file_template.c_str(), vv.c_str()); parser.Reset(); env_->WriteToNewFile(vv, buffer); - ASSERT_OK(parser.Parse(vv, env_.get())); + ASSERT_OK(parser.Parse(vv, fs_.get())); } } @@ -1547,10 +1551,10 @@ TEST_F(OptionsParserTest, DumpAndParse) { const std::string kOptionsFileName = "test-persisted-options.ini"; ASSERT_OK(PersistRocksDBOptions(base_db_opt, cf_names, base_cf_opts, - kOptionsFileName, env_.get())); + kOptionsFileName, fs_.get())); RocksDBOptionsParser parser; - ASSERT_OK(parser.Parse(kOptionsFileName, env_.get())); + ASSERT_OK(parser.Parse(kOptionsFileName, fs_.get())); // Make sure block-based table factory options was deserialized correctly std::shared_ptr ttf = (*parser.cf_opts())[4].table_factory; @@ -1562,7 +1566,7 @@ TEST_F(OptionsParserTest, DumpAndParse) { parsed_bbto.cache_index_and_filter_blocks); ASSERT_OK(RocksDBOptionsParser::VerifyRocksDBOptionsFromFile( - base_db_opt, cf_names, base_cf_opts, kOptionsFileName, env_.get())); + base_db_opt, cf_names, base_cf_opts, kOptionsFileName, fs_.get())); ASSERT_OK( RocksDBOptionsParser::VerifyDBOptions(*parser.db_opt(), base_db_opt)); @@ -1585,7 +1589,7 @@ TEST_F(OptionsParserTest, DumpAndParse) { base_db_opt.max_open_files++; ASSERT_NOK(RocksDBOptionsParser::VerifyRocksDBOptionsFromFile( - base_db_opt, cf_names, base_cf_opts, kOptionsFileName, env_.get())); + base_db_opt, cf_names, base_cf_opts, kOptionsFileName, fs_.get())); for (int c = 0; c < num_cf; ++c) { if (base_cf_opts[c].compaction_filter) { @@ -1606,10 +1610,10 @@ TEST_F(OptionsParserTest, DifferentDefault) { ASSERT_OK(PersistRocksDBOptions(DBOptions(), {"default", "universal"}, {cf_level_opts, cf_univ_opts}, - kOptionsFileName, env_.get())); + kOptionsFileName, fs_.get())); RocksDBOptionsParser parser; - ASSERT_OK(parser.Parse(kOptionsFileName, env_.get())); + ASSERT_OK(parser.Parse(kOptionsFileName, fs_.get())); { Options old_default_opts; @@ -1692,7 +1696,7 @@ class OptionsSanityCheckTest : public OptionsParserTest { Status SanityCheckCFOptions(const ColumnFamilyOptions& cf_opts, OptionsSanityCheckLevel level) { return RocksDBOptionsParser::VerifyRocksDBOptionsFromFile( - DBOptions(), {"default"}, {cf_opts}, kOptionsFileName, env_.get(), + DBOptions(), {"default"}, {cf_opts}, kOptionsFileName, fs_.get(), level); } @@ -1702,7 +1706,7 @@ class OptionsSanityCheckTest : public OptionsParserTest { return s; } return PersistRocksDBOptions(DBOptions(), {"default"}, {cf_opts}, - kOptionsFileName, env_.get()); + kOptionsFileName, fs_.get()); } const std::string kOptionsFileName = "OPTIONS"; diff --git a/src.mk b/src.mk index 9498d45e3..db91fc417 100644 --- a/src.mk +++ b/src.mk @@ -68,6 +68,8 @@ LIB_SOURCES = \ env/env_encryption.cc \ env/env_hdfs.cc \ env/env_posix.cc \ + env/file_system.cc \ + env/fs_posix.cc \ env/io_posix.cc \ env/mock_env.cc \ file/delete_scheduler.cc \ diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index ffdbdb1bd..be7c8e139 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -24,6 +24,7 @@ #include "rocksdb/cache.h" #include "rocksdb/comparator.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/filter_policy.h" #include "rocksdb/iterator.h" #include "rocksdb/options.h" @@ -994,7 +995,7 @@ void BlockBasedTable::SetupCacheKeyPrefix(Rep* rep) { } } -void BlockBasedTable::GenerateCachePrefix(Cache* cc, RandomAccessFile* file, +void BlockBasedTable::GenerateCachePrefix(Cache* cc, FSRandomAccessFile* file, char* buffer, size_t* size) { // generate an id from the file *size = file->GetUniqueId(buffer, kMaxCacheKeyPrefixSize); @@ -1007,7 +1008,7 @@ void BlockBasedTable::GenerateCachePrefix(Cache* cc, RandomAccessFile* file, } } -void BlockBasedTable::GenerateCachePrefix(Cache* cc, WritableFile* file, +void BlockBasedTable::GenerateCachePrefix(Cache* cc, FSWritableFile* file, char* buffer, size_t* size) { // generate an id from the file *size = file->GetUniqueId(buffer, kMaxCacheKeyPrefixSize); @@ -1603,13 +1604,13 @@ void BlockBasedTable::SetupForCompaction() { case Options::NONE: break; case Options::NORMAL: - rep_->file->file()->Hint(RandomAccessFile::NORMAL); + rep_->file->file()->Hint(FSRandomAccessFile::kNormal); break; case Options::SEQUENTIAL: - rep_->file->file()->Hint(RandomAccessFile::SEQUENTIAL); + rep_->file->file()->Hint(FSRandomAccessFile::kSequential); break; case Options::WILLNEED: - rep_->file->file()->Hint(RandomAccessFile::WILLNEED); + rep_->file->file()->Hint(FSRandomAccessFile::kWillNeed); break; default: assert(false); @@ -2316,7 +2317,7 @@ void BlockBasedTable::RetrieveMultipleBlocks( return; } - autovector read_reqs; + autovector read_reqs; size_t buf_offset = 0; size_t idx_in_batch = 0; for (auto mget_iter = batch->begin(); mget_iter != batch->end(); @@ -2326,7 +2327,7 @@ void BlockBasedTable::RetrieveMultipleBlocks( continue; } - ReadRequest req; + FSReadRequest req; req.len = block_size(handle); if (scratch == nullptr) { req.scratch = new char[req.len]; @@ -2335,7 +2336,7 @@ void BlockBasedTable::RetrieveMultipleBlocks( buf_offset += req.len; } req.offset = handle.offset(); - req.status = Status::OK(); + req.status = IOStatus::OK(); read_reqs.emplace_back(req); } @@ -2351,7 +2352,7 @@ void BlockBasedTable::RetrieveMultipleBlocks( continue; } - ReadRequest& req = read_reqs[read_req_idx++]; + FSReadRequest& req = read_reqs[read_req_idx++]; Status s = req.status; if (s.ok()) { if (req.result.size() != req.len) { diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index bcc7fd1a3..59bd1fdf6 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -51,7 +51,7 @@ class FullFilterBlockReader; class Footer; class InternalKeyComparator; class Iterator; -class RandomAccessFile; +class FSRandomAccessFile; class TableCache; class TableReader; class WritableFile; @@ -446,9 +446,9 @@ class BlockBasedTable : public TableReader { static void SetupCacheKeyPrefix(Rep* rep); // Generate a cache key prefix from the file - static void GenerateCachePrefix(Cache* cc, RandomAccessFile* file, + static void GenerateCachePrefix(Cache* cc, FSRandomAccessFile* file, char* buffer, size_t* size); - static void GenerateCachePrefix(Cache* cc, WritableFile* file, char* buffer, + static void GenerateCachePrefix(Cache* cc, FSWritableFile* file, char* buffer, size_t* size); // Given an iterator return its offset in file. diff --git a/table/block_based/data_block_hash_index_test.cc b/table/block_based/data_block_hash_index_test.cc index ae23f6ef2..a77e079cb 100644 --- a/table/block_based/data_block_hash_index_test.cc +++ b/table/block_based/data_block_hash_index_test.cc @@ -570,14 +570,13 @@ void TestBoundary(InternalKey& ik1, std::string& v1, InternalKey& ik2, file_writer->Flush(); EXPECT_TRUE(s.ok()) << s.ToString(); - EXPECT_EQ(static_cast(file_writer->writable_file()) - ->contents() - .size(), - builder->FileSize()); + EXPECT_EQ( + test::GetStringSinkFromLegacyWriter(file_writer.get())->contents().size(), + builder->FileSize()); // Open the table file_reader.reset(test::GetRandomAccessFileReader(new test::StringSource( - static_cast(file_writer->writable_file())->contents(), + test::GetStringSinkFromLegacyWriter(file_writer.get())->contents(), 0 /*uniq_id*/, ioptions.allow_mmap_reads))); const bool kSkipFilters = true; const bool kImmortal = true; @@ -586,9 +585,7 @@ void TestBoundary(InternalKey& ik1, std::string& v1, InternalKey& ik2, internal_comparator, !kSkipFilters, !kImmortal, level_), std::move(file_reader), - static_cast(file_writer->writable_file()) - ->contents() - .size(), + test::GetStringSinkFromLegacyWriter(file_writer.get())->contents().size(), &table_reader); // Search using Get() ReadOptions ro; diff --git a/table/cuckoo/cuckoo_table_builder_test.cc b/table/cuckoo/cuckoo_table_builder_test.cc index b84cc9f5b..4f4c15a60 100644 --- a/table/cuckoo/cuckoo_table_builder_test.cc +++ b/table/cuckoo/cuckoo_table_builder_test.cc @@ -65,7 +65,8 @@ class CuckooBuilderTest : public testing::Test { // Assert Table Properties. TableProperties* props = nullptr; std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(read_file), fname)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(read_file), + fname)); ASSERT_OK(ReadTableProperties(file_reader.get(), read_file_size, kCuckooTableMagicNumber, ioptions, &props, true /* compression_type_missing */)); @@ -165,8 +166,9 @@ TEST_F(CuckooBuilderTest, SuccessWithEmptyFile) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("EmptyFile"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(writable_file)), fname, + EnvOptions())); CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, 4, 100, BytewiseComparator(), 1, false, false, GetSliceHash, 0 /* column_family_id */, @@ -207,8 +209,9 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionFullKey) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("NoCollisionFullKey"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -256,8 +259,9 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionFullKey) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("WithCollisionFullKey"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -305,8 +309,9 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionAndCuckooBlock) { uint32_t cuckoo_block_size = 2; fname = test::PerThreadDBPath("WithCollisionFullKey2"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(writable_file)), fname, + EnvOptions())); CuckooTableBuilder builder( file_writer.get(), kHashTableRatio, num_hash_fun, 100, BytewiseComparator(), cuckoo_block_size, false, false, GetSliceHash, @@ -358,8 +363,9 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKey) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("WithCollisionPathFullKey"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -408,8 +414,9 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKeyAndCuckooBlock) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("WithCollisionPathFullKeyAndCuckooBlock"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -451,8 +458,9 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionUserKey) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("NoCollisionUserKey"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -495,8 +503,9 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionUserKey) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("WithCollisionUserKey"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -541,8 +550,9 @@ TEST_F(CuckooBuilderTest, WithCollisionPathUserKey) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("WithCollisionPathUserKey"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -586,8 +596,9 @@ TEST_F(CuckooBuilderTest, FailWhenCollisionPathTooLong) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("WithCollisionPathUserKey"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -614,8 +625,9 @@ TEST_F(CuckooBuilderTest, FailWhenSameKeyInserted) { std::unique_ptr writable_file; fname = test::PerThreadDBPath("FailWhenSameKeyInserted"); ASSERT_OK(env_->NewWritableFile(fname, &writable_file, env_options_)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, diff --git a/table/cuckoo/cuckoo_table_reader_test.cc b/table/cuckoo/cuckoo_table_reader_test.cc index 2dfe887ba..65c7f4d65 100644 --- a/table/cuckoo/cuckoo_table_reader_test.cc +++ b/table/cuckoo/cuckoo_table_reader_test.cc @@ -91,8 +91,9 @@ class CuckooReaderTest : public testing::Test { const Comparator* ucomp = BytewiseComparator()) { std::unique_ptr writable_file; ASSERT_OK(env->NewWritableFile(fname, &writable_file, env_options)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, env_options)); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(writable_file)), fname, + env_options)); CuckooTableBuilder builder( file_writer.get(), 0.9, kNumHashFunc, 100, ucomp, 2, false, false, @@ -112,7 +113,8 @@ class CuckooReaderTest : public testing::Test { std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(read_file), fname)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(read_file), + fname)); const ImmutableCFOptions ioptions(options); CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucomp, GetSliceHash); @@ -141,7 +143,8 @@ class CuckooReaderTest : public testing::Test { std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(read_file), fname)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(read_file), + fname)); const ImmutableCFOptions ioptions(options); CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucomp, GetSliceHash); @@ -323,7 +326,8 @@ TEST_F(CuckooReaderTest, WhenKeyNotFound) { std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(read_file), fname)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(read_file), + fname)); const ImmutableCFOptions ioptions(options); CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucmp, GetSliceHash); @@ -410,8 +414,9 @@ void WriteFile(const std::vector& keys, std::unique_ptr writable_file; ASSERT_OK(env->NewWritableFile(fname, &writable_file, env_options)); - std::unique_ptr file_writer( - new WritableFileWriter(std::move(writable_file), fname, env_options)); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(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 */, @@ -432,7 +437,8 @@ void WriteFile(const std::vector& keys, std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(read_file), fname)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(read_file), + fname)); const ImmutableCFOptions ioptions(options); CuckooTableReader reader(ioptions, std::move(file_reader), file_size, @@ -464,7 +470,8 @@ void ReadKeys(uint64_t num, uint32_t batch_size) { std::unique_ptr read_file; ASSERT_OK(env->NewRandomAccessFile(fname, &read_file, env_options)); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(read_file), fname)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(read_file), + fname)); const ImmutableCFOptions ioptions(options); CuckooTableReader reader(ioptions, std::move(file_reader), file_size, diff --git a/table/mock_table.cc b/table/mock_table.cc index 50cbb2024..be520cac3 100644 --- a/table/mock_table.cc +++ b/table/mock_table.cc @@ -6,6 +6,7 @@ #include "table/mock_table.h" #include "db/dbformat.h" +#include "env/composite_env_wrapper.h" #include "file/random_access_file_reader.h" #include "port/port.h" #include "rocksdb/table_properties.h" @@ -94,7 +95,8 @@ Status MockTableFactory::CreateMockTable(Env* env, const std::string& fname, return s; } - WritableFileWriter file_writer(std::move(file), fname, EnvOptions()); + WritableFileWriter file_writer(NewLegacyWritableFileWrapper(std::move(file)), + fname, EnvOptions()); uint32_t id = GetAndWriteNextID(&file_writer); file_system_.files.insert({id, std::move(file_contents)}); diff --git a/table/sst_file_reader.cc b/table/sst_file_reader.cc index 48db1d8b4..f9f17865b 100644 --- a/table/sst_file_reader.cc +++ b/table/sst_file_reader.cc @@ -9,6 +9,7 @@ #include "db/db_iter.h" #include "db/dbformat.h" +#include "env/composite_env_wrapper.h" #include "file/random_access_file_reader.h" #include "options/cf_options.h" #include "table/get_context.h" @@ -47,7 +48,8 @@ Status SstFileReader::Open(const std::string& file_path) { s = r->options.env->NewRandomAccessFile(file_path, &file, r->soptions); } if (s.ok()) { - file_reader.reset(new RandomAccessFileReader(std::move(file), file_path)); + file_reader.reset(new RandomAccessFileReader( + NewLegacyRandomAccessFileWrapper(file), file_path)); } if (s.ok()) { TableReaderOptions t_opt(r->ioptions, r->moptions.prefix_extractor.get(), diff --git a/table/sst_file_writer.cc b/table/sst_file_writer.cc index dc2c589f2..e72564bed 100644 --- a/table/sst_file_writer.cc +++ b/table/sst_file_writer.cc @@ -8,6 +8,7 @@ #include #include "db/dbformat.h" +#include "env/composite_env_wrapper.h" #include "file/writable_file_writer.h" #include "rocksdb/table.h" #include "table/block_based/block_based_table_builder.h" @@ -240,9 +241,10 @@ Status SstFileWriter::Open(const std::string& file_path) { &int_tbl_prop_collector_factories, compression_type, sample_for_compression, compression_opts, r->skip_filters, r->column_family_name, unknown_level); - r->file_writer.reset(new WritableFileWriter( - std::move(sst_file), file_path, r->env_options, r->ioptions.env, - nullptr /* stats */, r->ioptions.listeners)); + r->file_writer.reset( + new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(sst_file)), + file_path, r->env_options, r->ioptions.env, + nullptr /* stats */, r->ioptions.listeners)); // TODO(tec) : If table_factory is using compressed block cache, we will // be adding the external sst file blocks into it, which is wasteful. diff --git a/table/table_reader_bench.cc b/table/table_reader_bench.cc index 05bb2ea25..fec893097 100644 --- a/table/table_reader_bench.cc +++ b/table/table_reader_bench.cc @@ -13,6 +13,7 @@ int main() { #include "db/db_impl/db_impl.h" #include "db/dbformat.h" +#include "env/composite_env_wrapper.h" #include "file/random_access_file_reader.h" #include "monitoring/histogram.h" #include "rocksdb/db.h" @@ -94,8 +95,8 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, std::vector > int_tbl_prop_collector_factories; - file_writer.reset( - new WritableFileWriter(std::move(file), file_name, env_options)); + file_writer.reset(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(file)), file_name, env_options)); int unknown_level = -1; tb = opts.table_factory->NewTableBuilder( TableBuilderOptions( @@ -138,7 +139,8 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options, uint64_t file_size; env->GetFileSize(file_name, &file_size); std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(raf), file_name)); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(raf), + file_name)); s = opts.table_factory->NewTableReader( TableReaderOptions(ioptions, moptions.prefix_extractor.get(), env_options, ikc), diff --git a/table/table_test.cc b/table/table_test.cc index 77b962598..5ab19e9f9 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -28,6 +28,7 @@ #include "rocksdb/cache.h" #include "rocksdb/db.h" #include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/iterator.h" #include "rocksdb/memtablerep.h" #include "rocksdb/perf_context.h" @@ -426,7 +427,7 @@ class TableConstructor: public Constructor { bool ConvertToInternalKey() { return convert_to_internal_key_; } test::StringSink* TEST_GetSink() { - return static_cast(file_writer_->writable_file()); + return rocksdb::test::GetStringSinkFromLegacyWriter(file_writer_.get()); } BlockCacheTracer block_cache_tracer_; @@ -3067,7 +3068,7 @@ TEST_F(PlainTableTest, BasicPlainTableProperties) { file_writer->Flush(); test::StringSink* ss = - static_cast(file_writer->writable_file()); + rocksdb::test::GetStringSinkFromLegacyWriter(file_writer.get()); std::unique_ptr file_reader( test::GetRandomAccessFileReader( new test::StringSource(ss->contents(), 72242, true))); diff --git a/test_util/testutil.cc b/test_util/testutil.cc index 7c90c14ef..e309df088 100644 --- a/test_util/testutil.cc +++ b/test_util/testutil.cc @@ -15,6 +15,7 @@ #include #include "db/memtable_list.h" +#include "env/composite_env_wrapper.h" #include "file/random_access_file_reader.h" #include "file/sequence_file_reader.h" #include "file/writable_file_writer.h" @@ -127,19 +128,20 @@ const Comparator* Uint64Comparator() { WritableFileWriter* GetWritableFileWriter(WritableFile* wf, const std::string& fname) { std::unique_ptr file(wf); - return new WritableFileWriter(std::move(file), fname, EnvOptions()); + return new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(file)), + fname, EnvOptions()); } RandomAccessFileReader* GetRandomAccessFileReader(RandomAccessFile* raf) { std::unique_ptr file(raf); - return new RandomAccessFileReader(std::move(file), + return new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(file), "[test RandomAccessFileReader]"); } SequentialFileReader* GetSequentialFileReader(SequentialFile* se, const std::string& fname) { std::unique_ptr file(se); - return new SequentialFileReader(std::move(file), fname); + return new SequentialFileReader(NewLegacySequentialFileWrapper(file), fname); } void CorruptKeyType(InternalKey* ikey) { diff --git a/test_util/testutil.h b/test_util/testutil.h index 716ae7d26..996649c1b 100644 --- a/test_util/testutil.h +++ b/test_util/testutil.h @@ -13,6 +13,8 @@ #include #include +#include "env/composite_env_wrapper.h" +#include "file/writable_file_writer.h" #include "rocksdb/compaction_filter.h" #include "rocksdb/env.h" #include "rocksdb/iterator.h" @@ -377,6 +379,13 @@ class StringSource: public RandomAccessFile { mutable int total_reads_; }; +inline StringSink* GetStringSinkFromLegacyWriter( + const WritableFileWriter* writer) { + LegacyWritableFileWrapper* file = + static_cast(writer->writable_file()); + return static_cast(file->target()); +} + class NullLogger : public Logger { public: using Logger::Logv; diff --git a/tools/db_bench_tool_test.cc b/tools/db_bench_tool_test.cc index 87f8f1943..1f7fa9316 100644 --- a/tools/db_bench_tool_test.cc +++ b/tools/db_bench_tool_test.cc @@ -66,8 +66,8 @@ class DBBenchTest : public testing::Test { void VerifyOptions(const Options& opt) { DBOptions loaded_db_opts; std::vector cf_descs; - ASSERT_OK(LoadLatestOptions(db_path_, Env::Default(), &loaded_db_opts, - &cf_descs)); + ASSERT_OK(LoadLatestOptions(db_path_, FileSystem::Default(), + &loaded_db_opts, &cf_descs)); ASSERT_OK( RocksDBOptionsParser::VerifyDBOptions(DBOptions(opt), loaded_db_opts)); diff --git a/tools/ldb_cmd.cc b/tools/ldb_cmd.cc index 1965c6911..dbf09fe59 100644 --- a/tools/ldb_cmd.cc +++ b/tools/ldb_cmd.cc @@ -13,6 +13,7 @@ #include "db/dbformat.h" #include "db/log_reader.h" #include "db/write_batch_internal.h" +#include "env/composite_env_wrapper.h" #include "file/filename.h" #include "port/port_dirent.h" #include "rocksdb/cache.h" @@ -286,6 +287,8 @@ void LDBCommand::Run() { options_.env = env; } + options_.file_system.reset(new LegacyFileSystemWrapper(options_.env)); + if (db_ == nullptr && !NoDBOpen()) { OpenDB(); if (exec_state_.IsFailed() && try_load_options_) { @@ -705,6 +708,7 @@ Options LDBCommand::PrepareOptionsForOpenDB() { LDBCommandExecuteResult::Failed(ARG_FIX_PREFIX_LEN + " must be > 0."); } } + // TODO(ajkr): this return value doesn't reflect the CF options changed, so // subcommands that rely on this won't see the effect of CF-related CLI args. // Such subcommands need to be changed to properly support CFs. @@ -1806,6 +1810,8 @@ void ReduceDBLevelsCommand::DoCommand() { Status st; Options opt = PrepareOptionsForOpenDB(); int old_level_num = -1; + opt.file_system.reset(new LegacyFileSystemWrapper(opt.env)); + ; st = GetOldNumOfLevels(opt, &old_level_num); if (!st.ok()) { exec_state_ = LDBCommandExecuteResult::Failed(st.ToString()); @@ -2092,8 +2098,8 @@ void DumpWalFile(Options options, std::string wal_file, bool print_header, std::unique_ptr file; status = env->NewSequentialFile(wal_file, &file, soptions); if (status.ok()) { - wal_file_reader.reset( - new SequentialFileReader(std::move(file), wal_file)); + wal_file_reader.reset(new SequentialFileReader( + NewLegacySequentialFileWrapper(file), wal_file)); } } if (!status.ok()) { diff --git a/tools/ldb_cmd_test.cc b/tools/ldb_cmd_test.cc index 85afb1183..3b88ed7ac 100644 --- a/tools/ldb_cmd_test.cc +++ b/tools/ldb_cmd_test.cc @@ -6,6 +6,7 @@ #ifndef ROCKSDB_LITE #include "rocksdb/utilities/ldb_cmd.h" +#include "env/composite_env_wrapper.h" #include "port/stack_trace.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" @@ -74,6 +75,8 @@ TEST_F(LdbCmdTest, MemEnv) { opts.env = env.get(); opts.create_if_missing = true; + opts.file_system.reset(new LegacyFileSystemWrapper(opts.env)); + DB* db = nullptr; std::string dbname = test::TmpDir(); ASSERT_OK(DB::Open(opts, dbname, &db)); diff --git a/tools/sst_dump_test.cc b/tools/sst_dump_test.cc index 0fc72d738..57bf3ac8b 100644 --- a/tools/sst_dump_test.cc +++ b/tools/sst_dump_test.cc @@ -53,8 +53,8 @@ void createSST(const Options& opts, const std::string& file_name) { std::vector > int_tbl_prop_collector_factories; - std::unique_ptr file_writer( - new WritableFileWriter(std::move(file), file_name, EnvOptions())); + std::unique_ptr file_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(file)), file_name, EnvOptions())); std::string column_family_name; int unknown_level = -1; tb.reset(opts.table_factory->NewTableBuilder( diff --git a/tools/sst_dump_tool.cc b/tools/sst_dump_tool.cc index 71bec339e..7c864ea5d 100644 --- a/tools/sst_dump_tool.cc +++ b/tools/sst_dump_tool.cc @@ -18,6 +18,7 @@ #include "db/blob_index.h" #include "db/memtable.h" #include "db/write_batch_internal.h" +#include "env/composite_env_wrapper.h" #include "options/cf_options.h" #include "rocksdb/db.h" #include "rocksdb/env.h" @@ -91,7 +92,8 @@ Status SstFileDumper::GetTableReader(const std::string& file_path) { s = options_.env->GetFileSize(file_path, &file_size); } - file_.reset(new RandomAccessFileReader(std::move(file), file_path)); + file_.reset(new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(file), + file_path)); if (s.ok()) { s = ReadFooterFromFile(file_.get(), nullptr /* prefetch_buffer */, @@ -106,7 +108,8 @@ Status SstFileDumper::GetTableReader(const std::string& file_path) { magic_number == kLegacyPlainTableMagicNumber) { soptions_.use_mmap_reads = true; options_.env->NewRandomAccessFile(file_path, &file, soptions_); - file_.reset(new RandomAccessFileReader(std::move(file), file_path)); + file_.reset(new RandomAccessFileReader( + NewLegacyRandomAccessFileWrapper(file), file_path)); } options_.comparator = &internal_comparator_; // For old sst format, ReadTableProperties might fail but file can be read @@ -167,7 +170,8 @@ uint64_t SstFileDumper::CalculateCompressedTableSize( env->NewWritableFile(testFileName, &out_file, soptions_); std::unique_ptr dest_writer; dest_writer.reset( - new WritableFileWriter(std::move(out_file), testFileName, soptions_)); + new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(out_file)), + testFileName, soptions_)); BlockBasedTableOptions table_options; table_options.block_size = block_size; BlockBasedTableFactory block_based_tf(table_options); diff --git a/tools/trace_analyzer_test.cc b/tools/trace_analyzer_test.cc index f27792241..1ef100fe4 100644 --- a/tools/trace_analyzer_test.cc +++ b/tools/trace_analyzer_test.cc @@ -130,7 +130,9 @@ class TraceAnalyzerTest : public testing::Test { std::vector result; uint32_t count; Status s; - for (count = 0; ReadOneLine(&iss, f_ptr.get(), &get_line, &has_data, &s); + std::unique_ptr file = + NewLegacySequentialFileWrapper(f_ptr); + for (count = 0; ReadOneLine(&iss, file.get(), &get_line, &has_data, &s); ++count) { ASSERT_OK(s); result.push_back(get_line); diff --git a/tools/trace_analyzer_tool.cc b/tools/trace_analyzer_tool.cc index ae0b20bea..27d097c2c 100644 --- a/tools/trace_analyzer_tool.cc +++ b/tools/trace_analyzer_tool.cc @@ -26,6 +26,7 @@ #include "db/db_impl/db_impl.h" #include "db/memtable.h" #include "db/write_batch_internal.h" +#include "env/composite_env_wrapper.h" #include "file/read_write_util.h" #include "file/writable_file_writer.h" #include "options/cf_options.h" @@ -1049,15 +1050,19 @@ Status TraceAnalyzer::ReProcessing() { std::vector prefix(kTaTypeNum); std::istringstream iss; bool has_data = true; - s = env_->NewSequentialFile(whole_key_path, &wkey_input_f_, env_options_); + std::unique_ptr wkey_input_f; + + s = env_->NewSequentialFile(whole_key_path, &wkey_input_f, env_options_); if (!s.ok()) { fprintf(stderr, "Cannot open the whole key space file of CF: %u\n", cf_id); - wkey_input_f_.reset(); + wkey_input_f.reset(); } - if (wkey_input_f_) { + if (wkey_input_f) { + std::unique_ptr file; + file = NewLegacySequentialFileWrapper(wkey_input_f); for (cfs_[cf_id].w_count = 0; - ReadOneLine(&iss, wkey_input_f_.get(), &get_key, &has_data, &s); + ReadOneLine(&iss, file.get(), &get_key, &has_data, &s); ++cfs_[cf_id].w_count) { if (!s.ok()) { fprintf(stderr, "Read whole key space file failed\n"); diff --git a/tools/trace_analyzer_tool.h b/tools/trace_analyzer_tool.h index 4c3b973b7..0082b2bba 100644 --- a/tools/trace_analyzer_tool.h +++ b/tools/trace_analyzer_tool.h @@ -218,7 +218,6 @@ class TraceAnalyzer { std::unique_ptr trace_sequence_f_; // readable trace std::unique_ptr qps_f_; // overall qps std::unique_ptr cf_qps_f_; // The qps of each CF> - std::unique_ptr wkey_input_f_; std::vector ta_; // The main statistic collecting data structure std::map cfs_; // All the cf_id appears in this trace; std::vector qps_peak_; diff --git a/util/file_reader_writer_test.cc b/util/file_reader_writer_test.cc index 9a07bcccc..dd40e044b 100644 --- a/util/file_reader_writer_test.cc +++ b/util/file_reader_writer_test.cc @@ -5,6 +5,7 @@ // #include #include +#include "env/composite_env_wrapper.h" #include "file/random_access_file_reader.h" #include "file/readahead_raf.h" #include "file/sequence_file_reader.h" @@ -76,7 +77,8 @@ TEST_F(WritableFileWriterTest, RangeSync) { env_options.bytes_per_sync = kMb; std::unique_ptr wf(new FakeWF); std::unique_ptr writer( - new WritableFileWriter(std::move(wf), "" /* don't care */, env_options)); + new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(wf)), + "" /* don't care */, env_options)); Random r(301); std::unique_ptr large_buf(new char[10 * kMb]); for (int i = 0; i < 1000; i++) { @@ -157,8 +159,9 @@ TEST_F(WritableFileWriterTest, IncrementalBuffer) { false, #endif no_flush)); - std::unique_ptr writer(new WritableFileWriter( - std::move(wf), "" /* don't care */, env_options)); + std::unique_ptr writer( + new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(wf)), + "" /* don't care */, env_options)); std::string target; for (int i = 0; i < 20; i++) { @@ -212,12 +215,15 @@ TEST_F(WritableFileWriterTest, AppendStatusReturn) { std::unique_ptr wf(new FakeWF()); wf->Setuse_direct_io(true); std::unique_ptr writer( - new WritableFileWriter(std::move(wf), "" /* don't care */, EnvOptions())); + new WritableFileWriter(NewLegacyWritableFileWrapper(std::move(wf)), + "" /* don't care */, EnvOptions())); ASSERT_OK(writer->Append(std::string(2 * kMb, 'a'))); // Next call to WritableFile::Append() should fail - dynamic_cast(writer->writable_file())->SetIOError(true); + LegacyWritableFileWrapper* file = + static_cast(writer->writable_file()); + static_cast(file->target())->SetIOError(true); ASSERT_NOK(writer->Append(std::string(2 * kMb, 'b'))); } #endif @@ -345,8 +351,8 @@ class ReadaheadSequentialFileTest : public testing::Test, void ResetSourceStr(const std::string& str = "") { auto read_holder = std::unique_ptr(new test::SeqStringSource(str)); - test_read_holder_.reset(new SequentialFileReader(std::move(read_holder), - "test", readahead_size_)); + test_read_holder_.reset(new SequentialFileReader( + NewLegacySequentialFileWrapper(read_holder), "test", readahead_size_)); } size_t GetReadaheadSize() const { return readahead_size_; } diff --git a/utilities/backupable/backupable_db.cc b/utilities/backupable/backupable_db.cc index e85e4625a..af121f0dd 100644 --- a/utilities/backupable/backupable_db.cc +++ b/utilities/backupable/backupable_db.cc @@ -25,6 +25,7 @@ #include #include +#include "env/composite_env_wrapper.h" #include "file/filename.h" #include "file/sequence_file_reader.h" #include "file/writable_file_writer.h" @@ -1297,12 +1298,13 @@ Status BackupEngineImpl::CopyOrCreateFile( return s; } - std::unique_ptr dest_writer( - new WritableFileWriter(std::move(dst_file), dst, dst_env_options)); + std::unique_ptr dest_writer(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(dst_file)), dst, dst_env_options)); std::unique_ptr src_reader; std::unique_ptr buf; if (!src.empty()) { - src_reader.reset(new SequentialFileReader(std::move(src_file), src)); + src_reader.reset(new SequentialFileReader( + NewLegacySequentialFileWrapper(src_file), src)); buf.reset(new char[copy_file_buffer_size_]); } @@ -1499,7 +1501,7 @@ Status BackupEngineImpl::CalculateChecksum(const std::string& src, Env* src_env, } std::unique_ptr src_reader( - new SequentialFileReader(std::move(src_file), src)); + new SequentialFileReader(NewLegacySequentialFileWrapper(src_file), src)); std::unique_ptr buf(new char[copy_file_buffer_size_]); Slice data; @@ -1746,7 +1748,8 @@ Status BackupEngineImpl::BackupMeta::LoadFromFile( } std::unique_ptr backup_meta_reader( - new SequentialFileReader(std::move(backup_meta_file), meta_filename_)); + new SequentialFileReader(NewLegacySequentialFileWrapper(backup_meta_file), + meta_filename_)); std::unique_ptr buf(new char[max_backup_meta_file_size_ + 1]); Slice data; s = backup_meta_reader->Read(max_backup_meta_file_size_, &data, buf.get()); diff --git a/utilities/blob_db/blob_db_impl.cc b/utilities/blob_db/blob_db_impl.cc index d0c85c48f..bb597f305 100644 --- a/utilities/blob_db/blob_db_impl.cc +++ b/utilities/blob_db/blob_db_impl.cc @@ -15,6 +15,7 @@ #include "db/blob_index.h" #include "db/db_impl/db_impl.h" #include "db/write_batch_internal.h" +#include "env/composite_env_wrapper.h" #include "file/file_util.h" #include "file/filename.h" #include "file/random_access_file_reader.h" @@ -648,7 +649,8 @@ Status BlobDBImpl::CreateWriterLocked(const std::shared_ptr& bfile) { } std::unique_ptr fwriter; - fwriter.reset(new WritableFileWriter(std::move(wfile), fpath, env_options_)); + fwriter.reset(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(wfile)), fpath, env_options_)); uint64_t boffset = bfile->GetFileSize(); if (debug_level_ >= 2 && boffset) { diff --git a/utilities/blob_db/blob_db_test.cc b/utilities/blob_db/blob_db_test.cc index 0bb207167..0edcb9bb9 100644 --- a/utilities/blob_db/blob_db_test.cc +++ b/utilities/blob_db/blob_db_test.cc @@ -17,6 +17,7 @@ #include "db/blob_index.h" #include "db/db_test_util.h" +#include "env/composite_env_wrapper.h" #include "file/file_util.h" #include "file/sst_file_manager_impl.h" #include "port/port.h" @@ -904,10 +905,10 @@ TEST_F(BlobDBTest, SstFileManagerRestart) { Close(); // Create 3 dummy trash files under the blob_dir - CreateFile(db_options.env, blob_dir + "/000666.blob.trash", "", false); - CreateFile(db_options.env, blob_dir + "/000888.blob.trash", "", true); - CreateFile(db_options.env, blob_dir + "/something_not_match.trash", "", - false); + LegacyFileSystemWrapper fs(db_options.env); + CreateFile(&fs, blob_dir + "/000666.blob.trash", "", false); + CreateFile(&fs, blob_dir + "/000888.blob.trash", "", true); + CreateFile(&fs, blob_dir + "/something_not_match.trash", "", false); // Make sure that reopening the DB rescan the existing trash files Open(bdb_options, db_options); diff --git a/utilities/blob_db/blob_dump_tool.cc b/utilities/blob_db/blob_dump_tool.cc index 9905ce12e..69e7f19b5 100644 --- a/utilities/blob_db/blob_dump_tool.cc +++ b/utilities/blob_db/blob_dump_tool.cc @@ -10,6 +10,7 @@ #include #include #include +#include "env/composite_env_wrapper.h" #include "file/random_access_file_reader.h" #include "file/readahead_raf.h" #include "port/port.h" @@ -50,7 +51,8 @@ Status BlobDumpTool::Run(const std::string& filename, DisplayType show_key, if (file_size == 0) { return Status::Corruption("File is empty."); } - reader_.reset(new RandomAccessFileReader(std::move(file), filename)); + reader_.reset(new RandomAccessFileReader( + NewLegacyRandomAccessFileWrapper(file), filename)); uint64_t offset = 0; uint64_t footer_offset = 0; CompressionType compression = kNoCompression; diff --git a/utilities/blob_db/blob_file.cc b/utilities/blob_db/blob_file.cc index 2aa0922dc..999f8d5b2 100644 --- a/utilities/blob_db/blob_file.cc +++ b/utilities/blob_db/blob_file.cc @@ -15,6 +15,7 @@ #include "db/column_family.h" #include "db/db_impl/db_impl.h" #include "db/dbformat.h" +#include "env/composite_env_wrapper.h" #include "file/filename.h" #include "file/readahead_raf.h" #include "logging/logging.h" @@ -74,7 +75,8 @@ std::shared_ptr BlobFile::OpenRandomAccessReader( sfile = NewReadaheadRandomAccessFile(std::move(sfile), kReadaheadSize); std::unique_ptr sfile_reader; - sfile_reader.reset(new RandomAccessFileReader(std::move(sfile), path_name)); + sfile_reader.reset(new RandomAccessFileReader( + NewLegacyRandomAccessFileWrapper(sfile), path_name)); std::shared_ptr log_reader = std::make_shared( std::move(sfile_reader), db_options.env, db_options.statistics.get()); @@ -209,8 +211,8 @@ Status BlobFile::GetReader(Env* env, const EnvOptions& env_options, return s; } - ra_file_reader_ = std::make_shared(std::move(rfile), - PathName()); + ra_file_reader_ = std::make_shared( + NewLegacyRandomAccessFileWrapper(rfile), PathName()); *reader = ra_file_reader_; *fresh_open = true; return s; @@ -248,7 +250,8 @@ Status BlobFile::ReadMetadata(Env* env, const EnvOptions& env_options) { return s; } std::unique_ptr file_reader( - new RandomAccessFileReader(std::move(file), PathName())); + new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(file), + PathName())); // Read file header. char header_buf[BlobLogHeader::kSize]; diff --git a/utilities/checkpoint/checkpoint_impl.cc b/utilities/checkpoint/checkpoint_impl.cc index 35344a988..33cf9ae1d 100644 --- a/utilities/checkpoint/checkpoint_impl.cc +++ b/utilities/checkpoint/checkpoint_impl.cc @@ -111,20 +111,21 @@ Status CheckpointImpl::CreateCheckpoint(const std::string& checkpoint_dir, [&](const std::string& src_dirname, const std::string& fname, FileType) { ROCKS_LOG_INFO(db_options.info_log, "Hard Linking %s", fname.c_str()); - return db_->GetEnv()->LinkFile(src_dirname + fname, - full_private_path + fname); + return db_->GetFileSystem()->LinkFile(src_dirname + fname, + full_private_path + fname, + IOOptions(), nullptr); } /* link_file_cb */, [&](const std::string& src_dirname, const std::string& fname, uint64_t size_limit_bytes, FileType) { ROCKS_LOG_INFO(db_options.info_log, "Copying %s", fname.c_str()); - return CopyFile(db_->GetEnv(), src_dirname + fname, + return CopyFile(db_->GetFileSystem(), src_dirname + fname, full_private_path + fname, size_limit_bytes, db_options.use_fsync); } /* copy_file_cb */, [&](const std::string& fname, const std::string& contents, FileType) { ROCKS_LOG_INFO(db_options.info_log, "Creating %s", fname.c_str()); - return CreateFile(db_->GetEnv(), full_private_path + fname, contents, - db_options.use_fsync); + return CreateFile(db_->GetFileSystem(), full_private_path + fname, + contents, db_options.use_fsync); } /* create_file_cb */, &sequence_number, log_size_for_flush); // we copied all the files, enable file deletions @@ -383,7 +384,7 @@ Status CheckpointImpl::ExportColumnFamily( [&](const std::string& src_dirname, const std::string& fname) { ROCKS_LOG_INFO(db_options.info_log, "[%s] Copying %s", cf_name.c_str(), fname.c_str()); - return CopyFile(db_->GetEnv(), src_dirname + fname, + return CopyFile(db_->GetFileSystem(), src_dirname + fname, tmp_export_dir + fname, 0, db_options.use_fsync); } /*copy_file_cb*/); diff --git a/utilities/options/options_util.cc b/utilities/options/options_util.cc index 561e925eb..53acf7bc1 100644 --- a/utilities/options/options_util.cc +++ b/utilities/options/options_util.cc @@ -7,6 +7,7 @@ #include "rocksdb/utilities/options_util.h" +#include "env/composite_env_wrapper.h" #include "file/filename.h" #include "options/options_parser.h" #include "rocksdb/options.h" @@ -18,7 +19,8 @@ Status LoadOptionsFromFile(const std::string& file_name, Env* env, bool ignore_unknown_options, std::shared_ptr* cache) { RocksDBOptionsParser parser; - Status s = parser.Parse(file_name, env, ignore_unknown_options); + LegacyFileSystemWrapper fs(env); + Status s = parser.Parse(file_name, &fs, ignore_unknown_options); if (!s.ok()) { return s; } @@ -100,9 +102,10 @@ Status CheckOptionsCompatibility( } const OptionsSanityCheckLevel kDefaultLevel = kSanityLevelLooselyCompatible; + LegacyFileSystemWrapper fs(env); return RocksDBOptionsParser::VerifyRocksDBOptionsFromFile( - db_options, cf_names, cf_opts, dbpath + "/" + options_file_name, env, + db_options, cf_names, cf_opts, dbpath + "/" + options_file_name, &fs, kDefaultLevel, ignore_unknown_options); } diff --git a/utilities/options/options_util_test.cc b/utilities/options/options_util_test.cc index 3926275af..cebaf9e3f 100644 --- a/utilities/options/options_util_test.cc +++ b/utilities/options/options_util_test.cc @@ -31,11 +31,13 @@ class OptionsUtilTest : public testing::Test { public: OptionsUtilTest() : rnd_(0xFB) { env_.reset(new test::StringEnv(Env::Default())); + fs_.reset(new LegacyFileSystemWrapper(env_.get())); dbname_ = test::PerThreadDBPath("options_util_test"); } protected: std::unique_ptr env_; + std::unique_ptr fs_; std::string dbname_; Random rnd_; }; @@ -59,7 +61,7 @@ TEST_F(OptionsUtilTest, SaveAndLoad) { } const std::string kFileName = "OPTIONS-123456"; - PersistRocksDBOptions(db_opt, cf_names, cf_opts, kFileName, env_.get()); + PersistRocksDBOptions(db_opt, cf_names, cf_opts, kFileName, fs_.get()); DBOptions loaded_db_opt; std::vector loaded_cf_descs; @@ -120,7 +122,7 @@ TEST_F(OptionsUtilTest, SaveAndLoadWithCacheCheck) { cf_names.push_back("cf_plain_table_sample"); // Saving DB in file const std::string kFileName = "OPTIONS-LOAD_CACHE_123456"; - PersistRocksDBOptions(db_opt, cf_names, cf_opts, kFileName, env_.get()); + PersistRocksDBOptions(db_opt, cf_names, cf_opts, kFileName, fs_.get()); DBOptions loaded_db_opt; std::vector loaded_cf_descs; ASSERT_OK(LoadOptionsFromFile(kFileName, env_.get(), &loaded_db_opt, diff --git a/utilities/persistent_cache/block_cache_tier_file.cc b/utilities/persistent_cache/block_cache_tier_file.cc index 58671e2bb..8e6fb0f81 100644 --- a/utilities/persistent_cache/block_cache_tier_file.cc +++ b/utilities/persistent_cache/block_cache_tier_file.cc @@ -13,6 +13,7 @@ #include #include +#include "env/composite_env_wrapper.h" #include "logging/logging.h" #include "port/port.h" #include "util/crc32c.h" @@ -217,7 +218,8 @@ bool RandomAccessCacheFile::OpenImpl(const bool enable_direct_reads) { status.ToString().c_str()); return false; } - freader_.reset(new RandomAccessFileReader(std::move(file), Path(), env_)); + freader_.reset(new RandomAccessFileReader( + NewLegacyRandomAccessFileWrapper(file), Path(), env_)); return true; } diff --git a/utilities/persistent_cache/block_cache_tier_file.h b/utilities/persistent_cache/block_cache_tier_file.h index ddea4032c..b93f8f4b5 100644 --- a/utilities/persistent_cache/block_cache_tier_file.h +++ b/utilities/persistent_cache/block_cache_tier_file.h @@ -136,7 +136,7 @@ class BlockCacheFile : public LRUElement { protected: port::RWMutex rwlock_; // synchronization mutex - Env* const env_ = nullptr; // Env for IO + Env* const env_ = nullptr; // Env for OS const std::string dir_; // Directory name const uint32_t cache_id_; // Cache id for the file std::list block_infos_; // List of index entries mapping to the diff --git a/utilities/simulator_cache/sim_cache.cc b/utilities/simulator_cache/sim_cache.cc index cf0390a1b..cf534dddb 100644 --- a/utilities/simulator_cache/sim_cache.cc +++ b/utilities/simulator_cache/sim_cache.cc @@ -5,6 +5,7 @@ #include "rocksdb/utilities/sim_cache.h" #include +#include "env/composite_env_wrapper.h" #include "file/writable_file_writer.h" #include "monitoring/statistics.h" #include "port/port.h" @@ -46,8 +47,9 @@ class CacheActivityLogger { if (!status.ok()) { return status; } - file_writer_.reset(new WritableFileWriter(std::move(log_file), - activity_log_file, env_opts)); + file_writer_.reset(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(log_file)), activity_log_file, + env_opts)); max_logging_size_ = max_logging_size; activity_logging_enabled_.store(true); diff --git a/utilities/trace/file_trace_reader_writer.cc b/utilities/trace/file_trace_reader_writer.cc index 2910745e2..17506c7ff 100644 --- a/utilities/trace/file_trace_reader_writer.cc +++ b/utilities/trace/file_trace_reader_writer.cc @@ -5,6 +5,7 @@ #include "utilities/trace/file_trace_reader_writer.h" +#include "env/composite_env_wrapper.h" #include "file/random_access_file_reader.h" #include "file/writable_file_writer.h" #include "trace_replay/trace_replay.h" @@ -96,8 +97,8 @@ Status NewFileTraceReader(Env* env, const EnvOptions& env_options, } std::unique_ptr file_reader; - file_reader.reset( - new RandomAccessFileReader(std::move(trace_file), trace_filename)); + file_reader.reset(new RandomAccessFileReader( + NewLegacyRandomAccessFileWrapper(trace_file), trace_filename)); trace_reader->reset(new FileTraceReader(std::move(file_reader))); return s; } @@ -112,8 +113,9 @@ Status NewFileTraceWriter(Env* env, const EnvOptions& env_options, } std::unique_ptr file_writer; - file_writer.reset(new WritableFileWriter(std::move(trace_file), - trace_filename, env_options)); + file_writer.reset(new WritableFileWriter( + NewLegacyWritableFileWrapper(std::move(trace_file)), trace_filename, + env_options)); trace_writer->reset(new FileTraceWriter(std::move(file_writer))); return s; }