From 4aa46ac5dc12e9e49cf916542df1648133f7479b Mon Sep 17 00:00:00 2001 From: Peter Dillinger Date: Fri, 10 Jan 2025 13:28:05 -0800 Subject: [PATCH] Clean up some CFOptions code hygiene, unnecessary copies Summary: To start, I wanted to remove the unnecessary new_options parameter of `InstallSuperVersionAndScheduleWork()`. Passing it something other than the current mutable options would be inconsistent/outdated. There was even a comment "Use latest MutableCFOptions" on a place that was using the saved options in effect for the compaction. And remove some unnecessary copying of entire MutableCFOptions while holding DB mutex, as in `DBImpl::ReFitLevel()`, `DBImplFollower::TryCatchUpWithLeader()`, `DBImplSecondary::RecoverLogFiles()`. (`ConstructNewMemtable()` doesn't save a reference.) Intended follow-up: Clarify/simplify the crazy calling conventions of LogAndApply (see new FIXME) Test Plan: existing tests and CI --- db/arena_wrapped_db_iter.cc | 2 +- db/column_family.cc | 13 +- db/column_family.h | 13 +- db/compaction/compaction.cc | 24 +-- db/compaction/compaction.h | 8 +- db/compaction/compaction_iterator.h | 8 +- db/compaction/compaction_job.cc | 72 ++++----- db/compaction/compaction_job_test.cc | 12 +- db/compaction/compaction_outputs.cc | 22 ++- db/compaction/compaction_service_job.cc | 24 +-- db/compaction/compaction_service_test.cc | 4 +- db/compaction/subcompaction_state.cc | 2 +- db/db_filesnapshot.cc | 2 +- db/db_impl/db_impl.cc | 100 ++++++------ db/db_impl/db_impl.h | 7 +- db/db_impl/db_impl_compaction_flush.cc | 172 ++++++++++----------- db/db_impl/db_impl_debug.cc | 8 +- db/db_impl/db_impl_experimental.cc | 9 +- db/db_impl/db_impl_files.cc | 2 +- db/db_impl/db_impl_follower.cc | 4 +- db/db_impl/db_impl_open.cc | 47 +++--- db/db_impl/db_impl_readonly.cc | 4 +- db/db_impl/db_impl_secondary.cc | 23 ++- db/db_impl/db_impl_write.cc | 20 +-- db/external_sst_file_ingestion_job.cc | 14 +- db/flush_job.cc | 34 ++-- db/flush_job_test.cc | 36 ++--- db/forward_iterator.cc | 2 +- db/import_column_family_job.cc | 14 +- db/internal_stats.cc | 13 +- db/obsolete_files_test.cc | 7 +- db/repair.cc | 20 +-- db/version_builder.cc | 4 +- db/version_edit_handler.cc | 29 ++-- db/version_set.cc | 107 +++++++------ db/version_set.h | 15 +- db/version_set_test.cc | 7 +- db/version_util.h | 2 +- db/write_batch.cc | 3 +- file/sst_file_manager_impl.cc | 2 +- include/rocksdb/utilities/ldb_cmd.h | 2 +- tools/ldb_tool.cc | 2 +- utilities/blob_db/blob_db_impl.cc | 7 +- utilities/transactions/transaction_test.cc | 6 +- 44 files changed, 446 insertions(+), 482 deletions(-) diff --git a/db/arena_wrapped_db_iter.cc b/db/arena_wrapped_db_iter.cc index 22be567fdb0..27cae5eb199 100644 --- a/db/arena_wrapped_db_iter.cc +++ b/db/arena_wrapped_db_iter.cc @@ -97,7 +97,7 @@ Status ArenaWrappedDBIter::Refresh(const Snapshot* snapshot) { if (read_callback_) { read_callback_->Refresh(read_seq); } - Init(env, read_options_, *(cfd->ioptions()), sv->mutable_cf_options, + Init(env, read_options_, cfd->ioptions(), sv->mutable_cf_options, sv->current, read_seq, sv->mutable_cf_options.max_sequential_skip_in_iterations, sv->version_number, read_callback_, cfh_, expose_blob_index_, diff --git a/db/column_family.cc b/db/column_family.cc index 8983c1f0e0f..f85e7d6431a 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -53,7 +53,7 @@ ColumnFamilyHandleImpl::ColumnFamilyHandleImpl( ColumnFamilyHandleImpl::~ColumnFamilyHandleImpl() { if (cfd_ != nullptr) { - for (auto& listener : cfd_->ioptions()->listeners) { + for (auto& listener : cfd_->ioptions().listeners) { listener->OnColumnFamilyHandleDeletionStarted(this); } // Job id == 0 means that this is not our background process, but rather @@ -593,7 +593,7 @@ ColumnFamilyData::ColumnFamilyData( block_cache_tracer, io_tracer, db_session_id)); blob_file_cache_.reset( - new BlobFileCache(_table_cache, ioptions(), soptions(), id_, + new BlobFileCache(_table_cache, &ioptions(), soptions(), id_, internal_stats_->GetBlobFileReadHist(), io_tracer)); blob_source_.reset(new BlobSource(ioptions_, mutable_cf_options_, db_id, db_session_id, blob_file_cache_.get())); @@ -968,7 +968,7 @@ WriteStallCondition ColumnFamilyData::RecalculateWriteStallConditions( auto write_stall_condition_and_cause = GetWriteStallConditionAndCause( imm()->NumNotFlushed(), vstorage->l0_delay_trigger_count(), vstorage->estimated_compaction_needed_bytes(), mutable_cf_options, - *ioptions()); + ioptions()); write_stall_condition = write_stall_condition_and_cause.first; auto write_stall_cause = write_stall_condition_and_cause.second; @@ -1384,7 +1384,7 @@ void ColumnFamilyData::InstallSuperVersion( new_superversion->write_stall_condition) { sv_context->PushWriteStallNotification( old_superversion->write_stall_condition, - new_superversion->write_stall_condition, GetName(), ioptions()); + new_superversion->write_stall_condition, GetName(), &ioptions()); } if (old_superversion->Unref()) { old_superversion->Cleanup(); @@ -1844,10 +1844,7 @@ const ImmutableOptions& GetImmutableOptions(ColumnFamilyHandle* column_family) { const ColumnFamilyData* const cfd = handle->cfd(); assert(cfd); - const ImmutableOptions* ioptions = cfd->ioptions(); - assert(ioptions); - - return *ioptions; + return cfd->ioptions(); } } // namespace ROCKSDB_NAMESPACE diff --git a/db/column_family.h b/db/column_family.h index 4cb6ab59658..2d9fb28a3d5 100644 --- a/db/column_family.h +++ b/db/column_family.h @@ -221,7 +221,8 @@ struct SuperVersion { // enable UDT feature, this is an empty string. std::string full_history_ts_low; - // A shared copy of the DB's seqno to time mapping. + // An immutable snapshot of the DB's seqno to time mapping, usually shared + // between SuperVersions. std::shared_ptr seqno_to_time_mapping{nullptr}; // should be called outside the mutex @@ -342,17 +343,17 @@ class ColumnFamilyData { // thread-safe const FileOptions* soptions() const; - const ImmutableOptions* ioptions() const { return &ioptions_; } + const ImmutableOptions& ioptions() const { return ioptions_; } // REQUIRES: DB mutex held // This returns the MutableCFOptions used by current SuperVersion // You should use this API to reference MutableCFOptions most of the time. - const MutableCFOptions* GetCurrentMutableCFOptions() const { - return &(super_version_->mutable_cf_options); + const MutableCFOptions& GetCurrentMutableCFOptions() const { + return super_version_->mutable_cf_options; } // REQUIRES: DB mutex held // This returns the latest MutableCFOptions, which may be not in effect yet. - const MutableCFOptions* GetLatestMutableCFOptions() const { - return &mutable_cf_options_; + const MutableCFOptions& GetLatestMutableCFOptions() const { + return mutable_cf_options_; } // REQUIRES: DB mutex held diff --git a/db/compaction/compaction.cc b/db/compaction/compaction.cc index b6b9a5922a9..313e2998aec 100644 --- a/db/compaction/compaction.cc +++ b/db/compaction/compaction.cc @@ -332,11 +332,11 @@ Compaction::Compaction( : (_blob_garbage_collection_policy == BlobGarbageCollectionPolicy::kDisable ? false - : mutable_cf_options()->enable_blob_garbage_collection)), + : mutable_cf_options().enable_blob_garbage_collection)), blob_garbage_collection_age_cutoff_( _blob_garbage_collection_age_cutoff < 0 || _blob_garbage_collection_age_cutoff > 1 - ? mutable_cf_options()->blob_garbage_collection_age_cutoff + ? mutable_cf_options().blob_garbage_collection_age_cutoff : _blob_garbage_collection_age_cutoff), penultimate_level_( // For simplicity, we don't support the concept of "penultimate level" @@ -592,7 +592,7 @@ bool Compaction::IsTrivialMove() const { // input files are non overlapping if ((mutable_cf_options_.compaction_options_universal.allow_trivial_move) && (output_level_ != 0) && - (cfd_->ioptions()->compaction_style == kCompactionStyleUniversal)) { + (cfd_->ioptions().compaction_style == kCompactionStyleUniversal)) { return is_trivial_move_; } @@ -650,7 +650,7 @@ bool Compaction::KeyNotExistsBeyondOutputLevel( if (bottommost_level_) { return true; } else if (output_level_ != 0 && - cfd_->ioptions()->compaction_style == kCompactionStyleLevel) { + cfd_->ioptions().compaction_style == kCompactionStyleLevel) { // Maybe use binary search to find right entry instead of linear search? const Comparator* user_cmp = cfd_->user_comparator(); for (int lvl = output_level_ + 1; lvl < number_levels_; lvl++) { @@ -691,7 +691,7 @@ bool Compaction::KeyRangeNotExistsBeyondOutputLevel( if (bottommost_level_) { return true /* does not overlap */; } else if (output_level_ != 0 && - cfd_->ioptions()->compaction_style == kCompactionStyleLevel) { + cfd_->ioptions().compaction_style == kCompactionStyleLevel) { const Comparator* user_cmp = cfd_->user_comparator(); for (int lvl = output_level_ + 1; lvl < number_levels_; lvl++) { const std::vector& files = @@ -867,12 +867,12 @@ uint64_t Compaction::OutputFilePreallocationSize() const { } std::unique_ptr Compaction::CreateCompactionFilter() const { - if (!cfd_->ioptions()->compaction_filter_factory) { + if (!cfd_->ioptions().compaction_filter_factory) { return nullptr; } if (!cfd_->ioptions() - ->compaction_filter_factory->ShouldFilterTableFileCreation( + .compaction_filter_factory->ShouldFilterTableFileCreation( TableFileCreationReason::kCompaction)) { return nullptr; } @@ -891,7 +891,7 @@ std::unique_ptr Compaction::CreateCompactionFilter() const { "for compaction."); } - return cfd_->ioptions()->compaction_filter_factory->CreateCompactionFilter( + return cfd_->ioptions().compaction_filter_factory->CreateCompactionFilter( context); } @@ -925,8 +925,8 @@ bool Compaction::ShouldFormSubcompactions() const { // Round-Robin pri under leveled compaction allows subcompactions by default // and the number of subcompactions can be larger than max_subcompactions_ - if (cfd_->ioptions()->compaction_pri == kRoundRobin && - cfd_->ioptions()->compaction_style == kCompactionStyleLevel) { + if (cfd_->ioptions().compaction_pri == kRoundRobin && + cfd_->ioptions().compaction_style == kCompactionStyleLevel) { return output_level_ > 0; } @@ -934,9 +934,9 @@ bool Compaction::ShouldFormSubcompactions() const { return false; } - if (cfd_->ioptions()->compaction_style == kCompactionStyleLevel) { + if (cfd_->ioptions().compaction_style == kCompactionStyleLevel) { return (start_level_ == 0 || is_manual_compaction_) && output_level_ > 0; - } else if (cfd_->ioptions()->compaction_style == kCompactionStyleUniversal) { + } else if (cfd_->ioptions().compaction_style == kCompactionStyleUniversal) { return number_levels_ > 1 && output_level_ > 0; } else { return false; diff --git a/db/compaction/compaction.h b/db/compaction/compaction.h index d707b49c419..534b13c6a8f 100644 --- a/db/compaction/compaction.h +++ b/db/compaction/compaction.h @@ -288,14 +288,14 @@ class Compaction { // Return the ImmutableOptions that should be used throughout the compaction // procedure - const ImmutableOptions* immutable_options() const { - return &immutable_options_; + const ImmutableOptions& immutable_options() const { + return immutable_options_; } // Return the MutableCFOptions that should be used throughout the compaction // procedure - const MutableCFOptions* mutable_cf_options() const { - return &mutable_cf_options_; + const MutableCFOptions& mutable_cf_options() const { + return mutable_cf_options_; } // Returns the size in bytes that the output file should be preallocated to. diff --git a/db/compaction/compaction_iterator.h b/db/compaction/compaction_iterator.h index 3cf0ce0d3bd..c3e4942ac34 100644 --- a/db/compaction/compaction_iterator.h +++ b/db/compaction/compaction_iterator.h @@ -124,8 +124,6 @@ class CompactionIterator { explicit RealCompaction(const Compaction* compaction) : compaction_(compaction) { assert(compaction_); - assert(compaction_->immutable_options()); - assert(compaction_->mutable_cf_options()); } int level() const override { return compaction_->level(); } @@ -147,11 +145,11 @@ class CompactionIterator { } bool allow_ingest_behind() const override { - return compaction_->immutable_options()->allow_ingest_behind; + return compaction_->immutable_options().allow_ingest_behind; } bool allow_mmap_reads() const override { - return compaction_->immutable_options()->allow_mmap_reads; + return compaction_->immutable_options().allow_mmap_reads; } bool enable_blob_garbage_collection() const override { @@ -163,7 +161,7 @@ class CompactionIterator { } uint64_t blob_compaction_readahead_size() const override { - return compaction_->mutable_cf_options()->blob_compaction_readahead_size; + return compaction_->mutable_cf_options().blob_compaction_readahead_size; } const Version* input_version() const override { diff --git a/db/compaction/compaction_job.cc b/db/compaction/compaction_job.cc index 887688ad90f..97721551bd3 100644 --- a/db/compaction/compaction_job.cc +++ b/db/compaction/compaction_job.cc @@ -301,8 +301,8 @@ void CompactionJob::Prepare( SequenceNumber preserve_time_min_seqno = kMaxSequenceNumber; SequenceNumber preclude_last_level_min_seqno = kMaxSequenceNumber; uint64_t preserve_time_duration = - std::max(c->mutable_cf_options()->preserve_internal_time_seconds, - c->mutable_cf_options()->preclude_last_level_data_seconds); + std::max(c->mutable_cf_options().preserve_internal_time_seconds, + c->mutable_cf_options().preclude_last_level_data_seconds); if (preserve_time_duration > 0) { const ReadOptions read_options(Env::IOActivity::kCompaction); @@ -339,8 +339,8 @@ void CompactionJob::Prepare( seqno_to_time_mapping_.Enforce(_current_time); seqno_to_time_mapping_.GetCurrentTieringCutoffSeqnos( static_cast(_current_time), - c->mutable_cf_options()->preserve_internal_time_seconds, - c->mutable_cf_options()->preclude_last_level_data_seconds, + c->mutable_cf_options().preserve_internal_time_seconds, + c->mutable_cf_options().preclude_last_level_data_seconds, &preserve_time_min_seqno, &preclude_last_level_min_seqno); } // For accuracy of the GetProximalSeqnoBeforeTime queries above, we only @@ -519,14 +519,14 @@ void CompactionJob::GenSubcompactionBoundaries() { ReadOptions read_options(Env::IOActivity::kCompaction); read_options.rate_limiter_priority = GetRateLimiterPriority(); auto* c = compact_->compaction; - if (c->mutable_cf_options()->table_factory->Name() == + if (c->mutable_cf_options().table_factory->Name() == TableFactory::kPlainTableName()) { return; } if (c->max_subcompactions() <= 1 && - !(c->immutable_options()->compaction_pri == kRoundRobin && - c->immutable_options()->compaction_style == kCompactionStyleLevel)) { + !(c->immutable_options().compaction_pri == kRoundRobin && + c->immutable_options().compaction_style == kCompactionStyleLevel)) { return; } auto* cfd = c->column_family_data(); @@ -556,7 +556,7 @@ void CompactionJob::GenSubcompactionBoundaries() { FileMetaData* f = flevel->files[i].file_metadata; std::vector my_anchors; Status s = cfd->table_cache()->ApproximateKeyAnchors( - read_options, icomp, *f, *c->mutable_cf_options(), my_anchors); + read_options, icomp, *f, c->mutable_cf_options(), my_anchors); if (!s.ok() || my_anchors.empty()) { my_anchors.emplace_back(f->largest.user_key(), f->fd.GetFileSize()); } @@ -595,8 +595,8 @@ void CompactionJob::GenSubcompactionBoundaries() { // Get the number of planned subcompactions, may update reserve threads // and update extra_num_subcompaction_threads_reserved_ for round-robin uint64_t num_planned_subcompactions; - if (c->immutable_options()->compaction_pri == kRoundRobin && - c->immutable_options()->compaction_style == kCompactionStyleLevel) { + if (c->immutable_options().compaction_pri == kRoundRobin && + c->immutable_options().compaction_style == kCompactionStyleLevel) { // For round-robin compaction prioity, we need to employ more // subcompactions (may exceed the max_subcompaction limit). The extra // subcompactions will be executed using reserved threads and taken into @@ -635,9 +635,9 @@ void CompactionJob::GenSubcompactionBoundaries() { uint64_t target_range_size = std::max( total_size / num_planned_subcompactions, MaxFileSizeForLevel( - *(c->mutable_cf_options()), out_lvl, - c->immutable_options()->compaction_style, base_level, - c->immutable_options()->level_compaction_dynamic_level_bytes)); + c->mutable_cf_options(), out_lvl, + c->immutable_options().compaction_style, base_level, + c->immutable_options().level_compaction_dynamic_level_bytes)); if (target_range_size >= total_size) { return; @@ -780,14 +780,13 @@ Status CompactionJob::Run() { verify_table_read_options, file_options_, cfd->internal_comparator(), files_output[file_idx]->meta, /*range_del_agg=*/nullptr, - *compact_->compaction->mutable_cf_options(), + compact_->compaction->mutable_cf_options(), /*table_reader_ptr=*/nullptr, cfd->internal_stats()->GetFileReadHist( compact_->compaction->output_level()), TableReaderCaller::kCompactionRefill, /*arena=*/nullptr, /*skip_filters=*/false, compact_->compaction->output_level(), - MaxFileSizeForL0MetaPin( - *compact_->compaction->mutable_cf_options()), + MaxFileSizeForL0MetaPin(compact_->compaction->mutable_cf_options()), /*smallest_compaction_key=*/nullptr, /*largest_compaction_key=*/nullptr, /*allow_unprepared_value=*/false); @@ -844,7 +843,7 @@ Status CompactionJob::Run() { for (const auto& state : compact_->sub_compact_states) { for (const auto& output : state.GetOutputs()) { auto fn = - TableFileName(state.compaction->immutable_options()->cf_paths, + TableFileName(state.compaction->immutable_options().cf_paths, output.meta.fd.GetNumber(), output.meta.fd.GetPathId()); compact_->compaction->SetOutputTableProperties(fn, output.table_properties); @@ -1145,8 +1144,7 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) { // Create compaction filter and fail the compaction if // IgnoreSnapshots() = false because it is not supported anymore - const CompactionFilter* compaction_filter = - cfd->ioptions()->compaction_filter; + const CompactionFilter* compaction_filter = cfd->ioptions().compaction_filter; std::unique_ptr compaction_filter_from_factory = nullptr; if (compaction_filter == nullptr) { compaction_filter_from_factory = @@ -1304,28 +1302,27 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) { } MergeHelper merge( - env_, cfd->user_comparator(), cfd->ioptions()->merge_operator.get(), + env_, cfd->user_comparator(), cfd->ioptions().merge_operator.get(), compaction_filter, db_options_.info_log.get(), false /* internal key corruption is expected */, existing_snapshots_.empty() ? 0 : existing_snapshots_.back(), snapshot_checker_, compact_->compaction->level(), db_options_.stats); - const MutableCFOptions* mutable_cf_options = + const auto& mutable_cf_options = sub_compact->compaction->mutable_cf_options(); - assert(mutable_cf_options); std::vector blob_file_paths; // TODO: BlobDB to support output_to_penultimate_level compaction, which needs // 2 builders, so may need to move to `CompactionOutputs` std::unique_ptr blob_file_builder( - (mutable_cf_options->enable_blob_files && + (mutable_cf_options.enable_blob_files && sub_compact->compaction->output_level() >= - mutable_cf_options->blob_file_starting_level) + mutable_cf_options.blob_file_starting_level) ? new BlobFileBuilder( versions_, fs_.get(), - sub_compact->compaction->immutable_options(), - mutable_cf_options, &file_options_, &write_options, db_id_, + &sub_compact->compaction->immutable_options(), + &mutable_cf_options, &file_options_, &write_options, db_id_, db_session_id_, job_id_, cfd->GetID(), cfd->GetName(), write_hint_, io_tracer_, blob_callback_, BlobFileCreationReason::kCompaction, &blob_file_paths, @@ -1728,7 +1725,7 @@ Status CompactionJob::FinishCompactionOutputFile( // This happens when the output level is bottom level, at the same time // the sub_compact output nothing. std::string fname = - TableFileName(sub_compact->compaction->immutable_options()->cf_paths, + TableFileName(sub_compact->compaction->immutable_options().cf_paths, meta->fd.GetNumber(), meta->fd.GetPathId()); // TODO(AR) it is not clear if there are any larger implications if @@ -1775,7 +1772,7 @@ Status CompactionJob::FinishCompactionOutputFile( } } EventHelpers::LogAndNotifyTableFileCreationFinished( - event_logger_, cfd->ioptions()->listeners, dbname_, cfd->GetName(), fname, + event_logger_, cfd->ioptions().listeners, dbname_, cfd->GetName(), fname, job_id_, output_fd, oldest_blob_file_number, tp, TableFileCreationReason::kCompaction, status_for_listener, file_checksum, file_checksum_func_name); @@ -1878,7 +1875,7 @@ Status CompactionJob::InstallCompactionResults( if ((compaction->compaction_reason() == CompactionReason::kLevelMaxLevelSize || compaction->compaction_reason() == CompactionReason::kRoundRobinTtl) && - compaction->immutable_options()->compaction_pri == kRoundRobin) { + compaction->immutable_options().compaction_pri == kRoundRobin) { int start_level = compaction->start_level(); if (start_level > 0) { auto vstorage = compaction->input_version()->storage_info(); @@ -1933,7 +1930,7 @@ Status CompactionJob::OpenCompactionOutputFile(SubcompactionState* sub_compact, // Fire events. ColumnFamilyData* cfd = sub_compact->compaction->column_family_data(); EventHelpers::NotifyTableFileCreationStarted( - cfd->ioptions()->listeners, dbname_, cfd->GetName(), fname, job_id_, + cfd->ioptions().listeners, dbname_, cfd->GetName(), fname, job_id_, TableFileCreationReason::kCompaction); // Make the output file std::unique_ptr writable_file; @@ -1947,7 +1944,7 @@ Status CompactionJob::OpenCompactionOutputFile(SubcompactionState* sub_compact, FileOptions fo_copy = file_options_; Temperature temperature = sub_compact->compaction->output_temperature(); Temperature last_level_temp = - sub_compact->compaction->mutable_cf_options()->last_level_temperature; + sub_compact->compaction->mutable_cf_options().last_level_temperature; // Here last_level_temperature supersedes default_write_temperature, when // enabled and applicable if (last_level_temp != Temperature::kUnknown && @@ -1975,7 +1972,7 @@ Status CompactionJob::OpenCompactionOutputFile(SubcompactionState* sub_compact, job_id_, file_number, s.ToString().c_str()); LogFlush(db_options_.info_log); EventHelpers::LogAndNotifyTableFileCreationFinished( - event_logger_, cfd->ioptions()->listeners, dbname_, cfd->GetName(), + event_logger_, cfd->ioptions().listeners, dbname_, cfd->GetName(), fname, job_id_, FileDescriptor(), kInvalidBlobFileNumber, TableProperties(), TableFileCreationReason::kCompaction, s, kUnknownFileChecksum, kUnknownFileChecksumFuncName); @@ -2046,7 +2043,7 @@ Status CompactionJob::OpenCompactionOutputFile(SubcompactionState* sub_compact, writable_file->SetPreallocationBlockSize(static_cast( sub_compact->compaction->OutputFilePreallocationSize())); const auto& listeners = - sub_compact->compaction->immutable_options()->listeners; + sub_compact->compaction->immutable_options().listeners; outputs.AssignFileWriter(new WritableFileWriter( std::move(writable_file), fname, fo_copy, db_options_.clock, io_tracer_, db_options_.stats, Histograms::SST_WRITE_MICROS, listeners, @@ -2057,7 +2054,7 @@ Status CompactionJob::OpenCompactionOutputFile(SubcompactionState* sub_compact, const ReadOptions read_options(Env::IOActivity::kCompaction); const WriteOptions write_options(Env::IOActivity::kCompaction); TableBuilderOptions tboptions( - *cfd->ioptions(), *(sub_compact->compaction->mutable_cf_options()), + cfd->ioptions(), sub_compact->compaction->mutable_cf_options(), read_options, write_options, cfd->internal_comparator(), cfd->internal_tbl_prop_coll_factories(), sub_compact->compaction->output_compression(), @@ -2123,9 +2120,8 @@ bool CompactionJob::UpdateCompactionStats(uint64_t* num_input_range_del) { if (file_input_entries == 0) { uint64_t file_number = file_meta->fd.GetNumber(); // Try getting info from table property - std::string fn = - TableFileName(compaction->immutable_options()->cf_paths, - file_number, file_meta->fd.GetPathId()); + std::string fn = TableFileName(compaction->immutable_options().cf_paths, + file_number, file_meta->fd.GetPathId()); const auto& tp = input_table_properties.find(fn); if (tp != input_table_properties.end()) { file_input_entries = tp->second->num_entries; @@ -2258,7 +2254,7 @@ void CompactionJob::LogCompaction() { } std::string CompactionJob::GetTableFileName(uint64_t file_number) { - return TableFileName(compact_->compaction->immutable_options()->cf_paths, + return TableFileName(compact_->compaction->immutable_options().cf_paths, file_number, compact_->compaction->output_path_id()); } diff --git a/db/compaction/compaction_job_test.cc b/db/compaction/compaction_job_test.cc index f8a6b006c9a..470c4445cc7 100644 --- a/db/compaction/compaction_job_test.cc +++ b/db/compaction/compaction_job_test.cc @@ -301,7 +301,7 @@ class CompactionJobTestBase : public testing::Test { std::unique_ptr table_builder( cf_options_.table_factory->NewTableBuilder( TableBuilderOptions( - *cfd_->ioptions(), mutable_cf_options_, read_options, + cfd_->ioptions(), mutable_cf_options_, read_options, write_options, cfd_->internal_comparator(), cfd_->internal_tbl_prop_coll_factories(), CompressionType::kNoCompression, CompressionOptions(), @@ -460,7 +460,7 @@ class CompactionJobTestBase : public testing::Test { ReadOptions read_opts; Status s = cf_options_.table_factory->NewTableReader( read_opts, - TableReaderOptions(*cfd->ioptions(), nullptr, FileOptions(), + TableReaderOptions(cfd->ioptions(), nullptr, FileOptions(), cfd_->internal_comparator(), 0 /* block_protection_bytes_per_key */), std::move(freader), file_size, &table_reader, false); @@ -645,12 +645,12 @@ class CompactionJobTestBase : public testing::Test { } Compaction compaction( - cfd->current()->storage_info(), *cfd->ioptions(), - *cfd->GetLatestMutableCFOptions(), mutable_db_options_, + cfd->current()->storage_info(), cfd->ioptions(), + cfd->GetLatestMutableCFOptions(), mutable_db_options_, compaction_input_files, output_level, mutable_cf_options_.target_file_size_base, mutable_cf_options_.max_compaction_bytes, 0, kNoCompression, - cfd->GetLatestMutableCFOptions()->compression_opts, + cfd->GetLatestMutableCFOptions().compression_opts, Temperature::kUnknown, max_subcompactions, grandparents, /*earliest_snapshot*/ std::nullopt, /*snapshot_checker*/ nullptr, true); compaction.FinalizeInputInfo(cfd->current()); @@ -684,7 +684,7 @@ class CompactionJobTestBase : public testing::Test { ASSERT_OK(compaction_job.io_status()); mutex_.Lock(); bool compaction_released = false; - ASSERT_OK(compaction_job.Install(*cfd->GetLatestMutableCFOptions(), + ASSERT_OK(compaction_job.Install(cfd->GetLatestMutableCFOptions(), &compaction_released)); ASSERT_OK(compaction_job.io_status()); mutex_.Unlock(); diff --git a/db/compaction/compaction_outputs.cc b/db/compaction/compaction_outputs.cc index b724dd39357..3e1c4402cea 100644 --- a/db/compaction/compaction_outputs.cc +++ b/db/compaction/compaction_outputs.cc @@ -320,7 +320,7 @@ bool CompactionOutputs::ShouldStopBefore(const CompactionIterator& c_iter) { // More details, check PR #1963 const size_t num_skippable_boundaries_crossed = being_grandparent_gap_ ? 2 : 3; - if (compaction_->immutable_options()->compaction_style == + if (compaction_->immutable_options().compaction_style == kCompactionStyleLevel && num_grandparent_boundaries_crossed >= num_skippable_boundaries_crossed && @@ -341,7 +341,7 @@ bool CompactionOutputs::ShouldStopBefore(const CompactionIterator& c_iter) { // target file size. The test shows it can generate larger files than a // static threshold like 75% and has a similar write amplification // improvement. - if (compaction_->immutable_options()->compaction_style == + if (compaction_->immutable_options().compaction_style == kCompactionStyleLevel && current_output_file_size_ >= ((compaction_->target_output_file_size() + 99) / 100) * @@ -752,11 +752,10 @@ Status CompactionOutputs::AddRangeDels( } void CompactionOutputs::FillFilesToCutForTtl() { - if (compaction_->immutable_options()->compaction_style != + if (compaction_->immutable_options().compaction_style != kCompactionStyleLevel || - compaction_->immutable_options()->compaction_pri != - kMinOverlappingRatio || - compaction_->mutable_cf_options()->ttl == 0 || + compaction_->immutable_options().compaction_pri != kMinOverlappingRatio || + compaction_->mutable_cf_options().ttl == 0 || compaction_->num_input_levels() < 2 || compaction_->bottommost_level()) { return; } @@ -764,20 +763,19 @@ void CompactionOutputs::FillFilesToCutForTtl() { // We define new file with the oldest ancestor time to be younger than 1/4 // TTL, and an old one to be older than 1/2 TTL time. int64_t temp_current_time; - auto get_time_status = - compaction_->immutable_options()->clock->GetCurrentTime( - &temp_current_time); + auto get_time_status = compaction_->immutable_options().clock->GetCurrentTime( + &temp_current_time); if (!get_time_status.ok()) { return; } auto current_time = static_cast(temp_current_time); - if (current_time < compaction_->mutable_cf_options()->ttl) { + if (current_time < compaction_->mutable_cf_options().ttl) { return; } uint64_t old_age_thres = - current_time - compaction_->mutable_cf_options()->ttl / 2; + current_time - compaction_->mutable_cf_options().ttl / 2; const std::vector& olevel = *(compaction_->inputs(compaction_->num_input_levels() - 1)); for (FileMetaData* file : olevel) { @@ -787,7 +785,7 @@ void CompactionOutputs::FillFilesToCutForTtl() { // of small files. if (oldest_ancester_time < old_age_thres && file->fd.GetFileSize() > - compaction_->mutable_cf_options()->target_file_size_base / 2) { + compaction_->mutable_cf_options().target_file_size_base / 2) { files_to_cut_for_ttl_.push_back(file); } } diff --git a/db/compaction/compaction_service_job.cc b/db/compaction/compaction_service_job.cc index 2302521fe9a..528a131e44e 100644 --- a/db/compaction/compaction_service_job.cc +++ b/db/compaction/compaction_service_job.cc @@ -177,7 +177,7 @@ CompactionJob::ProcessKeyValueCompactionWithCompactionService( for (const auto& file : compaction_result.output_files) { uint64_t file_num = versions_->NewFileNumber(); auto src_file = compaction_result.output_path + "/" + file.file_name; - auto tgt_file = TableFileName(compaction->immutable_options()->cf_paths, + auto tgt_file = TableFileName(compaction->immutable_options().cf_paths, file_num, compaction->output_path_id()); s = fs_->RenameFile(src_file, tgt_file, IOOptions(), nullptr); if (!s.ok()) { @@ -269,17 +269,17 @@ CompactionServiceCompactionJob::CompactionServiceCompactionJob( std::string output_path, const CompactionServiceInput& compaction_service_input, CompactionServiceResult* compaction_service_result) - : CompactionJob( - job_id, compaction, db_options, mutable_db_options, file_options, - versions, shutting_down, log_buffer, nullptr, output_directory, - nullptr, stats, db_mutex, db_error_handler, - std::move(existing_snapshots), kMaxSequenceNumber, nullptr, nullptr, - std::move(table_cache), event_logger, - compaction->mutable_cf_options()->paranoid_file_checks, - compaction->mutable_cf_options()->report_bg_io_stats, dbname, - &(compaction_service_result->stats), Env::Priority::USER, io_tracer, - manual_compaction_canceled, db_id, db_session_id, - compaction->column_family_data()->GetFullHistoryTsLow()), + : CompactionJob(job_id, compaction, db_options, mutable_db_options, + file_options, versions, shutting_down, log_buffer, nullptr, + output_directory, nullptr, stats, db_mutex, + db_error_handler, std::move(existing_snapshots), + kMaxSequenceNumber, nullptr, nullptr, + std::move(table_cache), event_logger, + compaction->mutable_cf_options().paranoid_file_checks, + compaction->mutable_cf_options().report_bg_io_stats, dbname, + &(compaction_service_result->stats), Env::Priority::USER, + io_tracer, manual_compaction_canceled, db_id, db_session_id, + compaction->column_family_data()->GetFullHistoryTsLow()), output_path_(std::move(output_path)), compaction_input_(compaction_service_input), compaction_result_(compaction_service_result) {} diff --git a/db/compaction/compaction_service_test.cc b/db/compaction/compaction_service_test.cc index d3ee76aab8a..b022152c250 100644 --- a/db/compaction/compaction_service_test.cc +++ b/db/compaction/compaction_service_test.cc @@ -487,7 +487,7 @@ TEST_F(CompactionServiceTest, PreservedOptionsLocalCompaction) { ASSERT_TRUE(s.IsNotFound()); // Should be old value ASSERT_EQ(2, compaction->mutable_cf_options() - ->level0_file_num_compaction_trigger); + .level0_file_num_compaction_trigger); ASSERT_TRUE(dbfull()->min_options_file_numbers_.empty()); }); @@ -573,7 +573,7 @@ TEST_F(CompactionServiceTest, PreservedOptionsRemoteCompaction) { "CompactionJob::ProcessKeyValueCompaction()::Processing", [&](void* arg) { auto compaction = static_cast(arg); ASSERT_EQ(2, compaction->mutable_cf_options() - ->level0_file_num_compaction_trigger); + .level0_file_num_compaction_trigger); }); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); diff --git a/db/compaction/subcompaction_state.cc b/db/compaction/subcompaction_state.cc index 5b823cb3bac..13f40f63f0c 100644 --- a/db/compaction/subcompaction_state.cc +++ b/db/compaction/subcompaction_state.cc @@ -54,7 +54,7 @@ void SubcompactionState::Cleanup(Cache* cache) { // be picked up by scanning the DB directory. TableCache::ReleaseObsolete( cache, out.meta.fd.GetNumber(), nullptr /*handle*/, - compaction->mutable_cf_options()->uncache_aggressiveness); + compaction->mutable_cf_options().uncache_aggressiveness); } } // TODO: sub_compact.io_status is not checked like status. Not sure if thats diff --git a/db/db_filesnapshot.cc b/db/db_filesnapshot.cc index 87f4aad4035..e9ae7981ae2 100644 --- a/db/db_filesnapshot.cc +++ b/db/db_filesnapshot.cc @@ -265,7 +265,7 @@ Status DBImpl::GetLiveFilesStorageInfo( continue; } VersionStorageInfo& vsi = *cfd->current()->storage_info(); - auto& cf_paths = cfd->ioptions()->cf_paths; + auto& cf_paths = cfd->ioptions().cf_paths; auto GetDir = [&](size_t path_id) { // Matching TableFileName() behavior diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index 140d7535916..9baf913bf0a 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -386,9 +386,9 @@ Status DBImpl::ResumeImpl(DBRecoverContext context) { static_cast_with_check(default_cf_handle_); assert(cfh); ColumnFamilyData* cfd = cfh->cfd(); - const MutableCFOptions& cf_opts = *cfd->GetLatestMutableCFOptions(); - s = versions_->LogAndApply(cfd, cf_opts, read_options, write_options, - &edit, &mutex_, directories_.GetDbDir()); + s = versions_->LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), + read_options, write_options, &edit, &mutex_, + directories_.GetDbDir()); if (!s.ok()) { io_s = versions_->io_status(); if (!io_s.ok()) { @@ -851,7 +851,7 @@ Status DBImpl::RegisterRecordSeqnoTimeWorker(const ReadOptions& read_options, InstrumentedMutexLock l(&mutex_); for (auto cfd : *versions_->GetColumnFamilySet()) { - auto& mopts = *cfd->GetLatestMutableCFOptions(); + auto& mopts = cfd->GetLatestMutableCFOptions(); // preserve time is the max of 2 options. uint64_t preserve_seconds = std::max(mopts.preserve_internal_time_seconds, @@ -1166,7 +1166,7 @@ void DBImpl::DumpStats() { } auto* table_factory = - cfd->GetCurrentMutableCFOptions()->table_factory.get(); + cfd->GetCurrentMutableCFOptions().table_factory.get(); assert(table_factory != nullptr); // FIXME: need to a shared_ptr if/when block_cache is going to be mutable Cache* cache = @@ -1285,7 +1285,7 @@ Status DBImpl::SetOptions( } InstrumentedMutexLock ol(&options_mutex_); - MutableCFOptions new_options; + MutableCFOptions new_options_copy; Status s; Status persist_options_status; SuperVersionContext sv_context(/* create_superversion */ true); @@ -1294,11 +1294,12 @@ Status DBImpl::SetOptions( InstrumentedMutexLock l(&mutex_); s = cfd->SetOptions(db_options, options_map); if (s.ok()) { - new_options = *cfd->GetLatestMutableCFOptions(); + new_options_copy = cfd->GetLatestMutableCFOptions(); // Append new version to recompute compaction score. VersionEdit dummy_edit; - s = versions_->LogAndApply(cfd, new_options, read_options, write_options, - &dummy_edit, &mutex_, directories_.GetDbDir()); + s = versions_->LogAndApply(cfd, new_options_copy, read_options, + write_options, &dummy_edit, &mutex_, + directories_.GetDbDir()); if (!versions_->io_status().ok()) { assert(!s.ok()); error_handler_.SetBGError(versions_->io_status(), @@ -1307,7 +1308,7 @@ Status DBImpl::SetOptions( // Trigger possible flush/compactions. This has to be before we persist // options to file, otherwise there will be a deadlock with writer // thread. - InstallSuperVersionAndScheduleWork(cfd, &sv_context, new_options); + InstallSuperVersionAndScheduleWork(cfd, &sv_context); persist_options_status = WriteOptionsFile(write_options, true /*db_mutex_already_held*/); bg_cv_.SignalAll(); @@ -1331,7 +1332,7 @@ Status DBImpl::SetOptions( if (s.ok()) { ROCKS_LOG_INFO(immutable_db_options_.info_log, "[%s] SetOptions() succeeded", cfd->GetName().c_str()); - new_options.Dump(immutable_db_options_.info_log.get()); + new_options_copy.Dump(immutable_db_options_.info_log.get()); if (!persist_options_status.ok()) { // NOTE: WriteOptionsFile already logs on failure s = persist_options_status; @@ -2113,7 +2114,7 @@ InternalIterator* DBImpl::NewInternalIterator( } else { mem_tombstone_iter = std::make_unique( std::unique_ptr(range_del_iter), - &cfd->ioptions()->internal_comparator, nullptr /* smallest */, + &cfd->ioptions().internal_comparator, nullptr /* smallest */, nullptr /* largest */); } merge_iter_builder.AddPointAndTombstoneIterator( @@ -3662,8 +3663,7 @@ Status DBImpl::CreateColumnFamilyImpl(const ReadOptions& read_options, auto* cfd = versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name); assert(cfd != nullptr); - InstallSuperVersionAndScheduleWork(cfd, &sv_context, - *cfd->GetLatestMutableCFOptions()); + InstallSuperVersionAndScheduleWork(cfd, &sv_context); if (!cfd->mem()->IsSnapshotSupported()) { is_snapshot_supported_ = false; @@ -3757,13 +3757,13 @@ Status DBImpl::DropColumnFamilyImpl(ColumnFamilyHandle* column_family) { // we drop column family from a single write thread WriteThread::Writer w; write_thread_.EnterUnbatched(&w, &mutex_); - s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + s = versions_->LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), read_options, write_options, &edit, &mutex_, directories_.GetDbDir()); write_thread_.ExitUnbatched(&w); } if (s.ok()) { - auto& moptions = *cfd->GetLatestMutableCFOptions(); + auto& moptions = cfd->GetLatestMutableCFOptions(); max_total_in_memory_state_ -= moptions.write_buffer_size * moptions.max_write_buffer_number; used_preserve_preclude = moptions.preserve_internal_time_seconds > 0 || @@ -3894,7 +3894,7 @@ Iterator* DBImpl::NewIterator(const ReadOptions& _read_options, auto iter = new ForwardIterator(this, read_options, cfd, sv, /* allow_unprepared_value */ true); result = NewDBIterator( - env_, read_options, *cfd->ioptions(), sv->mutable_cf_options, + env_, read_options, cfd->ioptions(), sv->mutable_cf_options, cfd->user_comparator(), iter, sv->current, kMaxSequenceNumber, sv->mutable_cf_options.max_sequential_skip_in_iterations, nullptr /* read_callback */, cfh); @@ -3976,7 +3976,7 @@ ArenaWrappedDBIter* DBImpl::NewIteratorImpl( // likely that any iterator pointer is close to the iterator it points to so // that they are likely to be in the same cache line and/or page. ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator( - env_, read_options, *cfh->cfd()->ioptions(), sv->mutable_cf_options, + env_, read_options, cfh->cfd()->ioptions(), sv->mutable_cf_options, sv->current, snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations, sv->version_number, read_callback, cfh, expose_blob_index, allow_refresh); @@ -4111,7 +4111,7 @@ Status DBImpl::NewIterators( cf_sv_pair.super_version, /* allow_unprepared_value */ true); iterators->push_back( - NewDBIterator(env_, read_options, *cf_sv_pair.cfd->ioptions(), + NewDBIterator(env_, read_options, cf_sv_pair.cfd->ioptions(), cf_sv_pair.super_version->mutable_cf_options, cf_sv_pair.cfd->user_comparator(), iter, cf_sv_pair.super_version->current, kMaxSequenceNumber, @@ -4349,7 +4349,7 @@ void DBImpl::ReleaseSnapshot(const Snapshot* s) { CfdList cf_scheduled; if (oldest_snapshot > bottommost_files_mark_threshold_) { for (auto* cfd : *versions_->GetColumnFamilySet()) { - if (!cfd->ioptions()->allow_ingest_behind) { + if (!cfd->ioptions().allow_ingest_behind) { cfd->current()->storage_info()->UpdateOldestSnapshot( oldest_snapshot, /*allow_ingest_behind=*/false); if (!cfd->current() @@ -4370,7 +4370,7 @@ void DBImpl::ReleaseSnapshot(const Snapshot* s) { SequenceNumber new_bottommost_files_mark_threshold = kMaxSequenceNumber; for (auto* cfd : *versions_->GetColumnFamilySet()) { if (CfdListContains(cf_scheduled, cfd) || - cfd->ioptions()->allow_ingest_behind) { + cfd->ioptions().allow_ingest_behind) { continue; } new_bottommost_files_mark_threshold = std::min( @@ -4964,13 +4964,12 @@ Status DBImpl::DEPRECATED_DeleteFile(std::string name) { } edit.SetColumnFamily(cfd->GetID()); edit.DeleteFile(level, number); - status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + status = versions_->LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), read_options, write_options, &edit, &mutex_, directories_.GetDbDir()); if (status.ok()) { InstallSuperVersionAndScheduleWork( - cfd, job_context.superversion_contexts.data(), - *cfd->GetLatestMutableCFOptions()); + cfd, job_context.superversion_contexts.data()); } FindObsoleteFiles(&job_context, false); } // lock released here @@ -5069,21 +5068,20 @@ Status DBImpl::DeleteFilesInRanges(ColumnFamilyHandle* column_family, } } if (!deleted_files.empty()) { - vstorage->ComputeCompactionScore(*cfd->ioptions(), - *cfd->GetLatestMutableCFOptions()); + vstorage->ComputeCompactionScore(cfd->ioptions(), + cfd->GetLatestMutableCFOptions()); } if (edit.GetDeletedFiles().empty()) { job_context.Clean(); return status; } input_version->Ref(); - status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + status = versions_->LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), read_options, write_options, &edit, &mutex_, directories_.GetDbDir()); if (status.ok()) { InstallSuperVersionAndScheduleWork( - cfd, job_context.superversion_contexts.data(), - *cfd->GetLatestMutableCFOptions()); + cfd, job_context.superversion_contexts.data()); } for (auto* deleted_file : deleted_files) { deleted_file->being_compacted = false; @@ -5682,7 +5680,7 @@ Status DBImpl::RenameTempFileToOptionsFile(const std::string& file_name, void DBImpl::NewThreadStatusCfInfo(ColumnFamilyData* cfd) const { if (immutable_db_options_.enable_thread_tracking) { ThreadStatusUtil::NewColumnFamilyInfo(this, cfd, cfd->GetName(), - cfd->ioptions()->env); + cfd->ioptions().env); } } @@ -6133,7 +6131,7 @@ Status DBImpl::IngestExternalFiles( auto* cfd = ingestion_jobs[i].GetColumnFamilyData(); assert(!cfd->IsDropped()); cfds_to_commit.push_back(cfd); - mutable_cf_options_list.push_back(cfd->GetLatestMutableCFOptions()); + mutable_cf_options_list.push_back(&cfd->GetLatestMutableCFOptions()); autovector edit_list; edit_list.push_back(ingestion_jobs[i].edit()); edit_lists.push_back(edit_list); @@ -6183,8 +6181,7 @@ Status DBImpl::IngestExternalFiles( for (size_t i = 0; i != num_cfs; ++i) { auto* cfd = ingestion_jobs[i].GetColumnFamilyData(); assert(!cfd->IsDropped()); - InstallSuperVersionAndScheduleWork(cfd, &sv_ctxs[i], - *cfd->GetLatestMutableCFOptions()); + InstallSuperVersionAndScheduleWork(cfd, &sv_ctxs[i]); #ifndef NDEBUG if (0 == i && num_cfs > 1) { TEST_SYNC_POINT("DBImpl::IngestExternalFiles:InstallSVForFirstCF:0"); @@ -6303,12 +6300,13 @@ Status DBImpl::CreateColumnFamilyWithImport( // and this will overwrite the external file. To protect the external // file, we have to make sure the file number will never being reused. next_file_number = versions_->FetchAddFileNumber(total_file_num); - auto cf_options = cfd->GetLatestMutableCFOptions(); - status = - versions_->LogAndApply(cfd, *cf_options, read_options, write_options, - &dummy_edit, &mutex_, directories_.GetDbDir()); + MutableCFOptions mutable_cf_options_copy = + cfd->GetLatestMutableCFOptions(); + status = versions_->LogAndApply(cfd, mutable_cf_options_copy, + read_options, write_options, &dummy_edit, + &mutex_, directories_.GetDbDir()); if (status.ok()) { - InstallSuperVersionAndScheduleWork(cfd, &dummy_sv_ctx, *cf_options); + InstallSuperVersionAndScheduleWork(cfd, &dummy_sv_ctx); } } } @@ -6341,12 +6339,13 @@ Status DBImpl::CreateColumnFamilyWithImport( // Install job edit [Mutex will be unlocked here] if (status.ok()) { - auto cf_options = cfd->GetLatestMutableCFOptions(); - status = versions_->LogAndApply(cfd, *cf_options, read_options, - write_options, import_job.edit(), - &mutex_, directories_.GetDbDir()); + MutableCFOptions mutable_cf_options_copy = + cfd->GetLatestMutableCFOptions(); + status = versions_->LogAndApply( + cfd, mutable_cf_options_copy, read_options, write_options, + import_job.edit(), &mutex_, directories_.GetDbDir()); if (status.ok()) { - InstallSuperVersionAndScheduleWork(cfd, &sv_context, *cf_options); + InstallSuperVersionAndScheduleWork(cfd, &sv_context); } } @@ -6547,7 +6546,7 @@ Status DBImpl::VerifyChecksumInternal(const ReadOptions& read_options, const auto& fd = fd_with_krange.fd; const FileMetaData* fmeta = fd_with_krange.file_metadata; assert(fmeta); - std::string fname = TableFileName(cfd->ioptions()->cf_paths, + std::string fname = TableFileName(cfd->ioptions().cf_paths, fd.GetNumber(), fd.GetPathId()); if (use_file_checksum) { s = VerifyFullFileChecksum(fmeta->file_checksum, @@ -6571,7 +6570,7 @@ Status DBImpl::VerifyChecksumInternal(const ReadOptions& read_options, const uint64_t blob_file_number = meta->GetBlobFileNumber(); const std::string blob_file_name = BlobFileName( - cfd->ioptions()->cf_paths.front().path, blob_file_number); + cfd->ioptions().cf_paths.front().path, blob_file_number); s = VerifyFullFileChecksum(meta->GetChecksumValue(), meta->GetChecksumMethod(), blob_file_name, read_options); @@ -6764,16 +6763,17 @@ Status DBImpl::ReserveFileNumbersBeforeIngestion( pending_output_elem.reset(new std::list::iterator( CaptureCurrentFileNumberInPendingOutputs())); *next_file_number = versions_->FetchAddFileNumber(static_cast(num)); - auto cf_options = cfd->GetLatestMutableCFOptions(); + MutableCFOptions mutable_cf_options_copy = cfd->GetLatestMutableCFOptions(); VersionEdit dummy_edit; // If crash happen after a hard link established, Recover function may // reuse the file number that has already assigned to the internal file, // and this will overwrite the external file. To protect the external // file, we have to make sure the file number will never being reused. - s = versions_->LogAndApply(cfd, *cf_options, read_options, write_options, - &dummy_edit, &mutex_, directories_.GetDbDir()); + s = versions_->LogAndApply(cfd, mutable_cf_options_copy, read_options, + write_options, &dummy_edit, &mutex_, + directories_.GetDbDir()); if (s.ok()) { - InstallSuperVersionAndScheduleWork(cfd, &dummy_sv_ctx, *cf_options); + InstallSuperVersionAndScheduleWork(cfd, &dummy_sv_ctx); } dummy_sv_ctx.Clean(); return s; @@ -6944,7 +6944,7 @@ void DBImpl::InstallSeqnoToTimeMappingInSV( sv_contexts->emplace_back(/*create_superversion=*/true); sv_contexts->back().new_seqno_to_time_mapping = new_seqno_to_time_mapping; cfd->InstallSuperVersion(&sv_contexts->back(), - *(cfd->GetLatestMutableCFOptions())); + cfd->GetLatestMutableCFOptions()); } bg_cv_.SignalAll(); } diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index f39c26a4279..e4a638ad6a6 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -1451,7 +1451,7 @@ class DBImpl : public DB { uint32_t size = static_cast(map_.size()); map_.emplace(cfd->GetID(), size); cfds_.emplace_back(cfd); - mutable_cf_opts_.emplace_back(cfd->GetLatestMutableCFOptions()); + mutable_cf_opts_.emplace_back(&cfd->GetLatestMutableCFOptions()); edit_lists_.emplace_back(autovector()); } uint32_t i = map_[cfd->GetID()]; @@ -2522,9 +2522,8 @@ class DBImpl : public DB { // All ColumnFamily state changes go through this function. Here we analyze // the new state and we schedule background work if we detect that the new // state needs flush or compaction. - void InstallSuperVersionAndScheduleWork( - ColumnFamilyData* cfd, SuperVersionContext* sv_context, - const MutableCFOptions& mutable_cf_options); + void InstallSuperVersionAndScheduleWork(ColumnFamilyData* cfd, + SuperVersionContext* sv_context); bool GetIntPropertyInternal(ColumnFamilyData* cfd, const DBPropertyInfo& property_info, diff --git a/db/db_impl/db_impl_compaction_flush.cc b/db/db_impl/db_impl_compaction_flush.cc index b45f4cae745..e13baa58cab 100644 --- a/db/db_impl/db_impl_compaction_flush.cc +++ b/db/db_impl/db_impl_compaction_flush.cc @@ -66,7 +66,7 @@ bool DBImpl::RequestCompactionToken(ColumnFamilyData* cfd, bool force, LogBuffer* log_buffer) { assert(*token == nullptr); auto limiter = static_cast( - cfd->ioptions()->compaction_thread_limiter.get()); + cfd->ioptions().compaction_thread_limiter.get()); if (limiter == nullptr) { return true; } @@ -99,7 +99,7 @@ bool DBImpl::ShouldRescheduleFlushRequestToRetainUDT( // Check if holding on the flush will cause entering write stall mode. // Write stall entered because of the accumulation of write buffers can be // alleviated if we continue with the flush instead of postponing it. - const auto& mutable_cf_options = *cfd->GetLatestMutableCFOptions(); + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); // Use the same criteria as WaitUntilFlushWouldNotStallWrites does w.r.t // defining what a write stall is about to happen means. If this uses a @@ -116,7 +116,7 @@ bool DBImpl::ShouldRescheduleFlushRequestToRetainUDT( cfd->GetUnflushedMemTableCountForWriteStallCheck(), /*num_l0_files=*/0, /*num_compaction_needed_bytes=*/0, mutable_cf_options, - *cfd->ioptions()) + cfd->ioptions()) .first; if (write_stall != WriteStallCondition::kNormal) { return false; @@ -213,7 +213,7 @@ Status DBImpl::FlushMemTableToOutputFile( snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, job_context, flush_reason, log_buffer, directories_.GetDbDir(), GetDataDir(cfd, 0U), - GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_, + GetCompressionFlush(cfd->ioptions(), mutable_cf_options), stats_, &event_logger_, mutable_cf_options.report_bg_io_stats, true /* sync_output_directory */, true /* write_manifest */, thread_pri, io_tracer_, cfd->GetSuperVersion()->ShareSeqnoToTimeMapping(), db_id_, @@ -299,8 +299,7 @@ Status DBImpl::FlushMemTableToOutputFile( } if (s.ok()) { - InstallSuperVersionAndScheduleWork(cfd, superversion_context, - mutable_cf_options); + InstallSuperVersionAndScheduleWork(cfd, superversion_context); if (made_progress) { *made_progress = true; } @@ -369,7 +368,7 @@ Status DBImpl::FlushMemTableToOutputFile( if (sfm) { // Notify sst_file_manager that a new file was added std::string file_path = MakeTableFileName( - cfd->ioptions()->cf_paths[0].path, file_meta.fd.GetNumber()); + cfd->ioptions().cf_paths[0].path, file_meta.fd.GetNumber()); // TODO (PR7798). We should only add the file to the FileManager if it // exists. Otherwise, some tests may fail. Ignore the error in the // interim. @@ -404,7 +403,7 @@ Status DBImpl::FlushMemTablesToOutputFiles( const auto& bg_flush_arg = bg_flush_args[0]; ColumnFamilyData* cfd = bg_flush_arg.cfd_; // intentional infrequent copy for each flush - MutableCFOptions mutable_cf_options_copy = *cfd->GetLatestMutableCFOptions(); + MutableCFOptions mutable_cf_options_copy = cfd->GetLatestMutableCFOptions(); SuperVersionContext* superversion_context = bg_flush_arg.superversion_context_; FlushReason flush_reason = bg_flush_arg.flush_reason_; @@ -461,7 +460,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles( for (int i = 0; i < num_cfs; ++i) { auto cfd = cfds[i]; FSDirectory* data_dir = GetDataDir(cfd, 0U); - const std::string& curr_path = cfd->ioptions()->cf_paths[0].path; + const std::string& curr_path = cfd->ioptions().cf_paths[0].path; // Add to distinct output directories if eligible. Use linear search. Since // the number of elements in the vector is not large, performance should be @@ -478,7 +477,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles( distinct_output_dirs.emplace_back(data_dir); } - all_mutable_cf_options.emplace_back(*cfd->GetLatestMutableCFOptions()); + all_mutable_cf_options.emplace_back(cfd->GetLatestMutableCFOptions()); const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.back(); uint64_t max_memtable_id = bg_flush_args[i].max_memtable_id_; FlushReason flush_reason = bg_flush_args[i].flush_reason_; @@ -488,7 +487,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles( &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, job_context, flush_reason, log_buffer, directories_.GetDbDir(), data_dir, - GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_, + GetCompressionFlush(cfd->ioptions(), mutable_cf_options), stats_, &event_logger_, mutable_cf_options.report_bg_io_stats, false /* sync_output_directory */, false /* write_manifest */, thread_pri, io_tracer_, @@ -786,9 +785,8 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles( if (cfds[i]->IsDropped()) { continue; } - InstallSuperVersionAndScheduleWork(cfds[i], - &job_context->superversion_contexts[i], - all_mutable_cf_options[i]); + InstallSuperVersionAndScheduleWork( + cfds[i], &job_context->superversion_contexts[i]); const std::string& column_family_name = cfds[i]->GetName(); @@ -834,7 +832,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles( jobs[i]->GetCommittedFlushJobsInfo()); if (sfm) { std::string file_path = MakeTableFileName( - cfds[i]->ioptions()->cf_paths[0].path, file_meta[i].fd.GetNumber()); + cfds[i]->ioptions().cf_paths[0].path, file_meta[i].fd.GetNumber()); // TODO (PR7798). We should only add the file to the FileManager if it // exists. Otherwise, some tests may fail. Ignore the error in the // interim. @@ -908,7 +906,7 @@ void DBImpl::NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta, // go to L0 in the future. const uint64_t file_number = file_meta->fd.GetNumber(); info.file_path = - MakeTableFileName(cfd->ioptions()->cf_paths[0].path, file_number); + MakeTableFileName(cfd->ioptions().cf_paths[0].path, file_number); info.file_number = file_number; info.thread_id = env_->GetThreadID(); info.job_id = job_id; @@ -1040,7 +1038,7 @@ Status DBImpl::IncreaseFullHistoryTsLowImpl(ColumnFamilyData* cfd, return Status::InvalidArgument(oss.str()); } - Status s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + Status s = versions_->LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), read_options, write_options, &edit, &mutex_, directories_.GetDbDir()); if (!s.ok()) { @@ -1066,11 +1064,11 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, auto cfh = static_cast_with_check(column_family); auto cfd = cfh->cfd(); - if (options.target_path_id >= cfd->ioptions()->cf_paths.size()) { + if (options.target_path_id >= cfd->ioptions().cf_paths.size()) { return Status::InvalidArgument("Invalid target path ID"); } if (options.change_level && - cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + cfd->ioptions().compaction_style == kCompactionStyleFIFO) { return Status::NotSupported( "FIFO compaction does not support change_level."); } @@ -1122,7 +1120,7 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, constexpr int kInvalidLevel = -1; int final_output_level = kInvalidLevel; bool exclusive = options.exclusive_manual_compaction; - if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal && + if (cfd->ioptions().compaction_style == kCompactionStyleUniversal && cfd->NumberLevels() > 1) { // Always compact all files together. final_output_level = cfd->NumberLevels() - 1; @@ -1134,7 +1132,7 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, final_output_level, options, begin, end, exclusive, false /* disable_trivial_move */, std::numeric_limits::max(), trim_ts); - } else if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + } else if (cfd->ioptions().compaction_style == kCompactionStyleFIFO) { // FIFOCompactionPicker::CompactRange() will ignore the input key range // [begin, end] and just try to pick compaction based on the configured // option `compaction_options_fifo`. So we skip checking if [begin, end] @@ -1152,7 +1150,7 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, // Might need to query the partitioner SstPartitionerFactory* partitioner_factory = - current_version->cfd()->ioptions()->sst_partitioner_factory.get(); + current_version->cfd()->ioptions().sst_partitioner_factory.get(); std::unique_ptr partitioner; if (partitioner_factory && begin != nullptr && end != nullptr) { SstPartitioner::Context context; @@ -1228,7 +1226,7 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, CleanupSuperVersion(super_version); } if (s.ok() && first_overlapped_level != kInvalidLevel) { - if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal) { + if (cfd->ioptions().compaction_style == kCompactionStyleUniversal) { assert(first_overlapped_level == 0); s = RunManualCompaction( cfd, first_overlapped_level, first_overlapped_level, options, begin, @@ -1237,7 +1235,7 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, trim_ts); final_output_level = first_overlapped_level; } else { - assert(cfd->ioptions()->compaction_style == kCompactionStyleLevel); + assert(cfd->ioptions().compaction_style == kCompactionStyleLevel); uint64_t next_file_number = versions_->current_next_file_number(); // Start compaction from `first_overlapped_level`, one level down at a // time, until output level >= max_overlapped_level. @@ -1250,9 +1248,9 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, for (;;) { // Always allow L0 -> L1 compaction if (level > 0) { - if (cfd->ioptions()->level_compaction_dynamic_level_bytes) { - assert(final_output_level < cfd->ioptions()->num_levels); - if (final_output_level + 1 == cfd->ioptions()->num_levels) { + if (cfd->ioptions().level_compaction_dynamic_level_bytes) { + assert(final_output_level < cfd->ioptions().num_levels); + if (final_output_level + 1 == cfd->ioptions().num_levels) { break; } } else { @@ -1271,7 +1269,7 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, } } output_level = level + 1; - if (cfd->ioptions()->level_compaction_dynamic_level_bytes && + if (cfd->ioptions().level_compaction_dynamic_level_bytes && level == 0) { output_level = ColumnFamilyData::kCompactToBaseLevel; } @@ -1303,8 +1301,8 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, // bottommost level intra-level compaction if ((options.bottommost_level_compaction == BottommostLevelCompaction::kIfHaveCompactionFilter && - (cfd->ioptions()->compaction_filter != nullptr || - cfd->ioptions()->compaction_filter_factory != nullptr)) || + (cfd->ioptions().compaction_filter != nullptr || + cfd->ioptions().compaction_filter_factory != nullptr)) || options.bottommost_level_compaction == BottommostLevelCompaction::kForceOptimized || options.bottommost_level_compaction == @@ -1458,7 +1456,7 @@ Status DBImpl::CompactFilesImpl( } if (output_path_id < 0) { - if (cfd->ioptions()->cf_paths.size() == 1U) { + if (cfd->ioptions().cf_paths.size() == 1U) { output_path_id = 0; } else { return Status::NotSupported( @@ -1467,12 +1465,12 @@ Status DBImpl::CompactFilesImpl( } } - if (cfd->ioptions()->allow_ingest_behind && - output_level >= cfd->ioptions()->num_levels - 1) { + if (cfd->ioptions().allow_ingest_behind && + output_level >= cfd->ioptions().num_levels - 1) { return Status::InvalidArgument( "Exceed the maximum output level defined by " "the current compaction algorithm with ingest_behind --- " + - std::to_string(cfd->ioptions()->num_levels - 1)); + std::to_string(cfd->ioptions().num_levels - 1)); } std::vector input_files; @@ -1509,7 +1507,7 @@ Status DBImpl::CompactFilesImpl( assert(cfd->compaction_picker()); c.reset(cfd->compaction_picker()->CompactFiles( compact_options, input_files, output_level, version->storage_info(), - *cfd->GetLatestMutableCFOptions(), mutable_db_options_, output_path_id)); + cfd->GetLatestMutableCFOptions(), mutable_db_options_, output_path_id)); // we already sanitized the set of input files and checked for conflicts // without releasing the lock, so we're guaranteed a compaction can be formed. assert(c != nullptr); @@ -1539,8 +1537,8 @@ Status DBImpl::CompactFilesImpl( GetDataDir(c->column_family_data(), 0), stats_, &mutex_, &error_handler_, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, job_context, table_cache_, &event_logger_, - c->mutable_cf_options()->paranoid_file_checks, - c->mutable_cf_options()->report_bg_io_stats, dbname_, + c->mutable_cf_options().paranoid_file_checks, + c->mutable_cf_options().report_bg_io_stats, dbname_, &compaction_job_stats, Env::Priority::USER, io_tracer_, kManualCompactionCanceledFalse_, db_id_, db_session_id_, c->column_family_data()->GetFullHistoryTsLow(), c->trim_ts(), @@ -1551,8 +1549,8 @@ Status DBImpl::CompactFilesImpl( // takes running compactions into account (by skipping files that are already // being compacted). Since we just changed compaction score, we recalculate it // here. - version->storage_info()->ComputeCompactionScore(*cfd->ioptions(), - *c->mutable_cf_options()); + version->storage_info()->ComputeCompactionScore(cfd->ioptions(), + c->mutable_cf_options()); compaction_job.Prepare(std::nullopt /*subcompact to be computed*/); @@ -1577,15 +1575,14 @@ Status DBImpl::CompactFilesImpl( bool compaction_released = false; Status status = - compaction_job.Install(*c->mutable_cf_options(), &compaction_released); + compaction_job.Install(c->mutable_cf_options(), &compaction_released); if (!compaction_released) { c->ReleaseCompactionFiles(s); } if (status.ok()) { assert(compaction_job.io_status().ok()); InstallSuperVersionAndScheduleWork( - c->column_family_data(), job_context->superversion_contexts.data(), - *c->mutable_cf_options()); + c->column_family_data(), job_context->superversion_contexts.data()); } // status above captures any error during compaction_job.Install, so its ok // not check compaction_job.io_status() explicitly if we're not calling @@ -1633,13 +1630,13 @@ Status DBImpl::CompactFilesImpl( if (output_file_names != nullptr) { for (const auto& newf : c->edit()->GetNewFiles()) { output_file_names->push_back(TableFileName( - c->immutable_options()->cf_paths, newf.second.fd.GetNumber(), + c->immutable_options().cf_paths, newf.second.fd.GetNumber(), newf.second.fd.GetPathId())); } for (const auto& blob_file : c->edit()->GetBlobFileAdditions()) { output_file_names->push_back( - BlobFileName(c->immutable_options()->cf_paths.front().path, + BlobFileName(c->immutable_options().cf_paths.front().path, blob_file.GetBlobFileNumber())); } } @@ -1774,7 +1771,7 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { } refitting_level_ = true; - const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions(); + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); // move to a smaller level int to_level = target_level; if (target_level < 0) { @@ -1845,12 +1842,12 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { cfd->current()->DebugString().data()); std::unique_ptr c(new Compaction( - vstorage, *cfd->ioptions(), mutable_cf_options, mutable_db_options_, + vstorage, cfd->ioptions(), mutable_cf_options, mutable_db_options_, {input}, to_level, MaxFileSizeForLevel( mutable_cf_options, to_level, cfd->ioptions() - ->compaction_style) /* output file size limit, not applicable */ + .compaction_style) /* output file size limit, not applicable */ , LLONG_MAX /* max compaction bytes, not applicable */, 0 /* output path ID, not applicable */, mutable_cf_options.compression, @@ -1890,7 +1887,7 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { cfd->compaction_picker()->UnregisterCompaction(c.get()); c.reset(); - InstallSuperVersionAndScheduleWork(cfd, &sv_context, mutable_cf_options); + InstallSuperVersionAndScheduleWork(cfd, &sv_context); ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] LogAndApply: %s\n", cfd->GetName().c_str(), status.ToString().data()); @@ -2043,16 +2040,16 @@ Status DBImpl::RunManualCompaction( // For universal compaction, we enforce every manual compaction to compact // all files. if (begin == nullptr || - cfd->ioptions()->compaction_style == kCompactionStyleUniversal || - cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + cfd->ioptions().compaction_style == kCompactionStyleUniversal || + cfd->ioptions().compaction_style == kCompactionStyleFIFO) { manual.begin = nullptr; } else { begin_storage.SetMinPossibleForUserKey(*begin); manual.begin = &begin_storage; } if (end == nullptr || - cfd->ioptions()->compaction_style == kCompactionStyleUniversal || - cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + cfd->ioptions().compaction_style == kCompactionStyleUniversal || + cfd->ioptions().compaction_style == kCompactionStyleFIFO) { manual.end = nullptr; } else { end_storage.SetMaxPossibleForUserKey(*end); @@ -2133,7 +2130,7 @@ Status DBImpl::RunManualCompaction( scheduled || (((manual.manual_end = &manual.tmp_storage1) != nullptr) && ((compaction = manual.cfd->CompactRange( - *manual.cfd->GetLatestMutableCFOptions(), mutable_db_options_, + manual.cfd->GetLatestMutableCFOptions(), mutable_db_options_, manual.input_level, manual.output_level, compact_range_options, manual.begin, manual.end, &manual.manual_end, &manual_conflict, max_file_num_to_ignore, trim_ts)) == nullptr && @@ -2676,7 +2673,7 @@ Status DBImpl::WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd, return Status::OK(); } - const auto& mutable_cf_options = *cfd->GetLatestMutableCFOptions(); + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); const auto* vstorage = cfd->current()->storage_info(); // Skip stalling check if we're below auto-flush and auto-compaction @@ -2684,7 +2681,7 @@ Status DBImpl::WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd, // triggers are so low that stalling is needed for any background work. In // that case we shouldn't wait since background work won't be scheduled. if (cfd->imm()->NumNotFlushed() < - cfd->ioptions()->min_write_buffer_number_to_merge && + cfd->ioptions().min_write_buffer_number_to_merge && vstorage->l0_delay_trigger_count() < mutable_cf_options.level0_file_num_compaction_trigger) { break; @@ -2700,7 +2697,7 @@ Status DBImpl::WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd, cfd->GetUnflushedMemTableCountForWriteStallCheck(), vstorage->l0_delay_trigger_count() + 1, vstorage->estimated_compaction_needed_bytes(), mutable_cf_options, - *cfd->ioptions()) + cfd->ioptions()) .first; } while (write_stall_condition != WriteStallCondition::kNormal); } @@ -3679,8 +3676,8 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, // Compaction makes a copy of the latest MutableCFOptions. It should be used // throughout the compaction procedure to make sure consistency. It will // eventually be installed into SuperVersion - auto* mutable_cf_options = cfd->GetLatestMutableCFOptions(); - if (!mutable_cf_options->disable_auto_compactions && !cfd->IsDropped()) { + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); + if (!mutable_cf_options.disable_auto_compactions && !cfd->IsDropped()) { // NOTE: try to avoid unnecessary copy of MutableCFOptions if // compaction is not necessary. Need to make sure mutex is held // until we make a copy in the following code @@ -3689,7 +3686,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, std::vector snapshot_seqs; // This info is not useful for other scenarios, so save querying existing // snapshots for those cases. - if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal && + if (cfd->ioptions().compaction_style == kCompactionStyleUniversal && cfd->user_comparator()->timestamp_size() == 0) { SequenceNumber earliest_write_conflict_snapshot; GetSnapshotContext(job_context, &snapshot_seqs, @@ -3697,7 +3694,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, &snapshot_checker); assert(is_snapshot_supported_ || snapshots_.empty()); } - c.reset(cfd->PickCompaction(*mutable_cf_options, mutable_db_options_, + c.reset(cfd->PickCompaction(mutable_cf_options, mutable_db_options_, snapshot_seqs, snapshot_checker, log_buffer)); TEST_SYNC_POINT("DBImpl::BackgroundCompaction():AfterPickCompaction"); @@ -3711,8 +3708,8 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, c->column_family_data() ->current() ->storage_info() - ->ComputeCompactionScore(*(c->immutable_options()), - *(c->mutable_cf_options())); + ->ComputeCompactionScore(c->immutable_options(), + c->mutable_cf_options()); AddToCompactionQueue(cfd); c.reset(); @@ -3760,7 +3757,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction", c->column_family_data()); assert(c->num_input_files(1) == 0); - assert(c->column_family_data()->ioptions()->compaction_style == + assert(c->column_family_data()->ioptions().compaction_style == kCompactionStyleFIFO); compaction_job_stats.num_input_files = c->num_input_files(0); @@ -3772,7 +3769,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, c->edit()->DeleteFile(c->level(), f->fd.GetNumber()); } status = versions_->LogAndApply( - c->column_family_data(), *c->mutable_cf_options(), read_options, + c->column_family_data(), c->mutable_cf_options(), read_options, write_options, c->edit(), &mutex_, directories_.GetDbDir(), /*new_descriptor_log=*/false, /*column_family_options=*/nullptr, [&c, &compaction_released](const Status& s) { @@ -3781,8 +3778,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, }); io_s = versions_->io_status(); InstallSuperVersionAndScheduleWork( - c->column_family_data(), job_context->superversion_contexts.data(), - *c->mutable_cf_options()); + c->column_family_data(), job_context->superversion_contexts.data()); ROCKS_LOG_BUFFER(log_buffer, "[%s] Deleted %d files\n", c->column_family_data()->GetName().c_str(), c->num_input_files(0)); @@ -3838,7 +3834,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, } } if (c->compaction_reason() == CompactionReason::kLevelMaxLevelSize && - c->immutable_options()->compaction_pri == kRoundRobin) { + c->immutable_options().compaction_pri == kRoundRobin) { int start_level = c->start_level(); if (start_level > 0) { auto vstorage = c->input_version()->storage_info(); @@ -3848,7 +3844,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, } } status = versions_->LogAndApply( - c->column_family_data(), *c->mutable_cf_options(), read_options, + c->column_family_data(), c->mutable_cf_options(), read_options, write_options, c->edit(), &mutex_, directories_.GetDbDir(), /*new_descriptor_log=*/false, /*column_family_options=*/nullptr, [&c, &compaction_released](const Status& s) { @@ -3856,10 +3852,8 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, compaction_released = true; }); io_s = versions_->io_status(); - // Use latest MutableCFOptions InstallSuperVersionAndScheduleWork( - c->column_family_data(), job_context->superversion_contexts.data(), - *c->mutable_cf_options()); + c->column_family_data(), job_context->superversion_contexts.data()); VersionStorageInfo::LevelSummaryStorage tmp; c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(), @@ -3928,8 +3922,8 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, GetDataDir(c->column_family_data(), 0), stats_, &mutex_, &error_handler_, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, job_context, table_cache_, &event_logger_, - c->mutable_cf_options()->paranoid_file_checks, - c->mutable_cf_options()->report_bg_io_stats, dbname_, + c->mutable_cf_options().paranoid_file_checks, + c->mutable_cf_options().report_bg_io_stats, dbname_, &compaction_job_stats, thread_pri, io_tracer_, is_manual ? manual_compaction->canceled : kManualCompactionCanceledFalse_, @@ -3959,12 +3953,11 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, } status = - compaction_job.Install(*c->mutable_cf_options(), &compaction_released); + compaction_job.Install(c->mutable_cf_options(), &compaction_released); io_s = compaction_job.io_status(); if (status.ok()) { InstallSuperVersionAndScheduleWork( - c->column_family_data(), job_context->superversion_contexts.data(), - *c->mutable_cf_options()); + c->column_family_data(), job_context->superversion_contexts.data()); } *made_progress = true; TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction", @@ -4042,8 +4035,8 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, c->column_family_data() ->current() ->storage_info() - ->ComputeCompactionScore(*(c->immutable_options()), - *(c->mutable_cf_options())); + ->ComputeCompactionScore(c->immutable_options(), + c->mutable_cf_options()); if (!cfd->queued_for_compaction()) { AddToCompactionQueue(cfd); } @@ -4078,10 +4071,9 @@ Status DBImpl::BackgroundCompaction(bool* made_progress, // We only compacted part of the requested range. Update *m // to the range that is left to be compacted. // Universal and FIFO compactions should always compact the whole range - assert(m->cfd->ioptions()->compaction_style != - kCompactionStyleUniversal || - m->cfd->ioptions()->num_levels > 1); - assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO); + assert(m->cfd->ioptions().compaction_style != kCompactionStyleUniversal || + m->cfd->ioptions().num_levels > 1); + assert(m->cfd->ioptions().compaction_style != kCompactionStyleFIFO); m->tmp_storage = *m->manual_end; m->begin = &m->tmp_storage; m->incomplete = true; @@ -4228,7 +4220,7 @@ void DBImpl::BuildCompactionJobInfo( for (const auto fmd : *c->inputs(i)) { const FileDescriptor& desc = fmd->fd; const uint64_t file_number = desc.GetNumber(); - auto fn = TableFileName(c->immutable_options()->cf_paths, file_number, + auto fn = TableFileName(c->immutable_options().cf_paths, file_number, desc.GetPathId()); compaction_job_info->input_files.push_back(fn); compaction_job_info->input_file_infos.push_back(CompactionFileInfo{ @@ -4241,17 +4233,17 @@ void DBImpl::BuildCompactionJobInfo( const FileDescriptor& desc = meta.fd; const uint64_t file_number = desc.GetNumber(); compaction_job_info->output_files.push_back(TableFileName( - c->immutable_options()->cf_paths, file_number, desc.GetPathId())); + c->immutable_options().cf_paths, file_number, desc.GetPathId())); compaction_job_info->output_file_infos.push_back(CompactionFileInfo{ newf.first, file_number, meta.oldest_blob_file_number}); } compaction_job_info->blob_compression_type = - c->mutable_cf_options()->blob_compression_type; + c->mutable_cf_options().blob_compression_type; // Update BlobFilesInfo. for (const auto& blob_file : c->edit()->GetBlobFileAdditions()) { BlobFileAdditionInfo blob_file_addition_info( - BlobFileName(c->immutable_options()->cf_paths.front().path, + BlobFileName(c->immutable_options().cf_paths.front().path, blob_file.GetBlobFileNumber()) /*blob_file_path*/, blob_file.GetBlobFileNumber(), blob_file.GetTotalBlobCount(), blob_file.GetTotalBlobBytes()); @@ -4262,7 +4254,7 @@ void DBImpl::BuildCompactionJobInfo( // Update BlobFilesGarbageInfo. for (const auto& blob_file : c->edit()->GetBlobFileGarbages()) { BlobFileGarbageInfo blob_file_garbage_info( - BlobFileName(c->immutable_options()->cf_paths.front().path, + BlobFileName(c->immutable_options().cf_paths.front().path, blob_file.GetBlobFileNumber()) /*blob_file_path*/, blob_file.GetBlobFileNumber(), blob_file.GetGarbageBlobCount(), blob_file.GetGarbageBlobBytes()); @@ -4284,9 +4276,9 @@ void DBImpl::BuildCompactionJobInfo( // for superversion_to_free void DBImpl::InstallSuperVersionAndScheduleWork( - ColumnFamilyData* cfd, SuperVersionContext* sv_context, - const MutableCFOptions& mutable_cf_options) { + ColumnFamilyData* cfd, SuperVersionContext* sv_context) { mutex_.AssertHeld(); + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); // Update max_total_in_memory_state_ size_t old_memtable_size = 0; @@ -4309,7 +4301,7 @@ void DBImpl::InstallSuperVersionAndScheduleWork( bottommost_files_mark_threshold_ = kMaxSequenceNumber; standalone_range_deletion_files_mark_threshold_ = kMaxSequenceNumber; for (auto* my_cfd : *versions_->GetColumnFamilySet()) { - if (!my_cfd->ioptions()->allow_ingest_behind) { + if (!my_cfd->ioptions().allow_ingest_behind) { bottommost_files_mark_threshold_ = std::min( bottommost_files_mark_threshold_, my_cfd->current()->storage_info()->bottommost_files_mark_threshold()); diff --git a/db/db_impl/db_impl_debug.cc b/db/db_impl/db_impl_debug.cc index f8b2c6a757a..38873b0e321 100644 --- a/db/db_impl/db_impl_debug.cc +++ b/db/db_impl/db_impl_debug.cc @@ -103,8 +103,8 @@ Status DBImpl::TEST_CompactRange(int level, const Slice* begin, cfd = cfh->cfd(); } int output_level = - (cfd->ioptions()->compaction_style == kCompactionStyleUniversal || - cfd->ioptions()->compaction_style == kCompactionStyleFIFO) + (cfd->ioptions().compaction_style == kCompactionStyleUniversal || + cfd->ioptions().compaction_style == kCompactionStyleFIFO) ? level : level + 1; return RunManualCompaction( @@ -239,7 +239,7 @@ void DBImpl::TEST_GetAllBlockCaches( for (auto cfd : *versions_->GetColumnFamilySet()) { if (const auto bbto = cfd->GetCurrentMutableCFOptions() - ->table_factory->GetOptions()) { + .table_factory->GetOptions()) { cache_set->insert(bbto->block_cache.get()); } } @@ -267,7 +267,7 @@ Status DBImpl::TEST_GetLatestMutableCFOptions( InstrumentedMutexLock l(&mutex_); auto cfh = static_cast_with_check(column_family); - *mutable_cf_options = *cfh->cfd()->GetLatestMutableCFOptions(); + *mutable_cf_options = cfh->cfd()->GetLatestMutableCFOptions(); return Status::OK(); } diff --git a/db/db_impl/db_impl_experimental.cc b/db/db_impl/db_impl_experimental.cc index f802fb95688..b29bd9efad3 100644 --- a/db/db_impl/db_impl_experimental.cc +++ b/db/db_impl/db_impl_experimental.cc @@ -45,8 +45,8 @@ Status DBImpl::SuggestCompactRange(ColumnFamilyHandle* column_family, } // Since we have some more files to compact, we should also recompute // compaction score - vstorage->ComputeCompactionScore(*cfd->ioptions(), - *cfd->GetLatestMutableCFOptions()); + vstorage->ComputeCompactionScore(cfd->ioptions(), + cfd->GetLatestMutableCFOptions()); EnqueuePendingCompaction(cfd); MaybeScheduleFlushOrCompaction(); } @@ -146,13 +146,12 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { f->user_defined_timestamps_persisted); } - status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + status = versions_->LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), read_options, write_options, &edit, &mutex_, directories_.GetDbDir()); if (status.ok()) { InstallSuperVersionAndScheduleWork( - cfd, job_context.superversion_contexts.data(), - *cfd->GetLatestMutableCFOptions()); + cfd, job_context.superversion_contexts.data()); } } // lock released here LogFlush(immutable_db_options_.info_log); diff --git a/db/db_impl/db_impl_files.cc b/db/db_impl/db_impl_files.cc index b13563ee5b5..c1ef7b96b16 100644 --- a/db/db_impl/db_impl_files.cc +++ b/db/db_impl/db_impl_files.cc @@ -1040,7 +1040,7 @@ std::set DBImpl::CollectAllDBPaths() { all_db_paths.insert(NormalizePath(db_path.path)); } for (const auto* cfd : *versions_->GetColumnFamilySet()) { - for (const auto& cf_path : cfd->ioptions()->cf_paths) { + for (const auto& cf_path : cfd->ioptions().cf_paths) { all_db_paths.insert(NormalizePath(cf_path.path)); } } diff --git a/db/db_impl/db_impl_follower.cc b/db/db_impl/db_impl_follower.cc index a104a83d6bf..90c4326ceb1 100644 --- a/db/db_impl/db_impl_follower.cc +++ b/db/db_impl/db_impl_follower.cc @@ -140,10 +140,8 @@ Status DBImplFollower::TryCatchUpWithLeader() { // to super versions in a lock free manner, it checks the earliest // sequence number to detect if there was a change in version in // the meantime. - const MutableCFOptions mutable_cf_options = - *cfd->GetLatestMutableCFOptions(); MemTable* new_mem = cfd->ConstructNewMemtable( - mutable_cf_options, versions_->LastSequence()); + cfd->GetLatestMutableCFOptions(), versions_->LastSequence()); cfd->mem()->SetNextLogNumber(cfd->GetLogNumber()); cfd->mem()->ConstructFragmentedRangeTombstones(); cfd->imm()->Add(cfd->mem(), &job_context.memtables_to_free); diff --git a/db/db_impl/db_impl_open.cc b/db/db_impl/db_impl_open.cc index 065ea4446d8..4082c6847b9 100644 --- a/db/db_impl/db_impl_open.cc +++ b/db/db_impl/db_impl_open.cc @@ -575,7 +575,7 @@ Status DBImpl::Recover( } if (s.ok() && !read_only) { for (auto cfd : *versions_->GetColumnFamilySet()) { - auto& moptions = *cfd->GetLatestMutableCFOptions(); + const auto& moptions = cfd->GetLatestMutableCFOptions(); // Try to trivially move files down the LSM tree to start from bottommost // level when level_compaction_dynamic_level_bytes is enabled. This should // only be useful when user is migrating to turning on this option. @@ -590,16 +590,16 @@ Status DBImpl::Recover( // the user wants to partition SST files. // Note that files moved in this step may not respect the compression // option in target level. - if (cfd->ioptions()->compaction_style == + if (cfd->ioptions().compaction_style == CompactionStyle::kCompactionStyleLevel && - cfd->ioptions()->level_compaction_dynamic_level_bytes && + cfd->ioptions().level_compaction_dynamic_level_bytes && !moptions.disable_auto_compactions) { - int to_level = cfd->ioptions()->num_levels - 1; + int to_level = cfd->ioptions().num_levels - 1; // last level is reserved // allow_ingest_behind does not support Level Compaction, // and per_key_placement can have infinite compaction loop for Level // Compaction. Adjust to_level here just to be safe. - if (cfd->ioptions()->allow_ingest_behind || + if (cfd->ioptions().allow_ingest_behind || moptions.preclude_last_level_data_seconds > 0) { to_level -= 1; } @@ -622,10 +622,10 @@ Status DBImpl::Recover( // lsm_state will look like "[1,2,3,4,5,6,0]" for an LSM with // 7 levels std::string lsm_state = "["; - for (int i = 0; i < cfd->ioptions()->num_levels; ++i) { + for (int i = 0; i < cfd->ioptions().num_levels; ++i) { lsm_state += std::to_string( cfd->current()->storage_info()->NumLevelFiles(i)); - if (i < cfd->ioptions()->num_levels - 1) { + if (i < cfd->ioptions().num_levels - 1) { lsm_state += ","; } } @@ -708,9 +708,9 @@ Status DBImpl::Recover( // may check this value to decide whether to flush. max_total_in_memory_state_ = 0; for (auto cfd : *versions_->GetColumnFamilySet()) { - auto* mutable_cf_options = cfd->GetLatestMutableCFOptions(); - max_total_in_memory_state_ += mutable_cf_options->write_buffer_size * - mutable_cf_options->max_write_buffer_number; + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); + max_total_in_memory_state_ += mutable_cf_options.write_buffer_size * + mutable_cf_options.max_write_buffer_number; } SequenceNumber next_sequence(kMaxSequenceNumber); @@ -821,7 +821,7 @@ Status DBImpl::Recover( if (!s.ok()) { // Clear memtables if recovery failed for (auto cfd : *versions_->GetColumnFamilySet()) { - cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(), + cfd->CreateNewMemtable(cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); } } @@ -1601,7 +1601,7 @@ Status DBImpl::MaybeWriteLevel0TableForRecovery( } *flushed = true; - cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(), + cfd->CreateNewMemtable(cfd->GetLatestMutableCFOptions(), *next_sequence - 1); } } @@ -1788,7 +1788,7 @@ Status DBImpl::MaybeFlushFinalMemtableOrRestoreActiveLogFiles( } flushed = true; - cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(), + cfd->CreateNewMemtable(cfd->GetLatestMutableCFOptions(), versions_->LastSequence()); } data_seen = true; @@ -1956,7 +1956,7 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, assert(ucmp); const size_t ts_sz = ucmp->timestamp_size(); const bool logical_strip_timestamp = - ts_sz > 0 && !cfd->ioptions()->persist_user_defined_timestamps; + ts_sz > 0 && !cfd->ioptions().persist_user_defined_timestamps; { ScopedArenaPtr iter( logical_strip_timestamp @@ -1972,10 +1972,10 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, cfd->GetName().c_str(), meta.fd.GetNumber()); // Get the latest mutable cf options while the mutex is still locked - const MutableCFOptions mutable_cf_options = - *cfd->GetLatestMutableCFOptions(); + const MutableCFOptions mutable_cf_options_copy = + cfd->GetLatestMutableCFOptions(); bool paranoid_file_checks = - cfd->GetLatestMutableCFOptions()->paranoid_file_checks; + cfd->GetLatestMutableCFOptions().paranoid_file_checks; int64_t _current_time = 0; immutable_db_options_.clock->GetCurrentTime(&_current_time) @@ -2017,11 +2017,11 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, const WriteOptions write_option(Env::IO_HIGH, Env::IOActivity::kDBOpen); TableBuilderOptions tboptions( - *cfd->ioptions(), mutable_cf_options, read_option, write_option, + cfd->ioptions(), mutable_cf_options_copy, read_option, write_option, cfd->internal_comparator(), cfd->internal_tbl_prop_coll_factories(), - GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), - mutable_cf_options.compression_opts, cfd->GetID(), cfd->GetName(), - 0 /* level */, current_time /* newest_key_time */, + GetCompressionFlush(cfd->ioptions(), mutable_cf_options_copy), + mutable_cf_options_copy.compression_opts, cfd->GetID(), + cfd->GetName(), 0 /* level */, current_time /* newest_key_time */, false /* is_bottommost */, TableFileCreationReason::kRecovery, 0 /* oldest_key_time */, 0 /* file_creation_time */, db_id_, db_session_id_, 0 /* target_file_size */, meta.fd.GetNumber(), @@ -2497,8 +2497,7 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, if (s.ok()) { SuperVersionContext sv_context(/* create_superversion */ true); for (auto cfd : *impl->versions_->GetColumnFamilySet()) { - impl->InstallSuperVersionAndScheduleWork( - cfd, &sv_context, *cfd->GetLatestMutableCFOptions()); + impl->InstallSuperVersionAndScheduleWork(cfd, &sv_context); } sv_context.Clean(); } @@ -2513,7 +2512,7 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, if (!cfd->mem()->IsSnapshotSupported()) { impl->is_snapshot_supported_ = false; } - if (cfd->ioptions()->merge_operator != nullptr && + if (cfd->ioptions().merge_operator != nullptr && !cfd->mem()->IsMergeOperatorSupported()) { s = Status::InvalidArgument( "The memtable of column family %s does not support merge operator " diff --git a/db/db_impl/db_impl_readonly.cc b/db/db_impl/db_impl_readonly.cc index 2fb8472af93..4a49870f6b3 100644 --- a/db/db_impl/db_impl_readonly.cc +++ b/db/db_impl/db_impl_readonly.cc @@ -172,7 +172,7 @@ Iterator* DBImplReadOnly::NewIterator(const ReadOptions& _read_options, : latest_snapshot; ReadCallback* read_callback = nullptr; // No read callback provided. auto db_iter = NewArenaWrappedDbIterator( - env_, read_options, *cfd->ioptions(), super_version->mutable_cf_options, + env_, read_options, cfd->ioptions(), super_version->mutable_cf_options, super_version->current, read_seq, super_version->mutable_cf_options.max_sequential_skip_in_iterations, super_version->version_number, read_callback); @@ -239,7 +239,7 @@ Status DBImplReadOnly::NewIterators( assert(cfd_to_sv.size() == column_families.size()); for (auto [cfd, sv] : cfd_to_sv) { auto* db_iter = NewArenaWrappedDbIterator( - env_, read_options, *cfd->ioptions(), sv->mutable_cf_options, + env_, read_options, cfd->ioptions(), sv->mutable_cf_options, sv->current, read_seq, sv->mutable_cf_options.max_sequential_skip_in_iterations, sv->version_number, read_callback); diff --git a/db/db_impl/db_impl_secondary.cc b/db/db_impl/db_impl_secondary.cc index 3c1a559ccfc..7586bc3d128 100644 --- a/db/db_impl/db_impl_secondary.cc +++ b/db/db_impl/db_impl_secondary.cc @@ -55,9 +55,9 @@ Status DBImplSecondary::Recover( // Initial max_total_in_memory_state_ before recovery logs. max_total_in_memory_state_ = 0; for (auto cfd : *versions_->GetColumnFamilySet()) { - auto* mutable_cf_options = cfd->GetLatestMutableCFOptions(); - max_total_in_memory_state_ += mutable_cf_options->write_buffer_size * - mutable_cf_options->max_write_buffer_number; + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); + max_total_in_memory_state_ += mutable_cf_options.write_buffer_size * + mutable_cf_options.max_write_buffer_number; } if (s.ok()) { default_cf_handle_ = new ColumnFamilyHandleImpl( @@ -270,10 +270,8 @@ Status DBImplSecondary::RecoverLogFiles( if (!cfd->mem()->IsEmpty() && (curr_log_num == std::numeric_limits::max() || curr_log_num != log_number)) { - const MutableCFOptions mutable_cf_options = - *cfd->GetLatestMutableCFOptions(); - MemTable* new_mem = - cfd->ConstructNewMemtable(mutable_cf_options, seq_of_batch); + MemTable* new_mem = cfd->ConstructNewMemtable( + cfd->GetLatestMutableCFOptions(), seq_of_batch); cfd->mem()->SetNextLogNumber(log_number); cfd->mem()->ConstructFragmentedRangeTombstones(); cfd->imm()->Add(cfd->mem(), &job_context->memtables_to_free); @@ -533,7 +531,7 @@ ArenaWrappedDBIter* DBImplSecondary::NewIteratorImpl( snapshot = versions_->LastSequence(); assert(snapshot != kMaxSequenceNumber); auto db_iter = NewArenaWrappedDbIterator( - env_, read_options, *cfh->cfd()->ioptions(), + env_, read_options, cfh->cfd()->ioptions(), super_version->mutable_cf_options, super_version->current, snapshot, super_version->mutable_cf_options.max_sequential_skip_in_iterations, super_version->version_number, read_callback, cfh, expose_blob_index, @@ -864,16 +862,15 @@ Status DBImplSecondary::CompactWithoutInstallation( ColumnFamilyMetaData cf_meta; version->GetColumnFamilyMetaData(&cf_meta); - const MutableCFOptions* mutable_cf_options = cfd->GetLatestMutableCFOptions(); - ColumnFamilyOptions cf_options = cfd->GetLatestCFOptions(); VersionStorageInfo* vstorage = version->storage_info(); // Use comp_options to reuse some CompactFiles functions CompactionOptions comp_options; comp_options.compression = kDisableCompressionOption; comp_options.output_file_size_limit = MaxFileSizeForLevel( - *mutable_cf_options, input.output_level, cf_options.compaction_style, - vstorage->base_level(), cf_options.level_compaction_dynamic_level_bytes); + cfd->GetLatestMutableCFOptions(), input.output_level, + cfd->ioptions().compaction_style, vstorage->base_level(), + cfd->ioptions().level_compaction_dynamic_level_bytes); std::vector input_files; Status s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers( @@ -886,7 +883,7 @@ Status DBImplSecondary::CompactWithoutInstallation( assert(cfd->compaction_picker()); c.reset(cfd->compaction_picker()->CompactFiles( comp_options, input_files, input.output_level, vstorage, - *mutable_cf_options, mutable_db_options_, 0)); + cfd->GetLatestMutableCFOptions(), mutable_db_options_, 0)); assert(c != nullptr); c->FinalizeInputInfo(version); diff --git a/db/db_impl/db_impl_write.cc b/db/db_impl/db_impl_write.cc index 39b03dc7e3a..a85f23a3213 100644 --- a/db/db_impl/db_impl_write.cc +++ b/db/db_impl/db_impl_write.cc @@ -67,7 +67,7 @@ Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family, return s; } auto cfh = static_cast_with_check(column_family); - if (!cfh->cfd()->ioptions()->merge_operator) { + if (!cfh->cfd()->ioptions().merge_operator) { return Status::NotSupported("Provide a merge_operator when opening DB"); } else { return DB::Merge(o, column_family, key, val); @@ -232,8 +232,8 @@ Status DBImpl::IngestWBWI(std::shared_ptr wbwi, return s; } WBWIMemTable* wbwi_memtable = - new WBWIMemTable(wbwi, cfd->user_comparator(), cf_id, cfd->ioptions(), - cfd->GetLatestMutableCFOptions(), stat); + new WBWIMemTable(wbwi, cfd->user_comparator(), cf_id, &cfd->ioptions(), + &cfd->GetLatestMutableCFOptions(), stat); wbwi_memtable->Ref(); wbwi_memtable->AssignSequenceNumbers(assigned_seqno); // This is needed to keep the WAL that contains Prepare alive until @@ -2422,7 +2422,8 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context, } uint64_t new_log_number = creating_new_log ? versions_->NewFileNumber() : logfile_number_; - const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions(); + // For use outside of holding DB mutex + MutableCFOptions mutable_cf_options_copy = cfd->GetLatestMutableCFOptions(); // Set memtable_info for memtable sealed callback // TODO: memtable_info for `new_imm` @@ -2432,7 +2433,7 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context, memtable_info.earliest_seqno = cfd->mem()->GetEarliestSequenceNumber(); memtable_info.num_entries = cfd->mem()->NumEntries(); memtable_info.num_deletes = cfd->mem()->NumDeletion(); - if (!cfd->ioptions()->persist_user_defined_timestamps && + if (!cfd->ioptions().persist_user_defined_timestamps && cfd->user_comparator()->timestamp_size() > 0) { const Slice& newest_udt = cfd->mem()->GetNewestUDT(); memtable_info.newest_udt.assign(newest_udt.data(), newest_udt.size()); @@ -2441,7 +2442,7 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context, // flush happens before logging, but that should be ok. int num_imm_unflushed = cfd->imm()->NumNotFlushed(); const auto preallocate_block_size = - GetWalPreallocateBlockSize(mutable_cf_options.write_buffer_size); + GetWalPreallocateBlockSize(mutable_cf_options_copy.write_buffer_size); mutex_.Unlock(); if (creating_new_log) { PredecessorWALInfo info; @@ -2474,8 +2475,8 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context, } else { seq = versions_->LastSequence(); } - new_mem = - cfd->ConstructNewMemtable(mutable_cf_options, /*earliest_seq=*/seq); + new_mem = cfd->ConstructNewMemtable(mutable_cf_options_copy, + /*earliest_seq=*/seq); context->superversion_context.NewSuperVersion(); ROCKS_LOG_INFO(immutable_db_options_.info_log, @@ -2630,8 +2631,7 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context, } new_mem->Ref(); cfd->SetMemtable(new_mem); - InstallSuperVersionAndScheduleWork(cfd, &context->superversion_context, - mutable_cf_options); + InstallSuperVersionAndScheduleWork(cfd, &context->superversion_context); // Notify client that memtable is sealed, now that we have successfully // installed a new memtable diff --git a/db/external_sst_file_ingestion_job.cc b/db/external_sst_file_ingestion_job.cc index 2fbb3ff7e3d..a439189afa7 100644 --- a/db/external_sst_file_ingestion_job.cc +++ b/db/external_sst_file_ingestion_job.cc @@ -115,7 +115,7 @@ Status ExternalSstFileIngestionJob::Prepare( f.copy_file = false; const std::string path_outside_db = f.external_file_path; const std::string path_inside_db = TableFileName( - cfd_->ioptions()->cf_paths, f.fd.GetNumber(), f.fd.GetPathId()); + cfd_->ioptions().cf_paths, f.fd.GetNumber(), f.fd.GetPathId()); if (ingestion_options_.move_files || ingestion_options_.link_files) { status = fs_->LinkFile(path_outside_db, path_inside_db, IOOptions(), nullptr); @@ -463,7 +463,7 @@ Status ExternalSstFileIngestionJob::AssignLevelsForOneBatch( status = CheckLevelForIngestedBehindFile(file); } else { status = AssignLevelAndSeqnoForIngestedFile( - super_version, force_global_seqno, cfd_->ioptions()->compaction_style, + super_version, force_global_seqno, cfd_->ioptions().compaction_style, *last_seqno, file, &assigned_seqno, prev_batch_uppermost_level); } @@ -584,13 +584,13 @@ void ExternalSstFileIngestionJob::CreateEquivalentFileIngestingCompactions() { int output_level = pair.first; const CompactionInputFiles& input = pair.second; - const auto& mutable_cf_options = *(cfd_->GetLatestMutableCFOptions()); + const auto& mutable_cf_options = cfd_->GetLatestMutableCFOptions(); file_ingesting_compactions_.push_back(new Compaction( - cfd_->current()->storage_info(), *cfd_->ioptions(), mutable_cf_options, + cfd_->current()->storage_info(), cfd_->ioptions(), mutable_cf_options, mutable_db_options_, {input}, output_level, /* output file size limit not applicable */ MaxFileSizeForLevel(mutable_cf_options, output_level, - cfd_->ioptions()->compaction_style), + cfd_->ioptions().compaction_style), LLONG_MAX /* max compaction bytes, not applicable */, 0 /* output path ID, not applicable */, mutable_cf_options.compression, mutable_cf_options.compression_opts, @@ -749,7 +749,7 @@ Status ExternalSstFileIngestionJob::ResetTableReader( status = sv->mutable_cf_options.table_factory->NewTableReader( ro, TableReaderOptions( - *cfd_->ioptions(), sv->mutable_cf_options.prefix_extractor, + cfd_->ioptions(), sv->mutable_cf_options.prefix_extractor, env_options_, cfd_->internal_comparator(), sv->mutable_cf_options.block_protection_bytes_per_key, /*skip_filters*/ false, /*immortal*/ false, @@ -843,7 +843,7 @@ Status ExternalSstFileIngestionJob::SanityCheckTableProperties( bool mark_sst_file_has_no_udt = false; Status s = ValidateUserDefinedTimestampsOptions( cfd_->user_comparator(), props->comparator_name, - cfd_->ioptions()->persist_user_defined_timestamps, + cfd_->ioptions().persist_user_defined_timestamps, file_to_ingest->user_defined_timestamps_persisted, &mark_sst_file_has_no_udt); if (s.ok() && mark_sst_file_has_no_udt) { diff --git a/db/flush_job.cc b/db/flush_job.cc index e4955a4e25a..3e573d5ee08 100644 --- a/db/flush_job.cc +++ b/db/flush_job.cc @@ -448,7 +448,7 @@ Status FlushJob::MemPurge() { NewMergingIterator(&(cfd_->internal_comparator()), memtables.data(), static_cast(memtables.size()), &arena)); - auto* ioptions = cfd_->ioptions(); + const auto& ioptions = cfd_->ioptions(); // Place iterator at the First (meaning most recent) key node. iter->SeekToFirst(); @@ -469,8 +469,8 @@ Status FlushJob::MemPurge() { // MaxSize is the size of a memtable. size_t maxSize = mutable_cf_options_.write_buffer_size; std::unique_ptr compaction_filter; - if (ioptions->compaction_filter_factory != nullptr && - ioptions->compaction_filter_factory->ShouldFilterTableFileCreation( + if (ioptions.compaction_filter_factory != nullptr && + ioptions.compaction_filter_factory->ShouldFilterTableFileCreation( TableFileCreationReason::kFlush)) { CompactionFilter::Context ctx; ctx.is_full_compaction = false; @@ -478,7 +478,7 @@ Status FlushJob::MemPurge() { ctx.column_family_id = cfd_->GetID(); ctx.reason = TableFileCreationReason::kFlush; compaction_filter = - ioptions->compaction_filter_factory->CreateCompactionFilter(ctx); + ioptions.compaction_filter_factory->CreateCompactionFilter(ctx); if (compaction_filter != nullptr && !compaction_filter->IgnoreSnapshots()) { s = Status::NotSupported( @@ -488,7 +488,7 @@ Status FlushJob::MemPurge() { } } - new_mem = new MemTable((cfd_->internal_comparator()), *(cfd_->ioptions()), + new_mem = new MemTable(cfd_->internal_comparator(), cfd_->ioptions(), mutable_cf_options_, cfd_->write_buffer_mgr(), earliest_seqno, cfd_->GetID()); assert(new_mem != nullptr); @@ -497,8 +497,8 @@ Status FlushJob::MemPurge() { assert(env); MergeHelper merge( env, (cfd_->internal_comparator()).user_comparator(), - (ioptions->merge_operator).get(), compaction_filter.get(), - ioptions->logger, true /* internal key corruption is not ok */, + (ioptions.merge_operator).get(), compaction_filter.get(), + ioptions.logger, true /* internal key corruption is not ok */, existing_snapshots_.empty() ? 0 : existing_snapshots_.back(), snapshot_checker_); assert(job_context_); @@ -508,14 +508,14 @@ Status FlushJob::MemPurge() { iter.get(), (cfd_->internal_comparator()).user_comparator(), &merge, kMaxSequenceNumber, &existing_snapshots_, earliest_snapshot_, earliest_write_conflict_snapshot_, job_snapshot_seq, snapshot_checker_, - env, ShouldReportDetailedTime(env, ioptions->stats), + env, ShouldReportDetailedTime(env, ioptions.stats), true /* internal key corruption is not ok */, range_del_agg.get(), - nullptr, ioptions->allow_data_in_errors, - ioptions->enforce_single_del_contracts, + nullptr, ioptions.allow_data_in_errors, + ioptions.enforce_single_del_contracts, /*manual_compaction_canceled=*/kManualCompactionCanceledFalse, false /* must_count_input_entries */, /*compaction=*/nullptr, compaction_filter.get(), - /*shutting_down=*/nullptr, ioptions->info_log, full_history_ts_low); + /*shutting_down=*/nullptr, ioptions.info_log, full_history_ts_low); // Set earliest sequence number in the new memtable // to be equal to the earliest sequence number of the @@ -865,7 +865,7 @@ Status FlushJob::WriteLevel0Table() { assert(ucmp); const size_t ts_sz = ucmp->timestamp_size(); const bool logical_strip_timestamp = - ts_sz > 0 && !cfd_->ioptions()->persist_user_defined_timestamps; + ts_sz > 0 && !cfd_->ioptions().persist_user_defined_timestamps; std::vector blob_file_additions; @@ -986,7 +986,7 @@ Status FlushJob::WriteLevel0Table() { read_options.rate_limiter_priority = io_priority; const WriteOptions write_options(io_priority, Env::IOActivity::kFlush); TableBuilderOptions tboptions( - *cfd_->ioptions(), mutable_cf_options_, read_options, write_options, + cfd_->ioptions(), mutable_cf_options_, read_options, write_options, cfd_->internal_comparator(), cfd_->internal_tbl_prop_coll_factories(), output_compression_, mutable_cf_options_.compression_opts, cfd_->GetID(), cfd_->GetName(), 0 /* level */, @@ -1140,7 +1140,7 @@ std::unique_ptr FlushJob::GetFlushJobInfo() const { const uint64_t file_number = meta_.fd.GetNumber(); info->file_path = - MakeTableFileName(cfd_->ioptions()->cf_paths[0].path, file_number); + MakeTableFileName(cfd_->ioptions().cf_paths[0].path, file_number); info->file_number = file_number; info->oldest_blob_file_number = meta_.oldest_blob_file_number; info->thread_id = db_options_.env->GetThreadID(); @@ -1154,7 +1154,7 @@ std::unique_ptr FlushJob::GetFlushJobInfo() const { // Update BlobFilesInfo. for (const auto& blob_file : edit_->GetBlobFileAdditions()) { BlobFileAdditionInfo blob_file_addition_info( - BlobFileName(cfd_->ioptions()->cf_paths.front().path, + BlobFileName(cfd_->ioptions().cf_paths.front().path, blob_file.GetBlobFileNumber()) /*blob_file_path*/, blob_file.GetBlobFileNumber(), blob_file.GetTotalBlobCount(), blob_file.GetTotalBlobBytes()); @@ -1171,7 +1171,7 @@ void FlushJob::GetEffectiveCutoffUDTForPickedMemTables() { assert(ucmp); const size_t ts_sz = ucmp->timestamp_size(); if (db_options_.atomic_flush || ts_sz == 0 || - cfd_->ioptions()->persist_user_defined_timestamps) { + cfd_->ioptions().persist_user_defined_timestamps) { return; } // Find the newest user-defined timestamps from all the flushed memtables. @@ -1238,7 +1238,7 @@ Status FlushJob::MaybeIncreaseFullHistoryTsLowToAboveCutoffUDT() { VersionEdit edit; edit.SetColumnFamily(cfd_->GetID()); edit.SetFullHistoryTsLow(new_full_history_ts_low); - return versions_->LogAndApply(cfd_, *cfd_->GetLatestMutableCFOptions(), + return versions_->LogAndApply(cfd_, cfd_->GetLatestMutableCFOptions(), ReadOptions(Env::IOActivity::kFlush), WriteOptions(Env::IOActivity::kFlush), &edit, db_mutex_, output_file_directory_); diff --git a/db/flush_job_test.cc b/db/flush_job_test.cc index 45b7f73d9be..deb7d739f25 100644 --- a/db/flush_job_test.cc +++ b/db/flush_job_test.cc @@ -189,7 +189,7 @@ TEST_F(FlushJobTest, Empty) { SnapshotChecker* snapshot_checker = nullptr; // not relavant FlushJob flush_job( dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, - *cfd->GetLatestMutableCFOptions(), + cfd->GetLatestMutableCFOptions(), std::numeric_limits::max() /* memtable_id */, env_options_, versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, @@ -207,7 +207,7 @@ TEST_F(FlushJobTest, Empty) { TEST_F(FlushJobTest, NonEmpty) { JobContext job_context(0); auto cfd = versions_->GetColumnFamilySet()->GetDefault(); - auto new_mem = cfd->ConstructNewMemtable(*cfd->GetLatestMutableCFOptions(), + auto new_mem = cfd->ConstructNewMemtable(cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); new_mem->Ref(); auto inserted_keys = mock::MakeMockFile(); @@ -275,7 +275,7 @@ TEST_F(FlushJobTest, NonEmpty) { SnapshotChecker* snapshot_checker = nullptr; // not relavant FlushJob flush_job( dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, - *cfd->GetLatestMutableCFOptions(), + cfd->GetLatestMutableCFOptions(), std::numeric_limits::max() /* memtable_id */, env_options_, versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, @@ -310,7 +310,7 @@ TEST_F(FlushJobTest, FlushMemTablesSingleColumnFamily) { std::vector memtable_ids; std::vector new_mems; for (size_t i = 0; i != num_mems; ++i) { - MemTable* mem = cfd->ConstructNewMemtable(*cfd->GetLatestMutableCFOptions(), + MemTable* mem = cfd->ConstructNewMemtable(cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); mem->SetID(i); mem->Ref(); @@ -339,7 +339,7 @@ TEST_F(FlushJobTest, FlushMemTablesSingleColumnFamily) { uint64_t flush_memtable_id = smallest_memtable_id + num_mems_to_flush - 1; FlushJob flush_job( dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, - *cfd->GetLatestMutableCFOptions(), flush_memtable_id, env_options_, + cfd->GetLatestMutableCFOptions(), flush_memtable_id, env_options_, versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, nullptr, kNoCompression, db_options_.statistics.get(), &event_logger, @@ -387,7 +387,7 @@ TEST_F(FlushJobTest, FlushMemtablesMultipleColumnFamilies) { smallest_seqs.push_back(curr_seqno); for (size_t i = 0; i != num_memtables[k]; ++i) { MemTable* mem = cfd->ConstructNewMemtable( - *cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); + cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); mem->SetID(i); mem->Ref(); @@ -411,7 +411,7 @@ TEST_F(FlushJobTest, FlushMemtablesMultipleColumnFamilies) { for (auto cfd : all_cfds) { std::vector snapshot_seqs; flush_jobs.emplace_back(new FlushJob( - dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(), + dbname_, cfd, db_options_, cfd->GetLatestMutableCFOptions(), memtable_ids[k], env_options_, versions_.get(), &mutex_, &shutting_down_, snapshot_seqs, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, nullptr, @@ -446,7 +446,7 @@ TEST_F(FlushJobTest, FlushMemtablesMultipleColumnFamilies) { } autovector mutable_cf_options_list; for (auto cfd : all_cfds) { - mutable_cf_options_list.push_back(cfd->GetLatestMutableCFOptions()); + mutable_cf_options_list.push_back(&cfd->GetLatestMutableCFOptions()); } autovector>*> committed_flush_jobs_info; @@ -489,7 +489,7 @@ TEST_F(FlushJobTest, FlushMemtablesMultipleColumnFamilies) { TEST_F(FlushJobTest, Snapshots) { JobContext job_context(0); auto cfd = versions_->GetColumnFamilySet()->GetDefault(); - auto new_mem = cfd->ConstructNewMemtable(*cfd->GetLatestMutableCFOptions(), + auto new_mem = cfd->ConstructNewMemtable(cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); std::set snapshots_set; @@ -539,7 +539,7 @@ TEST_F(FlushJobTest, Snapshots) { SnapshotChecker* snapshot_checker = nullptr; // not relavant FlushJob flush_job( dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, - *cfd->GetLatestMutableCFOptions(), + cfd->GetLatestMutableCFOptions(), std::numeric_limits::max() /* memtable_id */, env_options_, versions_.get(), &mutex_, &shutting_down_, snapshots, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, @@ -567,7 +567,7 @@ TEST_F(FlushJobTest, GetRateLimiterPriorityForWrite) { std::vector memtable_ids; std::vector new_mems; for (size_t i = 0; i != num_mems; ++i) { - MemTable* mem = cfd->ConstructNewMemtable(*cfd->GetLatestMutableCFOptions(), + MemTable* mem = cfd->ConstructNewMemtable(cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); mem->SetID(i); mem->Ref(); @@ -596,7 +596,7 @@ TEST_F(FlushJobTest, GetRateLimiterPriorityForWrite) { uint64_t flush_memtable_id = smallest_memtable_id + num_mems_to_flush - 1; FlushJob flush_job( dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, - *cfd->GetLatestMutableCFOptions(), flush_memtable_id, env_options_, + cfd->GetLatestMutableCFOptions(), flush_memtable_id, env_options_, versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, nullptr, kNoCompression, db_options_.statistics.get(), &event_logger, @@ -627,7 +627,7 @@ TEST_F(FlushJobTest, GetRateLimiterPriorityForWrite) { TEST_F(FlushJobTest, ReplaceTimedPutWriteTimeWithPreferredSeqno) { JobContext job_context(0); auto cfd = versions_->GetColumnFamilySet()->GetDefault(); - auto new_mem = cfd->ConstructNewMemtable(*cfd->GetLatestMutableCFOptions(), + auto new_mem = cfd->ConstructNewMemtable(cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); new_mem->Ref(); std::shared_ptr seqno_to_time_mapping = @@ -665,7 +665,7 @@ TEST_F(FlushJobTest, ReplaceTimedPutWriteTimeWithPreferredSeqno) { SnapshotChecker* snapshot_checker = nullptr; // not relevant FlushJob flush_job( dbname_, versions_->GetColumnFamilySet()->GetDefault(), db_options_, - *cfd->GetLatestMutableCFOptions(), + cfd->GetLatestMutableCFOptions(), std::numeric_limits::max() /* memtable_id */, env_options_, versions_.get(), &mutex_, &shutting_down_, {}, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, @@ -748,7 +748,7 @@ TEST_P(FlushJobTimestampTest, AllKeysExpired) { { MemTable* new_mem = cfd->ConstructNewMemtable( - *cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); + cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); new_mem->Ref(); for (int i = 0; i < 100; ++i) { uint64_t ts = curr_ts_.fetch_add(1); @@ -772,7 +772,7 @@ TEST_P(FlushJobTimestampTest, AllKeysExpired) { PutFixed64(&full_history_ts_low, std::numeric_limits::max()); cfd->SetFullHistoryTsLow(full_history_ts_low); FlushJob flush_job( - dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(), + dbname_, cfd, db_options_, cfd->GetLatestMutableCFOptions(), std::numeric_limits::max() /* memtable_id */, env_options_, versions_.get(), &mutex_, &shutting_down_, snapshots, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, @@ -814,7 +814,7 @@ TEST_P(FlushJobTimestampTest, NoKeyExpired) { { MemTable* new_mem = cfd->ConstructNewMemtable( - *cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); + cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); new_mem->Ref(); for (int i = 0; i < 100; ++i) { uint64_t ts = curr_ts_.fetch_add(1); @@ -834,7 +834,7 @@ TEST_P(FlushJobTimestampTest, NoKeyExpired) { PutFixed64(&full_history_ts_low, 0); cfd->SetFullHistoryTsLow(full_history_ts_low); FlushJob flush_job( - dbname_, cfd, db_options_, *cfd->GetLatestMutableCFOptions(), + dbname_, cfd, db_options_, cfd->GetLatestMutableCFOptions(), std::numeric_limits::max() /* memtable_id */, env_options_, versions_.get(), &mutex_, &shutting_down_, snapshots, kMaxSequenceNumber, snapshot_checker, &job_context, FlushReason::kTest, nullptr, nullptr, diff --git a/db/forward_iterator.cc b/db/forward_iterator.cc index 6e7e1e6de50..f7c507d49fe 100644 --- a/db/forward_iterator.cc +++ b/db/forward_iterator.cc @@ -242,7 +242,7 @@ ForwardIterator::ForwardIterator(DBImpl* db, const ReadOptions& read_options, if (sv_) { RebuildIterators(false); } - if (!CheckFSFeatureSupport(cfd_->ioptions()->env->GetFileSystem().get(), + if (!CheckFSFeatureSupport(cfd_->ioptions().env->GetFileSystem().get(), FSSupportedOps::kAsyncIO)) { read_options_.async_io = false; } diff --git a/db/import_column_family_job.cc b/db/import_column_family_job.cc index f875a16a3e9..44a1c5d099a 100644 --- a/db/import_column_family_job.cc +++ b/db/import_column_family_job.cc @@ -108,7 +108,7 @@ Status ImportColumnFamilyJob::Prepare(uint64_t next_file_number, for (auto& f : files_to_import_per_cf) { const auto path_outside_db = f.external_file_path; const auto path_inside_db = TableFileName( - cfd_->ioptions()->cf_paths, f.fd.GetNumber(), f.fd.GetPathId()); + cfd_->ioptions().cf_paths, f.fd.GetNumber(), f.fd.GetPathId()); if (hardlink_files) { status = fs_->LinkFile(path_outside_db, path_inside_db, IOOptions(), @@ -186,16 +186,16 @@ Status ImportColumnFamilyJob::Run() { // RecoverEpochNumbers() below. for (size_t i = 0; s.ok() && i < files_to_import_.size(); ++i) { VersionBuilder dummy_version_builder( - cfd_->current()->version_set()->file_options(), cfd_->ioptions(), + cfd_->current()->version_set()->file_options(), &cfd_->ioptions(), cfd_->table_cache(), cfd_->current()->storage_info(), cfd_->current()->version_set(), cfd_->GetFileMetadataCacheReservationManager()); VersionStorageInfo dummy_vstorage( &cfd_->internal_comparator(), cfd_->user_comparator(), - cfd_->NumberLevels(), cfd_->ioptions()->compaction_style, - nullptr /* src_vstorage */, cfd_->ioptions()->force_consistency_checks, - EpochNumberRequirement::kMightMissing, cfd_->ioptions()->clock, - cfd_->GetLatestMutableCFOptions()->bottommost_file_compaction_delay, + cfd_->NumberLevels(), cfd_->ioptions().compaction_style, + nullptr /* src_vstorage */, cfd_->ioptions().force_consistency_checks, + EpochNumberRequirement::kMightMissing, cfd_->ioptions().clock, + cfd_->GetLatestMutableCFOptions().bottommost_file_compaction_delay, cfd_->current()->version_set()->offpeak_time_option()); for (size_t j = 0; s.ok() && j < files_to_import_[i].size(); ++j) { const auto& f = files_to_import_[i][j]; @@ -330,7 +330,7 @@ Status ImportColumnFamilyJob::GetIngestedFileInfo( // creating `TableReaderOptions` when the support is there. status = sv->mutable_cf_options.table_factory->NewTableReader( TableReaderOptions( - *cfd_->ioptions(), sv->mutable_cf_options.prefix_extractor, + cfd_->ioptions(), sv->mutable_cf_options.prefix_extractor, env_options_, cfd_->internal_comparator(), sv->mutable_cf_options.block_protection_bytes_per_key, /*skip_filters*/ false, /*immortal*/ false, diff --git a/db/internal_stats.cc b/db/internal_stats.cc index 47c7e1c9a91..bfc66731a1b 100644 --- a/db/internal_stats.cc +++ b/db/internal_stats.cc @@ -924,7 +924,7 @@ bool InternalStats::HandleLiveBlobFileGarbageSize(uint64_t* value, } Cache* InternalStats::GetBlobCacheForStats() { - return cfd_->ioptions()->blob_cache.get(); + return cfd_->ioptions().blob_cache.get(); } bool InternalStats::HandleBlobCacheCapacity(uint64_t* value, DBImpl* /*db*/, @@ -1468,9 +1468,9 @@ bool InternalStats::HandleEstimateOldestKeyTime(uint64_t* value, DBImpl* /*db*/, // TODO(yiwu): The property is currently available for fifo compaction // with allow_compaction = false. This is because we don't propagate // oldest_key_time on compaction. - if (cfd_->ioptions()->compaction_style != kCompactionStyleFIFO || + if (cfd_->ioptions().compaction_style != kCompactionStyleFIFO || cfd_->GetCurrentMutableCFOptions() - ->compaction_options_fifo.allow_compaction) { + .compaction_options_fifo.allow_compaction) { return false; } // TODO: plumb Env::IOActivity, Env::IOPriority @@ -1496,7 +1496,7 @@ bool InternalStats::HandleEstimateOldestKeyTime(uint64_t* value, DBImpl* /*db*/, Cache* InternalStats::GetBlockCacheForStats() { // NOTE: called in startup before GetCurrentMutableCFOptions() is ready - auto* table_factory = cfd_->GetLatestMutableCFOptions()->table_factory.get(); + auto* table_factory = cfd_->GetLatestMutableCFOptions().table_factory.get(); assert(table_factory != nullptr); // FIXME: need to a shared_ptr if/when block_cache is going to be mutable return table_factory->GetOptions(TableFactory::kBlockCacheOpts()); @@ -1752,7 +1752,7 @@ void InternalStats::DumpCFMapStats( assert(vstorage); int num_levels_to_check = - (cfd_->ioptions()->compaction_style == kCompactionStyleLevel) + (cfd_->ioptions().compaction_style == kCompactionStyleLevel) ? vstorage->num_levels() - 1 : 1; @@ -2163,8 +2163,7 @@ class BlockCachePropertyAggregator : public IntPropertyAggregator { virtual ~BlockCachePropertyAggregator() override = default; void Add(ColumnFamilyData* cfd, uint64_t value) override { - auto* table_factory = - cfd->GetCurrentMutableCFOptions()->table_factory.get(); + auto* table_factory = cfd->GetCurrentMutableCFOptions().table_factory.get(); assert(table_factory != nullptr); Cache* cache = table_factory->GetOptions(TableFactory::kBlockCacheOpts()); diff --git a/db/obsolete_files_test.cc b/db/obsolete_files_test.cc index 10fa6914330..eb3ed078c79 100644 --- a/db/obsolete_files_test.cc +++ b/db/obsolete_files_test.cc @@ -194,11 +194,10 @@ TEST_F(ObsoleteFilesTest, BlobFiles) { ColumnFamilyData* const cfd = versions->GetColumnFamilySet()->GetDefault(); assert(cfd); - const ImmutableCFOptions* const ioptions = cfd->ioptions(); - assert(ioptions); - assert(!ioptions->cf_paths.empty()); + const auto& cf_paths = cfd->ioptions().cf_paths; + assert(!cf_paths.empty()); - const std::string& path = ioptions->cf_paths.front().path; + const std::string& path = cf_paths.front().path; // Add an obsolete blob file. constexpr uint64_t first_blob_file_number = 234; diff --git a/db/repair.cc b/db/repair.cc index c207736038a..1713baf0a48 100644 --- a/db/repair.cc +++ b/db/repair.cc @@ -389,7 +389,7 @@ class Repairer { // Initialize per-column family memtables for (auto* cfd : *vset_.GetColumnFamilySet()) { - cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(), + cfd->CreateNewMemtable(cfd->GetLatestMutableCFOptions(), kMaxSequenceNumber); } auto cf_mems = new ColumnFamilyMemTablesImpl(vset_.GetColumnFamilySet()); @@ -472,7 +472,7 @@ class Repairer { const ReadOptions read_options; const WriteOptions write_option(Env::IO_HIGH); TableBuilderOptions tboptions( - *cfd->ioptions(), *cfd->GetLatestMutableCFOptions(), read_options, + cfd->ioptions(), cfd->GetLatestMutableCFOptions(), read_options, write_option, cfd->internal_comparator(), cfd->internal_tbl_prop_coll_factories(), kNoCompression, default_compression, cfd->GetID(), cfd->GetName(), -1 /* level */, @@ -605,7 +605,7 @@ class Repairer { ropts.total_order_seek = true; InternalIterator* iter = table_cache_->NewIterator( ropts, file_options_, cfd->internal_comparator(), t->meta, - nullptr /* range_del_agg */, *cfd->GetLatestMutableCFOptions(), + nullptr /* range_del_agg */, cfd->GetLatestMutableCFOptions(), /*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr, TableReaderCaller::kRepair, /*arena=*/nullptr, /*skip_filters=*/false, /*level=*/-1, /*max_file_size_for_l0_meta_pin=*/0, @@ -652,7 +652,7 @@ class Repairer { std::unique_ptr r_iter; status = table_cache_->GetRangeTombstoneIterator( ropts, cfd->internal_comparator(), t->meta, - *cfd->GetLatestMutableCFOptions(), &r_iter); + cfd->GetLatestMutableCFOptions(), &r_iter); if (r_iter) { r_iter->SeekToFirst(); @@ -693,15 +693,15 @@ class Repairer { // Recover files' epoch number using dummy VersionStorageInfo VersionBuilder dummy_version_builder( - cfd->current()->version_set()->file_options(), cfd->ioptions(), + cfd->current()->version_set()->file_options(), &cfd->ioptions(), cfd->table_cache(), cfd->current()->storage_info(), cfd->current()->version_set(), cfd->GetFileMetadataCacheReservationManager()); VersionStorageInfo dummy_vstorage( &cfd->internal_comparator(), cfd->user_comparator(), - cfd->NumberLevels(), cfd->ioptions()->compaction_style, - nullptr /* src_vstorage */, cfd->ioptions()->force_consistency_checks, - EpochNumberRequirement::kMightMissing, cfd->ioptions()->clock, + cfd->NumberLevels(), cfd->ioptions().compaction_style, + nullptr /* src_vstorage */, cfd->ioptions().force_consistency_checks, + EpochNumberRequirement::kMightMissing, cfd->ioptions().clock, /*bottommost_file_compaction_delay=*/0, cfd->current()->version_set()->offpeak_time_option()); Status s; @@ -733,7 +733,7 @@ class Repairer { VersionEdit edit; edit.SetComparatorName(cfd->user_comparator()->Name()); edit.SetPersistUserDefinedTimestamps( - cfd->ioptions()->persist_user_defined_timestamps); + cfd->ioptions().persist_user_defined_timestamps); edit.SetLogNumber(0); edit.SetNextFile(next_file_number_); edit.SetColumnFamily(cfd->GetID()); @@ -761,7 +761,7 @@ class Repairer { s = env_->GetFileSystem()->NewDirectory(dbname_, IOOptions(), &db_dir, nullptr); if (s.ok()) { - s = vset_.LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + s = vset_.LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), read_options, write_options, &edit, &mutex_, db_dir.get(), false /* new_descriptor_log */); } diff --git a/db/version_builder.cc b/db/version_builder.cc index acec67fd15e..aa7510c97f9 100644 --- a/db/version_builder.cc +++ b/db/version_builder.cc @@ -1831,7 +1831,7 @@ BaseReferencedVersionBuilder::BaseReferencedVersionBuilder( ColumnFamilyData* cfd, VersionEditHandler* version_edit_handler, bool track_found_and_missing_files, bool allow_incomplete_valid_version) : version_builder_(new VersionBuilder( - cfd->current()->version_set()->file_options(), cfd->ioptions(), + cfd->current()->version_set()->file_options(), &cfd->ioptions(), cfd->table_cache(), cfd->current()->storage_info(), cfd->current()->version_set(), cfd->GetFileMetadataCacheReservationManager(), cfd, @@ -1845,7 +1845,7 @@ BaseReferencedVersionBuilder::BaseReferencedVersionBuilder( ColumnFamilyData* cfd, Version* v, VersionEditHandler* version_edit_handler, bool track_found_and_missing_files, bool allow_incomplete_valid_version) : version_builder_(new VersionBuilder( - cfd->current()->version_set()->file_options(), cfd->ioptions(), + cfd->current()->version_set()->file_options(), &cfd->ioptions(), cfd->table_cache(), v->storage_info(), v->version_set(), cfd->GetFileMetadataCacheReservationManager(), cfd, version_edit_handler, track_found_and_missing_files, diff --git a/db/version_edit_handler.cc b/db/version_edit_handler.cc index ab21a055889..8b88bb4da3f 100644 --- a/db/version_edit_handler.cc +++ b/db/version_edit_handler.cc @@ -506,14 +506,14 @@ Status VersionEditHandler::MaybeCreateVersionBeforeApplyEdit( auto* builder = builder_iter->second->version_builder(); if (force_create_version) { auto* v = new Version(cfd, version_set_, version_set_->file_options_, - *cfd->GetLatestMutableCFOptions(), io_tracer_, + cfd->GetLatestMutableCFOptions(), io_tracer_, version_set_->current_version_number_++, epoch_number_requirement_); s = builder->SaveTo(v->storage_info()); if (s.ok()) { // Install new version v->PrepareAppend( - *cfd->GetLatestMutableCFOptions(), read_options_, + cfd->GetLatestMutableCFOptions(), read_options_, !(version_set_->db_options_->skip_stats_update_on_db_open)); version_set_->AppendVersion(cfd, v); } else { @@ -541,12 +541,12 @@ Status VersionEditHandler::LoadTables(ColumnFamilyData* cfd, assert(builder_iter->second != nullptr); VersionBuilder* builder = builder_iter->second->version_builder(); assert(builder); - const MutableCFOptions* moptions = cfd->GetLatestMutableCFOptions(); + const auto& moptions = cfd->GetLatestMutableCFOptions(); Status s = builder->LoadTableHandlers( cfd->internal_stats(), version_set_->db_options_->max_file_opening_threads, - prefetch_index_and_filter_in_cache, is_initial_load, *moptions, - MaxFileSizeForL0MetaPin(*moptions), read_options_); + prefetch_index_and_filter_in_cache, is_initial_load, moptions, + MaxFileSizeForL0MetaPin(moptions), read_options_); if ((s.IsPathNotFound() || s.IsCorruption()) && no_error_if_files_missing_) { s = Status::OK(); } @@ -582,7 +582,7 @@ Status VersionEditHandler::ExtractInfoFromVersionEdit(ColumnFamilyData* cfd, // it's not recorded and it should have default value true. s = ValidateUserDefinedTimestampsOptions( cfd->user_comparator(), edit.GetComparatorName(), - cfd->ioptions()->persist_user_defined_timestamps, + cfd->ioptions().persist_user_defined_timestamps, edit.GetPersistUserDefinedTimestamps(), &mark_sst_files_has_no_udt); if (!s.ok() && cf_to_cmp_names_) { cf_to_cmp_names_->emplace(cfd->GetID(), edit.GetComparatorName()); @@ -861,15 +861,14 @@ Status VersionEditHandlerPointInTime::MaybeCreateVersionBeforeApplyEdit( if (s.ok() && !missing_info && !in_atomic_group_ && ((!valid_pit_after_edit && valid_pit_before_edit) || (valid_pit_after_edit && force_create_version))) { - const MutableCFOptions* cf_opts_ptr = cfd->GetLatestMutableCFOptions(); - auto* version = new Version(cfd, version_set_, version_set_->file_options_, - *cf_opts_ptr, io_tracer_, - version_set_->current_version_number_++, - epoch_number_requirement_); + const auto& mopts = cfd->GetLatestMutableCFOptions(); + auto* version = new Version( + cfd, version_set_, version_set_->file_options_, mopts, io_tracer_, + version_set_->current_version_number_++, epoch_number_requirement_); s = builder->LoadSavePointTableHandlers( cfd->internal_stats(), - version_set_->db_options_->max_file_opening_threads, false, true, - *cf_opts_ptr, MaxFileSizeForL0MetaPin(*cf_opts_ptr), read_options_); + version_set_->db_options_->max_file_opening_threads, false, true, mopts, + MaxFileSizeForL0MetaPin(mopts), read_options_); if (!s.ok()) { delete version; if (s.IsCorruption()) { @@ -886,7 +885,7 @@ Status VersionEditHandlerPointInTime::MaybeCreateVersionBeforeApplyEdit( } } else { version->PrepareAppend( - *cfd->GetLatestMutableCFOptions(), read_options_, + cfd->GetLatestMutableCFOptions(), read_options_, !version_set_->db_options_->skip_stats_update_on_db_open); auto v_iter = versions_.find(cfd->GetID()); if (v_iter != versions_.end()) { @@ -986,7 +985,7 @@ void VersionEditHandlerPointInTime::AtomicUpdateVersionsApply() { Version* version = cfid_and_version.second; assert(version != nullptr); version->PrepareAppend( - *version->cfd()->GetLatestMutableCFOptions(), read_options_, + version->cfd()->GetLatestMutableCFOptions(), read_options_, !version_set_->db_options_->skip_stats_update_on_db_open); auto versions_iter = versions_.find(cfid); if (versions_iter != versions_.end()) { diff --git a/db/version_set.cc b/db/version_set.cc index d6235365b79..059d4584703 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -862,9 +862,9 @@ Version::~Version() { // to get current mutable options from auto* sv = cfd_->GetSuperVersion(); uint32_t path_id = f->fd.GetPathId(); - assert(path_id < cfd_->ioptions()->cf_paths.size()); + assert(path_id < cfd_->ioptions().cf_paths.size()); vset_->obsolete_files_.emplace_back( - f, cfd_->ioptions()->cf_paths[path_id].path, + f, cfd_->ioptions().cf_paths[path_id].path, sv ? sv->mutable_cf_options.uncache_aggressiveness : 0, cfd_->GetFileMetadataCacheReservationManager()); } @@ -1574,8 +1574,8 @@ Status Version::GetTableProperties(const ReadOptions& read_options, std::shared_ptr* tp, const FileMetaData* file_meta, const std::string* fname) const { - auto table_cache = cfd_->table_cache(); - auto ioptions = cfd_->ioptions(); + auto* table_cache = cfd_->table_cache(); + const auto& ioptions = cfd_->ioptions(); Status s = table_cache->GetTableProperties( file_options_, read_options, cfd_->internal_comparator(), *file_meta, tp, mutable_cf_options_, true /* no io */); @@ -1596,11 +1596,11 @@ Status Version::GetTableProperties(const ReadOptions& read_options, if (fname != nullptr) { file_name = *fname; } else { - file_name = TableFileName(ioptions->cf_paths, file_meta->fd.GetNumber(), + file_name = TableFileName(ioptions.cf_paths, file_meta->fd.GetNumber(), file_meta->fd.GetPathId()); } - s = ioptions->fs->NewRandomAccessFile(file_name, file_options_, &file, - nullptr); + s = ioptions.fs->NewRandomAccessFile(file_name, file_options_, &file, + nullptr); if (!s.ok()) { return s; } @@ -1609,21 +1609,21 @@ Status Version::GetTableProperties(const ReadOptions& read_options, // the magic number check in the footer. std::unique_ptr file_reader( new RandomAccessFileReader( - std::move(file), file_name, ioptions->clock /* clock */, io_tracer_, - ioptions->stats /* stats */, + std::move(file), file_name, ioptions.clock /* clock */, io_tracer_, + ioptions.stats /* stats */, Histograms::SST_READ_MICROS /* hist_type */, nullptr /* file_read_hist */, nullptr /* rate_limiter */, - ioptions->listeners)); + ioptions.listeners)); std::unique_ptr props; s = ReadTableProperties( file_reader.get(), file_meta->fd.GetFileSize(), - Footer::kNullTableMagicNumber /* table's magic number */, *ioptions, + Footer::kNullTableMagicNumber /* table's magic number */, ioptions, read_options, &props); if (!s.ok()) { return s; } *tp = std::move(props); - RecordTick(ioptions->stats, NUMBER_DIRECT_LOAD_TABLE_PROPERTIES); + RecordTick(ioptions.stats, NUMBER_DIRECT_LOAD_TABLE_PROPERTIES); return s; } @@ -1654,7 +1654,7 @@ Status Version::TablesRangeTombstoneSummary(int max_entries_to_print, for (int level = 0; level < storage_info_.num_levels_; level++) { for (const auto& file_meta : storage_info_.files_[level]) { auto fname = - TableFileName(cfd_->ioptions()->cf_paths, file_meta->fd.GetNumber(), + TableFileName(cfd_->ioptions().cf_paths, file_meta->fd.GetNumber(), file_meta->fd.GetPathId()); ss << "=== file : " << fname << " ===\n"; @@ -1702,7 +1702,7 @@ Status Version::GetPropertiesOfAllTables(const ReadOptions& read_options, int level) { for (const auto& file_meta : storage_info_.files_[level]) { auto fname = - TableFileName(cfd_->ioptions()->cf_paths, file_meta->fd.GetNumber(), + TableFileName(cfd_->ioptions().cf_paths, file_meta->fd.GetNumber(), file_meta->fd.GetPathId()); // 1. If the table is already present in table cache, load table // properties from there. @@ -1732,7 +1732,7 @@ Status Version::GetPropertiesOfTablesInRange( false); for (const auto& file_meta : files) { auto fname = - TableFileName(cfd_->ioptions()->cf_paths, file_meta->fd.GetNumber(), + TableFileName(cfd_->ioptions().cf_paths, file_meta->fd.GetNumber(), file_meta->fd.GetPathId()); if (props->count(fname) == 0) { // 1. If the table is already present in table cache, load table @@ -1800,7 +1800,7 @@ void Version::GetColumnFamilyMetaData(ColumnFamilyMetaData* cf_meta) { cf_meta->blob_file_count = 0; cf_meta->blob_files.clear(); - auto* ioptions = cfd_->ioptions(); + const auto& ioptions = cfd_->ioptions(); auto* vstorage = storage_info(); for (int level = 0; level < cfd_->NumberLevels(); level++) { @@ -1810,11 +1810,11 @@ void Version::GetColumnFamilyMetaData(ColumnFamilyMetaData* cf_meta) { for (const auto& file : vstorage->LevelFiles(level)) { uint32_t path_id = file->fd.GetPathId(); std::string file_path; - if (path_id < ioptions->cf_paths.size()) { - file_path = ioptions->cf_paths[path_id].path; + if (path_id < ioptions.cf_paths.size()) { + file_path = ioptions.cf_paths[path_id].path; } else { - assert(!ioptions->cf_paths.empty()); - file_path = ioptions->cf_paths.back().path; + assert(!ioptions.cf_paths.empty()); + file_path = ioptions.cf_paths.back().path; } const uint64_t file_number = file->fd.GetNumber(); files.emplace_back( @@ -1841,7 +1841,7 @@ void Version::GetColumnFamilyMetaData(ColumnFamilyMetaData* cf_meta) { cf_meta->blob_files.emplace_back( meta->GetBlobFileNumber(), BlobFileName("", meta->GetBlobFileNumber()), - ioptions->cf_paths.front().path, meta->GetBlobFileSize(), + ioptions.cf_paths.front().path, meta->GetBlobFileSize(), meta->GetTotalBlobCount(), meta->GetTotalBlobBytes(), meta->GetGarbageBlobCount(), meta->GetGarbageBlobBytes(), meta->GetChecksumMethod(), meta->GetChecksumValue()); @@ -2202,31 +2202,31 @@ VersionStorageInfo::VersionStorageInfo( Version::Version(ColumnFamilyData* column_family_data, VersionSet* vset, const FileOptions& file_opt, - const MutableCFOptions mutable_cf_options, + const MutableCFOptions& mutable_cf_options, const std::shared_ptr& io_tracer, uint64_t version_number, EpochNumberRequirement epoch_number_requirement) : env_(vset->env_), clock_(vset->clock_), cfd_(column_family_data), - info_log_((cfd_ == nullptr) ? nullptr : cfd_->ioptions()->logger), - db_statistics_((cfd_ == nullptr) ? nullptr : cfd_->ioptions()->stats), + info_log_((cfd_ == nullptr) ? nullptr : cfd_->ioptions().logger), + db_statistics_((cfd_ == nullptr) ? nullptr : cfd_->ioptions().stats), table_cache_((cfd_ == nullptr) ? nullptr : cfd_->table_cache()), blob_source_(cfd_ ? cfd_->blob_source() : nullptr), merge_operator_( - (cfd_ == nullptr) ? nullptr : cfd_->ioptions()->merge_operator.get()), + (cfd_ == nullptr) ? nullptr : cfd_->ioptions().merge_operator.get()), storage_info_( (cfd_ == nullptr) ? nullptr : &cfd_->internal_comparator(), (cfd_ == nullptr) ? nullptr : cfd_->user_comparator(), cfd_ == nullptr ? 0 : cfd_->NumberLevels(), cfd_ == nullptr ? kCompactionStyleLevel - : cfd_->ioptions()->compaction_style, + : cfd_->ioptions().compaction_style, (cfd_ == nullptr || cfd_->current() == nullptr) ? nullptr : cfd_->current()->storage_info(), - cfd_ == nullptr ? false : cfd_->ioptions()->force_consistency_checks, + cfd_ == nullptr ? false : cfd_->ioptions().force_consistency_checks, epoch_number_requirement, - cfd_ == nullptr ? nullptr : cfd_->ioptions()->clock, + cfd_ == nullptr ? nullptr : cfd_->ioptions().clock, cfd_ == nullptr ? 0 : mutable_cf_options.bottommost_file_compaction_delay, vset->offpeak_time_option()), @@ -3064,7 +3064,7 @@ Status Version::MultiGetAsync( bool Version::IsFilterSkipped(int level, bool is_file_last_in_level) { // Reaching the bottom level implies misses at all upper levels, so we'll // skip checking the filters when we predict a hit. - return cfd_->ioptions()->optimize_filters_for_hits && + return cfd_->ioptions().optimize_filters_for_hits && (level > 0 || is_file_last_in_level) && level == storage_info_.num_non_empty_levels() - 1; } @@ -3102,7 +3102,7 @@ void Version::PrepareAppend(const MutableCFOptions& mutable_cf_options, UpdateAccumulatedStats(read_options); } - storage_info_.PrepareForVersionAppend(*cfd_->ioptions(), mutable_cf_options); + storage_info_.PrepareForVersionAppend(cfd_->ioptions(), mutable_cf_options); } bool Version::MaybeInitializeFileMetaData(const ReadOptions& read_options, @@ -4613,12 +4613,12 @@ void VersionStorageInfo::RecoverEpochNumbers(ColumnFamilyData* cfd, cfd->ResetNextEpochNumber(); bool reserve_epoch_num_for_file_ingested_behind = - cfd->ioptions()->allow_ingest_behind; + cfd->ioptions().allow_ingest_behind; if (reserve_epoch_num_for_file_ingested_behind) { uint64_t reserved_epoch_number = cfd->NewEpochNumber(); assert(reserved_epoch_number == kReservedEpochNumberForFileIngestedBehind); - ROCKS_LOG_INFO(cfd->ioptions()->info_log.get(), + ROCKS_LOG_INFO(cfd->ioptions().info_log.get(), "[%s]CF has reserved epoch number %" PRIu64 " for files ingested " "behind since `Options::allow_ingest_behind` is true", @@ -4646,7 +4646,7 @@ void VersionStorageInfo::RecoverEpochNumbers(ColumnFamilyData* cfd, if (missing_epoch_number) { assert(epoch_number_requirement_ == EpochNumberRequirement::kMightMissing); - ROCKS_LOG_WARN(cfd->ioptions()->info_log.get(), + ROCKS_LOG_WARN(cfd->ioptions().info_log.get(), "[%s]CF's epoch numbers are inferred based on seqno", cfd->GetName().c_str()); epoch_number_requirement_ = EpochNumberRequirement::kMustPresent; @@ -5171,12 +5171,11 @@ Status VersionSet::Close(FSDirectory* db_dir, InstrumentedMutex* mu) { io_s = IOStatus::Corruption(); } ColumnFamilyData* cfd = GetColumnFamilySet()->GetDefault(); - const ImmutableOptions* ioptions = cfd->ioptions(); IOErrorInfo io_error_info(io_s, FileOperationType::kVerify, manifest_file_name, /*length=*/size, /*offset=*/0); - for (auto& listener : ioptions->listeners) { + for (auto& listener : cfd->ioptions().listeners) { listener->OnIOError(io_error_info); } io_s.PermitUncheckedError(); @@ -5190,9 +5189,8 @@ Status VersionSet::Close(FSDirectory* db_dir, InstrumentedMutex* mu) { io_s.ToString().c_str(), size); VersionEdit edit; assert(cfd); - const MutableCFOptions& cf_opts = *cfd->GetLatestMutableCFOptions(); - s = LogAndApply(cfd, cf_opts, ReadOptions(), WriteOptions(), &edit, mu, - db_dir); + s = LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), ReadOptions(), + WriteOptions(), &edit, mu, db_dir); } closed_ = true; @@ -5255,8 +5253,8 @@ void VersionSet::AppendVersion(ColumnFamilyData* column_family_data, Version* v) { // compute new compaction score v->storage_info()->ComputeCompactionScore( - *column_family_data->ioptions(), - *column_family_data->GetLatestMutableCFOptions()); + column_family_data->ioptions(), + column_family_data->GetLatestMutableCFOptions()); // Mark v finalized v->storage_info_.SetFinalized(); @@ -6633,7 +6631,7 @@ Status VersionSet::WriteCurrentStateToManifest( edit.SetComparatorName( cfd->internal_comparator().user_comparator()->Name()); edit.SetPersistUserDefinedTimestamps( - cfd->ioptions()->persist_user_defined_timestamps); + cfd->ioptions().persist_user_defined_timestamps); std::string record; if (!edit.EncodeTo(&record)) { return Status::Corruption("Unable to Encode VersionEdit:" + @@ -7085,13 +7083,13 @@ InternalIterator* VersionSet::MakeInputIterator( list[num++] = cfd->table_cache()->NewIterator( read_options, file_options_compactions, cfd->internal_comparator(), fmd, range_del_agg, - *c->mutable_cf_options(), + c->mutable_cf_options(), /*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr, TableReaderCaller::kCompaction, /*arena=*/nullptr, /*skip_filters=*/false, /*level=*/static_cast(c->level(which)), - MaxFileSizeForL0MetaPin(*c->mutable_cf_options()), + MaxFileSizeForL0MetaPin(c->mutable_cf_options()), /*smallest_compaction_key=*/nullptr, /*largest_compaction_key=*/nullptr, /*allow_unprepared_value=*/false, @@ -7106,7 +7104,7 @@ InternalIterator* VersionSet::MakeInputIterator( nullptr; list[num++] = new LevelIterator( cfd->table_cache(), read_options, file_options_compactions, - cfd->internal_comparator(), flevel, *c->mutable_cf_options(), + cfd->internal_comparator(), flevel, c->mutable_cf_options(), /*should_sample=*/false, /*no per level latency histogram=*/nullptr, TableReaderCaller::kCompaction, /*skip_filters=*/false, @@ -7175,11 +7173,11 @@ void VersionSet::GetLiveFilesMetaData(std::vector* metadata) { LiveFileMetaData filemetadata; filemetadata.column_family_name = cfd->GetName(); uint32_t path_id = file->fd.GetPathId(); - if (path_id < cfd->ioptions()->cf_paths.size()) { - filemetadata.db_path = cfd->ioptions()->cf_paths[path_id].path; + if (path_id < cfd->ioptions().cf_paths.size()) { + filemetadata.db_path = cfd->ioptions().cf_paths[path_id].path; } else { - assert(!cfd->ioptions()->cf_paths.empty()); - filemetadata.db_path = cfd->ioptions()->cf_paths.back().path; + assert(!cfd->ioptions().cf_paths.empty()); + filemetadata.db_path = cfd->ioptions().cf_paths.back().path; } filemetadata.directory = filemetadata.db_path; const uint64_t file_number = file->fd.GetNumber(); @@ -7270,18 +7268,18 @@ ColumnFamilyData* VersionSet::CreateColumnFamily( cf_options); Version* v = new Version(new_cfd, this, file_options_, - *new_cfd->GetLatestMutableCFOptions(), io_tracer_, + new_cfd->GetLatestMutableCFOptions(), io_tracer_, current_version_number_++); constexpr bool update_stats = false; - v->PrepareAppend(*new_cfd->GetLatestMutableCFOptions(), read_options, + v->PrepareAppend(new_cfd->GetLatestMutableCFOptions(), read_options, update_stats); AppendVersion(new_cfd, v); // GetLatestMutableCFOptions() is safe here without mutex since the // cfd is not available to client - new_cfd->CreateNewMemtable(*new_cfd->GetLatestMutableCFOptions(), + new_cfd->CreateNewMemtable(new_cfd->GetLatestMutableCFOptions(), LastSequence()); new_cfd->SetLogNumber(edit->GetLogNumber()); return new_cfd; @@ -7357,9 +7355,8 @@ Status VersionSet::VerifyFileMetadata(const ReadOptions& read_options, TableCache* table_cache = cfd->table_cache(); assert(table_cache); - const MutableCFOptions* const cf_opts = cfd->GetLatestMutableCFOptions(); - assert(cf_opts); - size_t max_sz_for_l0_meta_pin = MaxFileSizeForL0MetaPin(*cf_opts); + const auto& cf_opts = cfd->GetLatestMutableCFOptions(); + size_t max_sz_for_l0_meta_pin = MaxFileSizeForL0MetaPin(cf_opts); const FileOptions& file_opts = file_options(); @@ -7374,7 +7371,7 @@ Status VersionSet::VerifyFileMetadata(const ReadOptions& read_options, TableCache::TypedHandle* handle = nullptr; FileMetaData meta_copy = meta; status = table_cache->FindTable( - read_options, file_opts, *icmp, meta_copy, &handle, *cf_opts, + read_options, file_opts, *icmp, meta_copy, &handle, cf_opts, /*no_io=*/false, internal_stats->GetFileReadHist(level), false, level, /*prefetch_index_and_filter_in_cache*/ false, max_sz_for_l0_meta_pin, meta_copy.temperature); diff --git a/db/version_set.h b/db/version_set.h index bc1f53a20c9..a5226a84b8d 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -1139,7 +1139,7 @@ class Version { bool use_async_io_; Version(ColumnFamilyData* cfd, VersionSet* vset, const FileOptions& file_opt, - MutableCFOptions mutable_cf_options, + const MutableCFOptions& mutable_cf_options, const std::shared_ptr& io_tracer, uint64_t version_number = 0, EpochNumberRequirement epoch_number_requirement = @@ -1200,17 +1200,18 @@ class VersionSet { FSDirectory* dir_contains_current_file, bool new_descriptor_log = false, const ColumnFamilyOptions* column_family_options = nullptr) { ColumnFamilyData* default_cf = GetColumnFamilySet()->GetDefault(); - const MutableCFOptions* cf_options = - default_cf->GetLatestMutableCFOptions(); - return LogAndApply(default_cf, *cf_options, read_options, write_options, - edit, mu, dir_contains_current_file, new_descriptor_log, + return LogAndApply(default_cf, default_cf->GetLatestMutableCFOptions(), + read_options, write_options, edit, mu, + dir_contains_current_file, new_descriptor_log, column_family_options); } // Apply *edit to the current version to form a new descriptor that // is both saved to persistent state and installed as the new // current version. Will release *mu while actually writing to the file. - // column_family_options has to be set if edit is column family add + // column_family_options has to be set if edit is column family add. + // FIXME: mutable_cf_options must be a copy accessible without db mutex + // under some conditions; which conditions? // REQUIRES: *mu is held on entry. // REQUIRES: no other thread concurrently calls LogAndApply() Status LogAndApply( @@ -1573,7 +1574,7 @@ class VersionSet { void TEST_CreateAndAppendVersion(ColumnFamilyData* cfd) { assert(cfd); - const auto& mutable_cf_options = *cfd->GetLatestMutableCFOptions(); + const auto& mutable_cf_options = cfd->GetLatestMutableCFOptions(); Version* const version = new Version(cfd, this, file_options_, mutable_cf_options, io_tracer_); diff --git a/db/version_set_test.cc b/db/version_set_test.cc index 46a90703c87..ac1c1e167c7 100644 --- a/db/version_set_test.cc +++ b/db/version_set_test.cc @@ -2425,7 +2425,6 @@ class VersionSetWithTimestampTest : public VersionSetTest { options.comparator = test::BytewiseComparatorWithU64TsWrapper(); cfd_ = CreateColumnFamily(kNewCfName, options); EXPECT_NE(nullptr, cfd_); - EXPECT_NE(nullptr, cfd_->GetLatestMutableCFOptions()); column_families_.emplace_back(kNewCfName, options); } @@ -2475,7 +2474,7 @@ class VersionSetWithTimestampTest : public VersionSetTest { Status s; mutex_.Lock(); - s = versions_->LogAndApply(cfd_, *(cfd_->GetLatestMutableCFOptions()), + s = versions_->LogAndApply(cfd_, cfd_->GetLatestMutableCFOptions(), read_options_, write_options_, edits_, &mutex_, nullptr); mutex_.Unlock(); @@ -3371,7 +3370,7 @@ TEST_P(VersionSetTestDropOneCF, HandleDroppedColumnFamilyInAtomicGroup) { drop_cf_edit.SetColumnFamily(cfd_to_drop->GetID()); mutex_.Lock(); Status s = versions_->LogAndApply( - cfd_to_drop, *cfd_to_drop->GetLatestMutableCFOptions(), read_options, + cfd_to_drop, cfd_to_drop->GetLatestMutableCFOptions(), read_options, write_options, &drop_cf_edit, &mutex_, nullptr); mutex_.Unlock(); ASSERT_OK(s); @@ -3388,7 +3387,7 @@ TEST_P(VersionSetTestDropOneCF, HandleDroppedColumnFamilyInAtomicGroup) { : cfd_to_drop; ASSERT_NE(nullptr, cfd); cfds.push_back(cfd); - mutable_cf_options_list.emplace_back(cfd->GetLatestMutableCFOptions()); + mutable_cf_options_list.emplace_back(&cfd->GetLatestMutableCFOptions()); edits[i].SetColumnFamily(cfd->GetID()); edits[i].SetLogNumber(0); edits[i].SetNextFile(2); diff --git a/db/version_util.h b/db/version_util.h index e499b9e2ed1..25048b8f13d 100644 --- a/db/version_util.h +++ b/db/version_util.h @@ -43,7 +43,7 @@ class OfflineManifestWriter { // Use `mutex` to imitate a locked DB mutex when calling `LogAndApply()`. InstrumentedMutex mutex; mutex.Lock(); - Status s = versions_.LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + Status s = versions_.LogAndApply(cfd, cfd->GetLatestMutableCFOptions(), read_options, write_options, edit, &mutex, dir_contains_current_file, false /* new_descriptor_log */); diff --git a/db/write_batch.cc b/db/write_batch.cc index 7a5ad78192d..15034e5c3fc 100644 --- a/db/write_batch.cc +++ b/db/write_batch.cc @@ -2928,10 +2928,9 @@ class MemTableInserter : public WriteBatch::Handler { auto* cfd = cf_mems_->current(); assert(cfd); - assert(cfd->ioptions()); const size_t size_to_maintain = static_cast( - cfd->ioptions()->max_write_buffer_size_to_maintain); + cfd->ioptions().max_write_buffer_size_to_maintain); if (size_to_maintain > 0) { MemTableList* const imm = cfd->imm(); diff --git a/file/sst_file_manager_impl.cc b/file/sst_file_manager_impl.cc index 1db0c46f30c..bd42d04ac9e 100644 --- a/file/sst_file_manager_impl.cc +++ b/file/sst_file_manager_impl.cc @@ -184,7 +184,7 @@ bool SstFileManagerImpl::EnoughRoomForCompaction( // other DB instances if (bg_error.IsNoSpace() && CheckFreeSpace()) { auto fn = - TableFileName(cfd->ioptions()->cf_paths, inputs[0][0]->fd.GetNumber(), + TableFileName(cfd->ioptions().cf_paths, inputs[0][0]->fd.GetNumber(), inputs[0][0]->fd.GetPathId()); uint64_t free_space = 0; Status s = fs_->GetFreeSpace(fn, IOOptions(), &free_space, nullptr); diff --git a/include/rocksdb/utilities/ldb_cmd.h b/include/rocksdb/utilities/ldb_cmd.h index b6f896608a9..e0a1f06a7c8 100644 --- a/include/rocksdb/utilities/ldb_cmd.h +++ b/include/rocksdb/utilities/ldb_cmd.h @@ -330,7 +330,7 @@ class LDBCommandRunner { // Returns the status code to return. 0 is no error. static int RunCommand( - int argc, char const* const* argv, Options options, + int argc, char const* const* argv, const Options& options, const LDBOptions& ldb_options, const std::vector* column_families); }; diff --git a/tools/ldb_tool.cc b/tools/ldb_tool.cc index a3dd22dfcfc..3dd1905e83b 100644 --- a/tools/ldb_tool.cc +++ b/tools/ldb_tool.cc @@ -140,7 +140,7 @@ void LDBCommandRunner::PrintHelp(const LDBOptions& ldb_options, } int LDBCommandRunner::RunCommand( - int argc, char const* const* argv, Options options, + int argc, char const* const* argv, const Options& options, const LDBOptions& ldb_options, const std::vector* column_families) { if (argc <= 2) { diff --git a/utilities/blob_db/blob_db_impl.cc b/utilities/blob_db/blob_db_impl.cc index c88c3e8a709..00d15e90ccf 100644 --- a/utilities/blob_db/blob_db_impl.cc +++ b/utilities/blob_db/blob_db_impl.cc @@ -232,12 +232,9 @@ Status BlobDBImpl::Open(std::vector* handles) { static_cast(DefaultColumnFamily())->cfd(); assert(cfd); - const ImmutableCFOptions* const ioptions = cfd->ioptions(); - assert(ioptions); - assert(env_); - for (const auto& cf_path : ioptions->cf_paths) { + for (const auto& cf_path : cfd->ioptions().cf_paths) { bool blob_dir_same_as_cf_dir = false; s = env_->AreFilesSame(blob_dir_, cf_path.path, &blob_dir_same_as_cf_dir); if (!s.ok()) { @@ -1188,7 +1185,7 @@ Status BlobDBImpl::DecompressSlice(const Slice& compressed_value, compression_type); Status s = UncompressBlockData( info, compressed_value.data(), compressed_value.size(), &contents, - kBlockBasedTableVersionFormat, *(cfh->cfd()->ioptions())); + kBlockBasedTableVersionFormat, cfh->cfd()->ioptions()); if (!s.ok()) { return Status::Corruption("Unable to decompress blob."); } diff --git a/utilities/transactions/transaction_test.cc b/utilities/transactions/transaction_test.cc index 7ac4255bc34..9e590e1e808 100644 --- a/utilities/transactions/transaction_test.cc +++ b/utilities/transactions/transaction_test.cc @@ -5651,13 +5651,13 @@ TEST_P(TransactionTest, ToggleAutoCompactionTest) { ASSERT_OK(s); auto cfh_default = static_cast_with_check(handles[0]); - auto opt_default = *cfh_default->cfd()->GetLatestMutableCFOptions(); + auto& opt_default = cfh_default->cfd()->GetLatestMutableCFOptions(); auto cfh_a = static_cast_with_check(handles[1]); - auto opt_a = *cfh_a->cfd()->GetLatestMutableCFOptions(); + auto& opt_a = cfh_a->cfd()->GetLatestMutableCFOptions(); auto cfh_b = static_cast_with_check(handles[2]); - auto opt_b = *cfh_b->cfd()->GetLatestMutableCFOptions(); + auto& opt_b = cfh_b->cfd()->GetLatestMutableCFOptions(); ASSERT_EQ(opt_default.disable_auto_compactions, false); ASSERT_EQ(opt_a.disable_auto_compactions, true);