From e6918187568dbd01842d8d1d2c808ce16a894239 Mon Sep 17 00:00:00 2001 From: Daniel Baumann Date: Sun, 21 Apr 2024 13:54:28 +0200 Subject: Adding upstream version 18.2.2. Signed-off-by: Daniel Baumann --- src/rocksdb/db/db_impl/compacted_db_impl.cc | 257 + src/rocksdb/db/db_impl/compacted_db_impl.h | 154 + src/rocksdb/db/db_impl/db_impl.cc | 5918 ++++++++++++++++++++ src/rocksdb/db/db_impl/db_impl.h | 2804 ++++++++++ src/rocksdb/db/db_impl/db_impl_compaction_flush.cc | 3857 +++++++++++++ src/rocksdb/db/db_impl/db_impl_debug.cc | 312 ++ src/rocksdb/db/db_impl/db_impl_experimental.cc | 158 + src/rocksdb/db/db_impl/db_impl_files.cc | 1013 ++++ src/rocksdb/db/db_impl/db_impl_open.cc | 2106 +++++++ src/rocksdb/db/db_impl/db_impl_readonly.cc | 341 ++ src/rocksdb/db/db_impl/db_impl_readonly.h | 170 + src/rocksdb/db/db_impl/db_impl_secondary.cc | 967 ++++ src/rocksdb/db/db_impl/db_impl_secondary.h | 410 ++ src/rocksdb/db/db_impl/db_impl_write.cc | 2435 ++++++++ 14 files changed, 20902 insertions(+) create mode 100644 src/rocksdb/db/db_impl/compacted_db_impl.cc create mode 100644 src/rocksdb/db/db_impl/compacted_db_impl.h create mode 100644 src/rocksdb/db/db_impl/db_impl.cc create mode 100644 src/rocksdb/db/db_impl/db_impl.h create mode 100644 src/rocksdb/db/db_impl/db_impl_compaction_flush.cc create mode 100644 src/rocksdb/db/db_impl/db_impl_debug.cc create mode 100644 src/rocksdb/db/db_impl/db_impl_experimental.cc create mode 100644 src/rocksdb/db/db_impl/db_impl_files.cc create mode 100644 src/rocksdb/db/db_impl/db_impl_open.cc create mode 100644 src/rocksdb/db/db_impl/db_impl_readonly.cc create mode 100644 src/rocksdb/db/db_impl/db_impl_readonly.h create mode 100644 src/rocksdb/db/db_impl/db_impl_secondary.cc create mode 100644 src/rocksdb/db/db_impl/db_impl_secondary.h create mode 100644 src/rocksdb/db/db_impl/db_impl_write.cc (limited to 'src/rocksdb/db/db_impl') diff --git a/src/rocksdb/db/db_impl/compacted_db_impl.cc b/src/rocksdb/db/db_impl/compacted_db_impl.cc new file mode 100644 index 000000000..f18ee0d72 --- /dev/null +++ b/src/rocksdb/db/db_impl/compacted_db_impl.cc @@ -0,0 +1,257 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#ifndef ROCKSDB_LITE +#include "db/db_impl/compacted_db_impl.h" + +#include "db/db_impl/db_impl.h" +#include "db/version_set.h" +#include "logging/logging.h" +#include "table/get_context.h" +#include "util/cast_util.h" + +namespace ROCKSDB_NAMESPACE { + +extern void MarkKeyMayExist(void* arg); +extern bool SaveValue(void* arg, const ParsedInternalKey& parsed_key, + const Slice& v, bool hit_and_return); + +CompactedDBImpl::CompactedDBImpl(const DBOptions& options, + const std::string& dbname) + : DBImpl(options, dbname, /*seq_per_batch*/ false, +/*batch_per_txn*/ true, + /*read_only*/ true), + cfd_(nullptr), + version_(nullptr), + user_comparator_(nullptr) {} + +CompactedDBImpl::~CompactedDBImpl() {} + +size_t CompactedDBImpl::FindFile(const Slice& key) { + size_t right = files_.num_files - 1; + auto cmp = [&](const FdWithKeyRange& f, const Slice& k) -> bool { + return user_comparator_->Compare(ExtractUserKey(f.largest_key), k) < 0; + }; + return static_cast( + std::lower_bound(files_.files, files_.files + right, key, cmp) - + files_.files); +} + +Status CompactedDBImpl::Get(const ReadOptions& options, ColumnFamilyHandle*, + const Slice& key, PinnableSlice* value) { + return Get(options, /*column_family*/ nullptr, key, value, + /*timestamp*/ nullptr); +} + +Status CompactedDBImpl::Get(const ReadOptions& options, ColumnFamilyHandle*, + const Slice& key, PinnableSlice* value, + std::string* timestamp) { + assert(user_comparator_); + if (options.timestamp) { + const Status s = FailIfTsMismatchCf( + DefaultColumnFamily(), *(options.timestamp), /*ts_for_read=*/true); + if (!s.ok()) { + return s; + } + } else { + const Status s = FailIfCfHasTs(DefaultColumnFamily()); + if (!s.ok()) { + return s; + } + } + + // Clear the timestamps for returning results so that we can distinguish + // between tombstone or key that has never been written + if (timestamp) { + timestamp->clear(); + } + + GetWithTimestampReadCallback read_cb(kMaxSequenceNumber); + std::string* ts = + user_comparator_->timestamp_size() > 0 ? timestamp : nullptr; + LookupKey lkey(key, kMaxSequenceNumber, options.timestamp); + GetContext get_context(user_comparator_, nullptr, nullptr, nullptr, + GetContext::kNotFound, lkey.user_key(), value, + /*columns=*/nullptr, ts, nullptr, nullptr, true, + nullptr, nullptr, nullptr, nullptr, &read_cb); + + const FdWithKeyRange& f = files_.files[FindFile(lkey.user_key())]; + if (user_comparator_->CompareWithoutTimestamp( + key, /*a_has_ts=*/false, + ExtractUserKeyAndStripTimestamp(f.smallest_key, + user_comparator_->timestamp_size()), + /*b_has_ts=*/false) < 0) { + return Status::NotFound(); + } + Status s = f.fd.table_reader->Get(options, lkey.internal_key(), &get_context, + nullptr); + if (!s.ok() && !s.IsNotFound()) { + return s; + } + if (get_context.State() == GetContext::kFound) { + return Status::OK(); + } + return Status::NotFound(); +} + +std::vector CompactedDBImpl::MultiGet( + const ReadOptions& options, const std::vector&, + const std::vector& keys, std::vector* values) { + return MultiGet(options, keys, values, /*timestamps*/ nullptr); +} + +std::vector CompactedDBImpl::MultiGet( + const ReadOptions& options, const std::vector&, + const std::vector& keys, std::vector* values, + std::vector* timestamps) { + assert(user_comparator_); + size_t num_keys = keys.size(); + + if (options.timestamp) { + Status s = FailIfTsMismatchCf(DefaultColumnFamily(), *(options.timestamp), + /*ts_for_read=*/true); + if (!s.ok()) { + return std::vector(num_keys, s); + } + } else { + Status s = FailIfCfHasTs(DefaultColumnFamily()); + if (!s.ok()) { + return std::vector(num_keys, s); + } + } + + // Clear the timestamps for returning results so that we can distinguish + // between tombstone or key that has never been written + if (timestamps) { + for (auto& ts : *timestamps) { + ts.clear(); + } + } + + GetWithTimestampReadCallback read_cb(kMaxSequenceNumber); + autovector reader_list; + for (const auto& key : keys) { + LookupKey lkey(key, kMaxSequenceNumber, options.timestamp); + const FdWithKeyRange& f = files_.files[FindFile(lkey.user_key())]; + if (user_comparator_->CompareWithoutTimestamp( + key, /*a_has_ts=*/false, + ExtractUserKeyAndStripTimestamp(f.smallest_key, + user_comparator_->timestamp_size()), + /*b_has_ts=*/false) < 0) { + reader_list.push_back(nullptr); + } else { + f.fd.table_reader->Prepare(lkey.internal_key()); + reader_list.push_back(f.fd.table_reader); + } + } + std::vector statuses(num_keys, Status::NotFound()); + values->resize(num_keys); + if (timestamps) { + timestamps->resize(num_keys); + } + int idx = 0; + for (auto* r : reader_list) { + if (r != nullptr) { + PinnableSlice pinnable_val; + std::string& value = (*values)[idx]; + LookupKey lkey(keys[idx], kMaxSequenceNumber, options.timestamp); + std::string* timestamp = timestamps ? &(*timestamps)[idx] : nullptr; + GetContext get_context( + user_comparator_, nullptr, nullptr, nullptr, GetContext::kNotFound, + lkey.user_key(), &pinnable_val, /*columns=*/nullptr, + user_comparator_->timestamp_size() > 0 ? timestamp : nullptr, nullptr, + nullptr, true, nullptr, nullptr, nullptr, nullptr, &read_cb); + Status s = r->Get(options, lkey.internal_key(), &get_context, nullptr); + assert(static_cast(idx) < statuses.size()); + if (!s.ok() && !s.IsNotFound()) { + statuses[idx] = s; + } else { + value.assign(pinnable_val.data(), pinnable_val.size()); + if (get_context.State() == GetContext::kFound) { + statuses[idx] = Status::OK(); + } + } + } + ++idx; + } + return statuses; +} + +Status CompactedDBImpl::Init(const Options& options) { + SuperVersionContext sv_context(/* create_superversion */ true); + mutex_.Lock(); + ColumnFamilyDescriptor cf(kDefaultColumnFamilyName, + ColumnFamilyOptions(options)); + Status s = Recover({cf}, true /* read only */, false, true); + if (s.ok()) { + cfd_ = static_cast_with_check(DefaultColumnFamily()) + ->cfd(); + cfd_->InstallSuperVersion(&sv_context, &mutex_); + } + mutex_.Unlock(); + sv_context.Clean(); + if (!s.ok()) { + return s; + } + NewThreadStatusCfInfo(cfd_); + version_ = cfd_->GetSuperVersion()->current; + user_comparator_ = cfd_->user_comparator(); + auto* vstorage = version_->storage_info(); + if (vstorage->num_non_empty_levels() == 0) { + return Status::NotSupported("no file exists"); + } + const LevelFilesBrief& l0 = vstorage->LevelFilesBrief(0); + // L0 should not have files + if (l0.num_files > 1) { + return Status::NotSupported("L0 contain more than 1 file"); + } + if (l0.num_files == 1) { + if (vstorage->num_non_empty_levels() > 1) { + return Status::NotSupported("Both L0 and other level contain files"); + } + files_ = l0; + return Status::OK(); + } + + for (int i = 1; i < vstorage->num_non_empty_levels() - 1; ++i) { + if (vstorage->LevelFilesBrief(i).num_files > 0) { + return Status::NotSupported("Other levels also contain files"); + } + } + + int level = vstorage->num_non_empty_levels() - 1; + if (vstorage->LevelFilesBrief(level).num_files > 0) { + files_ = vstorage->LevelFilesBrief(level); + return Status::OK(); + } + return Status::NotSupported("no file exists"); +} + +Status CompactedDBImpl::Open(const Options& options, const std::string& dbname, + DB** dbptr) { + *dbptr = nullptr; + + if (options.max_open_files != -1) { + return Status::InvalidArgument("require max_open_files = -1"); + } + if (options.merge_operator.get() != nullptr) { + return Status::InvalidArgument("merge operator is not supported"); + } + DBOptions db_options(options); + std::unique_ptr db(new CompactedDBImpl(db_options, dbname)); + Status s = db->Init(options); + if (s.ok()) { + s = db->StartPeriodicTaskScheduler(); + } + if (s.ok()) { + ROCKS_LOG_INFO(db->immutable_db_options_.info_log, + "Opened the db as fully compacted mode"); + LogFlush(db->immutable_db_options_.info_log); + *dbptr = db.release(); + } + return s; +} + +} // namespace ROCKSDB_NAMESPACE +#endif // ROCKSDB_LITE diff --git a/src/rocksdb/db/db_impl/compacted_db_impl.h b/src/rocksdb/db/db_impl/compacted_db_impl.h new file mode 100644 index 000000000..eb458b85d --- /dev/null +++ b/src/rocksdb/db/db_impl/compacted_db_impl.h @@ -0,0 +1,154 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once +#ifndef ROCKSDB_LITE +#include +#include + +#include "db/db_impl/db_impl.h" + +namespace ROCKSDB_NAMESPACE { + +// TODO: Share common structure with DBImplSecondary and DBImplReadOnly +class CompactedDBImpl : public DBImpl { + public: + CompactedDBImpl(const DBOptions& options, const std::string& dbname); + // No copying allowed + CompactedDBImpl(const CompactedDBImpl&) = delete; + void operator=(const CompactedDBImpl&) = delete; + + ~CompactedDBImpl() override; + + static Status Open(const Options& options, const std::string& dbname, + DB** dbptr); + + // Implementations of the DB interface + using DB::Get; + virtual Status Get(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value) override; + + Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value, + std::string* timestamp) override; + + using DB::MultiGet; + // Note that CompactedDBImpl::MultiGet is not the optimized version of + // MultiGet to use. + // TODO: optimize CompactedDBImpl::MultiGet, see DBImpl::MultiGet for details. + virtual std::vector MultiGet( + const ReadOptions& options, const std::vector&, + const std::vector& keys, + std::vector* values) override; + + std::vector MultiGet(const ReadOptions& options, + const std::vector&, + const std::vector& keys, + std::vector* values, + std::vector* timestamps) override; + + using DBImpl::Put; + virtual Status Put(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/, const Slice& /*value*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + + using DBImpl::PutEntity; + Status PutEntity(const WriteOptions& /* options */, + ColumnFamilyHandle* /* column_family */, + const Slice& /* key */, + const WideColumns& /* columns */) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + + using DBImpl::Merge; + virtual Status Merge(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/, const Slice& /*value*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + + using DBImpl::Delete; + virtual Status Delete(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + virtual Status Write(const WriteOptions& /*options*/, + WriteBatch* /*updates*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + using DBImpl::CompactRange; + virtual Status CompactRange(const CompactRangeOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice* /*begin*/, + const Slice* /*end*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + + virtual Status DisableFileDeletions() override { + return Status::NotSupported("Not supported in compacted db mode."); + } + virtual Status EnableFileDeletions(bool /*force*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + virtual Status GetLiveFiles(std::vector& ret, + uint64_t* manifest_file_size, + bool /*flush_memtable*/) override { + return DBImpl::GetLiveFiles(ret, manifest_file_size, + false /* flush_memtable */); + } + using DBImpl::Flush; + virtual Status Flush(const FlushOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + + virtual Status SyncWAL() override { + return Status::NotSupported("Not supported in compacted db mode."); + } + + using DB::IngestExternalFile; + virtual Status IngestExternalFile( + ColumnFamilyHandle* /*column_family*/, + const std::vector& /*external_files*/, + const IngestExternalFileOptions& /*ingestion_options*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + using DB::CreateColumnFamilyWithImport; + virtual Status CreateColumnFamilyWithImport( + const ColumnFamilyOptions& /*options*/, + const std::string& /*column_family_name*/, + const ImportColumnFamilyOptions& /*import_options*/, + const ExportImportFilesMetaData& /*metadata*/, + ColumnFamilyHandle** /*handle*/) override { + return Status::NotSupported("Not supported in compacted db mode."); + } + + // FIXME: some missing overrides for more "write" functions + // Share with DBImplReadOnly? + + protected: +#ifndef ROCKSDB_LITE + Status FlushForGetLiveFiles() override { + // No-op for read-only DB + return Status::OK(); + } +#endif // !ROCKSDB_LITE + + private: + friend class DB; + inline size_t FindFile(const Slice& key); + Status Init(const Options& options); + + ColumnFamilyData* cfd_; + Version* version_; + const Comparator* user_comparator_; + LevelFilesBrief files_; +}; +} // namespace ROCKSDB_NAMESPACE +#endif // ROCKSDB_LITE diff --git a/src/rocksdb/db/db_impl/db_impl.cc b/src/rocksdb/db/db_impl/db_impl.cc new file mode 100644 index 000000000..a431111d4 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl.cc @@ -0,0 +1,5918 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +#include "db/db_impl/db_impl.h" + +#include +#ifdef OS_SOLARIS +#include +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "db/arena_wrapped_db_iter.h" +#include "db/builder.h" +#include "db/compaction/compaction_job.h" +#include "db/db_info_dumper.h" +#include "db/db_iter.h" +#include "db/dbformat.h" +#include "db/error_handler.h" +#include "db/event_helpers.h" +#include "db/external_sst_file_ingestion_job.h" +#include "db/flush_job.h" +#include "db/forward_iterator.h" +#include "db/import_column_family_job.h" +#include "db/job_context.h" +#include "db/log_reader.h" +#include "db/log_writer.h" +#include "db/malloc_stats.h" +#include "db/memtable.h" +#include "db/memtable_list.h" +#include "db/merge_context.h" +#include "db/merge_helper.h" +#include "db/periodic_task_scheduler.h" +#include "db/range_tombstone_fragmenter.h" +#include "db/table_cache.h" +#include "db/table_properties_collector.h" +#include "db/transaction_log_impl.h" +#include "db/version_set.h" +#include "db/write_batch_internal.h" +#include "db/write_callback.h" +#include "env/unique_id_gen.h" +#include "file/file_util.h" +#include "file/filename.h" +#include "file/random_access_file_reader.h" +#include "file/sst_file_manager_impl.h" +#include "logging/auto_roll_logger.h" +#include "logging/log_buffer.h" +#include "logging/logging.h" +#include "monitoring/in_memory_stats_history.h" +#include "monitoring/instrumented_mutex.h" +#include "monitoring/iostats_context_imp.h" +#include "monitoring/perf_context_imp.h" +#include "monitoring/persistent_stats_history.h" +#include "monitoring/thread_status_updater.h" +#include "monitoring/thread_status_util.h" +#include "options/cf_options.h" +#include "options/options_helper.h" +#include "options/options_parser.h" +#include "port/port.h" +#include "rocksdb/cache.h" +#include "rocksdb/compaction_filter.h" +#include "rocksdb/convenience.h" +#include "rocksdb/db.h" +#include "rocksdb/env.h" +#include "rocksdb/merge_operator.h" +#include "rocksdb/statistics.h" +#include "rocksdb/stats_history.h" +#include "rocksdb/status.h" +#include "rocksdb/table.h" +#include "rocksdb/version.h" +#include "rocksdb/write_buffer_manager.h" +#include "table/block_based/block.h" +#include "table/block_based/block_based_table_factory.h" +#include "table/get_context.h" +#include "table/merging_iterator.h" +#include "table/multiget_context.h" +#include "table/sst_file_dumper.h" +#include "table/table_builder.h" +#include "table/two_level_iterator.h" +#include "table/unique_id_impl.h" +#include "test_util/sync_point.h" +#include "trace_replay/trace_replay.h" +#include "util/autovector.h" +#include "util/cast_util.h" +#include "util/coding.h" +#include "util/compression.h" +#include "util/crc32c.h" +#include "util/defer.h" +#include "util/distributed_mutex.h" +#include "util/hash_containers.h" +#include "util/mutexlock.h" +#include "util/stop_watch.h" +#include "util/string_util.h" +#include "utilities/trace/replayer_impl.h" + +namespace ROCKSDB_NAMESPACE { + +const std::string kDefaultColumnFamilyName("default"); +const std::string kPersistentStatsColumnFamilyName( + "___rocksdb_stats_history___"); +void DumpRocksDBBuildVersion(Logger* log); + +CompressionType GetCompressionFlush( + const ImmutableCFOptions& ioptions, + const MutableCFOptions& mutable_cf_options) { + // Compressing memtable flushes might not help unless the sequential load + // optimization is used for leveled compaction. Otherwise the CPU and + // latency overhead is not offset by saving much space. + if (ioptions.compaction_style == kCompactionStyleUniversal && + mutable_cf_options.compaction_options_universal + .compression_size_percent >= 0) { + return kNoCompression; + } + if (mutable_cf_options.compression_per_level.empty()) { + return mutable_cf_options.compression; + } else { + // For leveled compress when min_level_to_compress != 0. + return mutable_cf_options.compression_per_level[0]; + } +} + +namespace { +void DumpSupportInfo(Logger* logger) { + ROCKS_LOG_HEADER(logger, "Compression algorithms supported:"); + for (auto& compression : OptionsHelper::compression_type_string_map) { + if (compression.second != kNoCompression && + compression.second != kDisableCompressionOption) { + ROCKS_LOG_HEADER(logger, "\t%s supported: %d", compression.first.c_str(), + CompressionTypeSupported(compression.second)); + } + } + ROCKS_LOG_HEADER(logger, "Fast CRC32 supported: %s", + crc32c::IsFastCrc32Supported().c_str()); + + ROCKS_LOG_HEADER(logger, "DMutex implementation: %s", DMutex::kName()); +} +} // namespace + +DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, + const bool seq_per_batch, const bool batch_per_txn, + bool read_only) + : dbname_(dbname), + own_info_log_(options.info_log == nullptr), + init_logger_creation_s_(), + initial_db_options_(SanitizeOptions(dbname, options, read_only, + &init_logger_creation_s_)), + env_(initial_db_options_.env), + io_tracer_(std::make_shared()), + immutable_db_options_(initial_db_options_), + fs_(immutable_db_options_.fs, io_tracer_), + mutable_db_options_(initial_db_options_), + stats_(immutable_db_options_.stats), +#ifdef COERCE_CONTEXT_SWITCH + mutex_(stats_, immutable_db_options_.clock, DB_MUTEX_WAIT_MICROS, &bg_cv_, + immutable_db_options_.use_adaptive_mutex), +#else // COERCE_CONTEXT_SWITCH + mutex_(stats_, immutable_db_options_.clock, DB_MUTEX_WAIT_MICROS, + immutable_db_options_.use_adaptive_mutex), +#endif // COERCE_CONTEXT_SWITCH + default_cf_handle_(nullptr), + error_handler_(this, immutable_db_options_, &mutex_), + event_logger_(immutable_db_options_.info_log.get()), + max_total_in_memory_state_(0), + file_options_(BuildDBOptions(immutable_db_options_, mutable_db_options_)), + file_options_for_compaction_(fs_->OptimizeForCompactionTableWrite( + file_options_, immutable_db_options_)), + seq_per_batch_(seq_per_batch), + batch_per_txn_(batch_per_txn), + next_job_id_(1), + shutting_down_(false), + db_lock_(nullptr), + manual_compaction_paused_(false), + bg_cv_(&mutex_), + logfile_number_(0), + log_dir_synced_(false), + log_empty_(true), + persist_stats_cf_handle_(nullptr), + log_sync_cv_(&log_write_mutex_), + total_log_size_(0), + is_snapshot_supported_(true), + write_buffer_manager_(immutable_db_options_.write_buffer_manager.get()), + write_thread_(immutable_db_options_), + nonmem_write_thread_(immutable_db_options_), + write_controller_(mutable_db_options_.delayed_write_rate), + last_batch_group_size_(0), + unscheduled_flushes_(0), + unscheduled_compactions_(0), + bg_bottom_compaction_scheduled_(0), + bg_compaction_scheduled_(0), + num_running_compactions_(0), + bg_flush_scheduled_(0), + num_running_flushes_(0), + bg_purge_scheduled_(0), + disable_delete_obsolete_files_(0), + pending_purge_obsolete_files_(0), + delete_obsolete_files_last_run_(immutable_db_options_.clock->NowMicros()), + last_stats_dump_time_microsec_(0), + has_unpersisted_data_(false), + unable_to_release_oldest_log_(false), + num_running_ingest_file_(0), +#ifndef ROCKSDB_LITE + wal_manager_(immutable_db_options_, file_options_, io_tracer_, + seq_per_batch), +#endif // ROCKSDB_LITE + bg_work_paused_(0), + bg_compaction_paused_(0), + refitting_level_(false), + opened_successfully_(false), +#ifndef ROCKSDB_LITE + periodic_task_scheduler_(), +#endif // ROCKSDB_LITE + two_write_queues_(options.two_write_queues), + manual_wal_flush_(options.manual_wal_flush), + // last_sequencee_ is always maintained by the main queue that also writes + // to the memtable. When two_write_queues_ is disabled last seq in + // memtable is the same as last seq published to the readers. When it is + // enabled but seq_per_batch_ is disabled, last seq in memtable still + // indicates last published seq since wal-only writes that go to the 2nd + // queue do not consume a sequence number. Otherwise writes performed by + // the 2nd queue could change what is visible to the readers. In this + // cases, last_seq_same_as_publish_seq_==false, the 2nd queue maintains a + // separate variable to indicate the last published sequence. + last_seq_same_as_publish_seq_( + !(seq_per_batch && options.two_write_queues)), + // Since seq_per_batch_ is currently set only by WritePreparedTxn which + // requires a custom gc for compaction, we use that to set use_custom_gc_ + // as well. + use_custom_gc_(seq_per_batch), + shutdown_initiated_(false), + own_sfm_(options.sst_file_manager == nullptr), + closed_(false), + atomic_flush_install_cv_(&mutex_), + blob_callback_(immutable_db_options_.sst_file_manager.get(), &mutex_, + &error_handler_, &event_logger_, + immutable_db_options_.listeners, dbname_) { + // !batch_per_trx_ implies seq_per_batch_ because it is only unset for + // WriteUnprepared, which should use seq_per_batch_. + assert(batch_per_txn_ || seq_per_batch_); + + // Reserve ten files or so for other uses and give the rest to TableCache. + // Give a large number for setting of "infinite" open files. + const int table_cache_size = (mutable_db_options_.max_open_files == -1) + ? TableCache::kInfiniteCapacity + : mutable_db_options_.max_open_files - 10; + LRUCacheOptions co; + co.capacity = table_cache_size; + co.num_shard_bits = immutable_db_options_.table_cache_numshardbits; + co.metadata_charge_policy = kDontChargeCacheMetadata; + table_cache_ = NewLRUCache(co); + SetDbSessionId(); + assert(!db_session_id_.empty()); + +#ifndef ROCKSDB_LITE + periodic_task_functions_.emplace(PeriodicTaskType::kDumpStats, + [this]() { this->DumpStats(); }); + periodic_task_functions_.emplace(PeriodicTaskType::kPersistStats, + [this]() { this->PersistStats(); }); + periodic_task_functions_.emplace(PeriodicTaskType::kFlushInfoLog, + [this]() { this->FlushInfoLog(); }); + periodic_task_functions_.emplace( + PeriodicTaskType::kRecordSeqnoTime, + [this]() { this->RecordSeqnoToTimeMapping(); }); +#endif // ROCKSDB_LITE + + versions_.reset(new VersionSet(dbname_, &immutable_db_options_, file_options_, + table_cache_.get(), write_buffer_manager_, + &write_controller_, &block_cache_tracer_, + io_tracer_, db_id_, db_session_id_)); + column_family_memtables_.reset( + new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet())); + + DumpRocksDBBuildVersion(immutable_db_options_.info_log.get()); + DumpDBFileSummary(immutable_db_options_, dbname_, db_session_id_); + immutable_db_options_.Dump(immutable_db_options_.info_log.get()); + mutable_db_options_.Dump(immutable_db_options_.info_log.get()); + DumpSupportInfo(immutable_db_options_.info_log.get()); + + max_total_wal_size_.store(mutable_db_options_.max_total_wal_size, + std::memory_order_relaxed); + if (write_buffer_manager_) { + wbm_stall_.reset(new WBMStallInterface()); + } +} + +Status DBImpl::Resume() { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Resuming DB"); + + InstrumentedMutexLock db_mutex(&mutex_); + + if (!error_handler_.IsDBStopped() && !error_handler_.IsBGWorkStopped()) { + // Nothing to do + return Status::OK(); + } + + if (error_handler_.IsRecoveryInProgress()) { + // Don't allow a mix of manual and automatic recovery + return Status::Busy(); + } + + mutex_.Unlock(); + Status s = error_handler_.RecoverFromBGError(true); + mutex_.Lock(); + return s; +} + +// This function implements the guts of recovery from a background error. It +// is eventually called for both manual as well as automatic recovery. It does +// the following - +// 1. Wait for currently scheduled background flush/compaction to exit, in +// order to inadvertently causing an error and thinking recovery failed +// 2. Flush memtables if there's any data for all the CFs. This may result +// another error, which will be saved by error_handler_ and reported later +// as the recovery status +// 3. Find and delete any obsolete files +// 4. Schedule compactions if needed for all the CFs. This is needed as the +// flush in the prior step might have been a no-op for some CFs, which +// means a new super version wouldn't have been installed +Status DBImpl::ResumeImpl(DBRecoverContext context) { + mutex_.AssertHeld(); + WaitForBackgroundWork(); + + Status s; + if (shutdown_initiated_) { + // Returning shutdown status to SFM during auto recovery will cause it + // to abort the recovery and allow the shutdown to progress + s = Status::ShutdownInProgress(); + } + + if (s.ok()) { + Status bg_error = error_handler_.GetBGError(); + if (bg_error.severity() > Status::Severity::kHardError) { + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "DB resume requested but failed due to Fatal/Unrecoverable error"); + s = bg_error; + } + } + + // Make sure the IO Status stored in version set is set to OK. + bool file_deletion_disabled = !IsFileDeletionsEnabled(); + if (s.ok()) { + IOStatus io_s = versions_->io_status(); + if (io_s.IsIOError()) { + // If resuming from IOError resulted from MANIFEST write, then assert + // that we must have already set the MANIFEST writer to nullptr during + // clean-up phase MANIFEST writing. We must have also disabled file + // deletions. + assert(!versions_->descriptor_log_); + assert(file_deletion_disabled); + // Since we are trying to recover from MANIFEST write error, we need to + // switch to a new MANIFEST anyway. The old MANIFEST can be corrupted. + // Therefore, force writing a dummy version edit because we do not know + // whether there are flush jobs with non-empty data to flush, triggering + // appends to MANIFEST. + VersionEdit edit; + auto cfh = + 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, &edit, &mutex_, + directories_.GetDbDir()); + if (!s.ok()) { + io_s = versions_->io_status(); + if (!io_s.ok()) { + s = error_handler_.SetBGError(io_s, + BackgroundErrorReason::kManifestWrite); + } + } + } + } + + // We cannot guarantee consistency of the WAL. So force flush Memtables of + // all the column families + if (s.ok()) { + FlushOptions flush_opts; + // We allow flush to stall write since we are trying to resume from error. + flush_opts.allow_write_stall = true; + if (immutable_db_options_.atomic_flush) { + autovector cfds; + SelectColumnFamiliesForAtomicFlush(&cfds); + mutex_.Unlock(); + s = AtomicFlushMemTables(cfds, flush_opts, context.flush_reason); + mutex_.Lock(); + } else { + for (auto cfd : versions_->GetRefedColumnFamilySet()) { + if (cfd->IsDropped()) { + continue; + } + InstrumentedMutexUnlock u(&mutex_); + s = FlushMemTable(cfd, flush_opts, context.flush_reason); + if (!s.ok()) { + break; + } + } + } + if (!s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "DB resume requested but failed due to Flush failure [%s]", + s.ToString().c_str()); + } + } + + JobContext job_context(0); + FindObsoleteFiles(&job_context, true); + mutex_.Unlock(); + + job_context.manifest_file_number = 1; + if (job_context.HaveSomethingToDelete()) { + PurgeObsoleteFiles(job_context); + } + job_context.Clean(); + + if (s.ok()) { + assert(versions_->io_status().ok()); + // If we reach here, we should re-enable file deletions if it was disabled + // during previous error handling. + if (file_deletion_disabled) { + // Always return ok + s = EnableFileDeletions(/*force=*/true); + if (!s.ok()) { + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "DB resume requested but could not enable file deletions [%s]", + s.ToString().c_str()); + assert(false); + } + } + } + + mutex_.Lock(); + if (s.ok()) { + // This will notify and unblock threads waiting for error recovery to + // finish. Those previouly waiting threads can now proceed, which may + // include closing the db. + s = error_handler_.ClearBGError(); + } else { + // NOTE: this is needed to pass ASSERT_STATUS_CHECKED + // in the DBSSTTest.DBWithMaxSpaceAllowedRandomized test. + // See https://github.com/facebook/rocksdb/pull/7715#issuecomment-754947952 + error_handler_.GetRecoveryError().PermitUncheckedError(); + } + + if (s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Successfully resumed DB"); + } else { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Failed to resume DB [%s]", + s.ToString().c_str()); + } + + // Check for shutdown again before scheduling further compactions, + // since we released and re-acquired the lock above + if (shutdown_initiated_) { + s = Status::ShutdownInProgress(); + } + if (s.ok()) { + for (auto cfd : *versions_->GetColumnFamilySet()) { + SchedulePendingCompaction(cfd); + } + MaybeScheduleFlushOrCompaction(); + } + + // Wake up any waiters - in this case, it could be the shutdown thread + bg_cv_.SignalAll(); + + // No need to check BGError again. If something happened, event listener would + // be notified and the operation causing it would have failed + return s; +} + +void DBImpl::WaitForBackgroundWork() { + // Wait for background work to finish + while (bg_bottom_compaction_scheduled_ || bg_compaction_scheduled_ || + bg_flush_scheduled_) { + bg_cv_.Wait(); + } +} + +// Will lock the mutex_, will wait for completion if wait is true +void DBImpl::CancelAllBackgroundWork(bool wait) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Shutdown: canceling all background work"); + +#ifndef ROCKSDB_LITE + for (uint8_t task_type = 0; + task_type < static_cast(PeriodicTaskType::kMax); task_type++) { + Status s = periodic_task_scheduler_.Unregister( + static_cast(task_type)); + if (!s.ok()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "Failed to unregister periodic task %d, status: %s", + task_type, s.ToString().c_str()); + } + } +#endif // !ROCKSDB_LITE + + InstrumentedMutexLock l(&mutex_); + if (!shutting_down_.load(std::memory_order_acquire) && + has_unpersisted_data_.load(std::memory_order_relaxed) && + !mutable_db_options_.avoid_flush_during_shutdown) { + if (immutable_db_options_.atomic_flush) { + autovector cfds; + SelectColumnFamiliesForAtomicFlush(&cfds); + mutex_.Unlock(); + Status s = + AtomicFlushMemTables(cfds, FlushOptions(), FlushReason::kShutDown); + s.PermitUncheckedError(); //**TODO: What to do on error? + mutex_.Lock(); + } else { + for (auto cfd : versions_->GetRefedColumnFamilySet()) { + if (!cfd->IsDropped() && cfd->initialized() && !cfd->mem()->IsEmpty()) { + InstrumentedMutexUnlock u(&mutex_); + Status s = FlushMemTable(cfd, FlushOptions(), FlushReason::kShutDown); + s.PermitUncheckedError(); //**TODO: What to do on error? + } + } + } + } + + shutting_down_.store(true, std::memory_order_release); + bg_cv_.SignalAll(); + if (!wait) { + return; + } + WaitForBackgroundWork(); +} + +Status DBImpl::MaybeReleaseTimestampedSnapshotsAndCheck() { + size_t num_snapshots = 0; + ReleaseTimestampedSnapshotsOlderThan(std::numeric_limits::max(), + &num_snapshots); + + // If there is unreleased snapshot, fail the close call + if (num_snapshots > 0) { + return Status::Aborted("Cannot close DB with unreleased snapshot."); + } + + return Status::OK(); +} + +Status DBImpl::CloseHelper() { + // Guarantee that there is no background error recovery in progress before + // continuing with the shutdown + mutex_.Lock(); + shutdown_initiated_ = true; + error_handler_.CancelErrorRecovery(); + while (error_handler_.IsRecoveryInProgress()) { + bg_cv_.Wait(); + } + mutex_.Unlock(); + + // Below check is added as recovery_error_ is not checked and it causes crash + // in DBSSTTest.DBWithMaxSpaceAllowedWithBlobFiles when space limit is + // reached. + error_handler_.GetRecoveryError().PermitUncheckedError(); + + // CancelAllBackgroundWork called with false means we just set the shutdown + // marker. After this we do a variant of the waiting and unschedule work + // (to consider: moving all the waiting into CancelAllBackgroundWork(true)) + CancelAllBackgroundWork(false); + + // Cancel manual compaction if there's any + if (HasPendingManualCompaction()) { + DisableManualCompaction(); + } + mutex_.Lock(); + // Unschedule all tasks for this DB + for (uint8_t i = 0; i < static_cast(TaskType::kCount); i++) { + env_->UnSchedule(GetTaskTag(i), Env::Priority::BOTTOM); + env_->UnSchedule(GetTaskTag(i), Env::Priority::LOW); + env_->UnSchedule(GetTaskTag(i), Env::Priority::HIGH); + } + + Status ret = Status::OK(); + + // Wait for background work to finish + while (bg_bottom_compaction_scheduled_ || bg_compaction_scheduled_ || + bg_flush_scheduled_ || bg_purge_scheduled_ || + pending_purge_obsolete_files_ || + error_handler_.IsRecoveryInProgress()) { + TEST_SYNC_POINT("DBImpl::~DBImpl:WaitJob"); + bg_cv_.Wait(); + } + TEST_SYNC_POINT_CALLBACK("DBImpl::CloseHelper:PendingPurgeFinished", + &files_grabbed_for_purge_); + EraseThreadStatusDbInfo(); + flush_scheduler_.Clear(); + trim_history_scheduler_.Clear(); + + while (!flush_queue_.empty()) { + const FlushRequest& flush_req = PopFirstFromFlushQueue(); + for (const auto& iter : flush_req) { + iter.first->UnrefAndTryDelete(); + } + } + + while (!compaction_queue_.empty()) { + auto cfd = PopFirstFromCompactionQueue(); + cfd->UnrefAndTryDelete(); + } + + if (default_cf_handle_ != nullptr || persist_stats_cf_handle_ != nullptr) { + // we need to delete handle outside of lock because it does its own locking + mutex_.Unlock(); + if (default_cf_handle_) { + delete default_cf_handle_; + default_cf_handle_ = nullptr; + } + if (persist_stats_cf_handle_) { + delete persist_stats_cf_handle_; + persist_stats_cf_handle_ = nullptr; + } + mutex_.Lock(); + } + + // Clean up obsolete files due to SuperVersion release. + // (1) Need to delete to obsolete files before closing because RepairDB() + // scans all existing files in the file system and builds manifest file. + // Keeping obsolete files confuses the repair process. + // (2) Need to check if we Open()/Recover() the DB successfully before + // deleting because if VersionSet recover fails (may be due to corrupted + // manifest file), it is not able to identify live files correctly. As a + // result, all "live" files can get deleted by accident. However, corrupted + // manifest is recoverable by RepairDB(). + if (opened_successfully_) { + JobContext job_context(next_job_id_.fetch_add(1)); + FindObsoleteFiles(&job_context, true); + + mutex_.Unlock(); + // manifest number starting from 2 + job_context.manifest_file_number = 1; + if (job_context.HaveSomethingToDelete()) { + PurgeObsoleteFiles(job_context); + } + job_context.Clean(); + mutex_.Lock(); + } + { + InstrumentedMutexLock lock(&log_write_mutex_); + for (auto l : logs_to_free_) { + delete l; + } + for (auto& log : logs_) { + uint64_t log_number = log.writer->get_log_number(); + Status s = log.ClearWriter(); + if (!s.ok()) { + ROCKS_LOG_WARN( + immutable_db_options_.info_log, + "Unable to Sync WAL file %s with error -- %s", + LogFileName(immutable_db_options_.GetWalDir(), log_number).c_str(), + s.ToString().c_str()); + // Retain the first error + if (ret.ok()) { + ret = s; + } + } + } + logs_.clear(); + } + + // Table cache may have table handles holding blocks from the block cache. + // We need to release them before the block cache is destroyed. The block + // cache may be destroyed inside versions_.reset(), when column family data + // list is destroyed, so leaving handles in table cache after + // versions_.reset() may cause issues. + // Here we clean all unreferenced handles in table cache. + // Now we assume all user queries have finished, so only version set itself + // can possibly hold the blocks from block cache. After releasing unreferenced + // handles here, only handles held by version set left and inside + // versions_.reset(), we will release them. There, we need to make sure every + // time a handle is released, we erase it from the cache too. By doing that, + // we can guarantee that after versions_.reset(), table cache is empty + // so the cache can be safely destroyed. + table_cache_->EraseUnRefEntries(); + + for (auto& txn_entry : recovered_transactions_) { + delete txn_entry.second; + } + + // versions need to be destroyed before table_cache since it can hold + // references to table_cache. + versions_.reset(); + mutex_.Unlock(); + if (db_lock_ != nullptr) { + // TODO: Check for unlock error + env_->UnlockFile(db_lock_).PermitUncheckedError(); + } + + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Shutdown complete"); + LogFlush(immutable_db_options_.info_log); + +#ifndef ROCKSDB_LITE + // If the sst_file_manager was allocated by us during DB::Open(), ccall + // Close() on it before closing the info_log. Otherwise, background thread + // in SstFileManagerImpl might try to log something + if (immutable_db_options_.sst_file_manager && own_sfm_) { + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); + sfm->Close(); + } +#endif // ROCKSDB_LITE + + if (immutable_db_options_.info_log && own_info_log_) { + Status s = immutable_db_options_.info_log->Close(); + if (!s.ok() && !s.IsNotSupported() && ret.ok()) { + ret = s; + } + } + + if (write_buffer_manager_ && wbm_stall_) { + write_buffer_manager_->RemoveDBFromQueue(wbm_stall_.get()); + } + + IOStatus io_s = directories_.Close(IOOptions(), nullptr /* dbg */); + if (!io_s.ok()) { + ret = io_s; + } + if (ret.IsAborted()) { + // Reserve IsAborted() error for those where users didn't release + // certain resource and they can release them and come back and + // retry. In this case, we wrap this exception to something else. + return Status::Incomplete(ret.ToString()); + } + + return ret; +} + +Status DBImpl::CloseImpl() { return CloseHelper(); } + +DBImpl::~DBImpl() { + // TODO: remove this. + init_logger_creation_s_.PermitUncheckedError(); + + InstrumentedMutexLock closing_lock_guard(&closing_mutex_); + if (closed_) { + return; + } + + closed_ = true; + + { + const Status s = MaybeReleaseTimestampedSnapshotsAndCheck(); + s.PermitUncheckedError(); + } + + closing_status_ = CloseImpl(); + closing_status_.PermitUncheckedError(); +} + +void DBImpl::MaybeIgnoreError(Status* s) const { + if (s->ok() || immutable_db_options_.paranoid_checks) { + // No change needed + } else { + ROCKS_LOG_WARN(immutable_db_options_.info_log, "Ignoring error %s", + s->ToString().c_str()); + *s = Status::OK(); + } +} + +const Status DBImpl::CreateArchivalDirectory() { + if (immutable_db_options_.WAL_ttl_seconds > 0 || + immutable_db_options_.WAL_size_limit_MB > 0) { + std::string archivalPath = + ArchivalDirectory(immutable_db_options_.GetWalDir()); + return env_->CreateDirIfMissing(archivalPath); + } + return Status::OK(); +} + +void DBImpl::PrintStatistics() { + auto dbstats = immutable_db_options_.stats; + if (dbstats) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "STATISTICS:\n %s", + dbstats->ToString().c_str()); + } +} + +Status DBImpl::StartPeriodicTaskScheduler() { +#ifndef ROCKSDB_LITE + +#ifndef NDEBUG + // It only used by test to disable scheduler + bool disable_scheduler = false; + TEST_SYNC_POINT_CALLBACK( + "DBImpl::StartPeriodicTaskScheduler:DisableScheduler", + &disable_scheduler); + if (disable_scheduler) { + return Status::OK(); + } + + { + InstrumentedMutexLock l(&mutex_); + TEST_SYNC_POINT_CALLBACK("DBImpl::StartPeriodicTaskScheduler:Init", + &periodic_task_scheduler_); + } + +#endif // !NDEBUG + if (mutable_db_options_.stats_dump_period_sec > 0) { + Status s = periodic_task_scheduler_.Register( + PeriodicTaskType::kDumpStats, + periodic_task_functions_.at(PeriodicTaskType::kDumpStats), + mutable_db_options_.stats_dump_period_sec); + if (!s.ok()) { + return s; + } + } + if (mutable_db_options_.stats_persist_period_sec > 0) { + Status s = periodic_task_scheduler_.Register( + PeriodicTaskType::kPersistStats, + periodic_task_functions_.at(PeriodicTaskType::kPersistStats), + mutable_db_options_.stats_persist_period_sec); + if (!s.ok()) { + return s; + } + } + + Status s = periodic_task_scheduler_.Register( + PeriodicTaskType::kFlushInfoLog, + periodic_task_functions_.at(PeriodicTaskType::kFlushInfoLog)); + + return s; +#else + return Status::OK(); +#endif // !ROCKSDB_LITE +} + +Status DBImpl::RegisterRecordSeqnoTimeWorker() { +#ifndef ROCKSDB_LITE + uint64_t min_time_duration = std::numeric_limits::max(); + uint64_t max_time_duration = std::numeric_limits::min(); + { + InstrumentedMutexLock l(&mutex_); + + for (auto cfd : *versions_->GetColumnFamilySet()) { + // preserve time is the max of 2 options. + uint64_t preserve_time_duration = + std::max(cfd->ioptions()->preserve_internal_time_seconds, + cfd->ioptions()->preclude_last_level_data_seconds); + if (!cfd->IsDropped() && preserve_time_duration > 0) { + min_time_duration = std::min(preserve_time_duration, min_time_duration); + max_time_duration = std::max(preserve_time_duration, max_time_duration); + } + } + if (min_time_duration == std::numeric_limits::max()) { + seqno_time_mapping_.Resize(0, 0); + } else { + seqno_time_mapping_.Resize(min_time_duration, max_time_duration); + } + } + + uint64_t seqno_time_cadence = 0; + if (min_time_duration != std::numeric_limits::max()) { + // round up to 1 when the time_duration is smaller than + // kMaxSeqnoTimePairsPerCF + seqno_time_cadence = + (min_time_duration + SeqnoToTimeMapping::kMaxSeqnoTimePairsPerCF - 1) / + SeqnoToTimeMapping::kMaxSeqnoTimePairsPerCF; + } + + Status s; + if (seqno_time_cadence == 0) { + s = periodic_task_scheduler_.Unregister(PeriodicTaskType::kRecordSeqnoTime); + } else { + s = periodic_task_scheduler_.Register( + PeriodicTaskType::kRecordSeqnoTime, + periodic_task_functions_.at(PeriodicTaskType::kRecordSeqnoTime), + seqno_time_cadence); + } + + return s; +#else + return Status::OK(); +#endif // !ROCKSDB_LITE +} + +// esitmate the total size of stats_history_ +size_t DBImpl::EstimateInMemoryStatsHistorySize() const { + size_t size_total = + sizeof(std::map>); + if (stats_history_.size() == 0) return size_total; + size_t size_per_slice = + sizeof(uint64_t) + sizeof(std::map); + // non-empty map, stats_history_.begin() guaranteed to exist + for (const auto& pairs : stats_history_.begin()->second) { + size_per_slice += + pairs.first.capacity() + sizeof(pairs.first) + sizeof(pairs.second); + } + size_total = size_per_slice * stats_history_.size(); + return size_total; +} + +void DBImpl::PersistStats() { + TEST_SYNC_POINT("DBImpl::PersistStats:Entry"); +#ifndef ROCKSDB_LITE + if (shutdown_initiated_) { + return; + } + TEST_SYNC_POINT("DBImpl::PersistStats:StartRunning"); + uint64_t now_seconds = + immutable_db_options_.clock->NowMicros() / kMicrosInSecond; + + Statistics* statistics = immutable_db_options_.stats; + if (!statistics) { + return; + } + size_t stats_history_size_limit = 0; + { + InstrumentedMutexLock l(&mutex_); + stats_history_size_limit = mutable_db_options_.stats_history_buffer_size; + } + + std::map stats_map; + if (!statistics->getTickerMap(&stats_map)) { + return; + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "------- PERSISTING STATS -------"); + + if (immutable_db_options_.persist_stats_to_disk) { + WriteBatch batch; + Status s = Status::OK(); + if (stats_slice_initialized_) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Reading %" ROCKSDB_PRIszt " stats from statistics\n", + stats_slice_.size()); + for (const auto& stat : stats_map) { + if (s.ok()) { + char key[100]; + int length = + EncodePersistentStatsKey(now_seconds, stat.first, 100, key); + // calculate the delta from last time + if (stats_slice_.find(stat.first) != stats_slice_.end()) { + uint64_t delta = stat.second - stats_slice_[stat.first]; + s = batch.Put(persist_stats_cf_handle_, + Slice(key, std::min(100, length)), + std::to_string(delta)); + } + } + } + } + stats_slice_initialized_ = true; + std::swap(stats_slice_, stats_map); + if (s.ok()) { + WriteOptions wo; + wo.low_pri = true; + wo.no_slowdown = true; + wo.sync = false; + s = Write(wo, &batch); + } + if (!s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Writing to persistent stats CF failed -- %s", + s.ToString().c_str()); + } else { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Writing %" ROCKSDB_PRIszt " stats with timestamp %" PRIu64 + " to persistent stats CF succeeded", + stats_slice_.size(), now_seconds); + } + // TODO(Zhongyi): add purging for persisted data + } else { + InstrumentedMutexLock l(&stats_history_mutex_); + // calculate the delta from last time + if (stats_slice_initialized_) { + std::map stats_delta; + for (const auto& stat : stats_map) { + if (stats_slice_.find(stat.first) != stats_slice_.end()) { + stats_delta[stat.first] = stat.second - stats_slice_[stat.first]; + } + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Storing %" ROCKSDB_PRIszt " stats with timestamp %" PRIu64 + " to in-memory stats history", + stats_slice_.size(), now_seconds); + stats_history_[now_seconds] = stats_delta; + } + stats_slice_initialized_ = true; + std::swap(stats_slice_, stats_map); + TEST_SYNC_POINT("DBImpl::PersistStats:StatsCopied"); + + // delete older stats snapshots to control memory consumption + size_t stats_history_size = EstimateInMemoryStatsHistorySize(); + bool purge_needed = stats_history_size > stats_history_size_limit; + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[Pre-GC] In-memory stats history size: %" ROCKSDB_PRIszt + " bytes, slice count: %" ROCKSDB_PRIszt, + stats_history_size, stats_history_.size()); + while (purge_needed && !stats_history_.empty()) { + stats_history_.erase(stats_history_.begin()); + purge_needed = + EstimateInMemoryStatsHistorySize() > stats_history_size_limit; + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[Post-GC] In-memory stats history size: %" ROCKSDB_PRIszt + " bytes, slice count: %" ROCKSDB_PRIszt, + stats_history_size, stats_history_.size()); + } + TEST_SYNC_POINT("DBImpl::PersistStats:End"); +#endif // !ROCKSDB_LITE +} + +bool DBImpl::FindStatsByTime(uint64_t start_time, uint64_t end_time, + uint64_t* new_time, + std::map* stats_map) { + assert(new_time); + assert(stats_map); + if (!new_time || !stats_map) return false; + // lock when search for start_time + { + InstrumentedMutexLock l(&stats_history_mutex_); + auto it = stats_history_.lower_bound(start_time); + if (it != stats_history_.end() && it->first < end_time) { + // make a copy for timestamp and stats_map + *new_time = it->first; + *stats_map = it->second; + return true; + } else { + return false; + } + } +} + +Status DBImpl::GetStatsHistory( + uint64_t start_time, uint64_t end_time, + std::unique_ptr* stats_iterator) { + if (!stats_iterator) { + return Status::InvalidArgument("stats_iterator not preallocated."); + } + if (immutable_db_options_.persist_stats_to_disk) { + stats_iterator->reset( + new PersistentStatsHistoryIterator(start_time, end_time, this)); + } else { + stats_iterator->reset( + new InMemoryStatsHistoryIterator(start_time, end_time, this)); + } + return (*stats_iterator)->status(); +} + +void DBImpl::DumpStats() { + TEST_SYNC_POINT("DBImpl::DumpStats:1"); +#ifndef ROCKSDB_LITE + std::string stats; + if (shutdown_initiated_) { + return; + } + + // Also probe block cache(s) for problems, dump to info log + UnorderedSet probed_caches; + TEST_SYNC_POINT("DBImpl::DumpStats:StartRunning"); + { + InstrumentedMutexLock l(&mutex_); + for (auto cfd : versions_->GetRefedColumnFamilySet()) { + if (!cfd->initialized()) { + continue; + } + + // Release DB mutex for gathering cache entry stats. Pass over all + // column families for this first so that other stats are dumped + // near-atomically. + InstrumentedMutexUnlock u(&mutex_); + cfd->internal_stats()->CollectCacheEntryStats(/*foreground=*/false); + + // Probe block cache for problems (if not already via another CF) + if (immutable_db_options_.info_log) { + auto* table_factory = cfd->ioptions()->table_factory.get(); + assert(table_factory != nullptr); + Cache* cache = + table_factory->GetOptions(TableFactory::kBlockCacheOpts()); + if (cache && probed_caches.insert(cache).second) { + cache->ReportProblems(immutable_db_options_.info_log); + } + } + } + + const std::string* property = &DB::Properties::kDBStats; + const DBPropertyInfo* property_info = GetPropertyInfo(*property); + assert(property_info != nullptr); + assert(!property_info->need_out_of_mutex); + default_cf_internal_stats_->GetStringProperty(*property_info, *property, + &stats); + + property = &InternalStats::kPeriodicCFStats; + property_info = GetPropertyInfo(*property); + assert(property_info != nullptr); + assert(!property_info->need_out_of_mutex); + for (auto cfd : *versions_->GetColumnFamilySet()) { + if (cfd->initialized()) { + cfd->internal_stats()->GetStringProperty(*property_info, *property, + &stats); + } + } + } + TEST_SYNC_POINT("DBImpl::DumpStats:2"); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "------- DUMPING STATS -------"); + ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s", stats.c_str()); + if (immutable_db_options_.dump_malloc_stats) { + stats.clear(); + DumpMallocStats(&stats); + if (!stats.empty()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "------- Malloc STATS -------"); + ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s", stats.c_str()); + } + } +#endif // !ROCKSDB_LITE + + PrintStatistics(); +} + +// Periodically flush info log out of application buffer at a low frequency. +// This improves debuggability in case of RocksDB hanging since it ensures the +// log messages leading up to the hang will eventually become visible in the +// log. +void DBImpl::FlushInfoLog() { + if (shutdown_initiated_) { + return; + } + TEST_SYNC_POINT("DBImpl::FlushInfoLog:StartRunning"); + LogFlush(immutable_db_options_.info_log); +} + +Status DBImpl::TablesRangeTombstoneSummary(ColumnFamilyHandle* column_family, + int max_entries_to_print, + std::string* out_str) { + auto* cfh = static_cast_with_check(column_family); + ColumnFamilyData* cfd = cfh->cfd(); + + SuperVersion* super_version = cfd->GetReferencedSuperVersion(this); + Version* version = super_version->current; + + Status s = + version->TablesRangeTombstoneSummary(max_entries_to_print, out_str); + + CleanupSuperVersion(super_version); + return s; +} + +void DBImpl::ScheduleBgLogWriterClose(JobContext* job_context) { + mutex_.AssertHeld(); + if (!job_context->logs_to_free.empty()) { + for (auto l : job_context->logs_to_free) { + AddToLogsToFreeQueue(l); + } + job_context->logs_to_free.clear(); + } +} + +FSDirectory* DBImpl::GetDataDir(ColumnFamilyData* cfd, size_t path_id) const { + assert(cfd); + FSDirectory* ret_dir = cfd->GetDataDir(path_id); + if (ret_dir == nullptr) { + return directories_.GetDataDir(path_id); + } + return ret_dir; +} + +Status DBImpl::SetOptions( + ColumnFamilyHandle* column_family, + const std::unordered_map& options_map) { +#ifdef ROCKSDB_LITE + (void)column_family; + (void)options_map; + return Status::NotSupported("Not supported in ROCKSDB LITE"); +#else + auto* cfd = + static_cast_with_check(column_family)->cfd(); + if (options_map.empty()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "SetOptions() on column family [%s], empty input", + cfd->GetName().c_str()); + return Status::InvalidArgument("empty input"); + } + + MutableCFOptions new_options; + Status s; + Status persist_options_status; + SuperVersionContext sv_context(/* create_superversion */ true); + { + auto db_options = GetDBOptions(); + InstrumentedMutexLock l(&mutex_); + s = cfd->SetOptions(db_options, options_map); + if (s.ok()) { + new_options = *cfd->GetLatestMutableCFOptions(); + // Append new version to recompute compaction score. + VersionEdit dummy_edit; + s = versions_->LogAndApply(cfd, new_options, &dummy_edit, &mutex_, + directories_.GetDbDir()); + // 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); + + persist_options_status = WriteOptionsFile( + false /*need_mutex_lock*/, true /*need_enter_write_thread*/); + bg_cv_.SignalAll(); + } + } + sv_context.Clean(); + + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "SetOptions() on column family [%s], inputs:", cfd->GetName().c_str()); + for (const auto& o : options_map) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s: %s\n", o.first.c_str(), + o.second.c_str()); + } + 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()); + if (!persist_options_status.ok()) { + // NOTE: WriteOptionsFile already logs on failure + s = persist_options_status; + } + } else { + persist_options_status.PermitUncheckedError(); // less important + ROCKS_LOG_WARN(immutable_db_options_.info_log, "[%s] SetOptions() failed", + cfd->GetName().c_str()); + } + LogFlush(immutable_db_options_.info_log); + return s; +#endif // ROCKSDB_LITE +} + +Status DBImpl::SetDBOptions( + const std::unordered_map& options_map) { +#ifdef ROCKSDB_LITE + (void)options_map; + return Status::NotSupported("Not supported in ROCKSDB LITE"); +#else + if (options_map.empty()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "SetDBOptions(), empty input."); + return Status::InvalidArgument("empty input"); + } + + MutableDBOptions new_options; + Status s; + Status persist_options_status = Status::OK(); + bool wal_changed = false; + WriteContext write_context; + { + InstrumentedMutexLock l(&mutex_); + s = GetMutableDBOptionsFromStrings(mutable_db_options_, options_map, + &new_options); + + if (new_options.bytes_per_sync == 0) { + new_options.bytes_per_sync = 1024 * 1024; + } + + if (MutableDBOptionsAreEqual(mutable_db_options_, new_options)) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "SetDBOptions(), input option value is not changed, " + "skipping updating."); + persist_options_status.PermitUncheckedError(); + return s; + } + + DBOptions new_db_options = + BuildDBOptions(immutable_db_options_, new_options); + if (s.ok()) { + s = ValidateOptions(new_db_options); + } + if (s.ok()) { + for (auto c : *versions_->GetColumnFamilySet()) { + if (!c->IsDropped()) { + auto cf_options = c->GetLatestCFOptions(); + s = ColumnFamilyData::ValidateOptions(new_db_options, cf_options); + if (!s.ok()) { + break; + } + } + } + } + if (s.ok()) { + const BGJobLimits current_bg_job_limits = + GetBGJobLimits(mutable_db_options_.max_background_flushes, + mutable_db_options_.max_background_compactions, + mutable_db_options_.max_background_jobs, + /* parallelize_compactions */ true); + const BGJobLimits new_bg_job_limits = GetBGJobLimits( + new_options.max_background_flushes, + new_options.max_background_compactions, + new_options.max_background_jobs, /* parallelize_compactions */ true); + + const bool max_flushes_increased = + new_bg_job_limits.max_flushes > current_bg_job_limits.max_flushes; + const bool max_compactions_increased = + new_bg_job_limits.max_compactions > + current_bg_job_limits.max_compactions; + + if (max_flushes_increased || max_compactions_increased) { + if (max_flushes_increased) { + env_->IncBackgroundThreadsIfNeeded(new_bg_job_limits.max_flushes, + Env::Priority::HIGH); + } + + if (max_compactions_increased) { + env_->IncBackgroundThreadsIfNeeded(new_bg_job_limits.max_compactions, + Env::Priority::LOW); + } + + MaybeScheduleFlushOrCompaction(); + } + + mutex_.Unlock(); + if (new_options.stats_dump_period_sec == 0) { + s = periodic_task_scheduler_.Unregister(PeriodicTaskType::kDumpStats); + } else { + s = periodic_task_scheduler_.Register( + PeriodicTaskType::kDumpStats, + periodic_task_functions_.at(PeriodicTaskType::kDumpStats), + new_options.stats_dump_period_sec); + } + if (new_options.max_total_wal_size != + mutable_db_options_.max_total_wal_size) { + max_total_wal_size_.store(new_options.max_total_wal_size, + std::memory_order_release); + } + if (s.ok()) { + if (new_options.stats_persist_period_sec == 0) { + s = periodic_task_scheduler_.Unregister( + PeriodicTaskType::kPersistStats); + } else { + s = periodic_task_scheduler_.Register( + PeriodicTaskType::kPersistStats, + periodic_task_functions_.at(PeriodicTaskType::kPersistStats), + new_options.stats_persist_period_sec); + } + } + mutex_.Lock(); + if (!s.ok()) { + return s; + } + + write_controller_.set_max_delayed_write_rate( + new_options.delayed_write_rate); + table_cache_.get()->SetCapacity(new_options.max_open_files == -1 + ? TableCache::kInfiniteCapacity + : new_options.max_open_files - 10); + wal_changed = mutable_db_options_.wal_bytes_per_sync != + new_options.wal_bytes_per_sync; + mutable_db_options_ = new_options; + file_options_for_compaction_ = FileOptions(new_db_options); + file_options_for_compaction_ = fs_->OptimizeForCompactionTableWrite( + file_options_for_compaction_, immutable_db_options_); + versions_->ChangeFileOptions(mutable_db_options_); + // TODO(xiez): clarify why apply optimize for read to write options + file_options_for_compaction_ = fs_->OptimizeForCompactionTableRead( + file_options_for_compaction_, immutable_db_options_); + file_options_for_compaction_.compaction_readahead_size = + mutable_db_options_.compaction_readahead_size; + WriteThread::Writer w; + write_thread_.EnterUnbatched(&w, &mutex_); + if (total_log_size_ > GetMaxTotalWalSize() || wal_changed) { + Status purge_wal_status = SwitchWAL(&write_context); + if (!purge_wal_status.ok()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "Unable to purge WAL files in SetDBOptions() -- %s", + purge_wal_status.ToString().c_str()); + } + } + persist_options_status = WriteOptionsFile( + false /*need_mutex_lock*/, false /*need_enter_write_thread*/); + write_thread_.ExitUnbatched(&w); + } else { + // To get here, we must have had invalid options and will not attempt to + // persist the options, which means the status is "OK/Uninitialized. + persist_options_status.PermitUncheckedError(); + } + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, "SetDBOptions(), inputs:"); + for (const auto& o : options_map) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s: %s\n", o.first.c_str(), + o.second.c_str()); + } + if (s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "SetDBOptions() succeeded"); + new_options.Dump(immutable_db_options_.info_log.get()); + if (!persist_options_status.ok()) { + if (immutable_db_options_.fail_if_options_file_error) { + s = Status::IOError( + "SetDBOptions() succeeded, but unable to persist options", + persist_options_status.ToString()); + } + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "Unable to persist options in SetDBOptions() -- %s", + persist_options_status.ToString().c_str()); + } + } else { + ROCKS_LOG_WARN(immutable_db_options_.info_log, "SetDBOptions failed"); + } + LogFlush(immutable_db_options_.info_log); + return s; +#endif // ROCKSDB_LITE +} + +// return the same level if it cannot be moved +int DBImpl::FindMinimumEmptyLevelFitting( + ColumnFamilyData* cfd, const MutableCFOptions& /*mutable_cf_options*/, + int level) { + mutex_.AssertHeld(); + const auto* vstorage = cfd->current()->storage_info(); + int minimum_level = level; + for (int i = level - 1; i > 0; --i) { + // stop if level i is not empty + if (vstorage->NumLevelFiles(i) > 0) break; + // stop if level i is too small (cannot fit the level files) + if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) { + break; + } + + minimum_level = i; + } + return minimum_level; +} + +Status DBImpl::FlushWAL(bool sync) { + if (manual_wal_flush_) { + IOStatus io_s; + { + // We need to lock log_write_mutex_ since logs_ might change concurrently + InstrumentedMutexLock wl(&log_write_mutex_); + log::Writer* cur_log_writer = logs_.back().writer; + io_s = cur_log_writer->WriteBuffer(); + } + if (!io_s.ok()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, "WAL flush error %s", + io_s.ToString().c_str()); + // In case there is a fs error we should set it globally to prevent the + // future writes + IOStatusCheck(io_s); + // whether sync or not, we should abort the rest of function upon error + return static_cast(io_s); + } + if (!sync) { + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "FlushWAL sync=false"); + return static_cast(io_s); + } + } + if (!sync) { + return Status::OK(); + } + // sync = true + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "FlushWAL sync=true"); + return SyncWAL(); +} + +bool DBImpl::WALBufferIsEmpty(bool lock) { + if (lock) { + log_write_mutex_.Lock(); + } + log::Writer* cur_log_writer = logs_.back().writer; + auto res = cur_log_writer->BufferIsEmpty(); + if (lock) { + log_write_mutex_.Unlock(); + } + return res; +} + +Status DBImpl::SyncWAL() { + TEST_SYNC_POINT("DBImpl::SyncWAL:Begin"); + autovector logs_to_sync; + bool need_log_dir_sync; + uint64_t current_log_number; + + { + InstrumentedMutexLock l(&log_write_mutex_); + assert(!logs_.empty()); + + // This SyncWAL() call only cares about logs up to this number. + current_log_number = logfile_number_; + + while (logs_.front().number <= current_log_number && + logs_.front().IsSyncing()) { + log_sync_cv_.Wait(); + } + // First check that logs are safe to sync in background. + for (auto it = logs_.begin(); + it != logs_.end() && it->number <= current_log_number; ++it) { + if (!it->writer->file()->writable_file()->IsSyncThreadSafe()) { + return Status::NotSupported( + "SyncWAL() is not supported for this implementation of WAL file", + immutable_db_options_.allow_mmap_writes + ? "try setting Options::allow_mmap_writes to false" + : Slice()); + } + } + for (auto it = logs_.begin(); + it != logs_.end() && it->number <= current_log_number; ++it) { + auto& log = *it; + log.PrepareForSync(); + logs_to_sync.push_back(log.writer); + } + + need_log_dir_sync = !log_dir_synced_; + } + + TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:1"); + RecordTick(stats_, WAL_FILE_SYNCED); + Status status; + IOStatus io_s; + for (log::Writer* log : logs_to_sync) { + io_s = log->file()->SyncWithoutFlush(immutable_db_options_.use_fsync); + if (!io_s.ok()) { + status = io_s; + break; + } + } + if (!io_s.ok()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, "WAL Sync error %s", + io_s.ToString().c_str()); + // In case there is a fs error we should set it globally to prevent the + // future writes + IOStatusCheck(io_s); + } + if (status.ok() && need_log_dir_sync) { + status = directories_.GetWalDir()->FsyncWithDirOptions( + IOOptions(), nullptr, + DirFsyncOptions(DirFsyncOptions::FsyncReason::kNewFileSynced)); + } + TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:2"); + + TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:1"); + VersionEdit synced_wals; + { + InstrumentedMutexLock l(&log_write_mutex_); + if (status.ok()) { + MarkLogsSynced(current_log_number, need_log_dir_sync, &synced_wals); + } else { + MarkLogsNotSynced(current_log_number); + } + } + if (status.ok() && synced_wals.IsWalAddition()) { + InstrumentedMutexLock l(&mutex_); + status = ApplyWALToManifest(&synced_wals); + } + + TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:2"); + + return status; +} + +Status DBImpl::ApplyWALToManifest(VersionEdit* synced_wals) { + // not empty, write to MANIFEST. + mutex_.AssertHeld(); + Status status = versions_->LogAndApplyToDefaultColumnFamily( + synced_wals, &mutex_, directories_.GetDbDir()); + if (!status.ok() && versions_->io_status().IsIOError()) { + status = error_handler_.SetBGError(versions_->io_status(), + BackgroundErrorReason::kManifestWrite); + } + return status; +} + +Status DBImpl::LockWAL() { + log_write_mutex_.Lock(); + auto cur_log_writer = logs_.back().writer; + IOStatus status = cur_log_writer->WriteBuffer(); + if (!status.ok()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, "WAL flush error %s", + status.ToString().c_str()); + // In case there is a fs error we should set it globally to prevent the + // future writes + WriteStatusCheck(status); + } + return static_cast(status); +} + +Status DBImpl::UnlockWAL() { + log_write_mutex_.Unlock(); + return Status::OK(); +} + +void DBImpl::MarkLogsSynced(uint64_t up_to, bool synced_dir, + VersionEdit* synced_wals) { + log_write_mutex_.AssertHeld(); + if (synced_dir && logfile_number_ == up_to) { + log_dir_synced_ = true; + } + for (auto it = logs_.begin(); it != logs_.end() && it->number <= up_to;) { + auto& wal = *it; + assert(wal.IsSyncing()); + + if (wal.number < logs_.back().number) { + // Inactive WAL + if (immutable_db_options_.track_and_verify_wals_in_manifest && + wal.GetPreSyncSize() > 0) { + synced_wals->AddWal(wal.number, WalMetadata(wal.GetPreSyncSize())); + } + if (wal.GetPreSyncSize() == wal.writer->file()->GetFlushedSize()) { + // Fully synced + logs_to_free_.push_back(wal.ReleaseWriter()); + it = logs_.erase(it); + } else { + assert(wal.GetPreSyncSize() < wal.writer->file()->GetFlushedSize()); + wal.FinishSync(); + ++it; + } + } else { + assert(wal.number == logs_.back().number); + // Active WAL + wal.FinishSync(); + ++it; + } + } + log_sync_cv_.SignalAll(); +} + +void DBImpl::MarkLogsNotSynced(uint64_t up_to) { + log_write_mutex_.AssertHeld(); + for (auto it = logs_.begin(); it != logs_.end() && it->number <= up_to; + ++it) { + auto& wal = *it; + wal.FinishSync(); + } + log_sync_cv_.SignalAll(); +} + +SequenceNumber DBImpl::GetLatestSequenceNumber() const { + return versions_->LastSequence(); +} + +void DBImpl::SetLastPublishedSequence(SequenceNumber seq) { + versions_->SetLastPublishedSequence(seq); +} + +Status DBImpl::GetFullHistoryTsLow(ColumnFamilyHandle* column_family, + std::string* ts_low) { + if (ts_low == nullptr) { + return Status::InvalidArgument("ts_low is nullptr"); + } + ColumnFamilyData* cfd = nullptr; + if (column_family == nullptr) { + cfd = default_cf_handle_->cfd(); + } else { + auto cfh = static_cast_with_check(column_family); + assert(cfh != nullptr); + cfd = cfh->cfd(); + } + assert(cfd != nullptr && cfd->user_comparator() != nullptr); + if (cfd->user_comparator()->timestamp_size() == 0) { + return Status::InvalidArgument( + "Timestamp is not enabled in this column family"); + } + InstrumentedMutexLock l(&mutex_); + *ts_low = cfd->GetFullHistoryTsLow(); + assert(cfd->user_comparator()->timestamp_size() == ts_low->size()); + return Status::OK(); +} + +InternalIterator* DBImpl::NewInternalIterator(const ReadOptions& read_options, + Arena* arena, + SequenceNumber sequence, + ColumnFamilyHandle* column_family, + bool allow_unprepared_value) { + ColumnFamilyData* cfd; + if (column_family == nullptr) { + cfd = default_cf_handle_->cfd(); + } else { + auto cfh = static_cast_with_check(column_family); + cfd = cfh->cfd(); + } + + mutex_.Lock(); + SuperVersion* super_version = cfd->GetSuperVersion()->Ref(); + mutex_.Unlock(); + return NewInternalIterator(read_options, cfd, super_version, arena, sequence, + allow_unprepared_value); +} + +void DBImpl::SchedulePurge() { + mutex_.AssertHeld(); + assert(opened_successfully_); + + // Purge operations are put into High priority queue + bg_purge_scheduled_++; + env_->Schedule(&DBImpl::BGWorkPurge, this, Env::Priority::HIGH, nullptr); +} + +void DBImpl::BackgroundCallPurge() { + mutex_.Lock(); + + while (!logs_to_free_queue_.empty()) { + assert(!logs_to_free_queue_.empty()); + log::Writer* log_writer = *(logs_to_free_queue_.begin()); + logs_to_free_queue_.pop_front(); + mutex_.Unlock(); + delete log_writer; + mutex_.Lock(); + } + while (!superversions_to_free_queue_.empty()) { + assert(!superversions_to_free_queue_.empty()); + SuperVersion* sv = superversions_to_free_queue_.front(); + superversions_to_free_queue_.pop_front(); + mutex_.Unlock(); + delete sv; + mutex_.Lock(); + } + + assert(bg_purge_scheduled_ > 0); + + // Can't use iterator to go over purge_files_ because inside the loop we're + // unlocking the mutex that protects purge_files_. + while (!purge_files_.empty()) { + auto it = purge_files_.begin(); + // Need to make a copy of the PurgeFilesInfo before unlocking the mutex. + PurgeFileInfo purge_file = it->second; + + const std::string& fname = purge_file.fname; + const std::string& dir_to_sync = purge_file.dir_to_sync; + FileType type = purge_file.type; + uint64_t number = purge_file.number; + int job_id = purge_file.job_id; + + purge_files_.erase(it); + + mutex_.Unlock(); + DeleteObsoleteFileImpl(job_id, fname, dir_to_sync, type, number); + mutex_.Lock(); + } + + bg_purge_scheduled_--; + + bg_cv_.SignalAll(); + // IMPORTANT:there should be no code after calling SignalAll. This call may + // signal the DB destructor that it's OK to proceed with destruction. In + // that case, all DB variables will be dealloacated and referencing them + // will cause trouble. + mutex_.Unlock(); +} + +namespace { + +// A `SuperVersionHandle` holds a non-null `SuperVersion*` pointing at a +// `SuperVersion` referenced once for this object. It also contains the state +// needed to clean up the `SuperVersion` reference from outside of `DBImpl` +// using `CleanupSuperVersionHandle()`. +struct SuperVersionHandle { + // `_super_version` must be non-nullptr and `Ref()`'d once as long as the + // `SuperVersionHandle` may use it. + SuperVersionHandle(DBImpl* _db, InstrumentedMutex* _mu, + SuperVersion* _super_version, bool _background_purge) + : db(_db), + mu(_mu), + super_version(_super_version), + background_purge(_background_purge) {} + + DBImpl* db; + InstrumentedMutex* mu; + SuperVersion* super_version; + bool background_purge; +}; + +static void CleanupSuperVersionHandle(void* arg1, void* /*arg2*/) { + SuperVersionHandle* sv_handle = reinterpret_cast(arg1); + + if (sv_handle->super_version->Unref()) { + // Job id == 0 means that this is not our background process, but rather + // user thread + JobContext job_context(0); + + sv_handle->mu->Lock(); + sv_handle->super_version->Cleanup(); + sv_handle->db->FindObsoleteFiles(&job_context, false, true); + if (sv_handle->background_purge) { + sv_handle->db->ScheduleBgLogWriterClose(&job_context); + sv_handle->db->AddSuperVersionsToFreeQueue(sv_handle->super_version); + sv_handle->db->SchedulePurge(); + } + sv_handle->mu->Unlock(); + + if (!sv_handle->background_purge) { + delete sv_handle->super_version; + } + if (job_context.HaveSomethingToDelete()) { + sv_handle->db->PurgeObsoleteFiles(job_context, + sv_handle->background_purge); + } + job_context.Clean(); + } + + delete sv_handle; +} + +struct GetMergeOperandsState { + MergeContext merge_context; + PinnedIteratorsManager pinned_iters_mgr; + SuperVersionHandle* sv_handle; +}; + +static void CleanupGetMergeOperandsState(void* arg1, void* /*arg2*/) { + GetMergeOperandsState* state = static_cast(arg1); + CleanupSuperVersionHandle(state->sv_handle /* arg1 */, nullptr /* arg2 */); + delete state; +} + +} // namespace + +InternalIterator* DBImpl::NewInternalIterator( + const ReadOptions& read_options, ColumnFamilyData* cfd, + SuperVersion* super_version, Arena* arena, SequenceNumber sequence, + bool allow_unprepared_value, ArenaWrappedDBIter* db_iter) { + InternalIterator* internal_iter; + assert(arena != nullptr); + // Need to create internal iterator from the arena. + MergeIteratorBuilder merge_iter_builder( + &cfd->internal_comparator(), arena, + !read_options.total_order_seek && + super_version->mutable_cf_options.prefix_extractor != nullptr, + read_options.iterate_upper_bound); + // Collect iterator for mutable memtable + auto mem_iter = super_version->mem->NewIterator(read_options, arena); + Status s; + if (!read_options.ignore_range_deletions) { + TruncatedRangeDelIterator* mem_tombstone_iter = nullptr; + auto range_del_iter = super_version->mem->NewRangeTombstoneIterator( + read_options, sequence, false /* immutable_memtable */); + if (range_del_iter == nullptr || range_del_iter->empty()) { + delete range_del_iter; + } else { + mem_tombstone_iter = new TruncatedRangeDelIterator( + std::unique_ptr(range_del_iter), + &cfd->ioptions()->internal_comparator, nullptr /* smallest */, + nullptr /* largest */); + } + merge_iter_builder.AddPointAndTombstoneIterator(mem_iter, + mem_tombstone_iter); + } else { + merge_iter_builder.AddIterator(mem_iter); + } + + // Collect all needed child iterators for immutable memtables + if (s.ok()) { + super_version->imm->AddIterators(read_options, &merge_iter_builder, + !read_options.ignore_range_deletions); + } + TEST_SYNC_POINT_CALLBACK("DBImpl::NewInternalIterator:StatusCallback", &s); + if (s.ok()) { + // Collect iterators for files in L0 - Ln + if (read_options.read_tier != kMemtableTier) { + super_version->current->AddIterators(read_options, file_options_, + &merge_iter_builder, + allow_unprepared_value); + } + internal_iter = merge_iter_builder.Finish( + read_options.ignore_range_deletions ? nullptr : db_iter); + SuperVersionHandle* cleanup = new SuperVersionHandle( + this, &mutex_, super_version, + read_options.background_purge_on_iterator_cleanup || + immutable_db_options_.avoid_unnecessary_blocking_io); + internal_iter->RegisterCleanup(CleanupSuperVersionHandle, cleanup, nullptr); + + return internal_iter; + } else { + CleanupSuperVersion(super_version); + } + return NewErrorInternalIterator(s, arena); +} + +ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const { + return default_cf_handle_; +} + +ColumnFamilyHandle* DBImpl::PersistentStatsColumnFamily() const { + return persist_stats_cf_handle_; +} + +Status DBImpl::Get(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value) { + return Get(read_options, column_family, key, value, /*timestamp=*/nullptr); +} + +Status DBImpl::Get(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value, std::string* timestamp) { + assert(value != nullptr); + value->Reset(); + GetImplOptions get_impl_options; + get_impl_options.column_family = column_family; + get_impl_options.value = value; + get_impl_options.timestamp = timestamp; + Status s = GetImpl(read_options, key, get_impl_options); + return s; +} + +Status DBImpl::GetEntity(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableWideColumns* columns) { + if (!column_family) { + return Status::InvalidArgument( + "Cannot call GetEntity without a column family handle"); + } + + if (!columns) { + return Status::InvalidArgument( + "Cannot call GetEntity without a PinnableWideColumns object"); + } + + columns->Reset(); + + GetImplOptions get_impl_options; + get_impl_options.column_family = column_family; + get_impl_options.columns = columns; + + return GetImpl(read_options, key, get_impl_options); +} + +bool DBImpl::ShouldReferenceSuperVersion(const MergeContext& merge_context) { + // If both thresholds are reached, a function returning merge operands as + // `PinnableSlice`s should reference the `SuperVersion` to avoid large and/or + // numerous `memcpy()`s. + // + // The below constants enable the optimization conservatively. They are + // verified to not regress `GetMergeOperands()` latency in the following + // scenarios. + // + // - CPU: two socket Intel(R) Xeon(R) Gold 6138 CPU @ 2.00GHz + // - `GetMergeOperands()` threads: 1 - 32 + // - Entry size: 32 bytes - 4KB + // - Merges per key: 1 - 16K + // - LSM component: memtable + // + // TODO(ajkr): expand measurement to SST files. + static const size_t kNumBytesForSvRef = 32768; + static const size_t kLog2AvgBytesForSvRef = 8; // 256 bytes + + size_t num_bytes = 0; + for (const Slice& sl : merge_context.GetOperands()) { + num_bytes += sl.size(); + } + return num_bytes >= kNumBytesForSvRef && + (num_bytes >> kLog2AvgBytesForSvRef) >= + merge_context.GetOperands().size(); +} + +Status DBImpl::GetImpl(const ReadOptions& read_options, const Slice& key, + GetImplOptions& get_impl_options) { + assert(get_impl_options.value != nullptr || + get_impl_options.merge_operands != nullptr || + get_impl_options.columns != nullptr); + + assert(get_impl_options.column_family); + + if (read_options.timestamp) { + const Status s = FailIfTsMismatchCf(get_impl_options.column_family, + *(read_options.timestamp), + /*ts_for_read=*/true); + if (!s.ok()) { + return s; + } + } else { + const Status s = FailIfCfHasTs(get_impl_options.column_family); + if (!s.ok()) { + return s; + } + } + + // Clear the timestamps for returning results so that we can distinguish + // between tombstone or key that has never been written + if (get_impl_options.timestamp) { + get_impl_options.timestamp->clear(); + } + + GetWithTimestampReadCallback read_cb(0); // Will call Refresh + + PERF_CPU_TIMER_GUARD(get_cpu_nanos, immutable_db_options_.clock); + StopWatch sw(immutable_db_options_.clock, stats_, DB_GET); + PERF_TIMER_GUARD(get_snapshot_time); + + auto cfh = static_cast_with_check( + get_impl_options.column_family); + auto cfd = cfh->cfd(); + + if (tracer_) { + // TODO: This mutex should be removed later, to improve performance when + // tracing is enabled. + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + // TODO: maybe handle the tracing status? + tracer_->Get(get_impl_options.column_family, key).PermitUncheckedError(); + } + } + + if (get_impl_options.get_merge_operands_options != nullptr) { + for (int i = 0; i < get_impl_options.get_merge_operands_options + ->expected_max_number_of_operands; + ++i) { + get_impl_options.merge_operands[i].Reset(); + } + } + + // Acquire SuperVersion + SuperVersion* sv = GetAndRefSuperVersion(cfd); + + TEST_SYNC_POINT("DBImpl::GetImpl:1"); + TEST_SYNC_POINT("DBImpl::GetImpl:2"); + + SequenceNumber snapshot; + if (read_options.snapshot != nullptr) { + if (get_impl_options.callback) { + // Already calculated based on read_options.snapshot + snapshot = get_impl_options.callback->max_visible_seq(); + } else { + snapshot = + reinterpret_cast(read_options.snapshot)->number_; + } + } else { + // Note that the snapshot is assigned AFTER referencing the super + // version because otherwise a flush happening in between may compact away + // data for the snapshot, so the reader would see neither data that was be + // visible to the snapshot before compaction nor the newer data inserted + // afterwards. + snapshot = GetLastPublishedSequence(); + if (get_impl_options.callback) { + // The unprep_seqs are not published for write unprepared, so it could be + // that max_visible_seq is larger. Seek to the std::max of the two. + // However, we still want our callback to contain the actual snapshot so + // that it can do the correct visibility filtering. + get_impl_options.callback->Refresh(snapshot); + + // Internally, WriteUnpreparedTxnReadCallback::Refresh would set + // max_visible_seq = max(max_visible_seq, snapshot) + // + // Currently, the commented out assert is broken by + // InvalidSnapshotReadCallback, but if write unprepared recovery followed + // the regular transaction flow, then this special read callback would not + // be needed. + // + // assert(callback->max_visible_seq() >= snapshot); + snapshot = get_impl_options.callback->max_visible_seq(); + } + } + // If timestamp is used, we use read callback to ensure is returned + // only if t <= read_opts.timestamp and s <= snapshot. + // HACK: temporarily overwrite input struct field but restore + SaveAndRestore restore_callback(&get_impl_options.callback); + const Comparator* ucmp = get_impl_options.column_family->GetComparator(); + assert(ucmp); + if (ucmp->timestamp_size() > 0) { + assert(!get_impl_options + .callback); // timestamp with callback is not supported + read_cb.Refresh(snapshot); + get_impl_options.callback = &read_cb; + } + TEST_SYNC_POINT("DBImpl::GetImpl:3"); + TEST_SYNC_POINT("DBImpl::GetImpl:4"); + + // Prepare to store a list of merge operations if merge occurs. + MergeContext merge_context; + SequenceNumber max_covering_tombstone_seq = 0; + + Status s; + // First look in the memtable, then in the immutable memtable (if any). + // s is both in/out. When in, s could either be OK or MergeInProgress. + // merge_operands will contain the sequence of merges in the latter case. + LookupKey lkey(key, snapshot, read_options.timestamp); + PERF_TIMER_STOP(get_snapshot_time); + + bool skip_memtable = (read_options.read_tier == kPersistedTier && + has_unpersisted_data_.load(std::memory_order_relaxed)); + bool done = false; + std::string* timestamp = + ucmp->timestamp_size() > 0 ? get_impl_options.timestamp : nullptr; + if (!skip_memtable) { + // Get value associated with key + if (get_impl_options.get_value) { + if (sv->mem->Get( + lkey, + get_impl_options.value ? get_impl_options.value->GetSelf() + : nullptr, + get_impl_options.columns, timestamp, &s, &merge_context, + &max_covering_tombstone_seq, read_options, + false /* immutable_memtable */, get_impl_options.callback, + get_impl_options.is_blob_index)) { + done = true; + + if (get_impl_options.value) { + get_impl_options.value->PinSelf(); + } + + RecordTick(stats_, MEMTABLE_HIT); + } else if ((s.ok() || s.IsMergeInProgress()) && + sv->imm->Get(lkey, + get_impl_options.value + ? get_impl_options.value->GetSelf() + : nullptr, + get_impl_options.columns, timestamp, &s, + &merge_context, &max_covering_tombstone_seq, + read_options, get_impl_options.callback, + get_impl_options.is_blob_index)) { + done = true; + + if (get_impl_options.value) { + get_impl_options.value->PinSelf(); + } + + RecordTick(stats_, MEMTABLE_HIT); + } + } else { + // Get Merge Operands associated with key, Merge Operands should not be + // merged and raw values should be returned to the user. + if (sv->mem->Get(lkey, /*value=*/nullptr, /*columns=*/nullptr, + /*timestamp=*/nullptr, &s, &merge_context, + &max_covering_tombstone_seq, read_options, + false /* immutable_memtable */, nullptr, nullptr, + false)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } else if ((s.ok() || s.IsMergeInProgress()) && + sv->imm->GetMergeOperands(lkey, &s, &merge_context, + &max_covering_tombstone_seq, + read_options)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } + } + if (!done && !s.ok() && !s.IsMergeInProgress()) { + ReturnAndCleanupSuperVersion(cfd, sv); + return s; + } + } + TEST_SYNC_POINT("DBImpl::GetImpl:PostMemTableGet:0"); + TEST_SYNC_POINT("DBImpl::GetImpl:PostMemTableGet:1"); + PinnedIteratorsManager pinned_iters_mgr; + if (!done) { + PERF_TIMER_GUARD(get_from_output_files_time); + sv->current->Get( + read_options, lkey, get_impl_options.value, get_impl_options.columns, + timestamp, &s, &merge_context, &max_covering_tombstone_seq, + &pinned_iters_mgr, + get_impl_options.get_value ? get_impl_options.value_found : nullptr, + nullptr, nullptr, + get_impl_options.get_value ? get_impl_options.callback : nullptr, + get_impl_options.get_value ? get_impl_options.is_blob_index : nullptr, + get_impl_options.get_value); + RecordTick(stats_, MEMTABLE_MISS); + } + + { + PERF_TIMER_GUARD(get_post_process_time); + + RecordTick(stats_, NUMBER_KEYS_READ); + size_t size = 0; + if (s.ok()) { + if (get_impl_options.get_value) { + if (get_impl_options.value) { + size = get_impl_options.value->size(); + } else if (get_impl_options.columns) { + size = get_impl_options.columns->serialized_size(); + } + } else { + // Return all merge operands for get_impl_options.key + *get_impl_options.number_of_operands = + static_cast(merge_context.GetNumOperands()); + if (*get_impl_options.number_of_operands > + get_impl_options.get_merge_operands_options + ->expected_max_number_of_operands) { + s = Status::Incomplete( + Status::SubCode::KMergeOperandsInsufficientCapacity); + } else { + // Each operand depends on one of the following resources: `sv`, + // `pinned_iters_mgr`, or `merge_context`. It would be crazy expensive + // to reference `sv` for each operand relying on it because `sv` is + // (un)ref'd in all threads using the DB. Furthermore, we do not track + // on which resource each operand depends. + // + // To solve this, we bundle the resources in a `GetMergeOperandsState` + // and manage them with a `SharedCleanablePtr` shared among the + // `PinnableSlice`s we return. This bundle includes one `sv` reference + // and ownership of the `merge_context` and `pinned_iters_mgr` + // objects. + bool ref_sv = ShouldReferenceSuperVersion(merge_context); + if (ref_sv) { + assert(!merge_context.GetOperands().empty()); + SharedCleanablePtr shared_cleanable; + GetMergeOperandsState* state = nullptr; + state = new GetMergeOperandsState(); + state->merge_context = std::move(merge_context); + state->pinned_iters_mgr = std::move(pinned_iters_mgr); + + sv->Ref(); + + state->sv_handle = new SuperVersionHandle( + this, &mutex_, sv, + immutable_db_options_.avoid_unnecessary_blocking_io); + + shared_cleanable.Allocate(); + shared_cleanable->RegisterCleanup(CleanupGetMergeOperandsState, + state /* arg1 */, + nullptr /* arg2 */); + for (size_t i = 0; i < state->merge_context.GetOperands().size(); + ++i) { + const Slice& sl = state->merge_context.GetOperands()[i]; + size += sl.size(); + + get_impl_options.merge_operands->PinSlice( + sl, nullptr /* cleanable */); + if (i == state->merge_context.GetOperands().size() - 1) { + shared_cleanable.MoveAsCleanupTo( + get_impl_options.merge_operands); + } else { + shared_cleanable.RegisterCopyWith( + get_impl_options.merge_operands); + } + get_impl_options.merge_operands++; + } + } else { + for (const Slice& sl : merge_context.GetOperands()) { + size += sl.size(); + get_impl_options.merge_operands->PinSelf(sl); + get_impl_options.merge_operands++; + } + } + } + } + RecordTick(stats_, BYTES_READ, size); + PERF_COUNTER_ADD(get_read_bytes, size); + } + + ReturnAndCleanupSuperVersion(cfd, sv); + + RecordInHistogram(stats_, BYTES_PER_READ, size); + } + return s; +} + +std::vector DBImpl::MultiGet( + const ReadOptions& read_options, + const std::vector& column_family, + const std::vector& keys, std::vector* values) { + return MultiGet(read_options, column_family, keys, values, + /*timestamps=*/nullptr); +} + +std::vector DBImpl::MultiGet( + const ReadOptions& read_options, + const std::vector& column_family, + const std::vector& keys, std::vector* values, + std::vector* timestamps) { + PERF_CPU_TIMER_GUARD(get_cpu_nanos, immutable_db_options_.clock); + StopWatch sw(immutable_db_options_.clock, stats_, DB_MULTIGET); + PERF_TIMER_GUARD(get_snapshot_time); + + size_t num_keys = keys.size(); + assert(column_family.size() == num_keys); + std::vector stat_list(num_keys); + + bool should_fail = false; + for (size_t i = 0; i < num_keys; ++i) { + assert(column_family[i]); + if (read_options.timestamp) { + stat_list[i] = FailIfTsMismatchCf( + column_family[i], *(read_options.timestamp), /*ts_for_read=*/true); + if (!stat_list[i].ok()) { + should_fail = true; + } + } else { + stat_list[i] = FailIfCfHasTs(column_family[i]); + if (!stat_list[i].ok()) { + should_fail = true; + } + } + } + + if (should_fail) { + for (auto& s : stat_list) { + if (s.ok()) { + s = Status::Incomplete( + "DB not queried due to invalid argument(s) in the same MultiGet"); + } + } + return stat_list; + } + + if (tracer_) { + // TODO: This mutex should be removed later, to improve performance when + // tracing is enabled. + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + // TODO: maybe handle the tracing status? + tracer_->MultiGet(column_family, keys).PermitUncheckedError(); + } + } + + SequenceNumber consistent_seqnum; + + UnorderedMap multiget_cf_data( + column_family.size()); + for (auto cf : column_family) { + auto cfh = static_cast_with_check(cf); + auto cfd = cfh->cfd(); + if (multiget_cf_data.find(cfd->GetID()) == multiget_cf_data.end()) { + multiget_cf_data.emplace(cfd->GetID(), + MultiGetColumnFamilyData(cfh, nullptr)); + } + } + + std::function::iterator&)> + iter_deref_lambda = + [](UnorderedMap::iterator& + cf_iter) { return &cf_iter->second; }; + + bool unref_only = + MultiCFSnapshot>( + read_options, nullptr, iter_deref_lambda, &multiget_cf_data, + &consistent_seqnum); + + TEST_SYNC_POINT("DBImpl::MultiGet:AfterGetSeqNum1"); + TEST_SYNC_POINT("DBImpl::MultiGet:AfterGetSeqNum2"); + + // Contain a list of merge operations if merge occurs. + MergeContext merge_context; + + // Note: this always resizes the values array + values->resize(num_keys); + if (timestamps) { + timestamps->resize(num_keys); + } + + // Keep track of bytes that we read for statistics-recording later + uint64_t bytes_read = 0; + PERF_TIMER_STOP(get_snapshot_time); + + // For each of the given keys, apply the entire "get" process as follows: + // First look in the memtable, then in the immutable memtable (if any). + // s is both in/out. When in, s could either be OK or MergeInProgress. + // merge_operands will contain the sequence of merges in the latter case. + size_t num_found = 0; + size_t keys_read; + uint64_t curr_value_size = 0; + + GetWithTimestampReadCallback timestamp_read_callback(0); + ReadCallback* read_callback = nullptr; + if (read_options.timestamp && read_options.timestamp->size() > 0) { + timestamp_read_callback.Refresh(consistent_seqnum); + read_callback = ×tamp_read_callback; + } + + for (keys_read = 0; keys_read < num_keys; ++keys_read) { + merge_context.Clear(); + Status& s = stat_list[keys_read]; + std::string* value = &(*values)[keys_read]; + std::string* timestamp = timestamps ? &(*timestamps)[keys_read] : nullptr; + + LookupKey lkey(keys[keys_read], consistent_seqnum, read_options.timestamp); + auto cfh = static_cast_with_check( + column_family[keys_read]); + SequenceNumber max_covering_tombstone_seq = 0; + auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID()); + assert(mgd_iter != multiget_cf_data.end()); + auto mgd = mgd_iter->second; + auto super_version = mgd.super_version; + bool skip_memtable = + (read_options.read_tier == kPersistedTier && + has_unpersisted_data_.load(std::memory_order_relaxed)); + bool done = false; + if (!skip_memtable) { + if (super_version->mem->Get( + lkey, value, /*columns=*/nullptr, timestamp, &s, &merge_context, + &max_covering_tombstone_seq, read_options, + false /* immutable_memtable */, read_callback)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } else if (super_version->imm->Get(lkey, value, /*columns=*/nullptr, + timestamp, &s, &merge_context, + &max_covering_tombstone_seq, + read_options, read_callback)) { + done = true; + RecordTick(stats_, MEMTABLE_HIT); + } + } + if (!done) { + PinnableSlice pinnable_val; + PERF_TIMER_GUARD(get_from_output_files_time); + PinnedIteratorsManager pinned_iters_mgr; + super_version->current->Get(read_options, lkey, &pinnable_val, + /*columns=*/nullptr, timestamp, &s, + &merge_context, &max_covering_tombstone_seq, + &pinned_iters_mgr, /*value_found=*/nullptr, + /*key_exists=*/nullptr, + /*seq=*/nullptr, read_callback); + value->assign(pinnable_val.data(), pinnable_val.size()); + RecordTick(stats_, MEMTABLE_MISS); + } + + if (s.ok()) { + bytes_read += value->size(); + num_found++; + curr_value_size += value->size(); + if (curr_value_size > read_options.value_size_soft_limit) { + while (++keys_read < num_keys) { + stat_list[keys_read] = Status::Aborted(); + } + break; + } + } + if (read_options.deadline.count() && + immutable_db_options_.clock->NowMicros() > + static_cast(read_options.deadline.count())) { + break; + } + } + + if (keys_read < num_keys) { + // The only reason to break out of the loop is when the deadline is + // exceeded + assert(immutable_db_options_.clock->NowMicros() > + static_cast(read_options.deadline.count())); + for (++keys_read; keys_read < num_keys; ++keys_read) { + stat_list[keys_read] = Status::TimedOut(); + } + } + + // Post processing (decrement reference counts and record statistics) + PERF_TIMER_GUARD(get_post_process_time); + autovector superversions_to_delete; + + for (auto mgd_iter : multiget_cf_data) { + auto mgd = mgd_iter.second; + if (!unref_only) { + ReturnAndCleanupSuperVersion(mgd.cfd, mgd.super_version); + } else { + mgd.cfd->GetSuperVersion()->Unref(); + } + } + RecordTick(stats_, NUMBER_MULTIGET_CALLS); + RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys); + RecordTick(stats_, NUMBER_MULTIGET_KEYS_FOUND, num_found); + RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read); + RecordInHistogram(stats_, BYTES_PER_MULTIGET, bytes_read); + PERF_COUNTER_ADD(multiget_read_bytes, bytes_read); + PERF_TIMER_STOP(get_post_process_time); + + return stat_list; +} + +template +bool DBImpl::MultiCFSnapshot( + const ReadOptions& read_options, ReadCallback* callback, + std::function& + iter_deref_func, + T* cf_list, SequenceNumber* snapshot) { + PERF_TIMER_GUARD(get_snapshot_time); + + bool last_try = false; + if (cf_list->size() == 1) { + // Fast path for a single column family. We can simply get the thread loca + // super version + auto cf_iter = cf_list->begin(); + auto node = iter_deref_func(cf_iter); + node->super_version = GetAndRefSuperVersion(node->cfd); + if (read_options.snapshot != nullptr) { + // Note: In WritePrepared txns this is not necessary but not harmful + // either. Because prep_seq > snapshot => commit_seq > snapshot so if + // a snapshot is specified we should be fine with skipping seq numbers + // that are greater than that. + // + // In WriteUnprepared, we cannot set snapshot in the lookup key because we + // may skip uncommitted data that should be visible to the transaction for + // reading own writes. + *snapshot = + static_cast(read_options.snapshot)->number_; + if (callback) { + *snapshot = std::max(*snapshot, callback->max_visible_seq()); + } + } else { + // Since we get and reference the super version before getting + // the snapshot number, without a mutex protection, it is possible + // that a memtable switch happened in the middle and not all the + // data for this snapshot is available. But it will contain all + // the data available in the super version we have, which is also + // a valid snapshot to read from. + // We shouldn't get snapshot before finding and referencing the super + // version because a flush happening in between may compact away data for + // the snapshot, but the snapshot is earlier than the data overwriting it, + // so users may see wrong results. + *snapshot = GetLastPublishedSequence(); + } + } else { + // If we end up with the same issue of memtable geting sealed during 2 + // consecutive retries, it means the write rate is very high. In that case + // its probably ok to take the mutex on the 3rd try so we can succeed for + // sure + constexpr int num_retries = 3; + for (int i = 0; i < num_retries; ++i) { + last_try = (i == num_retries - 1); + bool retry = false; + + if (i > 0) { + for (auto cf_iter = cf_list->begin(); cf_iter != cf_list->end(); + ++cf_iter) { + auto node = iter_deref_func(cf_iter); + SuperVersion* super_version = node->super_version; + ColumnFamilyData* cfd = node->cfd; + if (super_version != nullptr) { + ReturnAndCleanupSuperVersion(cfd, super_version); + } + node->super_version = nullptr; + } + } + if (read_options.snapshot == nullptr) { + if (last_try) { + TEST_SYNC_POINT("DBImpl::MultiGet::LastTry"); + // We're close to max number of retries. For the last retry, + // acquire the lock so we're sure to succeed + mutex_.Lock(); + } + *snapshot = GetLastPublishedSequence(); + } else { + *snapshot = + static_cast_with_check(read_options.snapshot) + ->number_; + } + for (auto cf_iter = cf_list->begin(); cf_iter != cf_list->end(); + ++cf_iter) { + auto node = iter_deref_func(cf_iter); + if (!last_try) { + node->super_version = GetAndRefSuperVersion(node->cfd); + } else { + node->super_version = node->cfd->GetSuperVersion()->Ref(); + } + TEST_SYNC_POINT("DBImpl::MultiGet::AfterRefSV"); + if (read_options.snapshot != nullptr || last_try) { + // If user passed a snapshot, then we don't care if a memtable is + // sealed or compaction happens because the snapshot would ensure + // that older key versions are kept around. If this is the last + // retry, then we have the lock so nothing bad can happen + continue; + } + // We could get the earliest sequence number for the whole list of + // memtables, which will include immutable memtables as well, but that + // might be tricky to maintain in case we decide, in future, to do + // memtable compaction. + if (!last_try) { + SequenceNumber seq = + node->super_version->mem->GetEarliestSequenceNumber(); + if (seq > *snapshot) { + retry = true; + break; + } + } + } + if (!retry) { + if (last_try) { + mutex_.Unlock(); + } + break; + } + } + } + + // Keep track of bytes that we read for statistics-recording later + PERF_TIMER_STOP(get_snapshot_time); + + return last_try; +} + +void DBImpl::MultiGet(const ReadOptions& read_options, const size_t num_keys, + ColumnFamilyHandle** column_families, const Slice* keys, + PinnableSlice* values, Status* statuses, + const bool sorted_input) { + return MultiGet(read_options, num_keys, column_families, keys, values, + /*timestamps=*/nullptr, statuses, sorted_input); +} + +void DBImpl::MultiGet(const ReadOptions& read_options, const size_t num_keys, + ColumnFamilyHandle** column_families, const Slice* keys, + PinnableSlice* values, std::string* timestamps, + Status* statuses, const bool sorted_input) { + if (num_keys == 0) { + return; + } + + bool should_fail = false; + for (size_t i = 0; i < num_keys; ++i) { + ColumnFamilyHandle* cfh = column_families[i]; + assert(cfh); + if (read_options.timestamp) { + statuses[i] = FailIfTsMismatchCf(cfh, *(read_options.timestamp), + /*ts_for_read=*/true); + if (!statuses[i].ok()) { + should_fail = true; + } + } else { + statuses[i] = FailIfCfHasTs(cfh); + if (!statuses[i].ok()) { + should_fail = true; + } + } + } + if (should_fail) { + for (size_t i = 0; i < num_keys; ++i) { + if (statuses[i].ok()) { + statuses[i] = Status::Incomplete( + "DB not queried due to invalid argument(s) in the same MultiGet"); + } + } + return; + } + + if (tracer_) { + // TODO: This mutex should be removed later, to improve performance when + // tracing is enabled. + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + // TODO: maybe handle the tracing status? + tracer_->MultiGet(num_keys, column_families, keys).PermitUncheckedError(); + } + } + + autovector key_context; + autovector sorted_keys; + sorted_keys.resize(num_keys); + for (size_t i = 0; i < num_keys; ++i) { + values[i].Reset(); + key_context.emplace_back(column_families[i], keys[i], &values[i], + timestamps ? ×tamps[i] : nullptr, + &statuses[i]); + } + for (size_t i = 0; i < num_keys; ++i) { + sorted_keys[i] = &key_context[i]; + } + PrepareMultiGetKeys(num_keys, sorted_input, &sorted_keys); + + autovector + multiget_cf_data; + size_t cf_start = 0; + ColumnFamilyHandle* cf = sorted_keys[0]->column_family; + + for (size_t i = 0; i < num_keys; ++i) { + KeyContext* key_ctx = sorted_keys[i]; + if (key_ctx->column_family != cf) { + multiget_cf_data.emplace_back(cf, cf_start, i - cf_start, nullptr); + cf_start = i; + cf = key_ctx->column_family; + } + } + + multiget_cf_data.emplace_back(cf, cf_start, num_keys - cf_start, nullptr); + + std::function::iterator&)> + iter_deref_lambda = + [](autovector::iterator& cf_iter) { + return &(*cf_iter); + }; + + SequenceNumber consistent_seqnum; + bool unref_only = MultiCFSnapshot< + autovector>( + read_options, nullptr, iter_deref_lambda, &multiget_cf_data, + &consistent_seqnum); + + GetWithTimestampReadCallback timestamp_read_callback(0); + ReadCallback* read_callback = nullptr; + if (read_options.timestamp && read_options.timestamp->size() > 0) { + timestamp_read_callback.Refresh(consistent_seqnum); + read_callback = ×tamp_read_callback; + } + + Status s; + auto cf_iter = multiget_cf_data.begin(); + for (; cf_iter != multiget_cf_data.end(); ++cf_iter) { + s = MultiGetImpl(read_options, cf_iter->start, cf_iter->num_keys, + &sorted_keys, cf_iter->super_version, consistent_seqnum, + read_callback); + if (!s.ok()) { + break; + } + } + if (!s.ok()) { + assert(s.IsTimedOut() || s.IsAborted()); + for (++cf_iter; cf_iter != multiget_cf_data.end(); ++cf_iter) { + for (size_t i = cf_iter->start; i < cf_iter->start + cf_iter->num_keys; + ++i) { + *sorted_keys[i]->s = s; + } + } + } + + for (const auto& iter : multiget_cf_data) { + if (!unref_only) { + ReturnAndCleanupSuperVersion(iter.cfd, iter.super_version); + } else { + iter.cfd->GetSuperVersion()->Unref(); + } + } +} + +namespace { +// Order keys by CF ID, followed by key contents +struct CompareKeyContext { + inline bool operator()(const KeyContext* lhs, const KeyContext* rhs) { + ColumnFamilyHandleImpl* cfh = + static_cast(lhs->column_family); + uint32_t cfd_id1 = cfh->cfd()->GetID(); + const Comparator* comparator = cfh->cfd()->user_comparator(); + cfh = static_cast(rhs->column_family); + uint32_t cfd_id2 = cfh->cfd()->GetID(); + + if (cfd_id1 < cfd_id2) { + return true; + } else if (cfd_id1 > cfd_id2) { + return false; + } + + // Both keys are from the same column family + int cmp = comparator->CompareWithoutTimestamp( + *(lhs->key), /*a_has_ts=*/false, *(rhs->key), /*b_has_ts=*/false); + if (cmp < 0) { + return true; + } + return false; + } +}; + +} // anonymous namespace + +void DBImpl::PrepareMultiGetKeys( + size_t num_keys, bool sorted_input, + autovector* sorted_keys) { + if (sorted_input) { +#ifndef NDEBUG + assert(std::is_sorted(sorted_keys->begin(), sorted_keys->end(), + CompareKeyContext())); +#endif + return; + } + + std::sort(sorted_keys->begin(), sorted_keys->begin() + num_keys, + CompareKeyContext()); +} + +void DBImpl::MultiGet(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const size_t num_keys, + const Slice* keys, PinnableSlice* values, + Status* statuses, const bool sorted_input) { + return MultiGet(read_options, column_family, num_keys, keys, values, + /*timestamp=*/nullptr, statuses, sorted_input); +} + +void DBImpl::MultiGet(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const size_t num_keys, + const Slice* keys, PinnableSlice* values, + std::string* timestamps, Status* statuses, + const bool sorted_input) { + if (tracer_) { + // TODO: This mutex should be removed later, to improve performance when + // tracing is enabled. + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + // TODO: maybe handle the tracing status? + tracer_->MultiGet(num_keys, column_family, keys).PermitUncheckedError(); + } + } + autovector key_context; + autovector sorted_keys; + sorted_keys.resize(num_keys); + for (size_t i = 0; i < num_keys; ++i) { + values[i].Reset(); + key_context.emplace_back(column_family, keys[i], &values[i], + timestamps ? ×tamps[i] : nullptr, + &statuses[i]); + } + for (size_t i = 0; i < num_keys; ++i) { + sorted_keys[i] = &key_context[i]; + } + PrepareMultiGetKeys(num_keys, sorted_input, &sorted_keys); + MultiGetWithCallback(read_options, column_family, nullptr, &sorted_keys); +} + +void DBImpl::MultiGetWithCallback( + const ReadOptions& read_options, ColumnFamilyHandle* column_family, + ReadCallback* callback, + autovector* sorted_keys) { + std::array multiget_cf_data; + multiget_cf_data[0] = MultiGetColumnFamilyData(column_family, nullptr); + std::function::iterator&)> + iter_deref_lambda = + [](std::array::iterator& cf_iter) { + return &(*cf_iter); + }; + + size_t num_keys = sorted_keys->size(); + SequenceNumber consistent_seqnum; + bool unref_only = MultiCFSnapshot>( + read_options, callback, iter_deref_lambda, &multiget_cf_data, + &consistent_seqnum); +#ifndef NDEBUG + assert(!unref_only); +#else + // Silence unused variable warning + (void)unref_only; +#endif // NDEBUG + + if (callback && read_options.snapshot == nullptr) { + // The unprep_seqs are not published for write unprepared, so it could be + // that max_visible_seq is larger. Seek to the std::max of the two. + // However, we still want our callback to contain the actual snapshot so + // that it can do the correct visibility filtering. + callback->Refresh(consistent_seqnum); + + // Internally, WriteUnpreparedTxnReadCallback::Refresh would set + // max_visible_seq = max(max_visible_seq, snapshot) + // + // Currently, the commented out assert is broken by + // InvalidSnapshotReadCallback, but if write unprepared recovery followed + // the regular transaction flow, then this special read callback would not + // be needed. + // + // assert(callback->max_visible_seq() >= snapshot); + consistent_seqnum = callback->max_visible_seq(); + } + + GetWithTimestampReadCallback timestamp_read_callback(0); + ReadCallback* read_callback = callback; + if (read_options.timestamp && read_options.timestamp->size() > 0) { + assert(!read_callback); // timestamp with callback is not supported + timestamp_read_callback.Refresh(consistent_seqnum); + read_callback = ×tamp_read_callback; + } + + Status s = MultiGetImpl(read_options, 0, num_keys, sorted_keys, + multiget_cf_data[0].super_version, consistent_seqnum, + read_callback); + assert(s.ok() || s.IsTimedOut() || s.IsAborted()); + ReturnAndCleanupSuperVersion(multiget_cf_data[0].cfd, + multiget_cf_data[0].super_version); +} + +// The actual implementation of batched MultiGet. Parameters - +// start_key - Index in the sorted_keys vector to start processing from +// num_keys - Number of keys to lookup, starting with sorted_keys[start_key] +// sorted_keys - The entire batch of sorted keys for this CF +// +// The per key status is returned in the KeyContext structures pointed to by +// sorted_keys. An overall Status is also returned, with the only possible +// values being Status::OK() and Status::TimedOut(). The latter indicates +// that the call exceeded read_options.deadline +Status DBImpl::MultiGetImpl( + const ReadOptions& read_options, size_t start_key, size_t num_keys, + autovector* sorted_keys, + SuperVersion* super_version, SequenceNumber snapshot, + ReadCallback* callback) { + PERF_CPU_TIMER_GUARD(get_cpu_nanos, immutable_db_options_.clock); + StopWatch sw(immutable_db_options_.clock, stats_, DB_MULTIGET); + + assert(sorted_keys); + // Clear the timestamps for returning results so that we can distinguish + // between tombstone or key that has never been written + for (auto* kctx : *sorted_keys) { + assert(kctx); + if (kctx->timestamp) { + kctx->timestamp->clear(); + } + } + + // For each of the given keys, apply the entire "get" process as follows: + // First look in the memtable, then in the immutable memtable (if any). + // s is both in/out. When in, s could either be OK or MergeInProgress. + // merge_operands will contain the sequence of merges in the latter case. + size_t keys_left = num_keys; + Status s; + uint64_t curr_value_size = 0; + while (keys_left) { + if (read_options.deadline.count() && + immutable_db_options_.clock->NowMicros() > + static_cast(read_options.deadline.count())) { + s = Status::TimedOut(); + break; + } + + size_t batch_size = (keys_left > MultiGetContext::MAX_BATCH_SIZE) + ? MultiGetContext::MAX_BATCH_SIZE + : keys_left; + MultiGetContext ctx(sorted_keys, start_key + num_keys - keys_left, + batch_size, snapshot, read_options, GetFileSystem(), + stats_); + MultiGetRange range = ctx.GetMultiGetRange(); + range.AddValueSize(curr_value_size); + bool lookup_current = false; + + keys_left -= batch_size; + for (auto mget_iter = range.begin(); mget_iter != range.end(); + ++mget_iter) { + mget_iter->merge_context.Clear(); + *mget_iter->s = Status::OK(); + } + + bool skip_memtable = + (read_options.read_tier == kPersistedTier && + has_unpersisted_data_.load(std::memory_order_relaxed)); + if (!skip_memtable) { + super_version->mem->MultiGet(read_options, &range, callback, + false /* immutable_memtable */); + if (!range.empty()) { + super_version->imm->MultiGet(read_options, &range, callback); + } + if (!range.empty()) { + lookup_current = true; + uint64_t left = range.KeysLeft(); + RecordTick(stats_, MEMTABLE_MISS, left); + } + } + if (lookup_current) { + PERF_TIMER_GUARD(get_from_output_files_time); + super_version->current->MultiGet(read_options, &range, callback); + } + curr_value_size = range.GetValueSize(); + if (curr_value_size > read_options.value_size_soft_limit) { + s = Status::Aborted(); + break; + } + } + + // Post processing (decrement reference counts and record statistics) + PERF_TIMER_GUARD(get_post_process_time); + size_t num_found = 0; + uint64_t bytes_read = 0; + for (size_t i = start_key; i < start_key + num_keys - keys_left; ++i) { + KeyContext* key = (*sorted_keys)[i]; + if (key->s->ok()) { + bytes_read += key->value->size(); + num_found++; + } + } + if (keys_left) { + assert(s.IsTimedOut() || s.IsAborted()); + for (size_t i = start_key + num_keys - keys_left; i < start_key + num_keys; + ++i) { + KeyContext* key = (*sorted_keys)[i]; + *key->s = s; + } + } + + RecordTick(stats_, NUMBER_MULTIGET_CALLS); + RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys); + RecordTick(stats_, NUMBER_MULTIGET_KEYS_FOUND, num_found); + RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read); + RecordInHistogram(stats_, BYTES_PER_MULTIGET, bytes_read); + PERF_COUNTER_ADD(multiget_read_bytes, bytes_read); + PERF_TIMER_STOP(get_post_process_time); + + return s; +} + +Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options, + const std::string& column_family, + ColumnFamilyHandle** handle) { + assert(handle != nullptr); + Status s = CreateColumnFamilyImpl(cf_options, column_family, handle); + if (s.ok()) { + s = WriteOptionsFile(true /*need_mutex_lock*/, + true /*need_enter_write_thread*/); + } + return s; +} + +Status DBImpl::CreateColumnFamilies( + const ColumnFamilyOptions& cf_options, + const std::vector& column_family_names, + std::vector* handles) { + assert(handles != nullptr); + handles->clear(); + size_t num_cf = column_family_names.size(); + Status s; + bool success_once = false; + for (size_t i = 0; i < num_cf; i++) { + ColumnFamilyHandle* handle; + s = CreateColumnFamilyImpl(cf_options, column_family_names[i], &handle); + if (!s.ok()) { + break; + } + handles->push_back(handle); + success_once = true; + } + if (success_once) { + Status persist_options_status = WriteOptionsFile( + true /*need_mutex_lock*/, true /*need_enter_write_thread*/); + if (s.ok() && !persist_options_status.ok()) { + s = persist_options_status; + } + } + return s; +} + +Status DBImpl::CreateColumnFamilies( + const std::vector& column_families, + std::vector* handles) { + assert(handles != nullptr); + handles->clear(); + size_t num_cf = column_families.size(); + Status s; + bool success_once = false; + for (size_t i = 0; i < num_cf; i++) { + ColumnFamilyHandle* handle; + s = CreateColumnFamilyImpl(column_families[i].options, + column_families[i].name, &handle); + if (!s.ok()) { + break; + } + handles->push_back(handle); + success_once = true; + } + if (success_once) { + Status persist_options_status = WriteOptionsFile( + true /*need_mutex_lock*/, true /*need_enter_write_thread*/); + if (s.ok() && !persist_options_status.ok()) { + s = persist_options_status; + } + } + return s; +} + +Status DBImpl::CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options, + const std::string& column_family_name, + ColumnFamilyHandle** handle) { + Status s; + *handle = nullptr; + + DBOptions db_options = + BuildDBOptions(immutable_db_options_, mutable_db_options_); + s = ColumnFamilyData::ValidateOptions(db_options, cf_options); + if (s.ok()) { + for (auto& cf_path : cf_options.cf_paths) { + s = env_->CreateDirIfMissing(cf_path.path); + if (!s.ok()) { + break; + } + } + } + if (!s.ok()) { + return s; + } + + SuperVersionContext sv_context(/* create_superversion */ true); + { + InstrumentedMutexLock l(&mutex_); + + if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) != + nullptr) { + return Status::InvalidArgument("Column family already exists"); + } + VersionEdit edit; + edit.AddColumnFamily(column_family_name); + uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID(); + edit.SetColumnFamily(new_id); + edit.SetLogNumber(logfile_number_); + edit.SetComparatorName(cf_options.comparator->Name()); + + // LogAndApply will both write the creation in MANIFEST and create + // ColumnFamilyData object + { // write thread + WriteThread::Writer w; + write_thread_.EnterUnbatched(&w, &mutex_); + // LogAndApply will both write the creation in MANIFEST and create + // ColumnFamilyData object + s = versions_->LogAndApply(nullptr, MutableCFOptions(cf_options), &edit, + &mutex_, directories_.GetDbDir(), false, + &cf_options); + write_thread_.ExitUnbatched(&w); + } + if (s.ok()) { + auto* cfd = + versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name); + assert(cfd != nullptr); + std::map> dummy_created_dirs; + s = cfd->AddDirectories(&dummy_created_dirs); + } + if (s.ok()) { + auto* cfd = + versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name); + assert(cfd != nullptr); + InstallSuperVersionAndScheduleWork(cfd, &sv_context, + *cfd->GetLatestMutableCFOptions()); + + if (!cfd->mem()->IsSnapshotSupported()) { + is_snapshot_supported_ = false; + } + + cfd->set_initialized(); + + *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Created column family [%s] (ID %u)", + column_family_name.c_str(), (unsigned)cfd->GetID()); + } else { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Creating column family [%s] FAILED -- %s", + column_family_name.c_str(), s.ToString().c_str()); + } + } // InstrumentedMutexLock l(&mutex_) + + if (cf_options.preserve_internal_time_seconds > 0 || + cf_options.preclude_last_level_data_seconds > 0) { + s = RegisterRecordSeqnoTimeWorker(); + } + sv_context.Clean(); + // this is outside the mutex + if (s.ok()) { + NewThreadStatusCfInfo( + static_cast_with_check(*handle)->cfd()); + } + return s; +} + +Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) { + assert(column_family != nullptr); + Status s = DropColumnFamilyImpl(column_family); + if (s.ok()) { + s = WriteOptionsFile(true /*need_mutex_lock*/, + true /*need_enter_write_thread*/); + } + return s; +} + +Status DBImpl::DropColumnFamilies( + const std::vector& column_families) { + Status s; + bool success_once = false; + for (auto* handle : column_families) { + s = DropColumnFamilyImpl(handle); + if (!s.ok()) { + break; + } + success_once = true; + } + if (success_once) { + Status persist_options_status = WriteOptionsFile( + true /*need_mutex_lock*/, true /*need_enter_write_thread*/); + if (s.ok() && !persist_options_status.ok()) { + s = persist_options_status; + } + } + return s; +} + +Status DBImpl::DropColumnFamilyImpl(ColumnFamilyHandle* column_family) { + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + if (cfd->GetID() == 0) { + return Status::InvalidArgument("Can't drop default column family"); + } + + bool cf_support_snapshot = cfd->mem()->IsSnapshotSupported(); + + VersionEdit edit; + edit.DropColumnFamily(); + edit.SetColumnFamily(cfd->GetID()); + + Status s; + { + InstrumentedMutexLock l(&mutex_); + if (cfd->IsDropped()) { + s = Status::InvalidArgument("Column family already dropped!\n"); + } + if (s.ok()) { + // we drop column family from a single write thread + WriteThread::Writer w; + write_thread_.EnterUnbatched(&w, &mutex_); + s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), &edit, + &mutex_, directories_.GetDbDir()); + write_thread_.ExitUnbatched(&w); + } + if (s.ok()) { + 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 (!cf_support_snapshot) { + // Dropped Column Family doesn't support snapshot. Need to recalculate + // is_snapshot_supported_. + bool new_is_snapshot_supported = true; + for (auto c : *versions_->GetColumnFamilySet()) { + if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) { + new_is_snapshot_supported = false; + break; + } + } + is_snapshot_supported_ = new_is_snapshot_supported; + } + bg_cv_.SignalAll(); + } + + if (cfd->ioptions()->preserve_internal_time_seconds > 0 || + cfd->ioptions()->preclude_last_level_data_seconds > 0) { + s = RegisterRecordSeqnoTimeWorker(); + } + + if (s.ok()) { + // Note that here we erase the associated cf_info of the to-be-dropped + // cfd before its ref-count goes to zero to avoid having to erase cf_info + // later inside db_mutex. + EraseThreadStatusCfInfo(cfd); + assert(cfd->IsDropped()); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Dropped column family with id %u\n", cfd->GetID()); + } else { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Dropping column family with id %u FAILED -- %s\n", + cfd->GetID(), s.ToString().c_str()); + } + + return s; +} + +bool DBImpl::KeyMayExist(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + std::string* value, std::string* timestamp, + bool* value_found) { + assert(value != nullptr); + if (value_found != nullptr) { + // falsify later if key-may-exist but can't fetch value + *value_found = true; + } + ReadOptions roptions = read_options; + roptions.read_tier = kBlockCacheTier; // read from block cache only + PinnableSlice pinnable_val; + GetImplOptions get_impl_options; + get_impl_options.column_family = column_family; + get_impl_options.value = &pinnable_val; + get_impl_options.value_found = value_found; + get_impl_options.timestamp = timestamp; + auto s = GetImpl(roptions, key, get_impl_options); + value->assign(pinnable_val.data(), pinnable_val.size()); + + // If block_cache is enabled and the index block of the table didn't + // not present in block_cache, the return value will be Status::Incomplete. + // In this case, key may still exist in the table. + return s.ok() || s.IsIncomplete(); +} + +Iterator* DBImpl::NewIterator(const ReadOptions& read_options, + ColumnFamilyHandle* column_family) { + if (read_options.managed) { + return NewErrorIterator( + Status::NotSupported("Managed iterator is not supported anymore.")); + } + Iterator* result = nullptr; + if (read_options.read_tier == kPersistedTier) { + return NewErrorIterator(Status::NotSupported( + "ReadTier::kPersistedData is not yet supported in iterators.")); + } + + assert(column_family); + + if (read_options.timestamp) { + const Status s = FailIfTsMismatchCf( + column_family, *(read_options.timestamp), /*ts_for_read=*/true); + if (!s.ok()) { + return NewErrorIterator(s); + } + } else { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return NewErrorIterator(s); + } + } + + auto cfh = static_cast_with_check(column_family); + ColumnFamilyData* cfd = cfh->cfd(); + assert(cfd != nullptr); + ReadCallback* read_callback = nullptr; // No read callback provided. + if (read_options.tailing) { +#ifdef ROCKSDB_LITE + // not supported in lite version + result = nullptr; + +#else + SuperVersion* sv = cfd->GetReferencedSuperVersion(this); + auto iter = new ForwardIterator(this, read_options, cfd, sv, + /* allow_unprepared_value */ true); + result = NewDBIterator( + 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, read_callback, + this, cfd); +#endif + } else { + // Note: no need to consider the special case of + // last_seq_same_as_publish_seq_==false since NewIterator is overridden in + // WritePreparedTxnDB + result = NewIteratorImpl(read_options, cfd, + (read_options.snapshot != nullptr) + ? read_options.snapshot->GetSequenceNumber() + : kMaxSequenceNumber, + read_callback); + } + return result; +} + +ArenaWrappedDBIter* DBImpl::NewIteratorImpl(const ReadOptions& read_options, + ColumnFamilyData* cfd, + SequenceNumber snapshot, + ReadCallback* read_callback, + bool expose_blob_index, + bool allow_refresh) { + SuperVersion* sv = cfd->GetReferencedSuperVersion(this); + + TEST_SYNC_POINT("DBImpl::NewIterator:1"); + TEST_SYNC_POINT("DBImpl::NewIterator:2"); + + if (snapshot == kMaxSequenceNumber) { + // Note that the snapshot is assigned AFTER referencing the super + // version because otherwise a flush happening in between may compact away + // data for the snapshot, so the reader would see neither data that was be + // visible to the snapshot before compaction nor the newer data inserted + // afterwards. + // Note that the super version might not contain all the data available + // to this snapshot, but in that case it can see all the data in the + // super version, which is a valid consistent state after the user + // calls NewIterator(). + snapshot = versions_->LastSequence(); + TEST_SYNC_POINT("DBImpl::NewIterator:3"); + TEST_SYNC_POINT("DBImpl::NewIterator:4"); + } + + // Try to generate a DB iterator tree in continuous memory area to be + // cache friendly. Here is an example of result: + // +-------------------------------+ + // | | + // | ArenaWrappedDBIter | + // | + | + // | +---> Inner Iterator ------------+ + // | | | | + // | | +-- -- -- -- -- -- -- --+ | + // | +--- | Arena | | + // | | | | + // | Allocated Memory: | | + // | | +-------------------+ | + // | | | DBIter | <---+ + // | | + | + // | | | +-> iter_ ------------+ + // | | | | | + // | | +-------------------+ | + // | | | MergingIterator | <---+ + // | | + | + // | | | +->child iter1 ------------+ + // | | | | | | + // | | +->child iter2 ----------+ | + // | | | | | | | + // | | | +->child iter3 --------+ | | + // | | | | | | + // | | +-------------------+ | | | + // | | | Iterator1 | <--------+ + // | | +-------------------+ | | + // | | | Iterator2 | <------+ + // | | +-------------------+ | + // | | | Iterator3 | <----+ + // | | +-------------------+ + // | | | + // +-------+-----------------------+ + // + // ArenaWrappedDBIter inlines an arena area where all the iterators in + // the iterator tree are allocated in the order of being accessed when + // querying. + // Laying out the iterators in the order of being accessed makes it more + // 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, *cfd->ioptions(), sv->mutable_cf_options, sv->current, + snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations, + sv->version_number, read_callback, this, cfd, expose_blob_index, + read_options.snapshot != nullptr ? false : allow_refresh); + + InternalIterator* internal_iter = NewInternalIterator( + db_iter->GetReadOptions(), cfd, sv, db_iter->GetArena(), snapshot, + /* allow_unprepared_value */ true, db_iter); + db_iter->SetIterUnderDBIter(internal_iter); + + return db_iter; +} + +Status DBImpl::NewIterators( + const ReadOptions& read_options, + const std::vector& column_families, + std::vector* iterators) { + if (read_options.managed) { + return Status::NotSupported("Managed iterator is not supported anymore."); + } + if (read_options.read_tier == kPersistedTier) { + return Status::NotSupported( + "ReadTier::kPersistedData is not yet supported in iterators."); + } + + if (read_options.timestamp) { + for (auto* cf : column_families) { + assert(cf); + const Status s = FailIfTsMismatchCf(cf, *(read_options.timestamp), + /*ts_for_read=*/true); + if (!s.ok()) { + return s; + } + } + } else { + for (auto* cf : column_families) { + assert(cf); + const Status s = FailIfCfHasTs(cf); + if (!s.ok()) { + return s; + } + } + } + + ReadCallback* read_callback = nullptr; // No read callback provided. + iterators->clear(); + iterators->reserve(column_families.size()); + if (read_options.tailing) { +#ifdef ROCKSDB_LITE + return Status::InvalidArgument( + "Tailing iterator not supported in RocksDB lite"); +#else + for (auto cfh : column_families) { + auto cfd = static_cast_with_check(cfh)->cfd(); + SuperVersion* sv = cfd->GetReferencedSuperVersion(this); + auto iter = new ForwardIterator(this, read_options, cfd, sv, + /* allow_unprepared_value */ true); + iterators->push_back(NewDBIterator( + 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, + read_callback, this, cfd)); + } +#endif + } else { + // Note: no need to consider the special case of + // last_seq_same_as_publish_seq_==false since NewIterators is overridden in + // WritePreparedTxnDB + auto snapshot = read_options.snapshot != nullptr + ? read_options.snapshot->GetSequenceNumber() + : versions_->LastSequence(); + for (size_t i = 0; i < column_families.size(); ++i) { + auto* cfd = + static_cast_with_check(column_families[i]) + ->cfd(); + iterators->push_back( + NewIteratorImpl(read_options, cfd, snapshot, read_callback)); + } + } + + return Status::OK(); +} + +const Snapshot* DBImpl::GetSnapshot() { return GetSnapshotImpl(false); } + +#ifndef ROCKSDB_LITE +const Snapshot* DBImpl::GetSnapshotForWriteConflictBoundary() { + return GetSnapshotImpl(true); +} +#endif // ROCKSDB_LITE + +std::pair> +DBImpl::CreateTimestampedSnapshot(SequenceNumber snapshot_seq, uint64_t ts) { + assert(ts != std::numeric_limits::max()); + + auto ret = CreateTimestampedSnapshotImpl(snapshot_seq, ts, /*lock=*/true); + return ret; +} + +std::shared_ptr DBImpl::GetTimestampedSnapshot( + uint64_t ts) const { + InstrumentedMutexLock lock_guard(&mutex_); + return timestamped_snapshots_.GetSnapshot(ts); +} + +void DBImpl::ReleaseTimestampedSnapshotsOlderThan(uint64_t ts, + size_t* remaining_total_ss) { + autovector> snapshots_to_release; + { + InstrumentedMutexLock lock_guard(&mutex_); + timestamped_snapshots_.ReleaseSnapshotsOlderThan(ts, snapshots_to_release); + } + snapshots_to_release.clear(); + + if (remaining_total_ss) { + InstrumentedMutexLock lock_guard(&mutex_); + *remaining_total_ss = static_cast(snapshots_.count()); + } +} + +Status DBImpl::GetTimestampedSnapshots( + uint64_t ts_lb, uint64_t ts_ub, + std::vector>& timestamped_snapshots) const { + if (ts_lb >= ts_ub) { + return Status::InvalidArgument( + "timestamp lower bound must be smaller than upper bound"); + } + timestamped_snapshots.clear(); + InstrumentedMutexLock lock_guard(&mutex_); + timestamped_snapshots_.GetSnapshots(ts_lb, ts_ub, timestamped_snapshots); + return Status::OK(); +} + +SnapshotImpl* DBImpl::GetSnapshotImpl(bool is_write_conflict_boundary, + bool lock) { + int64_t unix_time = 0; + immutable_db_options_.clock->GetCurrentTime(&unix_time) + .PermitUncheckedError(); // Ignore error + SnapshotImpl* s = new SnapshotImpl; + + if (lock) { + mutex_.Lock(); + } else { + mutex_.AssertHeld(); + } + // returns null if the underlying memtable does not support snapshot. + if (!is_snapshot_supported_) { + if (lock) { + mutex_.Unlock(); + } + delete s; + return nullptr; + } + auto snapshot_seq = GetLastPublishedSequence(); + SnapshotImpl* snapshot = + snapshots_.New(s, snapshot_seq, unix_time, is_write_conflict_boundary); + if (lock) { + mutex_.Unlock(); + } + return snapshot; +} + +std::pair> +DBImpl::CreateTimestampedSnapshotImpl(SequenceNumber snapshot_seq, uint64_t ts, + bool lock) { + int64_t unix_time = 0; + immutable_db_options_.clock->GetCurrentTime(&unix_time) + .PermitUncheckedError(); // Ignore error + SnapshotImpl* s = new SnapshotImpl; + + const bool need_update_seq = (snapshot_seq != kMaxSequenceNumber); + + if (lock) { + mutex_.Lock(); + } else { + mutex_.AssertHeld(); + } + // returns null if the underlying memtable does not support snapshot. + if (!is_snapshot_supported_) { + if (lock) { + mutex_.Unlock(); + } + delete s; + return std::make_pair( + Status::NotSupported("Memtable does not support snapshot"), nullptr); + } + + // Caller is not write thread, thus didn't provide a valid snapshot_seq. + // Obtain seq from db. + if (!need_update_seq) { + snapshot_seq = GetLastPublishedSequence(); + } + + std::shared_ptr latest = + timestamped_snapshots_.GetSnapshot(std::numeric_limits::max()); + + // If there is already a latest timestamped snapshot, then we need to do some + // checks. + if (latest) { + uint64_t latest_snap_ts = latest->GetTimestamp(); + SequenceNumber latest_snap_seq = latest->GetSequenceNumber(); + assert(latest_snap_seq <= snapshot_seq); + bool needs_create_snap = true; + Status status; + std::shared_ptr ret; + if (latest_snap_ts > ts) { + // A snapshot created later cannot have smaller timestamp than a previous + // timestamped snapshot. + needs_create_snap = false; + std::ostringstream oss; + oss << "snapshot exists with larger timestamp " << latest_snap_ts << " > " + << ts; + status = Status::InvalidArgument(oss.str()); + } else if (latest_snap_ts == ts) { + if (latest_snap_seq == snapshot_seq) { + // We are requesting the same sequence number and timestamp, thus can + // safely reuse (share) the current latest timestamped snapshot. + needs_create_snap = false; + ret = latest; + } else if (latest_snap_seq < snapshot_seq) { + // There may have been writes to the database since the latest + // timestamped snapshot, yet we are still requesting the same + // timestamp. In this case, we cannot create the new timestamped + // snapshot. + needs_create_snap = false; + std::ostringstream oss; + oss << "Allocated seq is " << snapshot_seq + << ", while snapshot exists with smaller seq " << latest_snap_seq + << " but same timestamp " << ts; + status = Status::InvalidArgument(oss.str()); + } + } + if (!needs_create_snap) { + if (lock) { + mutex_.Unlock(); + } + delete s; + return std::make_pair(status, ret); + } else { + status.PermitUncheckedError(); + } + } + + SnapshotImpl* snapshot = + snapshots_.New(s, snapshot_seq, unix_time, + /*is_write_conflict_boundary=*/true, ts); + + std::shared_ptr ret( + snapshot, + std::bind(&DBImpl::ReleaseSnapshot, this, std::placeholders::_1)); + timestamped_snapshots_.AddSnapshot(ret); + + // Caller is from write thread, and we need to update database's sequence + // number. + if (need_update_seq) { + assert(versions_); + if (last_seq_same_as_publish_seq_) { + versions_->SetLastSequence(snapshot_seq); + } else { + // TODO: support write-prepared/write-unprepared transactions with two + // write queues. + assert(false); + } + } + + if (lock) { + mutex_.Unlock(); + } + return std::make_pair(Status::OK(), ret); +} + +namespace { +using CfdList = autovector; +bool CfdListContains(const CfdList& list, ColumnFamilyData* cfd) { + for (const ColumnFamilyData* t : list) { + if (t == cfd) { + return true; + } + } + return false; +} +} // namespace + +void DBImpl::ReleaseSnapshot(const Snapshot* s) { + if (s == nullptr) { + // DBImpl::GetSnapshot() can return nullptr when snapshot + // not supported by specifying the condition: + // inplace_update_support enabled. + return; + } + const SnapshotImpl* casted_s = reinterpret_cast(s); + { + InstrumentedMutexLock l(&mutex_); + snapshots_.Delete(casted_s); + uint64_t oldest_snapshot; + if (snapshots_.empty()) { + oldest_snapshot = GetLastPublishedSequence(); + } else { + oldest_snapshot = snapshots_.oldest()->number_; + } + // Avoid to go through every column family by checking a global threshold + // first. + if (oldest_snapshot > bottommost_files_mark_threshold_) { + CfdList cf_scheduled; + for (auto* cfd : *versions_->GetColumnFamilySet()) { + if (!cfd->ioptions()->allow_ingest_behind) { + cfd->current()->storage_info()->UpdateOldestSnapshot(oldest_snapshot); + if (!cfd->current() + ->storage_info() + ->BottommostFilesMarkedForCompaction() + .empty()) { + SchedulePendingCompaction(cfd); + MaybeScheduleFlushOrCompaction(); + cf_scheduled.push_back(cfd); + } + } + } + + // Calculate a new threshold, skipping those CFs where compactions are + // scheduled. We do not do the same pass as the previous loop because + // mutex might be unlocked during the loop, making the result inaccurate. + SequenceNumber new_bottommost_files_mark_threshold = kMaxSequenceNumber; + for (auto* cfd : *versions_->GetColumnFamilySet()) { + if (CfdListContains(cf_scheduled, cfd) || + cfd->ioptions()->allow_ingest_behind) { + continue; + } + new_bottommost_files_mark_threshold = std::min( + new_bottommost_files_mark_threshold, + cfd->current()->storage_info()->bottommost_files_mark_threshold()); + } + bottommost_files_mark_threshold_ = new_bottommost_files_mark_threshold; + } + } + delete casted_s; +} + +#ifndef ROCKSDB_LITE +Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family, + TablePropertiesCollection* props) { + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + + // Increment the ref count + mutex_.Lock(); + auto version = cfd->current(); + version->Ref(); + mutex_.Unlock(); + + auto s = version->GetPropertiesOfAllTables(props); + + // Decrement the ref count + mutex_.Lock(); + version->Unref(); + mutex_.Unlock(); + + return s; +} + +Status DBImpl::GetPropertiesOfTablesInRange(ColumnFamilyHandle* column_family, + const Range* range, std::size_t n, + TablePropertiesCollection* props) { + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + + // Increment the ref count + mutex_.Lock(); + auto version = cfd->current(); + version->Ref(); + mutex_.Unlock(); + + auto s = version->GetPropertiesOfTablesInRange(range, n, props); + + // Decrement the ref count + mutex_.Lock(); + version->Unref(); + mutex_.Unlock(); + + return s; +} + +#endif // ROCKSDB_LITE + +const std::string& DBImpl::GetName() const { return dbname_; } + +Env* DBImpl::GetEnv() const { return env_; } + +FileSystem* DB::GetFileSystem() const { + const auto& fs = GetEnv()->GetFileSystem(); + return fs.get(); +} + +FileSystem* DBImpl::GetFileSystem() const { + return immutable_db_options_.fs.get(); +} + +SystemClock* DBImpl::GetSystemClock() const { + return immutable_db_options_.clock; +} + +#ifndef ROCKSDB_LITE + +Status DBImpl::StartIOTrace(const TraceOptions& trace_options, + std::unique_ptr&& trace_writer) { + assert(trace_writer != nullptr); + return io_tracer_->StartIOTrace(GetSystemClock(), trace_options, + std::move(trace_writer)); +} + +Status DBImpl::EndIOTrace() { + io_tracer_->EndIOTrace(); + return Status::OK(); +} + +#endif // ROCKSDB_LITE + +Options DBImpl::GetOptions(ColumnFamilyHandle* column_family) const { + InstrumentedMutexLock l(&mutex_); + auto cfh = static_cast_with_check(column_family); + return Options(BuildDBOptions(immutable_db_options_, mutable_db_options_), + cfh->cfd()->GetLatestCFOptions()); +} + +DBOptions DBImpl::GetDBOptions() const { + InstrumentedMutexLock l(&mutex_); + return BuildDBOptions(immutable_db_options_, mutable_db_options_); +} + +bool DBImpl::GetProperty(ColumnFamilyHandle* column_family, + const Slice& property, std::string* value) { + const DBPropertyInfo* property_info = GetPropertyInfo(property); + value->clear(); + auto cfd = + static_cast_with_check(column_family)->cfd(); + if (property_info == nullptr) { + return false; + } else if (property_info->handle_int) { + uint64_t int_value; + bool ret_value = + GetIntPropertyInternal(cfd, *property_info, false, &int_value); + if (ret_value) { + *value = std::to_string(int_value); + } + return ret_value; + } else if (property_info->handle_string) { + if (property_info->need_out_of_mutex) { + return cfd->internal_stats()->GetStringProperty(*property_info, property, + value); + } else { + InstrumentedMutexLock l(&mutex_); + return cfd->internal_stats()->GetStringProperty(*property_info, property, + value); + } + } else if (property_info->handle_string_dbimpl) { + if (property_info->need_out_of_mutex) { + return (this->*(property_info->handle_string_dbimpl))(value); + } else { + InstrumentedMutexLock l(&mutex_); + return (this->*(property_info->handle_string_dbimpl))(value); + } + } + // Shouldn't reach here since exactly one of handle_string and handle_int + // should be non-nullptr. + assert(false); + return false; +} + +bool DBImpl::GetMapProperty(ColumnFamilyHandle* column_family, + const Slice& property, + std::map* value) { + const DBPropertyInfo* property_info = GetPropertyInfo(property); + value->clear(); + auto cfd = + static_cast_with_check(column_family)->cfd(); + if (property_info == nullptr) { + return false; + } else if (property_info->handle_map) { + if (property_info->need_out_of_mutex) { + return cfd->internal_stats()->GetMapProperty(*property_info, property, + value); + } else { + InstrumentedMutexLock l(&mutex_); + return cfd->internal_stats()->GetMapProperty(*property_info, property, + value); + } + } + // If we reach this point it means that handle_map is not provided for the + // requested property + return false; +} + +bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family, + const Slice& property, uint64_t* value) { + const DBPropertyInfo* property_info = GetPropertyInfo(property); + if (property_info == nullptr || property_info->handle_int == nullptr) { + return false; + } + auto cfd = + static_cast_with_check(column_family)->cfd(); + return GetIntPropertyInternal(cfd, *property_info, false, value); +} + +bool DBImpl::GetIntPropertyInternal(ColumnFamilyData* cfd, + const DBPropertyInfo& property_info, + bool is_locked, uint64_t* value) { + assert(property_info.handle_int != nullptr); + if (!property_info.need_out_of_mutex) { + if (is_locked) { + mutex_.AssertHeld(); + return cfd->internal_stats()->GetIntProperty(property_info, value, this); + } else { + InstrumentedMutexLock l(&mutex_); + return cfd->internal_stats()->GetIntProperty(property_info, value, this); + } + } else { + SuperVersion* sv = nullptr; + if (is_locked) { + mutex_.Unlock(); + } + sv = GetAndRefSuperVersion(cfd); + + bool ret = cfd->internal_stats()->GetIntPropertyOutOfMutex( + property_info, sv->current, value); + + ReturnAndCleanupSuperVersion(cfd, sv); + if (is_locked) { + mutex_.Lock(); + } + + return ret; + } +} + +bool DBImpl::GetPropertyHandleOptionsStatistics(std::string* value) { + assert(value != nullptr); + Statistics* statistics = immutable_db_options_.stats; + if (!statistics) { + return false; + } + *value = statistics->ToString(); + return true; +} + +#ifndef ROCKSDB_LITE +Status DBImpl::ResetStats() { + InstrumentedMutexLock l(&mutex_); + for (auto* cfd : *versions_->GetColumnFamilySet()) { + if (cfd->initialized()) { + cfd->internal_stats()->Clear(); + } + } + return Status::OK(); +} +#endif // ROCKSDB_LITE + +bool DBImpl::GetAggregatedIntProperty(const Slice& property, + uint64_t* aggregated_value) { + const DBPropertyInfo* property_info = GetPropertyInfo(property); + if (property_info == nullptr || property_info->handle_int == nullptr) { + return false; + } + + uint64_t sum = 0; + bool ret = true; + { + // Needs mutex to protect the list of column families. + InstrumentedMutexLock l(&mutex_); + uint64_t value; + for (auto* cfd : versions_->GetRefedColumnFamilySet()) { + if (!cfd->initialized()) { + continue; + } + ret = GetIntPropertyInternal(cfd, *property_info, true, &value); + // GetIntPropertyInternal may release db mutex and re-acquire it. + mutex_.AssertHeld(); + if (ret) { + sum += value; + } else { + ret = false; + break; + } + } + } + *aggregated_value = sum; + return ret; +} + +SuperVersion* DBImpl::GetAndRefSuperVersion(ColumnFamilyData* cfd) { + // TODO(ljin): consider using GetReferencedSuperVersion() directly + return cfd->GetThreadLocalSuperVersion(this); +} + +// REQUIRED: this function should only be called on the write thread or if the +// mutex is held. +SuperVersion* DBImpl::GetAndRefSuperVersion(uint32_t column_family_id) { + auto column_family_set = versions_->GetColumnFamilySet(); + auto cfd = column_family_set->GetColumnFamily(column_family_id); + if (!cfd) { + return nullptr; + } + + return GetAndRefSuperVersion(cfd); +} + +void DBImpl::CleanupSuperVersion(SuperVersion* sv) { + // Release SuperVersion + if (sv->Unref()) { + bool defer_purge = immutable_db_options().avoid_unnecessary_blocking_io; + { + InstrumentedMutexLock l(&mutex_); + sv->Cleanup(); + if (defer_purge) { + AddSuperVersionsToFreeQueue(sv); + SchedulePurge(); + } + } + if (!defer_purge) { + delete sv; + } + RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS); + } + RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES); +} + +void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd, + SuperVersion* sv) { + if (!cfd->ReturnThreadLocalSuperVersion(sv)) { + CleanupSuperVersion(sv); + } +} + +// REQUIRED: this function should only be called on the write thread. +void DBImpl::ReturnAndCleanupSuperVersion(uint32_t column_family_id, + SuperVersion* sv) { + auto column_family_set = versions_->GetColumnFamilySet(); + auto cfd = column_family_set->GetColumnFamily(column_family_id); + + // If SuperVersion is held, and we successfully fetched a cfd using + // GetAndRefSuperVersion(), it must still exist. + assert(cfd != nullptr); + ReturnAndCleanupSuperVersion(cfd, sv); +} + +// REQUIRED: this function should only be called on the write thread or if the +// mutex is held. +ColumnFamilyHandle* DBImpl::GetColumnFamilyHandle(uint32_t column_family_id) { + ColumnFamilyMemTables* cf_memtables = column_family_memtables_.get(); + + if (!cf_memtables->Seek(column_family_id)) { + return nullptr; + } + + return cf_memtables->GetColumnFamilyHandle(); +} + +// REQUIRED: mutex is NOT held. +std::unique_ptr DBImpl::GetColumnFamilyHandleUnlocked( + uint32_t column_family_id) { + InstrumentedMutexLock l(&mutex_); + + auto* cfd = + versions_->GetColumnFamilySet()->GetColumnFamily(column_family_id); + if (cfd == nullptr) { + return nullptr; + } + + return std::unique_ptr( + new ColumnFamilyHandleImpl(cfd, this, &mutex_)); +} + +void DBImpl::GetApproximateMemTableStats(ColumnFamilyHandle* column_family, + const Range& range, + uint64_t* const count, + uint64_t* const size) { + ColumnFamilyHandleImpl* cfh = + static_cast_with_check(column_family); + ColumnFamilyData* cfd = cfh->cfd(); + SuperVersion* sv = GetAndRefSuperVersion(cfd); + + // Convert user_key into a corresponding internal key. + InternalKey k1(range.start, kMaxSequenceNumber, kValueTypeForSeek); + InternalKey k2(range.limit, kMaxSequenceNumber, kValueTypeForSeek); + MemTable::MemTableStats memStats = + sv->mem->ApproximateStats(k1.Encode(), k2.Encode()); + MemTable::MemTableStats immStats = + sv->imm->ApproximateStats(k1.Encode(), k2.Encode()); + *count = memStats.count + immStats.count; + *size = memStats.size + immStats.size; + + ReturnAndCleanupSuperVersion(cfd, sv); +} + +Status DBImpl::GetApproximateSizes(const SizeApproximationOptions& options, + ColumnFamilyHandle* column_family, + const Range* range, int n, uint64_t* sizes) { + if (!options.include_memtables && !options.include_files) { + return Status::InvalidArgument("Invalid options"); + } + + const Comparator* const ucmp = column_family->GetComparator(); + assert(ucmp); + size_t ts_sz = ucmp->timestamp_size(); + + Version* v; + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + SuperVersion* sv = GetAndRefSuperVersion(cfd); + v = sv->current; + + for (int i = 0; i < n; i++) { + Slice start = range[i].start; + Slice limit = range[i].limit; + + // Add timestamp if needed + std::string start_with_ts, limit_with_ts; + if (ts_sz > 0) { + // Maximum timestamp means including all key with any timestamp + AppendKeyWithMaxTimestamp(&start_with_ts, start, ts_sz); + // Append a maximum timestamp as the range limit is exclusive: + // [start, limit) + AppendKeyWithMaxTimestamp(&limit_with_ts, limit, ts_sz); + start = start_with_ts; + limit = limit_with_ts; + } + // Convert user_key into a corresponding internal key. + InternalKey k1(start, kMaxSequenceNumber, kValueTypeForSeek); + InternalKey k2(limit, kMaxSequenceNumber, kValueTypeForSeek); + sizes[i] = 0; + if (options.include_files) { + sizes[i] += versions_->ApproximateSize( + options, v, k1.Encode(), k2.Encode(), /*start_level=*/0, + /*end_level=*/-1, TableReaderCaller::kUserApproximateSize); + } + if (options.include_memtables) { + sizes[i] += sv->mem->ApproximateStats(k1.Encode(), k2.Encode()).size; + sizes[i] += sv->imm->ApproximateStats(k1.Encode(), k2.Encode()).size; + } + } + + ReturnAndCleanupSuperVersion(cfd, sv); + return Status::OK(); +} + +std::list::iterator +DBImpl::CaptureCurrentFileNumberInPendingOutputs() { + // We need to remember the iterator of our insert, because after the + // background job is done, we need to remove that element from + // pending_outputs_. + pending_outputs_.push_back(versions_->current_next_file_number()); + auto pending_outputs_inserted_elem = pending_outputs_.end(); + --pending_outputs_inserted_elem; + return pending_outputs_inserted_elem; +} + +void DBImpl::ReleaseFileNumberFromPendingOutputs( + std::unique_ptr::iterator>& v) { + if (v.get() != nullptr) { + pending_outputs_.erase(*v.get()); + v.reset(); + } +} + +#ifndef ROCKSDB_LITE +Status DBImpl::GetUpdatesSince( + SequenceNumber seq, std::unique_ptr* iter, + const TransactionLogIterator::ReadOptions& read_options) { + RecordTick(stats_, GET_UPDATES_SINCE_CALLS); + if (seq_per_batch_) { + return Status::NotSupported( + "This API is not yet compatible with write-prepared/write-unprepared " + "transactions"); + } + if (seq > versions_->LastSequence()) { + return Status::NotFound("Requested sequence not yet written in the db"); + } + return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get()); +} + +Status DBImpl::DeleteFile(std::string name) { + uint64_t number; + FileType type; + WalFileType log_type; + if (!ParseFileName(name, &number, &type, &log_type) || + (type != kTableFile && type != kWalFile)) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, "DeleteFile %s failed.\n", + name.c_str()); + return Status::InvalidArgument("Invalid file name"); + } + + if (type == kWalFile) { + // Only allow deleting archived log files + if (log_type != kArchivedLogFile) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "DeleteFile %s failed - not archived log.\n", + name.c_str()); + return Status::NotSupported("Delete only supported for archived logs"); + } + Status status = wal_manager_.DeleteFile(name, number); + if (!status.ok()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "DeleteFile %s failed -- %s.\n", name.c_str(), + status.ToString().c_str()); + } + return status; + } + + Status status; + int level; + FileMetaData* metadata; + ColumnFamilyData* cfd; + VersionEdit edit; + JobContext job_context(next_job_id_.fetch_add(1), true); + { + InstrumentedMutexLock l(&mutex_); + status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd); + if (!status.ok()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "DeleteFile %s failed. File not found\n", name.c_str()); + job_context.Clean(); + return Status::InvalidArgument("File not found"); + } + assert(level < cfd->NumberLevels()); + + // If the file is being compacted no need to delete. + if (metadata->being_compacted) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "DeleteFile %s Skipped. File about to be compacted\n", + name.c_str()); + job_context.Clean(); + return Status::OK(); + } + + // Only the files in the last level can be deleted externally. + // This is to make sure that any deletion tombstones are not + // lost. Check that the level passed is the last level. + auto* vstoreage = cfd->current()->storage_info(); + for (int i = level + 1; i < cfd->NumberLevels(); i++) { + if (vstoreage->NumLevelFiles(i) != 0) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "DeleteFile %s FAILED. File not in last level\n", + name.c_str()); + job_context.Clean(); + return Status::InvalidArgument("File not in last level"); + } + } + // if level == 0, it has to be the oldest file + if (level == 0 && + vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "DeleteFile %s failed ---" + " target file in level 0 must be the oldest.", + name.c_str()); + job_context.Clean(); + return Status::InvalidArgument("File in level 0, but not oldest"); + } + edit.SetColumnFamily(cfd->GetID()); + edit.DeleteFile(level, number); + status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + &edit, &mutex_, directories_.GetDbDir()); + if (status.ok()) { + InstallSuperVersionAndScheduleWork(cfd, + &job_context.superversion_contexts[0], + *cfd->GetLatestMutableCFOptions()); + } + FindObsoleteFiles(&job_context, false); + } // lock released here + + LogFlush(immutable_db_options_.info_log); + // remove files outside the db-lock + if (job_context.HaveSomethingToDelete()) { + // Call PurgeObsoleteFiles() without holding mutex. + PurgeObsoleteFiles(job_context); + } + job_context.Clean(); + return status; +} + +Status DBImpl::DeleteFilesInRanges(ColumnFamilyHandle* column_family, + const RangePtr* ranges, size_t n, + bool include_end) { + Status status = Status::OK(); + auto cfh = static_cast_with_check(column_family); + ColumnFamilyData* cfd = cfh->cfd(); + VersionEdit edit; + std::set deleted_files; + JobContext job_context(next_job_id_.fetch_add(1), true); + { + InstrumentedMutexLock l(&mutex_); + Version* input_version = cfd->current(); + + auto* vstorage = input_version->storage_info(); + for (size_t r = 0; r < n; r++) { + auto begin = ranges[r].start, end = ranges[r].limit; + for (int i = 1; i < cfd->NumberLevels(); i++) { + if (vstorage->LevelFiles(i).empty() || + !vstorage->OverlapInLevel(i, begin, end)) { + continue; + } + std::vector level_files; + InternalKey begin_storage, end_storage, *begin_key, *end_key; + if (begin == nullptr) { + begin_key = nullptr; + } else { + begin_storage.SetMinPossibleForUserKey(*begin); + begin_key = &begin_storage; + } + if (end == nullptr) { + end_key = nullptr; + } else { + end_storage.SetMaxPossibleForUserKey(*end); + end_key = &end_storage; + } + + vstorage->GetCleanInputsWithinInterval( + i, begin_key, end_key, &level_files, -1 /* hint_index */, + nullptr /* file_index */); + FileMetaData* level_file; + for (uint32_t j = 0; j < level_files.size(); j++) { + level_file = level_files[j]; + if (level_file->being_compacted) { + continue; + } + if (deleted_files.find(level_file) != deleted_files.end()) { + continue; + } + if (!include_end && end != nullptr && + cfd->user_comparator()->Compare(level_file->largest.user_key(), + *end) == 0) { + continue; + } + edit.SetColumnFamily(cfd->GetID()); + edit.DeleteFile(i, level_file->fd.GetNumber()); + deleted_files.insert(level_file); + level_file->being_compacted = true; + } + vstorage->ComputeCompactionScore(*cfd->ioptions(), + *cfd->GetLatestMutableCFOptions()); + } + } + if (edit.GetDeletedFiles().empty()) { + job_context.Clean(); + return status; + } + input_version->Ref(); + status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + &edit, &mutex_, directories_.GetDbDir()); + if (status.ok()) { + InstallSuperVersionAndScheduleWork(cfd, + &job_context.superversion_contexts[0], + *cfd->GetLatestMutableCFOptions()); + } + for (auto* deleted_file : deleted_files) { + deleted_file->being_compacted = false; + } + input_version->Unref(); + FindObsoleteFiles(&job_context, false); + } // lock released here + + LogFlush(immutable_db_options_.info_log); + // remove files outside the db-lock + if (job_context.HaveSomethingToDelete()) { + // Call PurgeObsoleteFiles() without holding mutex. + PurgeObsoleteFiles(job_context); + } + job_context.Clean(); + return status; +} + +void DBImpl::GetLiveFilesMetaData(std::vector* metadata) { + InstrumentedMutexLock l(&mutex_); + versions_->GetLiveFilesMetaData(metadata); +} + +Status DBImpl::GetLiveFilesChecksumInfo(FileChecksumList* checksum_list) { + InstrumentedMutexLock l(&mutex_); + return versions_->GetLiveFilesChecksumInfo(checksum_list); +} + +void DBImpl::GetColumnFamilyMetaData(ColumnFamilyHandle* column_family, + ColumnFamilyMetaData* cf_meta) { + assert(column_family); + auto* cfd = + static_cast_with_check(column_family)->cfd(); + auto* sv = GetAndRefSuperVersion(cfd); + { + // Without mutex, Version::GetColumnFamilyMetaData will have data race with + // Compaction::MarkFilesBeingCompacted. One solution is to use mutex, but + // this may cause regression. An alternative is to make + // FileMetaData::being_compacted atomic, but it will make FileMetaData + // non-copy-able. Another option is to separate these variables from + // original FileMetaData struct, and this requires re-organization of data + // structures. For now, we take the easy approach. If + // DB::GetColumnFamilyMetaData is not called frequently, the regression + // should not be big. We still need to keep an eye on it. + InstrumentedMutexLock l(&mutex_); + sv->current->GetColumnFamilyMetaData(cf_meta); + } + ReturnAndCleanupSuperVersion(cfd, sv); +} + +void DBImpl::GetAllColumnFamilyMetaData( + std::vector* metadata) { + InstrumentedMutexLock l(&mutex_); + for (auto cfd : *(versions_->GetColumnFamilySet())) { + { + metadata->emplace_back(); + cfd->current()->GetColumnFamilyMetaData(&metadata->back()); + } + } +} + +#endif // ROCKSDB_LITE + +Status DBImpl::CheckConsistency() { + mutex_.AssertHeld(); + std::vector metadata; + versions_->GetLiveFilesMetaData(&metadata); + TEST_SYNC_POINT("DBImpl::CheckConsistency:AfterGetLiveFilesMetaData"); + + std::string corruption_messages; + + if (immutable_db_options_.skip_checking_sst_file_sizes_on_db_open) { + // Instead of calling GetFileSize() for each expected file, call + // GetChildren() for the DB directory and check that all expected files + // are listed, without checking their sizes. + // Since sst files might be in different directories, do it for each + // directory separately. + std::map> files_by_directory; + for (const auto& md : metadata) { + // md.name has a leading "/". Remove it. + std::string fname = md.name; + if (!fname.empty() && fname[0] == '/') { + fname = fname.substr(1); + } + files_by_directory[md.db_path].push_back(fname); + } + + IOOptions io_opts; + io_opts.do_not_recurse = true; + for (const auto& dir_files : files_by_directory) { + std::string directory = dir_files.first; + std::vector existing_files; + Status s = fs_->GetChildren(directory, io_opts, &existing_files, + /*IODebugContext*=*/nullptr); + if (!s.ok()) { + corruption_messages += + "Can't list files in " + directory + ": " + s.ToString() + "\n"; + continue; + } + std::sort(existing_files.begin(), existing_files.end()); + + for (const std::string& fname : dir_files.second) { + if (!std::binary_search(existing_files.begin(), existing_files.end(), + fname) && + !std::binary_search(existing_files.begin(), existing_files.end(), + Rocks2LevelTableFileName(fname))) { + corruption_messages += + "Missing sst file " + fname + " in " + directory + "\n"; + } + } + } + } else { + for (const auto& md : metadata) { + // md.name has a leading "/". + std::string file_path = md.db_path + md.name; + + uint64_t fsize = 0; + TEST_SYNC_POINT("DBImpl::CheckConsistency:BeforeGetFileSize"); + Status s = env_->GetFileSize(file_path, &fsize); + if (!s.ok() && + env_->GetFileSize(Rocks2LevelTableFileName(file_path), &fsize).ok()) { + s = Status::OK(); + } + if (!s.ok()) { + corruption_messages += + "Can't access " + md.name + ": " + s.ToString() + "\n"; + } else if (fsize != md.size) { + corruption_messages += "Sst file size mismatch: " + file_path + + ". Size recorded in manifest " + + std::to_string(md.size) + ", actual size " + + std::to_string(fsize) + "\n"; + } + } + } + + if (corruption_messages.size() == 0) { + return Status::OK(); + } else { + return Status::Corruption(corruption_messages); + } +} + +Status DBImpl::GetDbIdentity(std::string& identity) const { + identity.assign(db_id_); + return Status::OK(); +} + +Status DBImpl::GetDbIdentityFromIdentityFile(std::string* identity) const { + std::string idfilename = IdentityFileName(dbname_); + const FileOptions soptions; + + Status s = ReadFileToString(fs_.get(), idfilename, identity); + if (!s.ok()) { + return s; + } + + // If last character is '\n' remove it from identity. (Old implementations + // of Env::GenerateUniqueId() would include a trailing '\n'.) + if (identity->size() > 0 && identity->back() == '\n') { + identity->pop_back(); + } + return s; +} + +Status DBImpl::GetDbSessionId(std::string& session_id) const { + session_id.assign(db_session_id_); + return Status::OK(); +} + +namespace { +SemiStructuredUniqueIdGen* DbSessionIdGen() { + static SemiStructuredUniqueIdGen gen; + return &gen; +} +} // namespace + +void DBImpl::TEST_ResetDbSessionIdGen() { DbSessionIdGen()->Reset(); } + +std::string DBImpl::GenerateDbSessionId(Env*) { + // See SemiStructuredUniqueIdGen for its desirable properties. + auto gen = DbSessionIdGen(); + + uint64_t lo, hi; + gen->GenerateNext(&hi, &lo); + if (lo == 0) { + // Avoid emitting session ID with lo==0, so that SST unique + // IDs can be more easily ensured non-zero + gen->GenerateNext(&hi, &lo); + assert(lo != 0); + } + return EncodeSessionId(hi, lo); +} + +void DBImpl::SetDbSessionId() { + db_session_id_ = GenerateDbSessionId(env_); + TEST_SYNC_POINT_CALLBACK("DBImpl::SetDbSessionId", &db_session_id_); +} + +// Default implementation -- returns not supported status +Status DB::CreateColumnFamily(const ColumnFamilyOptions& /*cf_options*/, + const std::string& /*column_family_name*/, + ColumnFamilyHandle** /*handle*/) { + return Status::NotSupported(""); +} + +Status DB::CreateColumnFamilies( + const ColumnFamilyOptions& /*cf_options*/, + const std::vector& /*column_family_names*/, + std::vector* /*handles*/) { + return Status::NotSupported(""); +} + +Status DB::CreateColumnFamilies( + const std::vector& /*column_families*/, + std::vector* /*handles*/) { + return Status::NotSupported(""); +} + +Status DB::DropColumnFamily(ColumnFamilyHandle* /*column_family*/) { + return Status::NotSupported(""); +} + +Status DB::DropColumnFamilies( + const std::vector& /*column_families*/) { + return Status::NotSupported(""); +} + +Status DB::DestroyColumnFamilyHandle(ColumnFamilyHandle* column_family) { + if (DefaultColumnFamily() == column_family) { + return Status::InvalidArgument( + "Cannot destroy the handle returned by DefaultColumnFamily()"); + } + delete column_family; + return Status::OK(); +} + +DB::~DB() {} + +Status DBImpl::Close() { + InstrumentedMutexLock closing_lock_guard(&closing_mutex_); + if (closed_) { + return closing_status_; + } + + { + const Status s = MaybeReleaseTimestampedSnapshotsAndCheck(); + if (!s.ok()) { + return s; + } + } + + closing_status_ = CloseImpl(); + closed_ = true; + return closing_status_; +} + +Status DB::ListColumnFamilies(const DBOptions& db_options, + const std::string& name, + std::vector* column_families) { + const std::shared_ptr& fs = db_options.env->GetFileSystem(); + return VersionSet::ListColumnFamilies(column_families, name, fs.get()); +} + +Snapshot::~Snapshot() {} + +Status DestroyDB(const std::string& dbname, const Options& options, + const std::vector& column_families) { + ImmutableDBOptions soptions(SanitizeOptions(dbname, options)); + Env* env = soptions.env; + std::vector filenames; + bool wal_in_db_path = soptions.IsWalDirSameAsDBPath(); + + // Reset the logger because it holds a handle to the + // log file and prevents cleanup and directory removal + soptions.info_log.reset(); + IOOptions io_opts; + // Ignore error in case directory does not exist + soptions.fs + ->GetChildren(dbname, io_opts, &filenames, + /*IODebugContext*=*/nullptr) + .PermitUncheckedError(); + + FileLock* lock; + const std::string lockname = LockFileName(dbname); + Status result = env->LockFile(lockname, &lock); + if (result.ok()) { + uint64_t number; + FileType type; + InfoLogPrefix info_log_prefix(!soptions.db_log_dir.empty(), dbname); + for (const auto& fname : filenames) { + if (ParseFileName(fname, &number, info_log_prefix.prefix, &type) && + type != kDBLockFile) { // Lock file will be deleted at end + Status del; + std::string path_to_delete = dbname + "/" + fname; + if (type == kMetaDatabase) { + del = DestroyDB(path_to_delete, options); + } else if (type == kTableFile || type == kWalFile || + type == kBlobFile) { + del = DeleteDBFile( + &soptions, path_to_delete, dbname, + /*force_bg=*/false, + /*force_fg=*/(type == kWalFile) ? !wal_in_db_path : false); + } else { + del = env->DeleteFile(path_to_delete); + } + if (!del.ok() && result.ok()) { + result = del; + } + } + } + + std::set paths; + for (const DbPath& db_path : options.db_paths) { + paths.insert(db_path.path); + } + for (const ColumnFamilyDescriptor& cf : column_families) { + for (const DbPath& cf_path : cf.options.cf_paths) { + paths.insert(cf_path.path); + } + } + + for (const auto& path : paths) { + if (soptions.fs + ->GetChildren(path, io_opts, &filenames, + /*IODebugContext*=*/nullptr) + .ok()) { + for (const auto& fname : filenames) { + if (ParseFileName(fname, &number, &type) && + (type == kTableFile || + type == kBlobFile)) { // Lock file will be deleted at end + std::string file_path = path + "/" + fname; + Status del = DeleteDBFile(&soptions, file_path, dbname, + /*force_bg=*/false, /*force_fg=*/false); + if (!del.ok() && result.ok()) { + result = del; + } + } + } + // TODO: Should we return an error if we cannot delete the directory? + env->DeleteDir(path).PermitUncheckedError(); + } + } + + std::vector walDirFiles; + std::string archivedir = ArchivalDirectory(dbname); + bool wal_dir_exists = false; + if (!soptions.IsWalDirSameAsDBPath(dbname)) { + wal_dir_exists = + soptions.fs + ->GetChildren(soptions.wal_dir, io_opts, &walDirFiles, + /*IODebugContext*=*/nullptr) + .ok(); + archivedir = ArchivalDirectory(soptions.wal_dir); + } + + // Archive dir may be inside wal dir or dbname and should be + // processed and removed before those otherwise we have issues + // removing them + std::vector archiveFiles; + if (soptions.fs + ->GetChildren(archivedir, io_opts, &archiveFiles, + /*IODebugContext*=*/nullptr) + .ok()) { + // Delete archival files. + for (const auto& file : archiveFiles) { + if (ParseFileName(file, &number, &type) && type == kWalFile) { + Status del = + DeleteDBFile(&soptions, archivedir + "/" + file, archivedir, + /*force_bg=*/false, /*force_fg=*/!wal_in_db_path); + if (!del.ok() && result.ok()) { + result = del; + } + } + } + // Ignore error in case dir contains other files + env->DeleteDir(archivedir).PermitUncheckedError(); + } + + // Delete log files in the WAL dir + if (wal_dir_exists) { + for (const auto& file : walDirFiles) { + if (ParseFileName(file, &number, &type) && type == kWalFile) { + Status del = + DeleteDBFile(&soptions, LogFileName(soptions.wal_dir, number), + soptions.wal_dir, /*force_bg=*/false, + /*force_fg=*/!wal_in_db_path); + if (!del.ok() && result.ok()) { + result = del; + } + } + } + // Ignore error in case dir contains other files + env->DeleteDir(soptions.wal_dir).PermitUncheckedError(); + } + + // Ignore error since state is already gone + env->UnlockFile(lock).PermitUncheckedError(); + env->DeleteFile(lockname).PermitUncheckedError(); + + // sst_file_manager holds a ref to the logger. Make sure the logger is + // gone before trying to remove the directory. + soptions.sst_file_manager.reset(); + + // Ignore error in case dir contains other files + env->DeleteDir(dbname).PermitUncheckedError(); + ; + } + return result; +} + +Status DBImpl::WriteOptionsFile(bool need_mutex_lock, + bool need_enter_write_thread) { +#ifndef ROCKSDB_LITE + WriteThread::Writer w; + if (need_mutex_lock) { + mutex_.Lock(); + } else { + mutex_.AssertHeld(); + } + if (need_enter_write_thread) { + write_thread_.EnterUnbatched(&w, &mutex_); + } + + std::vector cf_names; + std::vector cf_opts; + + // This part requires mutex to protect the column family options + for (auto cfd : *versions_->GetColumnFamilySet()) { + if (cfd->IsDropped()) { + continue; + } + cf_names.push_back(cfd->GetName()); + cf_opts.push_back(cfd->GetLatestCFOptions()); + } + + // Unlock during expensive operations. New writes cannot get here + // because the single write thread ensures all new writes get queued. + DBOptions db_options = + BuildDBOptions(immutable_db_options_, mutable_db_options_); + mutex_.Unlock(); + + TEST_SYNC_POINT("DBImpl::WriteOptionsFile:1"); + TEST_SYNC_POINT("DBImpl::WriteOptionsFile:2"); + TEST_SYNC_POINT_CALLBACK("DBImpl::WriteOptionsFile:PersistOptions", + &db_options); + + std::string file_name = + TempOptionsFileName(GetName(), versions_->NewFileNumber()); + Status s = PersistRocksDBOptions(db_options, cf_names, cf_opts, file_name, + fs_.get()); + + if (s.ok()) { + s = RenameTempFileToOptionsFile(file_name); + } + // restore lock + if (!need_mutex_lock) { + mutex_.Lock(); + } + if (need_enter_write_thread) { + write_thread_.ExitUnbatched(&w); + } + if (!s.ok()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "Unnable to persist options -- %s", s.ToString().c_str()); + if (immutable_db_options_.fail_if_options_file_error) { + return Status::IOError("Unable to persist options.", + s.ToString().c_str()); + } + } +#else + (void)need_mutex_lock; + (void)need_enter_write_thread; +#endif // !ROCKSDB_LITE + return Status::OK(); +} + +#ifndef ROCKSDB_LITE +namespace { +void DeleteOptionsFilesHelper(const std::map& filenames, + const size_t num_files_to_keep, + const std::shared_ptr& info_log, + Env* env) { + if (filenames.size() <= num_files_to_keep) { + return; + } + for (auto iter = std::next(filenames.begin(), num_files_to_keep); + iter != filenames.end(); ++iter) { + if (!env->DeleteFile(iter->second).ok()) { + ROCKS_LOG_WARN(info_log, "Unable to delete options file %s", + iter->second.c_str()); + } + } +} +} // namespace +#endif // !ROCKSDB_LITE + +Status DBImpl::DeleteObsoleteOptionsFiles() { +#ifndef ROCKSDB_LITE + std::vector filenames; + // use ordered map to store keep the filenames sorted from the newest + // to the oldest. + std::map options_filenames; + Status s; + IOOptions io_opts; + io_opts.do_not_recurse = true; + s = fs_->GetChildren(GetName(), io_opts, &filenames, + /*IODebugContext*=*/nullptr); + if (!s.ok()) { + return s; + } + for (auto& filename : filenames) { + uint64_t file_number; + FileType type; + if (ParseFileName(filename, &file_number, &type) && type == kOptionsFile) { + options_filenames.insert( + {std::numeric_limits::max() - file_number, + GetName() + "/" + filename}); + } + } + + // Keeps the latest 2 Options file + const size_t kNumOptionsFilesKept = 2; + DeleteOptionsFilesHelper(options_filenames, kNumOptionsFilesKept, + immutable_db_options_.info_log, GetEnv()); + return Status::OK(); +#else + return Status::OK(); +#endif // !ROCKSDB_LITE +} + +Status DBImpl::RenameTempFileToOptionsFile(const std::string& file_name) { +#ifndef ROCKSDB_LITE + Status s; + + uint64_t options_file_number = versions_->NewFileNumber(); + std::string options_file_name = + OptionsFileName(GetName(), options_file_number); + uint64_t options_file_size = 0; + s = GetEnv()->GetFileSize(file_name, &options_file_size); + if (s.ok()) { + // Retry if the file name happen to conflict with an existing one. + s = GetEnv()->RenameFile(file_name, options_file_name); + std::unique_ptr dir_obj; + if (s.ok()) { + s = fs_->NewDirectory(GetName(), IOOptions(), &dir_obj, nullptr); + } + if (s.ok()) { + s = dir_obj->FsyncWithDirOptions(IOOptions(), nullptr, + DirFsyncOptions(options_file_name)); + } + if (s.ok()) { + Status temp_s = dir_obj->Close(IOOptions(), nullptr); + // The default Close() could return "NotSupproted" and we bypass it + // if it is not impelmented. Detailed explanations can be found in + // db/db_impl/db_impl.h + if (!temp_s.ok()) { + if (temp_s.IsNotSupported()) { + temp_s.PermitUncheckedError(); + } else { + s = temp_s; + } + } + } + } + if (s.ok()) { + InstrumentedMutexLock l(&mutex_); + versions_->options_file_number_ = options_file_number; + versions_->options_file_size_ = options_file_size; + } + + if (0 == disable_delete_obsolete_files_) { + // TODO: Should we check for errors here? + DeleteObsoleteOptionsFiles().PermitUncheckedError(); + } + return s; +#else + (void)file_name; + return Status::OK(); +#endif // !ROCKSDB_LITE +} + +#ifdef ROCKSDB_USING_THREAD_STATUS + +void DBImpl::NewThreadStatusCfInfo(ColumnFamilyData* cfd) const { + if (immutable_db_options_.enable_thread_tracking) { + ThreadStatusUtil::NewColumnFamilyInfo(this, cfd, cfd->GetName(), + cfd->ioptions()->env); + } +} + +void DBImpl::EraseThreadStatusCfInfo(ColumnFamilyData* cfd) const { + if (immutable_db_options_.enable_thread_tracking) { + ThreadStatusUtil::EraseColumnFamilyInfo(cfd); + } +} + +void DBImpl::EraseThreadStatusDbInfo() const { + if (immutable_db_options_.enable_thread_tracking) { + ThreadStatusUtil::EraseDatabaseInfo(this); + } +} + +#else +void DBImpl::NewThreadStatusCfInfo(ColumnFamilyData* /*cfd*/) const {} + +void DBImpl::EraseThreadStatusCfInfo(ColumnFamilyData* /*cfd*/) const {} + +void DBImpl::EraseThreadStatusDbInfo() const {} +#endif // ROCKSDB_USING_THREAD_STATUS + +// +// A global method that can dump out the build version +void DumpRocksDBBuildVersion(Logger* log) { + ROCKS_LOG_HEADER(log, "RocksDB version: %s\n", + GetRocksVersionAsString().c_str()); + const auto& props = GetRocksBuildProperties(); + const auto& sha = props.find("rocksdb_build_git_sha"); + if (sha != props.end()) { + ROCKS_LOG_HEADER(log, "Git sha %s", sha->second.c_str()); + } + const auto date = props.find("rocksdb_build_date"); + if (date != props.end()) { + ROCKS_LOG_HEADER(log, "Compile date %s", date->second.c_str()); + } +} + +#ifndef ROCKSDB_LITE +SequenceNumber DBImpl::GetEarliestMemTableSequenceNumber(SuperVersion* sv, + bool include_history) { + // Find the earliest sequence number that we know we can rely on reading + // from the memtable without needing to check sst files. + SequenceNumber earliest_seq = + sv->imm->GetEarliestSequenceNumber(include_history); + if (earliest_seq == kMaxSequenceNumber) { + earliest_seq = sv->mem->GetEarliestSequenceNumber(); + } + assert(sv->mem->GetEarliestSequenceNumber() >= earliest_seq); + + return earliest_seq; +} + +Status DBImpl::GetLatestSequenceForKey( + SuperVersion* sv, const Slice& key, bool cache_only, + SequenceNumber lower_bound_seq, SequenceNumber* seq, std::string* timestamp, + bool* found_record_for_key, bool* is_blob_index) { + Status s; + MergeContext merge_context; + SequenceNumber max_covering_tombstone_seq = 0; + + ReadOptions read_options; + SequenceNumber current_seq = versions_->LastSequence(); + + ColumnFamilyData* cfd = sv->cfd; + assert(cfd); + const Comparator* const ucmp = cfd->user_comparator(); + assert(ucmp); + size_t ts_sz = ucmp->timestamp_size(); + std::string ts_buf; + if (ts_sz > 0) { + assert(timestamp); + ts_buf.assign(ts_sz, '\xff'); + } else { + assert(!timestamp); + } + Slice ts(ts_buf); + + LookupKey lkey(key, current_seq, ts_sz == 0 ? nullptr : &ts); + + *seq = kMaxSequenceNumber; + *found_record_for_key = false; + + // Check if there is a record for this key in the latest memtable + sv->mem->Get(lkey, /*value=*/nullptr, /*columns=*/nullptr, timestamp, &s, + &merge_context, &max_covering_tombstone_seq, seq, read_options, + false /* immutable_memtable */, nullptr /*read_callback*/, + is_blob_index); + + if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { + // unexpected error reading memtable. + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Unexpected status returned from MemTable::Get: %s\n", + s.ToString().c_str()); + + return s; + } + assert(!ts_sz || + (*seq != kMaxSequenceNumber && + *timestamp != std::string(ts_sz, '\xff')) || + (*seq == kMaxSequenceNumber && timestamp->empty())); + + TEST_SYNC_POINT_CALLBACK("DBImpl::GetLatestSequenceForKey:mem", timestamp); + + if (*seq != kMaxSequenceNumber) { + // Found a sequence number, no need to check immutable memtables + *found_record_for_key = true; + return Status::OK(); + } + + SequenceNumber lower_bound_in_mem = sv->mem->GetEarliestSequenceNumber(); + if (lower_bound_in_mem != kMaxSequenceNumber && + lower_bound_in_mem < lower_bound_seq) { + *found_record_for_key = false; + return Status::OK(); + } + + // Check if there is a record for this key in the immutable memtables + sv->imm->Get(lkey, /*value=*/nullptr, /*columns=*/nullptr, timestamp, &s, + &merge_context, &max_covering_tombstone_seq, seq, read_options, + nullptr /*read_callback*/, is_blob_index); + + if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { + // unexpected error reading memtable. + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Unexpected status returned from MemTableList::Get: %s\n", + s.ToString().c_str()); + + return s; + } + + assert(!ts_sz || + (*seq != kMaxSequenceNumber && + *timestamp != std::string(ts_sz, '\xff')) || + (*seq == kMaxSequenceNumber && timestamp->empty())); + + if (*seq != kMaxSequenceNumber) { + // Found a sequence number, no need to check memtable history + *found_record_for_key = true; + return Status::OK(); + } + + SequenceNumber lower_bound_in_imm = sv->imm->GetEarliestSequenceNumber(); + if (lower_bound_in_imm != kMaxSequenceNumber && + lower_bound_in_imm < lower_bound_seq) { + *found_record_for_key = false; + return Status::OK(); + } + + // Check if there is a record for this key in the immutable memtables + sv->imm->GetFromHistory(lkey, /*value=*/nullptr, /*columns=*/nullptr, + timestamp, &s, &merge_context, + &max_covering_tombstone_seq, seq, read_options, + is_blob_index); + + if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { + // unexpected error reading memtable. + ROCKS_LOG_ERROR( + immutable_db_options_.info_log, + "Unexpected status returned from MemTableList::GetFromHistory: %s\n", + s.ToString().c_str()); + + return s; + } + + assert(!ts_sz || + (*seq != kMaxSequenceNumber && + *timestamp != std::string(ts_sz, '\xff')) || + (*seq == kMaxSequenceNumber && timestamp->empty())); + + if (*seq != kMaxSequenceNumber) { + // Found a sequence number, no need to check SST files + assert(0 == ts_sz || *timestamp != std::string(ts_sz, '\xff')); + *found_record_for_key = true; + return Status::OK(); + } + + // We could do a sv->imm->GetEarliestSequenceNumber(/*include_history*/ true) + // check here to skip the history if possible. But currently the caller + // already does that. Maybe we should move the logic here later. + + // TODO(agiardullo): possible optimization: consider checking cached + // SST files if cache_only=true? + if (!cache_only) { + // Check tables + PinnedIteratorsManager pinned_iters_mgr; + sv->current->Get(read_options, lkey, /*value=*/nullptr, /*columns=*/nullptr, + timestamp, &s, &merge_context, &max_covering_tombstone_seq, + &pinned_iters_mgr, nullptr /* value_found */, + found_record_for_key, seq, nullptr /*read_callback*/, + is_blob_index); + + if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) { + // unexpected error reading SST files + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Unexpected status returned from Version::Get: %s\n", + s.ToString().c_str()); + } + } + + return s; +} + +Status DBImpl::IngestExternalFile( + ColumnFamilyHandle* column_family, + const std::vector& external_files, + const IngestExternalFileOptions& ingestion_options) { + IngestExternalFileArg arg; + arg.column_family = column_family; + arg.external_files = external_files; + arg.options = ingestion_options; + return IngestExternalFiles({arg}); +} + +Status DBImpl::IngestExternalFiles( + const std::vector& args) { + if (args.empty()) { + return Status::InvalidArgument("ingestion arg list is empty"); + } + { + std::unordered_set unique_cfhs; + for (const auto& arg : args) { + if (arg.column_family == nullptr) { + return Status::InvalidArgument("column family handle is null"); + } else if (unique_cfhs.count(arg.column_family) > 0) { + return Status::InvalidArgument( + "ingestion args have duplicate column families"); + } + unique_cfhs.insert(arg.column_family); + } + } + // Ingest multiple external SST files atomically. + const size_t num_cfs = args.size(); + for (size_t i = 0; i != num_cfs; ++i) { + if (args[i].external_files.empty()) { + char err_msg[128] = {0}; + snprintf(err_msg, 128, "external_files[%zu] is empty", i); + return Status::InvalidArgument(err_msg); + } + } + for (const auto& arg : args) { + const IngestExternalFileOptions& ingest_opts = arg.options; + if (ingest_opts.ingest_behind && + !immutable_db_options_.allow_ingest_behind) { + return Status::InvalidArgument( + "can't ingest_behind file in DB with allow_ingest_behind=false"); + } + } + + // TODO (yanqin) maybe handle the case in which column_families have + // duplicates + std::unique_ptr::iterator> pending_output_elem; + size_t total = 0; + for (const auto& arg : args) { + total += arg.external_files.size(); + } + uint64_t next_file_number = 0; + Status status = ReserveFileNumbersBeforeIngestion( + static_cast(args[0].column_family)->cfd(), total, + pending_output_elem, &next_file_number); + if (!status.ok()) { + InstrumentedMutexLock l(&mutex_); + ReleaseFileNumberFromPendingOutputs(pending_output_elem); + return status; + } + + std::vector ingestion_jobs; + for (const auto& arg : args) { + auto* cfd = static_cast(arg.column_family)->cfd(); + ingestion_jobs.emplace_back(versions_.get(), cfd, immutable_db_options_, + file_options_, &snapshots_, arg.options, + &directories_, &event_logger_, io_tracer_); + } + + // TODO(yanqin) maybe make jobs run in parallel + uint64_t start_file_number = next_file_number; + for (size_t i = 1; i != num_cfs; ++i) { + start_file_number += args[i - 1].external_files.size(); + auto* cfd = + static_cast(args[i].column_family)->cfd(); + SuperVersion* super_version = cfd->GetReferencedSuperVersion(this); + Status es = ingestion_jobs[i].Prepare( + args[i].external_files, args[i].files_checksums, + args[i].files_checksum_func_names, args[i].file_temperature, + start_file_number, super_version); + // capture first error only + if (!es.ok() && status.ok()) { + status = es; + } + CleanupSuperVersion(super_version); + } + TEST_SYNC_POINT("DBImpl::IngestExternalFiles:BeforeLastJobPrepare:0"); + TEST_SYNC_POINT("DBImpl::IngestExternalFiles:BeforeLastJobPrepare:1"); + { + auto* cfd = + static_cast(args[0].column_family)->cfd(); + SuperVersion* super_version = cfd->GetReferencedSuperVersion(this); + Status es = ingestion_jobs[0].Prepare( + args[0].external_files, args[0].files_checksums, + args[0].files_checksum_func_names, args[0].file_temperature, + next_file_number, super_version); + if (!es.ok()) { + status = es; + } + CleanupSuperVersion(super_version); + } + if (!status.ok()) { + for (size_t i = 0; i != num_cfs; ++i) { + ingestion_jobs[i].Cleanup(status); + } + InstrumentedMutexLock l(&mutex_); + ReleaseFileNumberFromPendingOutputs(pending_output_elem); + return status; + } + + std::vector sv_ctxs; + for (size_t i = 0; i != num_cfs; ++i) { + sv_ctxs.emplace_back(true /* create_superversion */); + } + TEST_SYNC_POINT("DBImpl::IngestExternalFiles:BeforeJobsRun:0"); + TEST_SYNC_POINT("DBImpl::IngestExternalFiles:BeforeJobsRun:1"); + TEST_SYNC_POINT("DBImpl::AddFile:Start"); + { + InstrumentedMutexLock l(&mutex_); + TEST_SYNC_POINT("DBImpl::AddFile:MutexLock"); + + // Stop writes to the DB by entering both write threads + WriteThread::Writer w; + write_thread_.EnterUnbatched(&w, &mutex_); + WriteThread::Writer nonmem_w; + if (two_write_queues_) { + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + } + + // When unordered_write is enabled, the keys are writing to memtable in an + // unordered way. If the ingestion job checks memtable key range before the + // key landing in memtable, the ingestion job may skip the necessary + // memtable flush. + // So wait here to ensure there is no pending write to memtable. + WaitForPendingWrites(); + + num_running_ingest_file_ += static_cast(num_cfs); + TEST_SYNC_POINT("DBImpl::IngestExternalFile:AfterIncIngestFileCounter"); + + bool at_least_one_cf_need_flush = false; + std::vector need_flush(num_cfs, false); + for (size_t i = 0; i != num_cfs; ++i) { + auto* cfd = + static_cast(args[i].column_family)->cfd(); + if (cfd->IsDropped()) { + // TODO (yanqin) investigate whether we should abort ingestion or + // proceed with other non-dropped column families. + status = Status::InvalidArgument( + "cannot ingest an external file into a dropped CF"); + break; + } + bool tmp = false; + status = ingestion_jobs[i].NeedsFlush(&tmp, cfd->GetSuperVersion()); + need_flush[i] = tmp; + at_least_one_cf_need_flush = (at_least_one_cf_need_flush || tmp); + if (!status.ok()) { + break; + } + } + TEST_SYNC_POINT_CALLBACK("DBImpl::IngestExternalFile:NeedFlush", + &at_least_one_cf_need_flush); + + if (status.ok() && at_least_one_cf_need_flush) { + FlushOptions flush_opts; + flush_opts.allow_write_stall = true; + if (immutable_db_options_.atomic_flush) { + autovector cfds_to_flush; + SelectColumnFamiliesForAtomicFlush(&cfds_to_flush); + mutex_.Unlock(); + status = AtomicFlushMemTables(cfds_to_flush, flush_opts, + FlushReason::kExternalFileIngestion, + true /* entered_write_thread */); + mutex_.Lock(); + } else { + for (size_t i = 0; i != num_cfs; ++i) { + if (need_flush[i]) { + mutex_.Unlock(); + auto* cfd = + static_cast(args[i].column_family) + ->cfd(); + status = FlushMemTable(cfd, flush_opts, + FlushReason::kExternalFileIngestion, + true /* entered_write_thread */); + mutex_.Lock(); + if (!status.ok()) { + break; + } + } + } + } + } + // Run ingestion jobs. + if (status.ok()) { + for (size_t i = 0; i != num_cfs; ++i) { + status = ingestion_jobs[i].Run(); + if (!status.ok()) { + break; + } + } + } + if (status.ok()) { + autovector cfds_to_commit; + autovector mutable_cf_options_list; + autovector> edit_lists; + uint32_t num_entries = 0; + for (size_t i = 0; i != num_cfs; ++i) { + auto* cfd = + static_cast(args[i].column_family)->cfd(); + if (cfd->IsDropped()) { + continue; + } + cfds_to_commit.push_back(cfd); + 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); + ++num_entries; + } + // Mark the version edits as an atomic group if the number of version + // edits exceeds 1. + if (cfds_to_commit.size() > 1) { + for (auto& edits : edit_lists) { + assert(edits.size() == 1); + edits[0]->MarkAtomicGroup(--num_entries); + } + assert(0 == num_entries); + } + status = + versions_->LogAndApply(cfds_to_commit, mutable_cf_options_list, + edit_lists, &mutex_, directories_.GetDbDir()); + // It is safe to update VersionSet last seqno here after LogAndApply since + // LogAndApply persists last sequence number from VersionEdits, + // which are from file's largest seqno and not from VersionSet. + // + // It is necessary to update last seqno here since LogAndApply releases + // mutex when persisting MANIFEST file, and the snapshots taken during + // that period will not be stable if VersionSet last seqno is updated + // before LogAndApply. + int consumed_seqno_count = + ingestion_jobs[0].ConsumedSequenceNumbersCount(); + for (size_t i = 1; i != num_cfs; ++i) { + consumed_seqno_count = + std::max(consumed_seqno_count, + ingestion_jobs[i].ConsumedSequenceNumbersCount()); + } + if (consumed_seqno_count > 0) { + const SequenceNumber last_seqno = versions_->LastSequence(); + versions_->SetLastAllocatedSequence(last_seqno + consumed_seqno_count); + versions_->SetLastPublishedSequence(last_seqno + consumed_seqno_count); + versions_->SetLastSequence(last_seqno + consumed_seqno_count); + } + } + + if (status.ok()) { + for (size_t i = 0; i != num_cfs; ++i) { + auto* cfd = + static_cast(args[i].column_family)->cfd(); + if (!cfd->IsDropped()) { + InstallSuperVersionAndScheduleWork(cfd, &sv_ctxs[i], + *cfd->GetLatestMutableCFOptions()); +#ifndef NDEBUG + if (0 == i && num_cfs > 1) { + TEST_SYNC_POINT( + "DBImpl::IngestExternalFiles:InstallSVForFirstCF:0"); + TEST_SYNC_POINT( + "DBImpl::IngestExternalFiles:InstallSVForFirstCF:1"); + } +#endif // !NDEBUG + } + } + } else if (versions_->io_status().IsIOError()) { + // Error while writing to MANIFEST. + // In fact, versions_->io_status() can also be the result of renaming + // CURRENT file. With current code, it's just difficult to tell. So just + // be pessimistic and try write to a new MANIFEST. + // TODO: distinguish between MANIFEST write and CURRENT renaming + const IOStatus& io_s = versions_->io_status(); + // Should handle return error? + error_handler_.SetBGError(io_s, BackgroundErrorReason::kManifestWrite); + } + + // Resume writes to the DB + if (two_write_queues_) { + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } + write_thread_.ExitUnbatched(&w); + + if (status.ok()) { + for (auto& job : ingestion_jobs) { + job.UpdateStats(); + } + } + ReleaseFileNumberFromPendingOutputs(pending_output_elem); + num_running_ingest_file_ -= static_cast(num_cfs); + if (0 == num_running_ingest_file_) { + bg_cv_.SignalAll(); + } + TEST_SYNC_POINT("DBImpl::AddFile:MutexUnlock"); + } + // mutex_ is unlocked here + + // Cleanup + for (size_t i = 0; i != num_cfs; ++i) { + sv_ctxs[i].Clean(); + // This may rollback jobs that have completed successfully. This is + // intended for atomicity. + ingestion_jobs[i].Cleanup(status); + } + if (status.ok()) { + for (size_t i = 0; i != num_cfs; ++i) { + auto* cfd = + static_cast(args[i].column_family)->cfd(); + if (!cfd->IsDropped()) { + NotifyOnExternalFileIngested(cfd, ingestion_jobs[i]); + } + } + } + return status; +} + +Status DBImpl::CreateColumnFamilyWithImport( + const ColumnFamilyOptions& options, const std::string& column_family_name, + const ImportColumnFamilyOptions& import_options, + const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle) { + assert(handle != nullptr); + assert(*handle == nullptr); + std::string cf_comparator_name = options.comparator->Name(); + if (cf_comparator_name != metadata.db_comparator_name) { + return Status::InvalidArgument("Comparator name mismatch"); + } + + // Create column family. + auto status = CreateColumnFamily(options, column_family_name, handle); + if (!status.ok()) { + return status; + } + + // Import sst files from metadata. + auto cfh = static_cast_with_check(*handle); + auto cfd = cfh->cfd(); + ImportColumnFamilyJob import_job(versions_.get(), cfd, immutable_db_options_, + file_options_, import_options, + metadata.files, io_tracer_); + + SuperVersionContext dummy_sv_ctx(/* create_superversion */ true); + VersionEdit dummy_edit; + uint64_t next_file_number = 0; + std::unique_ptr::iterator> pending_output_elem; + { + // Lock db mutex + InstrumentedMutexLock l(&mutex_); + if (error_handler_.IsDBStopped()) { + // Don't import files when there is a bg_error + status = error_handler_.GetBGError(); + } + + // Make sure that bg cleanup wont delete the files that we are importing + pending_output_elem.reset(new std::list::iterator( + CaptureCurrentFileNumberInPendingOutputs())); + + if (status.ok()) { + // 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. + next_file_number = versions_->FetchAddFileNumber(metadata.files.size()); + auto cf_options = cfd->GetLatestMutableCFOptions(); + status = versions_->LogAndApply(cfd, *cf_options, &dummy_edit, &mutex_, + directories_.GetDbDir()); + if (status.ok()) { + InstallSuperVersionAndScheduleWork(cfd, &dummy_sv_ctx, *cf_options); + } + } + } + dummy_sv_ctx.Clean(); + + if (status.ok()) { + SuperVersion* sv = cfd->GetReferencedSuperVersion(this); + status = import_job.Prepare(next_file_number, sv); + CleanupSuperVersion(sv); + } + + if (status.ok()) { + SuperVersionContext sv_context(true /*create_superversion*/); + { + // Lock db mutex + InstrumentedMutexLock l(&mutex_); + + // Stop writes to the DB by entering both write threads + WriteThread::Writer w; + write_thread_.EnterUnbatched(&w, &mutex_); + WriteThread::Writer nonmem_w; + if (two_write_queues_) { + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + } + + num_running_ingest_file_++; + assert(!cfd->IsDropped()); + status = import_job.Run(); + + // Install job edit [Mutex will be unlocked here] + if (status.ok()) { + auto cf_options = cfd->GetLatestMutableCFOptions(); + status = versions_->LogAndApply(cfd, *cf_options, import_job.edit(), + &mutex_, directories_.GetDbDir()); + if (status.ok()) { + InstallSuperVersionAndScheduleWork(cfd, &sv_context, *cf_options); + } + } + + // Resume writes to the DB + if (two_write_queues_) { + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } + write_thread_.ExitUnbatched(&w); + + num_running_ingest_file_--; + if (num_running_ingest_file_ == 0) { + bg_cv_.SignalAll(); + } + } + // mutex_ is unlocked here + + sv_context.Clean(); + } + + { + InstrumentedMutexLock l(&mutex_); + ReleaseFileNumberFromPendingOutputs(pending_output_elem); + } + + import_job.Cleanup(status); + if (!status.ok()) { + Status temp_s = DropColumnFamily(*handle); + if (!temp_s.ok()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "DropColumnFamily failed with error %s", + temp_s.ToString().c_str()); + } + // Always returns Status::OK() + temp_s = DestroyColumnFamilyHandle(*handle); + assert(temp_s.ok()); + *handle = nullptr; + } + return status; +} + +Status DBImpl::VerifyFileChecksums(const ReadOptions& read_options) { + return VerifyChecksumInternal(read_options, /*use_file_checksum=*/true); +} + +Status DBImpl::VerifyChecksum(const ReadOptions& read_options) { + return VerifyChecksumInternal(read_options, /*use_file_checksum=*/false); +} + +Status DBImpl::VerifyChecksumInternal(const ReadOptions& read_options, + bool use_file_checksum) { + // `bytes_read` stat is enabled based on compile-time support and cannot + // be dynamically toggled. So we do not need to worry about `PerfLevel` + // here, unlike many other `IOStatsContext` / `PerfContext` stats. + uint64_t prev_bytes_read = IOSTATS(bytes_read); + + Status s; + + if (use_file_checksum) { + FileChecksumGenFactory* const file_checksum_gen_factory = + immutable_db_options_.file_checksum_gen_factory.get(); + if (!file_checksum_gen_factory) { + s = Status::InvalidArgument( + "Cannot verify file checksum if options.file_checksum_gen_factory is " + "null"); + return s; + } + } + + // TODO: simplify using GetRefedColumnFamilySet? + std::vector cfd_list; + { + InstrumentedMutexLock l(&mutex_); + for (auto cfd : *versions_->GetColumnFamilySet()) { + if (!cfd->IsDropped() && cfd->initialized()) { + cfd->Ref(); + cfd_list.push_back(cfd); + } + } + } + std::vector sv_list; + for (auto cfd : cfd_list) { + sv_list.push_back(cfd->GetReferencedSuperVersion(this)); + } + + for (auto& sv : sv_list) { + VersionStorageInfo* vstorage = sv->current->storage_info(); + ColumnFamilyData* cfd = sv->current->cfd(); + Options opts; + if (!use_file_checksum) { + InstrumentedMutexLock l(&mutex_); + opts = Options(BuildDBOptions(immutable_db_options_, mutable_db_options_), + cfd->GetLatestCFOptions()); + } + for (int i = 0; i < vstorage->num_non_empty_levels() && s.ok(); i++) { + for (size_t j = 0; j < vstorage->LevelFilesBrief(i).num_files && s.ok(); + j++) { + const auto& fd_with_krange = vstorage->LevelFilesBrief(i).files[j]; + 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, + fd.GetNumber(), fd.GetPathId()); + if (use_file_checksum) { + s = VerifyFullFileChecksum(fmeta->file_checksum, + fmeta->file_checksum_func_name, fname, + read_options); + } else { + s = ROCKSDB_NAMESPACE::VerifySstFileChecksum( + opts, file_options_, read_options, fname, fd.largest_seqno); + } + RecordTick(stats_, VERIFY_CHECKSUM_READ_BYTES, + IOSTATS(bytes_read) - prev_bytes_read); + prev_bytes_read = IOSTATS(bytes_read); + } + } + + if (s.ok() && use_file_checksum) { + const auto& blob_files = vstorage->GetBlobFiles(); + for (const auto& meta : blob_files) { + assert(meta); + + const uint64_t blob_file_number = meta->GetBlobFileNumber(); + + const std::string blob_file_name = BlobFileName( + cfd->ioptions()->cf_paths.front().path, blob_file_number); + s = VerifyFullFileChecksum(meta->GetChecksumValue(), + meta->GetChecksumMethod(), blob_file_name, + read_options); + RecordTick(stats_, VERIFY_CHECKSUM_READ_BYTES, + IOSTATS(bytes_read) - prev_bytes_read); + prev_bytes_read = IOSTATS(bytes_read); + if (!s.ok()) { + break; + } + } + } + if (!s.ok()) { + break; + } + } + + bool defer_purge = immutable_db_options().avoid_unnecessary_blocking_io; + { + InstrumentedMutexLock l(&mutex_); + for (auto sv : sv_list) { + if (sv && sv->Unref()) { + sv->Cleanup(); + if (defer_purge) { + AddSuperVersionsToFreeQueue(sv); + } else { + delete sv; + } + } + } + if (defer_purge) { + SchedulePurge(); + } + for (auto cfd : cfd_list) { + cfd->UnrefAndTryDelete(); + } + } + RecordTick(stats_, VERIFY_CHECKSUM_READ_BYTES, + IOSTATS(bytes_read) - prev_bytes_read); + return s; +} + +Status DBImpl::VerifyFullFileChecksum(const std::string& file_checksum_expected, + const std::string& func_name_expected, + const std::string& fname, + const ReadOptions& read_options) { + Status s; + if (file_checksum_expected == kUnknownFileChecksum) { + return s; + } + std::string file_checksum; + std::string func_name; + s = ROCKSDB_NAMESPACE::GenerateOneFileChecksum( + fs_.get(), fname, immutable_db_options_.file_checksum_gen_factory.get(), + func_name_expected, &file_checksum, &func_name, + read_options.readahead_size, immutable_db_options_.allow_mmap_reads, + io_tracer_, immutable_db_options_.rate_limiter.get(), + read_options.rate_limiter_priority); + if (s.ok()) { + assert(func_name_expected == func_name); + if (file_checksum != file_checksum_expected) { + std::ostringstream oss; + oss << fname << " file checksum mismatch, "; + oss << "expecting " + << Slice(file_checksum_expected).ToString(/*hex=*/true); + oss << ", but actual " << Slice(file_checksum).ToString(/*hex=*/true); + s = Status::Corruption(oss.str()); + TEST_SYNC_POINT_CALLBACK("DBImpl::VerifyFullFileChecksum:mismatch", &s); + } + } + return s; +} + +void DBImpl::NotifyOnExternalFileIngested( + ColumnFamilyData* cfd, const ExternalSstFileIngestionJob& ingestion_job) { + if (immutable_db_options_.listeners.empty()) { + return; + } + + for (const IngestedFileInfo& f : ingestion_job.files_to_ingest()) { + ExternalFileIngestionInfo info; + info.cf_name = cfd->GetName(); + info.external_file_path = f.external_file_path; + info.internal_file_path = f.internal_file_path; + info.global_seqno = f.assigned_seqno; + info.table_properties = f.table_properties; + for (auto listener : immutable_db_options_.listeners) { + listener->OnExternalFileIngested(this, info); + } + } +} + +void DBImpl::WaitForIngestFile() { + mutex_.AssertHeld(); + while (num_running_ingest_file_ > 0) { + bg_cv_.Wait(); + } +} + +Status DBImpl::StartTrace(const TraceOptions& trace_options, + std::unique_ptr&& trace_writer) { + InstrumentedMutexLock lock(&trace_mutex_); + tracer_.reset(new Tracer(immutable_db_options_.clock, trace_options, + std::move(trace_writer))); + return Status::OK(); +} + +Status DBImpl::EndTrace() { + InstrumentedMutexLock lock(&trace_mutex_); + Status s; + if (tracer_ != nullptr) { + s = tracer_->Close(); + tracer_.reset(); + } else { + s = Status::IOError("No trace file to close"); + } + return s; +} + +Status DBImpl::NewDefaultReplayer( + const std::vector& handles, + std::unique_ptr&& reader, + std::unique_ptr* replayer) { + replayer->reset(new ReplayerImpl(this, handles, std::move(reader))); + return Status::OK(); +} + +Status DBImpl::StartBlockCacheTrace( + const TraceOptions& trace_options, + std::unique_ptr&& trace_writer) { + BlockCacheTraceOptions block_trace_opts; + block_trace_opts.sampling_frequency = trace_options.sampling_frequency; + + BlockCacheTraceWriterOptions trace_writer_opt; + trace_writer_opt.max_trace_file_size = trace_options.max_trace_file_size; + + std::unique_ptr block_cache_trace_writer = + NewBlockCacheTraceWriter(env_->GetSystemClock().get(), trace_writer_opt, + std::move(trace_writer)); + + return block_cache_tracer_.StartTrace(block_trace_opts, + std::move(block_cache_trace_writer)); +} + +Status DBImpl::StartBlockCacheTrace( + const BlockCacheTraceOptions& trace_options, + std::unique_ptr&& trace_writer) { + return block_cache_tracer_.StartTrace(trace_options, std::move(trace_writer)); +} + +Status DBImpl::EndBlockCacheTrace() { + block_cache_tracer_.EndTrace(); + return Status::OK(); +} + +Status DBImpl::TraceIteratorSeek(const uint32_t& cf_id, const Slice& key, + const Slice& lower_bound, + const Slice upper_bound) { + Status s; + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + s = tracer_->IteratorSeek(cf_id, key, lower_bound, upper_bound); + } + } + return s; +} + +Status DBImpl::TraceIteratorSeekForPrev(const uint32_t& cf_id, const Slice& key, + const Slice& lower_bound, + const Slice upper_bound) { + Status s; + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + s = tracer_->IteratorSeekForPrev(cf_id, key, lower_bound, upper_bound); + } + } + return s; +} + +Status DBImpl::ReserveFileNumbersBeforeIngestion( + ColumnFamilyData* cfd, uint64_t num, + std::unique_ptr::iterator>& pending_output_elem, + uint64_t* next_file_number) { + Status s; + SuperVersionContext dummy_sv_ctx(true /* create_superversion */); + assert(nullptr != next_file_number); + InstrumentedMutexLock l(&mutex_); + if (error_handler_.IsDBStopped()) { + // Do not ingest files when there is a bg_error + return error_handler_.GetBGError(); + } + pending_output_elem.reset(new std::list::iterator( + CaptureCurrentFileNumberInPendingOutputs())); + *next_file_number = versions_->FetchAddFileNumber(static_cast(num)); + auto cf_options = 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, &dummy_edit, &mutex_, + directories_.GetDbDir()); + if (s.ok()) { + InstallSuperVersionAndScheduleWork(cfd, &dummy_sv_ctx, *cf_options); + } + dummy_sv_ctx.Clean(); + return s; +} + +Status DBImpl::GetCreationTimeOfOldestFile(uint64_t* creation_time) { + if (mutable_db_options_.max_open_files == -1) { + uint64_t oldest_time = std::numeric_limits::max(); + for (auto cfd : *versions_->GetColumnFamilySet()) { + if (!cfd->IsDropped()) { + uint64_t ctime; + { + SuperVersion* sv = GetAndRefSuperVersion(cfd); + Version* version = sv->current; + version->GetCreationTimeOfOldestFile(&ctime); + ReturnAndCleanupSuperVersion(cfd, sv); + } + + if (ctime < oldest_time) { + oldest_time = ctime; + } + if (oldest_time == 0) { + break; + } + } + } + *creation_time = oldest_time; + return Status::OK(); + } else { + return Status::NotSupported("This API only works if max_open_files = -1"); + } +} + +void DBImpl::RecordSeqnoToTimeMapping() { + // Get time first then sequence number, so the actual time of seqno is <= + // unix_time recorded + int64_t unix_time = 0; + immutable_db_options_.clock->GetCurrentTime(&unix_time) + .PermitUncheckedError(); // Ignore error + SequenceNumber seqno = GetLatestSequenceNumber(); + bool appended = false; + { + InstrumentedMutexLock l(&mutex_); + appended = seqno_time_mapping_.Append(seqno, unix_time); + } + if (!appended) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "Failed to insert sequence number to time entry: %" PRIu64 + " -> %" PRIu64, + seqno, unix_time); + } +} +#endif // ROCKSDB_LITE + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl.h b/src/rocksdb/db/db_impl/db_impl.h new file mode 100644 index 000000000..725e77c18 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl.h @@ -0,0 +1,2804 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "db/column_family.h" +#include "db/compaction/compaction_iterator.h" +#include "db/compaction/compaction_job.h" +#include "db/error_handler.h" +#include "db/event_helpers.h" +#include "db/external_sst_file_ingestion_job.h" +#include "db/flush_job.h" +#include "db/flush_scheduler.h" +#include "db/import_column_family_job.h" +#include "db/internal_stats.h" +#include "db/log_writer.h" +#include "db/logs_with_prep_tracker.h" +#include "db/memtable_list.h" +#include "db/periodic_task_scheduler.h" +#include "db/post_memtable_callback.h" +#include "db/pre_release_callback.h" +#include "db/range_del_aggregator.h" +#include "db/read_callback.h" +#include "db/seqno_to_time_mapping.h" +#include "db/snapshot_checker.h" +#include "db/snapshot_impl.h" +#include "db/trim_history_scheduler.h" +#include "db/version_edit.h" +#include "db/wal_manager.h" +#include "db/write_controller.h" +#include "db/write_thread.h" +#include "logging/event_logger.h" +#include "monitoring/instrumented_mutex.h" +#include "options/db_options.h" +#include "port/port.h" +#include "rocksdb/db.h" +#include "rocksdb/env.h" +#include "rocksdb/memtablerep.h" +#include "rocksdb/status.h" +#ifndef ROCKSDB_LITE +#include "rocksdb/trace_reader_writer.h" +#endif // ROCKSDB_LITE +#include "rocksdb/transaction_log.h" +#ifndef ROCKSDB_LITE +#include "rocksdb/utilities/replayer.h" +#endif // ROCKSDB_LITE +#include "rocksdb/write_buffer_manager.h" +#include "table/merging_iterator.h" +#include "table/scoped_arena_iterator.h" +#include "util/autovector.h" +#include "util/hash.h" +#include "util/repeatable_thread.h" +#include "util/stop_watch.h" +#include "util/thread_local.h" + +namespace ROCKSDB_NAMESPACE { + +class Arena; +class ArenaWrappedDBIter; +class InMemoryStatsHistoryIterator; +class MemTable; +class PersistentStatsHistoryIterator; +class TableCache; +class TaskLimiterToken; +class Version; +class VersionEdit; +class VersionSet; +class WriteCallback; +struct JobContext; +struct ExternalSstFileInfo; +struct MemTableInfo; + +// Class to maintain directories for all database paths other than main one. +class Directories { + public: + IOStatus SetDirectories(FileSystem* fs, const std::string& dbname, + const std::string& wal_dir, + const std::vector& data_paths); + + FSDirectory* GetDataDir(size_t path_id) const { + assert(path_id < data_dirs_.size()); + FSDirectory* ret_dir = data_dirs_[path_id].get(); + if (ret_dir == nullptr) { + // Should use db_dir_ + return db_dir_.get(); + } + return ret_dir; + } + + FSDirectory* GetWalDir() { + if (wal_dir_) { + return wal_dir_.get(); + } + return db_dir_.get(); + } + + FSDirectory* GetDbDir() { return db_dir_.get(); } + + IOStatus Close(const IOOptions& options, IODebugContext* dbg) { + // close all directories for all database paths + IOStatus s = IOStatus::OK(); + + // The default implementation for Close() in Directory/FSDirectory class + // "NotSupported" status, the upper level interface should be able to + // handle this error so that Close() does not fail after upgrading when + // run on FileSystems that have not implemented `Directory::Close()` or + // `FSDirectory::Close()` yet + + if (db_dir_) { + IOStatus temp_s = db_dir_->Close(options, dbg); + if (!temp_s.ok() && !temp_s.IsNotSupported() && s.ok()) { + s = std::move(temp_s); + } + } + + // Attempt to close everything even if one fails + s.PermitUncheckedError(); + + if (wal_dir_) { + IOStatus temp_s = wal_dir_->Close(options, dbg); + if (!temp_s.ok() && !temp_s.IsNotSupported() && s.ok()) { + s = std::move(temp_s); + } + } + + s.PermitUncheckedError(); + + for (auto& data_dir_ptr : data_dirs_) { + if (data_dir_ptr) { + IOStatus temp_s = data_dir_ptr->Close(options, dbg); + if (!temp_s.ok() && !temp_s.IsNotSupported() && s.ok()) { + s = std::move(temp_s); + } + } + } + + // Ready for caller + s.MustCheck(); + return s; + } + + private: + std::unique_ptr db_dir_; + std::vector> data_dirs_; + std::unique_ptr wal_dir_; +}; + +// While DB is the public interface of RocksDB, and DBImpl is the actual +// class implementing it. It's the entrance of the core RocksdB engine. +// All other DB implementations, e.g. TransactionDB, BlobDB, etc, wrap a +// DBImpl internally. +// Other than functions implementing the DB interface, some public +// functions are there for other internal components to call. For +// example, TransactionDB directly calls DBImpl::WriteImpl() and +// BlobDB directly calls DBImpl::GetImpl(). Some other functions +// are for sub-components to call. For example, ColumnFamilyHandleImpl +// calls DBImpl::FindObsoleteFiles(). +// +// Since it's a very large class, the definition of the functions is +// divided in several db_impl_*.cc files, besides db_impl.cc. +class DBImpl : public DB { + public: + DBImpl(const DBOptions& options, const std::string& dbname, + const bool seq_per_batch = false, const bool batch_per_txn = true, + bool read_only = false); + // No copying allowed + DBImpl(const DBImpl&) = delete; + void operator=(const DBImpl&) = delete; + + virtual ~DBImpl(); + + // ---- Implementations of the DB interface ---- + + using DB::Resume; + Status Resume() override; + + using DB::Put; + Status Put(const WriteOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& value) override; + Status Put(const WriteOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts, const Slice& value) override; + + using DB::PutEntity; + Status PutEntity(const WriteOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns) override; + + using DB::Merge; + Status Merge(const WriteOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& value) override; + Status Merge(const WriteOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts, const Slice& value) override; + + using DB::Delete; + Status Delete(const WriteOptions& options, ColumnFamilyHandle* column_family, + const Slice& key) override; + Status Delete(const WriteOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts) override; + + using DB::SingleDelete; + Status SingleDelete(const WriteOptions& options, + ColumnFamilyHandle* column_family, + const Slice& key) override; + Status SingleDelete(const WriteOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + const Slice& ts) override; + + using DB::DeleteRange; + Status DeleteRange(const WriteOptions& options, + ColumnFamilyHandle* column_family, const Slice& begin_key, + const Slice& end_key) override; + Status DeleteRange(const WriteOptions& options, + ColumnFamilyHandle* column_family, const Slice& begin_key, + const Slice& end_key, const Slice& ts) override; + + using DB::Write; + virtual Status Write(const WriteOptions& options, + WriteBatch* updates) override; + + using DB::Get; + virtual Status Get(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value) override; + virtual Status Get(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value, std::string* timestamp) override; + + using DB::GetEntity; + Status GetEntity(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableWideColumns* columns) override; + + using DB::GetMergeOperands; + Status GetMergeOperands(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* merge_operands, + GetMergeOperandsOptions* get_merge_operands_options, + int* number_of_operands) override { + GetImplOptions get_impl_options; + get_impl_options.column_family = column_family; + get_impl_options.merge_operands = merge_operands; + get_impl_options.get_merge_operands_options = get_merge_operands_options; + get_impl_options.number_of_operands = number_of_operands; + get_impl_options.get_value = false; + return GetImpl(options, key, get_impl_options); + } + + using DB::MultiGet; + virtual std::vector MultiGet( + const ReadOptions& options, + const std::vector& column_family, + const std::vector& keys, + std::vector* values) override; + virtual std::vector MultiGet( + const ReadOptions& options, + const std::vector& column_family, + const std::vector& keys, std::vector* values, + std::vector* timestamps) override; + + // This MultiGet is a batched version, which may be faster than calling Get + // multiple times, especially if the keys have some spatial locality that + // enables them to be queried in the same SST files/set of files. The larger + // the batch size, the more scope for batching and performance improvement + // The values and statuses parameters are arrays with number of elements + // equal to keys.size(). This allows the storage for those to be alloacted + // by the caller on the stack for small batches + virtual void MultiGet(const ReadOptions& options, + ColumnFamilyHandle* column_family, + const size_t num_keys, const Slice* keys, + PinnableSlice* values, Status* statuses, + const bool sorted_input = false) override; + virtual void MultiGet(const ReadOptions& options, + ColumnFamilyHandle* column_family, + const size_t num_keys, const Slice* keys, + PinnableSlice* values, std::string* timestamps, + Status* statuses, + const bool sorted_input = false) override; + + virtual void MultiGet(const ReadOptions& options, const size_t num_keys, + ColumnFamilyHandle** column_families, const Slice* keys, + PinnableSlice* values, Status* statuses, + const bool sorted_input = false) override; + virtual void MultiGet(const ReadOptions& options, const size_t num_keys, + ColumnFamilyHandle** column_families, const Slice* keys, + PinnableSlice* values, std::string* timestamps, + Status* statuses, + const bool sorted_input = false) override; + + virtual void MultiGetWithCallback( + const ReadOptions& options, ColumnFamilyHandle* column_family, + ReadCallback* callback, + autovector* sorted_keys); + + virtual Status CreateColumnFamily(const ColumnFamilyOptions& cf_options, + const std::string& column_family, + ColumnFamilyHandle** handle) override; + virtual Status CreateColumnFamilies( + const ColumnFamilyOptions& cf_options, + const std::vector& column_family_names, + std::vector* handles) override; + virtual Status CreateColumnFamilies( + const std::vector& column_families, + std::vector* handles) override; + virtual Status DropColumnFamily(ColumnFamilyHandle* column_family) override; + virtual Status DropColumnFamilies( + const std::vector& column_families) override; + + // Returns false if key doesn't exist in the database and true if it may. + // If value_found is not passed in as null, then return the value if found in + // memory. On return, if value was found, then value_found will be set to true + // , otherwise false. + using DB::KeyMayExist; + virtual bool KeyMayExist(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + std::string* value, std::string* timestamp, + bool* value_found = nullptr) override; + + using DB::NewIterator; + virtual Iterator* NewIterator(const ReadOptions& options, + ColumnFamilyHandle* column_family) override; + virtual Status NewIterators( + const ReadOptions& options, + const std::vector& column_families, + std::vector* iterators) override; + + virtual const Snapshot* GetSnapshot() override; + virtual void ReleaseSnapshot(const Snapshot* snapshot) override; + // Create a timestamped snapshot. This snapshot can be shared by multiple + // readers. If any of them uses it for write conflict checking, then + // is_write_conflict_boundary is true. For simplicity, set it to true by + // default. + std::pair> CreateTimestampedSnapshot( + SequenceNumber snapshot_seq, uint64_t ts); + std::shared_ptr GetTimestampedSnapshot(uint64_t ts) const; + void ReleaseTimestampedSnapshotsOlderThan( + uint64_t ts, size_t* remaining_total_ss = nullptr); + Status GetTimestampedSnapshots(uint64_t ts_lb, uint64_t ts_ub, + std::vector>& + timestamped_snapshots) const; + + using DB::GetProperty; + virtual bool GetProperty(ColumnFamilyHandle* column_family, + const Slice& property, std::string* value) override; + using DB::GetMapProperty; + virtual bool GetMapProperty( + ColumnFamilyHandle* column_family, const Slice& property, + std::map* value) override; + using DB::GetIntProperty; + virtual bool GetIntProperty(ColumnFamilyHandle* column_family, + const Slice& property, uint64_t* value) override; + using DB::GetAggregatedIntProperty; + virtual bool GetAggregatedIntProperty(const Slice& property, + uint64_t* aggregated_value) override; + using DB::GetApproximateSizes; + virtual Status GetApproximateSizes(const SizeApproximationOptions& options, + ColumnFamilyHandle* column_family, + const Range* range, int n, + uint64_t* sizes) override; + using DB::GetApproximateMemTableStats; + virtual void GetApproximateMemTableStats(ColumnFamilyHandle* column_family, + const Range& range, + uint64_t* const count, + uint64_t* const size) override; + using DB::CompactRange; + virtual Status CompactRange(const CompactRangeOptions& options, + ColumnFamilyHandle* column_family, + const Slice* begin, const Slice* end) override; + + using DB::CompactFiles; + virtual Status CompactFiles( + const CompactionOptions& compact_options, + ColumnFamilyHandle* column_family, + const std::vector& input_file_names, const int output_level, + const int output_path_id = -1, + std::vector* const output_file_names = nullptr, + CompactionJobInfo* compaction_job_info = nullptr) override; + + virtual Status PauseBackgroundWork() override; + virtual Status ContinueBackgroundWork() override; + + virtual Status EnableAutoCompaction( + const std::vector& column_family_handles) override; + + virtual void EnableManualCompaction() override; + virtual void DisableManualCompaction() override; + + using DB::SetOptions; + Status SetOptions( + ColumnFamilyHandle* column_family, + const std::unordered_map& options_map) override; + + virtual Status SetDBOptions( + const std::unordered_map& options_map) override; + + using DB::NumberLevels; + virtual int NumberLevels(ColumnFamilyHandle* column_family) override; + using DB::MaxMemCompactionLevel; + virtual int MaxMemCompactionLevel(ColumnFamilyHandle* column_family) override; + using DB::Level0StopWriteTrigger; + virtual int Level0StopWriteTrigger( + ColumnFamilyHandle* column_family) override; + virtual const std::string& GetName() const override; + virtual Env* GetEnv() const override; + virtual FileSystem* GetFileSystem() const override; + using DB::GetOptions; + virtual Options GetOptions(ColumnFamilyHandle* column_family) const override; + using DB::GetDBOptions; + virtual DBOptions GetDBOptions() const override; + using DB::Flush; + virtual Status Flush(const FlushOptions& options, + ColumnFamilyHandle* column_family) override; + virtual Status Flush( + const FlushOptions& options, + const std::vector& column_families) override; + virtual Status FlushWAL(bool sync) override; + bool WALBufferIsEmpty(bool lock = true); + virtual Status SyncWAL() override; + virtual Status LockWAL() override; + virtual Status UnlockWAL() override; + + virtual SequenceNumber GetLatestSequenceNumber() const override; + + // IncreaseFullHistoryTsLow(ColumnFamilyHandle*, std::string) will acquire + // and release db_mutex + Status IncreaseFullHistoryTsLow(ColumnFamilyHandle* column_family, + std::string ts_low) override; + + // GetFullHistoryTsLow(ColumnFamilyHandle*, std::string*) will acquire and + // release db_mutex + Status GetFullHistoryTsLow(ColumnFamilyHandle* column_family, + std::string* ts_low) override; + + virtual Status GetDbIdentity(std::string& identity) const override; + + virtual Status GetDbIdentityFromIdentityFile(std::string* identity) const; + + virtual Status GetDbSessionId(std::string& session_id) const override; + + ColumnFamilyHandle* DefaultColumnFamily() const override; + + ColumnFamilyHandle* PersistentStatsColumnFamily() const; + + virtual Status Close() override; + + virtual Status DisableFileDeletions() override; + + virtual Status EnableFileDeletions(bool force) override; + + virtual bool IsFileDeletionsEnabled() const; + + Status GetStatsHistory( + uint64_t start_time, uint64_t end_time, + std::unique_ptr* stats_iterator) override; + +#ifndef ROCKSDB_LITE + using DB::ResetStats; + virtual Status ResetStats() override; + // All the returned filenames start with "/" + virtual Status GetLiveFiles(std::vector&, + uint64_t* manifest_file_size, + bool flush_memtable = true) override; + virtual Status GetSortedWalFiles(VectorLogPtr& files) override; + virtual Status GetCurrentWalFile( + std::unique_ptr* current_log_file) override; + virtual Status GetCreationTimeOfOldestFile( + uint64_t* creation_time) override; + + virtual Status GetUpdatesSince( + SequenceNumber seq_number, std::unique_ptr* iter, + const TransactionLogIterator::ReadOptions& read_options = + TransactionLogIterator::ReadOptions()) override; + virtual Status DeleteFile(std::string name) override; + Status DeleteFilesInRanges(ColumnFamilyHandle* column_family, + const RangePtr* ranges, size_t n, + bool include_end = true); + + virtual void GetLiveFilesMetaData( + std::vector* metadata) override; + + virtual Status GetLiveFilesChecksumInfo( + FileChecksumList* checksum_list) override; + + virtual Status GetLiveFilesStorageInfo( + const LiveFilesStorageInfoOptions& opts, + std::vector* files) override; + + // Obtains the meta data of the specified column family of the DB. + // TODO(yhchiang): output parameter is placed in the end in this codebase. + virtual void GetColumnFamilyMetaData(ColumnFamilyHandle* column_family, + ColumnFamilyMetaData* metadata) override; + + void GetAllColumnFamilyMetaData( + std::vector* metadata) override; + + Status SuggestCompactRange(ColumnFamilyHandle* column_family, + const Slice* begin, const Slice* end) override; + + Status PromoteL0(ColumnFamilyHandle* column_family, + int target_level) override; + + using DB::IngestExternalFile; + virtual Status IngestExternalFile( + ColumnFamilyHandle* column_family, + const std::vector& external_files, + const IngestExternalFileOptions& ingestion_options) override; + + using DB::IngestExternalFiles; + virtual Status IngestExternalFiles( + const std::vector& args) override; + + using DB::CreateColumnFamilyWithImport; + virtual Status CreateColumnFamilyWithImport( + const ColumnFamilyOptions& options, const std::string& column_family_name, + const ImportColumnFamilyOptions& import_options, + const ExportImportFilesMetaData& metadata, + ColumnFamilyHandle** handle) override; + + using DB::VerifyFileChecksums; + Status VerifyFileChecksums(const ReadOptions& read_options) override; + + using DB::VerifyChecksum; + virtual Status VerifyChecksum(const ReadOptions& /*read_options*/) override; + // Verify the checksums of files in db. Currently only tables are checked. + // + // read_options: controls file I/O behavior, e.g. read ahead size while + // reading all the live table files. + // + // use_file_checksum: if false, verify the block checksums of all live table + // in db. Otherwise, obtain the file checksums and compare + // with the MANIFEST. Currently, file checksums are + // recomputed by reading all table files. + // + // Returns: OK if there is no file whose file or block checksum mismatches. + Status VerifyChecksumInternal(const ReadOptions& read_options, + bool use_file_checksum); + + Status VerifyFullFileChecksum(const std::string& file_checksum_expected, + const std::string& func_name_expected, + const std::string& fpath, + const ReadOptions& read_options); + + using DB::StartTrace; + virtual Status StartTrace( + const TraceOptions& options, + std::unique_ptr&& trace_writer) override; + + using DB::EndTrace; + virtual Status EndTrace() override; + + using DB::NewDefaultReplayer; + virtual Status NewDefaultReplayer( + const std::vector& handles, + std::unique_ptr&& reader, + std::unique_ptr* replayer) override; + + using DB::StartBlockCacheTrace; + Status StartBlockCacheTrace( + const TraceOptions& trace_options, + std::unique_ptr&& trace_writer) override; + + Status StartBlockCacheTrace( + const BlockCacheTraceOptions& options, + std::unique_ptr&& trace_writer) override; + + using DB::EndBlockCacheTrace; + Status EndBlockCacheTrace() override; + + using DB::StartIOTrace; + Status StartIOTrace(const TraceOptions& options, + std::unique_ptr&& trace_writer) override; + + using DB::EndIOTrace; + Status EndIOTrace() override; + + using DB::GetPropertiesOfAllTables; + virtual Status GetPropertiesOfAllTables( + ColumnFamilyHandle* column_family, + TablePropertiesCollection* props) override; + virtual Status GetPropertiesOfTablesInRange( + ColumnFamilyHandle* column_family, const Range* range, std::size_t n, + TablePropertiesCollection* props) override; + +#endif // ROCKSDB_LITE + + // ---- End of implementations of the DB interface ---- + SystemClock* GetSystemClock() const; + + struct GetImplOptions { + ColumnFamilyHandle* column_family = nullptr; + PinnableSlice* value = nullptr; + PinnableWideColumns* columns = nullptr; + std::string* timestamp = nullptr; + bool* value_found = nullptr; + ReadCallback* callback = nullptr; + bool* is_blob_index = nullptr; + // If true return value associated with key via value pointer else return + // all merge operands for key via merge_operands pointer + bool get_value = true; + // Pointer to an array of size + // get_merge_operands_options.expected_max_number_of_operands allocated by + // user + PinnableSlice* merge_operands = nullptr; + GetMergeOperandsOptions* get_merge_operands_options = nullptr; + int* number_of_operands = nullptr; + }; + + // Function that Get and KeyMayExist call with no_io true or false + // Note: 'value_found' from KeyMayExist propagates here + // This function is also called by GetMergeOperands + // If get_impl_options.get_value = true get value associated with + // get_impl_options.key via get_impl_options.value + // If get_impl_options.get_value = false get merge operands associated with + // get_impl_options.key via get_impl_options.merge_operands + Status GetImpl(const ReadOptions& options, const Slice& key, + GetImplOptions& get_impl_options); + + // If `snapshot` == kMaxSequenceNumber, set a recent one inside the file. + ArenaWrappedDBIter* NewIteratorImpl(const ReadOptions& options, + ColumnFamilyData* cfd, + SequenceNumber snapshot, + ReadCallback* read_callback, + bool expose_blob_index = false, + bool allow_refresh = true); + + virtual SequenceNumber GetLastPublishedSequence() const { + if (last_seq_same_as_publish_seq_) { + return versions_->LastSequence(); + } else { + return versions_->LastPublishedSequence(); + } + } + + // REQUIRES: joined the main write queue if two_write_queues is disabled, and + // the second write queue otherwise. + virtual void SetLastPublishedSequence(SequenceNumber seq); + // Returns LastSequence in last_seq_same_as_publish_seq_ + // mode and LastAllocatedSequence otherwise. This is useful when visiblility + // depends also on data written to the WAL but not to the memtable. + SequenceNumber TEST_GetLastVisibleSequence() const; + +#ifndef ROCKSDB_LITE + // Similar to Write() but will call the callback once on the single write + // thread to determine whether it is safe to perform the write. + virtual Status WriteWithCallback(const WriteOptions& write_options, + WriteBatch* my_batch, + WriteCallback* callback); + + // Returns the sequence number that is guaranteed to be smaller than or equal + // to the sequence number of any key that could be inserted into the current + // memtables. It can then be assumed that any write with a larger(or equal) + // sequence number will be present in this memtable or a later memtable. + // + // If the earliest sequence number could not be determined, + // kMaxSequenceNumber will be returned. + // + // If include_history=true, will also search Memtables in MemTableList + // History. + SequenceNumber GetEarliestMemTableSequenceNumber(SuperVersion* sv, + bool include_history); + + // For a given key, check to see if there are any records for this key + // in the memtables, including memtable history. If cache_only is false, + // SST files will also be checked. + // + // `key` should NOT have user-defined timestamp appended to user key even if + // timestamp is enabled. + // + // If a key is found, *found_record_for_key will be set to true and + // *seq will be set to the stored sequence number for the latest + // operation on this key or kMaxSequenceNumber if unknown. If user-defined + // timestamp is enabled for this column family and timestamp is not nullptr, + // then *timestamp will be set to the stored timestamp for the latest + // operation on this key. + // If no key is found, *found_record_for_key will be set to false. + // + // Note: If cache_only=false, it is possible for *seq to be set to 0 if + // the sequence number has been cleared from the record. If the caller is + // holding an active db snapshot, we know the missing sequence must be less + // than the snapshot's sequence number (sequence numbers are only cleared + // when there are no earlier active snapshots). + // + // If NotFound is returned and found_record_for_key is set to false, then no + // record for this key was found. If the caller is holding an active db + // snapshot, we know that no key could have existing after this snapshot + // (since we do not compact keys that have an earlier snapshot). + // + // Only records newer than or at `lower_bound_seq` are guaranteed to be + // returned. Memtables and files may not be checked if it only contains data + // older than `lower_bound_seq`. + // + // Returns OK or NotFound on success, + // other status on unexpected error. + // TODO(andrewkr): this API need to be aware of range deletion operations + Status GetLatestSequenceForKey(SuperVersion* sv, const Slice& key, + bool cache_only, + SequenceNumber lower_bound_seq, + SequenceNumber* seq, std::string* timestamp, + bool* found_record_for_key, + bool* is_blob_index); + + Status TraceIteratorSeek(const uint32_t& cf_id, const Slice& key, + const Slice& lower_bound, const Slice upper_bound); + Status TraceIteratorSeekForPrev(const uint32_t& cf_id, const Slice& key, + const Slice& lower_bound, + const Slice upper_bound); +#endif // ROCKSDB_LITE + + // Similar to GetSnapshot(), but also lets the db know that this snapshot + // will be used for transaction write-conflict checking. The DB can then + // make sure not to compact any keys that would prevent a write-conflict from + // being detected. + const Snapshot* GetSnapshotForWriteConflictBoundary(); + + // checks if all live files exist on file system and that their file sizes + // match to our in-memory records + virtual Status CheckConsistency(); + + // max_file_num_to_ignore allows bottom level compaction to filter out newly + // compacted SST files. Setting max_file_num_to_ignore to kMaxUint64 will + // disable the filtering + Status RunManualCompaction(ColumnFamilyData* cfd, int input_level, + int output_level, + const CompactRangeOptions& compact_range_options, + const Slice* begin, const Slice* end, + bool exclusive, bool disallow_trivial_move, + uint64_t max_file_num_to_ignore, + const std::string& trim_ts); + + // Return an internal iterator over the current state of the database. + // The keys of this iterator are internal keys (see format.h). + // The returned iterator should be deleted when no longer needed. + // If allow_unprepared_value is true, the returned iterator may defer reading + // the value and so will require PrepareValue() to be called before value(); + // allow_unprepared_value = false is convenient when this optimization is not + // useful, e.g. when reading the whole column family. + // + // read_options.ignore_range_deletions determines whether range tombstones are + // processed in the returned interator internally, i.e., whether range + // tombstone covered keys are in this iterator's output. + // @param read_options Must outlive the returned iterator. + InternalIterator* NewInternalIterator( + const ReadOptions& read_options, Arena* arena, SequenceNumber sequence, + ColumnFamilyHandle* column_family = nullptr, + bool allow_unprepared_value = false); + + // Note: to support DB iterator refresh, memtable range tombstones in the + // underlying merging iterator needs to be refreshed. If db_iter is not + // nullptr, db_iter->SetMemtableRangetombstoneIter() is called with the + // memtable range tombstone iterator used by the underlying merging iterator. + // This range tombstone iterator can be refreshed later by db_iter. + // @param read_options Must outlive the returned iterator. + InternalIterator* NewInternalIterator(const ReadOptions& read_options, + ColumnFamilyData* cfd, + SuperVersion* super_version, + Arena* arena, SequenceNumber sequence, + bool allow_unprepared_value, + ArenaWrappedDBIter* db_iter = nullptr); + + LogsWithPrepTracker* logs_with_prep_tracker() { + return &logs_with_prep_tracker_; + } + + struct BGJobLimits { + int max_flushes; + int max_compactions; + }; + // Returns maximum background flushes and compactions allowed to be scheduled + BGJobLimits GetBGJobLimits() const; + // Need a static version that can be called during SanitizeOptions(). + static BGJobLimits GetBGJobLimits(int max_background_flushes, + int max_background_compactions, + int max_background_jobs, + bool parallelize_compactions); + + // move logs pending closing from job_context to the DB queue and + // schedule a purge + void ScheduleBgLogWriterClose(JobContext* job_context); + + uint64_t MinLogNumberToKeep(); + + // Returns the lower bound file number for SSTs that won't be deleted, even if + // they're obsolete. This lower bound is used internally to prevent newly + // created flush/compaction output files from being deleted before they're + // installed. This technique avoids the need for tracking the exact numbers of + // files pending creation, although it prevents more files than necessary from + // being deleted. + uint64_t MinObsoleteSstNumberToKeep(); + + // Returns the list of live files in 'live' and the list + // of all files in the filesystem in 'candidate_files'. + // If force == false and the last call was less than + // db_options_.delete_obsolete_files_period_micros microseconds ago, + // it will not fill up the job_context + void FindObsoleteFiles(JobContext* job_context, bool force, + bool no_full_scan = false); + + // Diffs the files listed in filenames and those that do not + // belong to live files are possibly removed. Also, removes all the + // files in sst_delete_files and log_delete_files. + // It is not necessary to hold the mutex when invoking this method. + // If FindObsoleteFiles() was run, we need to also run + // PurgeObsoleteFiles(), even if disable_delete_obsolete_files_ is true + void PurgeObsoleteFiles(JobContext& background_contet, + bool schedule_only = false); + + // Schedule a background job to actually delete obsolete files. + void SchedulePurge(); + + const SnapshotList& snapshots() const { return snapshots_; } + + // load list of snapshots to `snap_vector` that is no newer than `max_seq` + // in ascending order. + // `oldest_write_conflict_snapshot` is filled with the oldest snapshot + // which satisfies SnapshotImpl.is_write_conflict_boundary_ = true. + void LoadSnapshots(std::vector* snap_vector, + SequenceNumber* oldest_write_conflict_snapshot, + const SequenceNumber& max_seq) const { + InstrumentedMutexLock l(mutex()); + snapshots().GetAll(snap_vector, oldest_write_conflict_snapshot, max_seq); + } + + const ImmutableDBOptions& immutable_db_options() const { + return immutable_db_options_; + } + + // Cancel all background jobs, including flush, compaction, background + // purging, stats dumping threads, etc. If `wait` = true, wait for the + // running jobs to abort or finish before returning. Otherwise, only + // sends the signals. + void CancelAllBackgroundWork(bool wait); + + // Find Super version and reference it. Based on options, it might return + // the thread local cached one. + // Call ReturnAndCleanupSuperVersion() when it is no longer needed. + SuperVersion* GetAndRefSuperVersion(ColumnFamilyData* cfd); + + // Similar to the previous function but looks up based on a column family id. + // nullptr will be returned if this column family no longer exists. + // REQUIRED: this function should only be called on the write thread or if the + // mutex is held. + SuperVersion* GetAndRefSuperVersion(uint32_t column_family_id); + + // Un-reference the super version and clean it up if it is the last reference. + void CleanupSuperVersion(SuperVersion* sv); + + // Un-reference the super version and return it to thread local cache if + // needed. If it is the last reference of the super version. Clean it up + // after un-referencing it. + void ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd, SuperVersion* sv); + + // Similar to the previous function but looks up based on a column family id. + // nullptr will be returned if this column family no longer exists. + // REQUIRED: this function should only be called on the write thread. + void ReturnAndCleanupSuperVersion(uint32_t colun_family_id, SuperVersion* sv); + + // REQUIRED: this function should only be called on the write thread or if the + // mutex is held. Return value only valid until next call to this function or + // mutex is released. + ColumnFamilyHandle* GetColumnFamilyHandle(uint32_t column_family_id); + + // Same as above, should called without mutex held and not on write thread. + std::unique_ptr GetColumnFamilyHandleUnlocked( + uint32_t column_family_id); + + // Returns the number of currently running flushes. + // REQUIREMENT: mutex_ must be held when calling this function. + int num_running_flushes() { + mutex_.AssertHeld(); + return num_running_flushes_; + } + + // Returns the number of currently running compactions. + // REQUIREMENT: mutex_ must be held when calling this function. + int num_running_compactions() { + mutex_.AssertHeld(); + return num_running_compactions_; + } + + const WriteController& write_controller() { return write_controller_; } + + // hollow transactions shell used for recovery. + // these will then be passed to TransactionDB so that + // locks can be reacquired before writing can resume. + struct RecoveredTransaction { + std::string name_; + bool unprepared_; + + struct BatchInfo { + uint64_t log_number_; + // TODO(lth): For unprepared, the memory usage here can be big for + // unprepared transactions. This is only useful for rollbacks, and we + // can in theory just keep keyset for that. + WriteBatch* batch_; + // Number of sub-batches. A new sub-batch is created if txn attempts to + // insert a duplicate key,seq to memtable. This is currently used in + // WritePreparedTxn/WriteUnpreparedTxn. + size_t batch_cnt_; + }; + + // This maps the seq of the first key in the batch to BatchInfo, which + // contains WriteBatch and other information relevant to the batch. + // + // For WriteUnprepared, batches_ can have size greater than 1, but for + // other write policies, it must be of size 1. + std::map batches_; + + explicit RecoveredTransaction(const uint64_t log, const std::string& name, + WriteBatch* batch, SequenceNumber seq, + size_t batch_cnt, bool unprepared) + : name_(name), unprepared_(unprepared) { + batches_[seq] = {log, batch, batch_cnt}; + } + + ~RecoveredTransaction() { + for (auto& it : batches_) { + delete it.second.batch_; + } + } + + void AddBatch(SequenceNumber seq, uint64_t log_number, WriteBatch* batch, + size_t batch_cnt, bool unprepared) { + assert(batches_.count(seq) == 0); + batches_[seq] = {log_number, batch, batch_cnt}; + // Prior state must be unprepared, since the prepare batch must be the + // last batch. + assert(unprepared_); + unprepared_ = unprepared; + } + }; + + bool allow_2pc() const { return immutable_db_options_.allow_2pc; } + + std::unordered_map + recovered_transactions() { + return recovered_transactions_; + } + + RecoveredTransaction* GetRecoveredTransaction(const std::string& name) { + auto it = recovered_transactions_.find(name); + if (it == recovered_transactions_.end()) { + return nullptr; + } else { + return it->second; + } + } + + void InsertRecoveredTransaction(const uint64_t log, const std::string& name, + WriteBatch* batch, SequenceNumber seq, + size_t batch_cnt, bool unprepared_batch) { + // For WriteUnpreparedTxn, InsertRecoveredTransaction is called multiple + // times for every unprepared batch encountered during recovery. + // + // If the transaction is prepared, then the last call to + // InsertRecoveredTransaction will have unprepared_batch = false. + auto rtxn = recovered_transactions_.find(name); + if (rtxn == recovered_transactions_.end()) { + recovered_transactions_[name] = new RecoveredTransaction( + log, name, batch, seq, batch_cnt, unprepared_batch); + } else { + rtxn->second->AddBatch(seq, log, batch, batch_cnt, unprepared_batch); + } + logs_with_prep_tracker_.MarkLogAsContainingPrepSection(log); + } + + void DeleteRecoveredTransaction(const std::string& name) { + auto it = recovered_transactions_.find(name); + assert(it != recovered_transactions_.end()); + auto* trx = it->second; + recovered_transactions_.erase(it); + for (const auto& info : trx->batches_) { + logs_with_prep_tracker_.MarkLogAsHavingPrepSectionFlushed( + info.second.log_number_); + } + delete trx; + } + + void DeleteAllRecoveredTransactions() { + for (auto it = recovered_transactions_.begin(); + it != recovered_transactions_.end(); ++it) { + delete it->second; + } + recovered_transactions_.clear(); + } + + void AddToLogsToFreeQueue(log::Writer* log_writer) { + mutex_.AssertHeld(); + logs_to_free_queue_.push_back(log_writer); + } + + void AddSuperVersionsToFreeQueue(SuperVersion* sv) { + superversions_to_free_queue_.push_back(sv); + } + + void SetSnapshotChecker(SnapshotChecker* snapshot_checker); + + // Fill JobContext with snapshot information needed by flush and compaction. + void GetSnapshotContext(JobContext* job_context, + std::vector* snapshot_seqs, + SequenceNumber* earliest_write_conflict_snapshot, + SnapshotChecker** snapshot_checker); + + // Not thread-safe. + void SetRecoverableStatePreReleaseCallback(PreReleaseCallback* callback); + + InstrumentedMutex* mutex() const { return &mutex_; } + + // Initialize a brand new DB. The DB directory is expected to be empty before + // calling it. Push new manifest file name into `new_filenames`. + Status NewDB(std::vector* new_filenames); + + // This is to be used only by internal rocksdb classes. + static Status Open(const DBOptions& db_options, const std::string& name, + const std::vector& column_families, + std::vector* handles, DB** dbptr, + const bool seq_per_batch, const bool batch_per_txn); + + static IOStatus CreateAndNewDirectory( + FileSystem* fs, const std::string& dirname, + std::unique_ptr* directory); + + // find stats map from stats_history_ with smallest timestamp in + // the range of [start_time, end_time) + bool FindStatsByTime(uint64_t start_time, uint64_t end_time, + uint64_t* new_time, + std::map* stats_map); + + // Print information of all tombstones of all iterators to the std::string + // This is only used by ldb. The output might be capped. Tombstones + // printed out are not guaranteed to be in any order. + Status TablesRangeTombstoneSummary(ColumnFamilyHandle* column_family, + int max_entries_to_print, + std::string* out_str); + + VersionSet* GetVersionSet() const { return versions_.get(); } + + // Wait for any compaction + // We add a bool parameter to wait for unscheduledCompactions_ == 0, but this + // is only for the special test of CancelledCompactions + Status WaitForCompact(bool waitUnscheduled = false); + +#ifndef NDEBUG + // Compact any files in the named level that overlap [*begin, *end] + Status TEST_CompactRange(int level, const Slice* begin, const Slice* end, + ColumnFamilyHandle* column_family = nullptr, + bool disallow_trivial_move = false); + + Status TEST_SwitchWAL(); + + bool TEST_UnableToReleaseOldestLog() { return unable_to_release_oldest_log_; } + + bool TEST_IsLogGettingFlushed() { + return alive_log_files_.begin()->getting_flushed; + } + + Status TEST_SwitchMemtable(ColumnFamilyData* cfd = nullptr); + + // Force current memtable contents to be flushed. + Status TEST_FlushMemTable(bool wait = true, bool allow_write_stall = false, + ColumnFamilyHandle* cfh = nullptr); + + Status TEST_FlushMemTable(ColumnFamilyData* cfd, + const FlushOptions& flush_opts); + + // Flush (multiple) ColumnFamilyData without using ColumnFamilyHandle. This + // is because in certain cases, we can flush column families, wait for the + // flush to complete, but delete the column family handle before the wait + // finishes. For example in CompactRange. + Status TEST_AtomicFlushMemTables(const autovector& cfds, + const FlushOptions& flush_opts); + + // Wait for background threads to complete scheduled work. + Status TEST_WaitForBackgroundWork(); + + // Wait for memtable compaction + Status TEST_WaitForFlushMemTable(ColumnFamilyHandle* column_family = nullptr); + + // Wait for any compaction + // We add a bool parameter to wait for unscheduledCompactions_ == 0, but this + // is only for the special test of CancelledCompactions + Status TEST_WaitForCompact(bool waitUnscheduled = false); + + // Wait for any background purge + Status TEST_WaitForPurge(); + + // Get the background error status + Status TEST_GetBGError(); + + // Return the maximum overlapping data (in bytes) at next level for any + // file at a level >= 1. + uint64_t TEST_MaxNextLevelOverlappingBytes( + ColumnFamilyHandle* column_family = nullptr); + + // Return the current manifest file no. + uint64_t TEST_Current_Manifest_FileNo(); + + // Returns the number that'll be assigned to the next file that's created. + uint64_t TEST_Current_Next_FileNo(); + + // get total level0 file size. Only for testing. + uint64_t TEST_GetLevel0TotalSize(); + + void TEST_GetFilesMetaData( + ColumnFamilyHandle* column_family, + std::vector>* metadata, + std::vector>* blob_metadata = nullptr); + + void TEST_LockMutex(); + + void TEST_UnlockMutex(); + + // REQUIRES: mutex locked + void* TEST_BeginWrite(); + + // REQUIRES: mutex locked + // pass the pointer that you got from TEST_BeginWrite() + void TEST_EndWrite(void* w); + + uint64_t TEST_MaxTotalInMemoryState() const { + return max_total_in_memory_state_; + } + + size_t TEST_LogsToFreeSize(); + + uint64_t TEST_LogfileNumber(); + + uint64_t TEST_total_log_size() const { return total_log_size_; } + + // Returns column family name to ImmutableCFOptions map. + Status TEST_GetAllImmutableCFOptions( + std::unordered_map* iopts_map); + + // Return the lastest MutableCFOptions of a column family + Status TEST_GetLatestMutableCFOptions(ColumnFamilyHandle* column_family, + MutableCFOptions* mutable_cf_options); + + Cache* TEST_table_cache() { return table_cache_.get(); } + + WriteController& TEST_write_controler() { return write_controller_; } + + uint64_t TEST_FindMinLogContainingOutstandingPrep(); + uint64_t TEST_FindMinPrepLogReferencedByMemTable(); + size_t TEST_PreparedSectionCompletedSize(); + size_t TEST_LogsWithPrepSize(); + + int TEST_BGCompactionsAllowed() const; + int TEST_BGFlushesAllowed() const; + size_t TEST_GetWalPreallocateBlockSize(uint64_t write_buffer_size) const; + void TEST_WaitForPeridicTaskRun(std::function callback) const; + SeqnoToTimeMapping TEST_GetSeqnoToTimeMapping() const; + size_t TEST_EstimateInMemoryStatsHistorySize() const; + + uint64_t TEST_GetCurrentLogNumber() const { + InstrumentedMutexLock l(mutex()); + assert(!logs_.empty()); + return logs_.back().number; + } + + const std::unordered_set& TEST_GetFilesGrabbedForPurge() const { + return files_grabbed_for_purge_; + } + +#ifndef ROCKSDB_LITE + const PeriodicTaskScheduler& TEST_GetPeriodicTaskScheduler() const; +#endif // !ROCKSDB_LITE + +#endif // NDEBUG + + // persist stats to column family "_persistent_stats" + void PersistStats(); + + // dump rocksdb.stats to LOG + void DumpStats(); + + // flush LOG out of application buffer + void FlushInfoLog(); + + // record current sequence number to time mapping + void RecordSeqnoToTimeMapping(); + + // Interface to block and signal the DB in case of stalling writes by + // WriteBufferManager. Each DBImpl object contains ptr to WBMStallInterface. + // When DB needs to be blocked or signalled by WriteBufferManager, + // state_ is changed accordingly. + class WBMStallInterface : public StallInterface { + public: + enum State { + BLOCKED = 0, + RUNNING, + }; + + WBMStallInterface() : state_cv_(&state_mutex_) { + MutexLock lock(&state_mutex_); + state_ = State::RUNNING; + } + + void SetState(State state) { + MutexLock lock(&state_mutex_); + state_ = state; + } + + // Change the state_ to State::BLOCKED and wait until its state is + // changed by WriteBufferManager. When stall is cleared, Signal() is + // called to change the state and unblock the DB. + void Block() override { + MutexLock lock(&state_mutex_); + while (state_ == State::BLOCKED) { + TEST_SYNC_POINT("WBMStallInterface::BlockDB"); + state_cv_.Wait(); + } + } + + // Called from WriteBufferManager. This function changes the state_ + // to State::RUNNING indicating the stall is cleared and DB can proceed. + void Signal() override { + { + MutexLock lock(&state_mutex_); + state_ = State::RUNNING; + } + state_cv_.Signal(); + } + + private: + // Conditional variable and mutex to block and + // signal the DB during stalling process. + port::Mutex state_mutex_; + port::CondVar state_cv_; + // state represting whether DB is running or blocked because of stall by + // WriteBufferManager. + State state_; + }; + + static void TEST_ResetDbSessionIdGen(); + static std::string GenerateDbSessionId(Env* env); + + bool seq_per_batch() const { return seq_per_batch_; } + + protected: + const std::string dbname_; + // TODO(peterd): unify with VersionSet::db_id_ + std::string db_id_; + // db_session_id_ is an identifier that gets reset + // every time the DB is opened + std::string db_session_id_; + std::unique_ptr versions_; + // Flag to check whether we allocated and own the info log file + bool own_info_log_; + Status init_logger_creation_s_; + const DBOptions initial_db_options_; + Env* const env_; + std::shared_ptr io_tracer_; + const ImmutableDBOptions immutable_db_options_; + FileSystemPtr fs_; + MutableDBOptions mutable_db_options_; + Statistics* stats_; + std::unordered_map + recovered_transactions_; + std::unique_ptr tracer_; + InstrumentedMutex trace_mutex_; + BlockCacheTracer block_cache_tracer_; + + // constant false canceled flag, used when the compaction is not manual + const std::atomic kManualCompactionCanceledFalse_{false}; + + // State below is protected by mutex_ + // With two_write_queues enabled, some of the variables that accessed during + // WriteToWAL need different synchronization: log_empty_, alive_log_files_, + // logs_, logfile_number_. Refer to the definition of each variable below for + // more description. + // + // `mutex_` can be a hot lock in some workloads, so it deserves dedicated + // cachelines. + mutable CacheAlignedInstrumentedMutex mutex_; + + ColumnFamilyHandleImpl* default_cf_handle_; + InternalStats* default_cf_internal_stats_; + + // table_cache_ provides its own synchronization + std::shared_ptr table_cache_; + + ErrorHandler error_handler_; + + // Unified interface for logging events + EventLogger event_logger_; + + // only used for dynamically adjusting max_total_wal_size. it is a sum of + // [write_buffer_size * max_write_buffer_number] over all column families + std::atomic max_total_in_memory_state_; + + // The options to access storage files + const FileOptions file_options_; + + // Additonal options for compaction and flush + FileOptions file_options_for_compaction_; + + std::unique_ptr column_family_memtables_; + + // Increase the sequence number after writing each batch, whether memtable is + // disabled for that or not. Otherwise the sequence number is increased after + // writing each key into memtable. This implies that when disable_memtable is + // set, the seq is not increased at all. + // + // Default: false + const bool seq_per_batch_; + // This determines during recovery whether we expect one writebatch per + // recovered transaction, or potentially multiple writebatches per + // transaction. For WriteUnprepared, this is set to false, since multiple + // batches can exist per transaction. + // + // Default: true + const bool batch_per_txn_; + + // Each flush or compaction gets its own job id. this counter makes sure + // they're unique + std::atomic next_job_id_; + + std::atomic shutting_down_; + + // RecoveryContext struct stores the context about version edits along + // with corresponding column_family_data and column_family_options. + class RecoveryContext { + public: + ~RecoveryContext() { + for (auto& edit_list : edit_lists_) { + for (auto* edit : edit_list) { + delete edit; + } + } + } + + void UpdateVersionEdits(ColumnFamilyData* cfd, const VersionEdit& edit) { + assert(cfd != nullptr); + if (map_.find(cfd->GetID()) == map_.end()) { + uint32_t size = static_cast(map_.size()); + map_.emplace(cfd->GetID(), size); + cfds_.emplace_back(cfd); + mutable_cf_opts_.emplace_back(cfd->GetLatestMutableCFOptions()); + edit_lists_.emplace_back(autovector()); + } + uint32_t i = map_[cfd->GetID()]; + edit_lists_[i].emplace_back(new VersionEdit(edit)); + } + + std::unordered_map map_; // cf_id to index; + autovector cfds_; + autovector mutable_cf_opts_; + autovector> edit_lists_; + // files_to_delete_ contains sst files + std::unordered_set files_to_delete_; + }; + + // Except in DB::Open(), WriteOptionsFile can only be called when: + // Persist options to options file. + // If need_mutex_lock = false, the method will lock DB mutex. + // If need_enter_write_thread = false, the method will enter write thread. + Status WriteOptionsFile(bool need_mutex_lock, bool need_enter_write_thread); + + Status CompactRangeInternal(const CompactRangeOptions& options, + ColumnFamilyHandle* column_family, + const Slice* begin, const Slice* end, + const std::string& trim_ts); + + // The following two functions can only be called when: + // 1. WriteThread::Writer::EnterUnbatched() is used. + // 2. db_mutex is NOT held + Status RenameTempFileToOptionsFile(const std::string& file_name); + Status DeleteObsoleteOptionsFiles(); + + void NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta, + const MutableCFOptions& mutable_cf_options, + int job_id); + + void NotifyOnFlushCompleted( + ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, + std::list>* flush_jobs_info); + + void NotifyOnCompactionBegin(ColumnFamilyData* cfd, Compaction* c, + const Status& st, + const CompactionJobStats& job_stats, int job_id); + + void NotifyOnCompactionCompleted(ColumnFamilyData* cfd, Compaction* c, + const Status& st, + const CompactionJobStats& job_stats, + int job_id); + void NotifyOnMemTableSealed(ColumnFamilyData* cfd, + const MemTableInfo& mem_table_info); + +#ifndef ROCKSDB_LITE + void NotifyOnExternalFileIngested( + ColumnFamilyData* cfd, const ExternalSstFileIngestionJob& ingestion_job); + + virtual Status FlushForGetLiveFiles(); +#endif // !ROCKSDB_LITE + + void NewThreadStatusCfInfo(ColumnFamilyData* cfd) const; + + void EraseThreadStatusCfInfo(ColumnFamilyData* cfd) const; + + void EraseThreadStatusDbInfo() const; + + // If disable_memtable is set the application logic must guarantee that the + // batch will still be skipped from memtable during the recovery. An excption + // to this is seq_per_batch_ mode, in which since each batch already takes one + // seq, it is ok for the batch to write to memtable during recovery as long as + // it only takes one sequence number: i.e., no duplicate keys. + // In WriteCommitted it is guarnateed since disable_memtable is used for + // prepare batch which will be written to memtable later during the commit, + // and in WritePrepared it is guaranteed since it will be used only for WAL + // markers which will never be written to memtable. If the commit marker is + // accompanied with CommitTimeWriteBatch that is not written to memtable as + // long as it has no duplicate keys, it does not violate the one-seq-per-batch + // policy. + // batch_cnt is expected to be non-zero in seq_per_batch mode and + // indicates the number of sub-patches. A sub-patch is a subset of the write + // batch that does not have duplicate keys. + Status WriteImpl(const WriteOptions& options, WriteBatch* updates, + WriteCallback* callback = nullptr, + uint64_t* log_used = nullptr, uint64_t log_ref = 0, + bool disable_memtable = false, uint64_t* seq_used = nullptr, + size_t batch_cnt = 0, + PreReleaseCallback* pre_release_callback = nullptr, + PostMemTableCallback* post_memtable_callback = nullptr); + + Status PipelinedWriteImpl(const WriteOptions& options, WriteBatch* updates, + WriteCallback* callback = nullptr, + uint64_t* log_used = nullptr, uint64_t log_ref = 0, + bool disable_memtable = false, + uint64_t* seq_used = nullptr); + + // Write only to memtables without joining any write queue + Status UnorderedWriteMemtable(const WriteOptions& write_options, + WriteBatch* my_batch, WriteCallback* callback, + uint64_t log_ref, SequenceNumber seq, + const size_t sub_batch_cnt); + + // Whether the batch requires to be assigned with an order + enum AssignOrder : bool { kDontAssignOrder, kDoAssignOrder }; + // Whether it requires publishing last sequence or not + enum PublishLastSeq : bool { kDontPublishLastSeq, kDoPublishLastSeq }; + + // Join the write_thread to write the batch only to the WAL. It is the + // responsibility of the caller to also write the write batch to the memtable + // if it required. + // + // sub_batch_cnt is expected to be non-zero when assign_order = kDoAssignOrder + // indicating the number of sub-batches in my_batch. A sub-patch is a subset + // of the write batch that does not have duplicate keys. When seq_per_batch is + // not set, each key is a separate sub_batch. Otherwise each duplicate key + // marks start of a new sub-batch. + Status WriteImplWALOnly( + WriteThread* write_thread, const WriteOptions& options, + WriteBatch* updates, WriteCallback* callback, uint64_t* log_used, + const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt, + PreReleaseCallback* pre_release_callback, const AssignOrder assign_order, + const PublishLastSeq publish_last_seq, const bool disable_memtable); + + // write cached_recoverable_state_ to memtable if it is not empty + // The writer must be the leader in write_thread_ and holding mutex_ + Status WriteRecoverableState(); + + // Actual implementation of Close() + Status CloseImpl(); + + // Recover the descriptor from persistent storage. May do a significant + // amount of work to recover recently logged updates. Any changes to + // be made to the descriptor are added to *edit. + // recovered_seq is set to less than kMaxSequenceNumber if the log's tail is + // skipped. + // recovery_ctx stores the context about version edits and all those + // edits are persisted to new Manifest after successfully syncing the new WAL. + virtual Status Recover( + const std::vector& column_families, + bool read_only = false, bool error_if_wal_file_exists = false, + bool error_if_data_exists_in_wals = false, + uint64_t* recovered_seq = nullptr, + RecoveryContext* recovery_ctx = nullptr); + + virtual bool OwnTablesAndLogs() const { return true; } + + // Setup DB identity file, and write DB ID to manifest if necessary. + Status SetupDBId(bool read_only, RecoveryContext* recovery_ctx); + // Assign db_id_ and write DB ID to manifest if necessary. + void SetDBId(std::string&& id, bool read_only, RecoveryContext* recovery_ctx); + + // REQUIRES: db mutex held when calling this function, but the db mutex can + // be released and re-acquired. Db mutex will be held when the function + // returns. + // After recovery, there may be SST files in db/cf paths that are + // not referenced in the MANIFEST (e.g. + // 1. It's best effort recovery; + // 2. The VersionEdits referencing the SST files are appended to + // RecoveryContext, DB crashes when syncing the MANIFEST, the VersionEdits are + // still not synced to MANIFEST during recovery.) + // It stores the SST files to be deleted in RecoveryContext. In the + // meantime, we find out the largest file number present in the paths, and + // bump up the version set's next_file_number_ to be 1 + largest_file_number. + // recovery_ctx stores the context about version edits and files to be + // deleted. All those edits are persisted to new Manifest after successfully + // syncing the new WAL. + Status DeleteUnreferencedSstFiles(RecoveryContext* recovery_ctx); + + // SetDbSessionId() should be called in the constuctor DBImpl() + // to ensure that db_session_id_ gets updated every time the DB is opened + void SetDbSessionId(); + + Status FailIfCfHasTs(const ColumnFamilyHandle* column_family) const; + Status FailIfTsMismatchCf(ColumnFamilyHandle* column_family, const Slice& ts, + bool ts_for_read) const; + + // recovery_ctx stores the context about version edits and + // LogAndApplyForRecovery persist all those edits to new Manifest after + // successfully syncing new WAL. + // LogAndApplyForRecovery should be called only once during recovery and it + // should be called when RocksDB writes to a first new MANIFEST since this + // recovery. + Status LogAndApplyForRecovery(const RecoveryContext& recovery_ctx); + + void InvokeWalFilterIfNeededOnColumnFamilyToWalNumberMap(); + + // Return true to proceed with current WAL record whose content is stored in + // `batch`. Return false to skip current WAL record. + bool InvokeWalFilterIfNeededOnWalRecord(uint64_t wal_number, + const std::string& wal_fname, + log::Reader::Reporter& reporter, + Status& status, bool& stop_replay, + WriteBatch& batch); + + private: + friend class DB; + friend class ErrorHandler; + friend class InternalStats; + friend class PessimisticTransaction; + friend class TransactionBaseImpl; + friend class WriteCommittedTxn; + friend class WritePreparedTxn; + friend class WritePreparedTxnDB; + friend class WriteBatchWithIndex; + friend class WriteUnpreparedTxnDB; + friend class WriteUnpreparedTxn; + +#ifndef ROCKSDB_LITE + friend class ForwardIterator; +#endif + friend struct SuperVersion; + friend class CompactedDBImpl; + friend class DBTest_ConcurrentFlushWAL_Test; + friend class DBTest_MixedSlowdownOptionsStop_Test; + friend class DBCompactionTest_CompactBottomLevelFilesWithDeletions_Test; + friend class DBCompactionTest_CompactionDuringShutdown_Test; + friend class StatsHistoryTest_PersistentStatsCreateColumnFamilies_Test; +#ifndef NDEBUG + friend class DBTest2_ReadCallbackTest_Test; + friend class WriteCallbackPTest_WriteWithCallbackTest_Test; + friend class XFTransactionWriteHandler; + friend class DBBlobIndexTest; + friend class WriteUnpreparedTransactionTest_RecoveryTest_Test; +#endif + + struct CompactionState; + struct PrepickedCompaction; + struct PurgeFileInfo; + + struct WriteContext { + SuperVersionContext superversion_context; + autovector memtables_to_free_; + + explicit WriteContext(bool create_superversion = false) + : superversion_context(create_superversion) {} + + ~WriteContext() { + superversion_context.Clean(); + for (auto& m : memtables_to_free_) { + delete m; + } + } + }; + + struct LogFileNumberSize { + explicit LogFileNumberSize(uint64_t _number) : number(_number) {} + LogFileNumberSize() {} + void AddSize(uint64_t new_size) { size += new_size; } + uint64_t number; + uint64_t size = 0; + bool getting_flushed = false; + }; + + struct LogWriterNumber { + // pass ownership of _writer + LogWriterNumber(uint64_t _number, log::Writer* _writer) + : number(_number), writer(_writer) {} + + log::Writer* ReleaseWriter() { + auto* w = writer; + writer = nullptr; + return w; + } + Status ClearWriter() { + Status s = writer->WriteBuffer(); + delete writer; + writer = nullptr; + return s; + } + + bool IsSyncing() { return getting_synced; } + + uint64_t GetPreSyncSize() { + assert(getting_synced); + return pre_sync_size; + } + + void PrepareForSync() { + assert(!getting_synced); + // Size is expected to be monotonically increasing. + assert(writer->file()->GetFlushedSize() >= pre_sync_size); + getting_synced = true; + pre_sync_size = writer->file()->GetFlushedSize(); + } + + void FinishSync() { + assert(getting_synced); + getting_synced = false; + } + + uint64_t number; + // Visual Studio doesn't support deque's member to be noncopyable because + // of a std::unique_ptr as a member. + log::Writer* writer; // own + + private: + // true for some prefix of logs_ + bool getting_synced = false; + // The size of the file before the sync happens. This amount is guaranteed + // to be persisted even if appends happen during sync so it can be used for + // tracking the synced size in MANIFEST. + uint64_t pre_sync_size = 0; + }; + + struct LogContext { + explicit LogContext(bool need_sync = false) + : need_log_sync(need_sync), need_log_dir_sync(need_sync) {} + bool need_log_sync = false; + bool need_log_dir_sync = false; + log::Writer* writer = nullptr; + LogFileNumberSize* log_file_number_size = nullptr; + }; + + // PurgeFileInfo is a structure to hold information of files to be deleted in + // purge_files_ + struct PurgeFileInfo { + std::string fname; + std::string dir_to_sync; + FileType type; + uint64_t number; + int job_id; + PurgeFileInfo(std::string fn, std::string d, FileType t, uint64_t num, + int jid) + : fname(fn), dir_to_sync(d), type(t), number(num), job_id(jid) {} + }; + + // Argument required by background flush thread. + struct BGFlushArg { + BGFlushArg() + : cfd_(nullptr), max_memtable_id_(0), superversion_context_(nullptr) {} + BGFlushArg(ColumnFamilyData* cfd, uint64_t max_memtable_id, + SuperVersionContext* superversion_context) + : cfd_(cfd), + max_memtable_id_(max_memtable_id), + superversion_context_(superversion_context) {} + + // Column family to flush. + ColumnFamilyData* cfd_; + // Maximum ID of memtable to flush. In this column family, memtables with + // IDs smaller than this value must be flushed before this flush completes. + uint64_t max_memtable_id_; + // Pointer to a SuperVersionContext object. After flush completes, RocksDB + // installs a new superversion for the column family. This operation + // requires a SuperVersionContext object (currently embedded in JobContext). + SuperVersionContext* superversion_context_; + }; + + // Argument passed to flush thread. + struct FlushThreadArg { + DBImpl* db_; + + Env::Priority thread_pri_; + }; + + // Information for a manual compaction + struct ManualCompactionState { + ManualCompactionState(ColumnFamilyData* _cfd, int _input_level, + int _output_level, uint32_t _output_path_id, + bool _exclusive, bool _disallow_trivial_move, + std::atomic* _canceled) + : cfd(_cfd), + input_level(_input_level), + output_level(_output_level), + output_path_id(_output_path_id), + exclusive(_exclusive), + disallow_trivial_move(_disallow_trivial_move), + canceled(_canceled ? *_canceled : canceled_internal_storage) {} + // When _canceled is not provided by ther user, we assign the reference of + // canceled_internal_storage to it to consolidate canceled and + // manual_compaction_paused since DisableManualCompaction() might be + // called + + ColumnFamilyData* cfd; + int input_level; + int output_level; + uint32_t output_path_id; + Status status; + bool done = false; + bool in_progress = false; // compaction request being processed? + bool incomplete = false; // only part of requested range compacted + bool exclusive; // current behavior of only one manual + bool disallow_trivial_move; // Force actual compaction to run + const InternalKey* begin = nullptr; // nullptr means beginning of key range + const InternalKey* end = nullptr; // nullptr means end of key range + InternalKey* manual_end = nullptr; // how far we are compacting + InternalKey tmp_storage; // Used to keep track of compaction progress + InternalKey tmp_storage1; // Used to keep track of compaction progress + + // When the user provides a canceled pointer in CompactRangeOptions, the + // above varaibe is the reference of the user-provided + // `canceled`, otherwise, it is the reference of canceled_internal_storage + std::atomic canceled_internal_storage = false; + std::atomic& canceled; // Compaction canceled pointer reference + }; + struct PrepickedCompaction { + // background compaction takes ownership of `compaction`. + Compaction* compaction; + // caller retains ownership of `manual_compaction_state` as it is reused + // across background compactions. + ManualCompactionState* manual_compaction_state; // nullptr if non-manual + // task limiter token is requested during compaction picking. + std::unique_ptr task_token; + }; + + struct CompactionArg { + // caller retains ownership of `db`. + DBImpl* db; + // background compaction takes ownership of `prepicked_compaction`. + PrepickedCompaction* prepicked_compaction; + Env::Priority compaction_pri_; + }; + + // Initialize the built-in column family for persistent stats. Depending on + // whether on-disk persistent stats have been enabled before, it may either + // create a new column family and column family handle or just a column family + // handle. + // Required: DB mutex held + Status InitPersistStatsColumnFamily(); + + // Persistent Stats column family has two format version key which are used + // for compatibility check. Write format version if it's created for the + // first time, read format version and check compatibility if recovering + // from disk. This function requires DB mutex held at entrance but may + // release and re-acquire DB mutex in the process. + // Required: DB mutex held + Status PersistentStatsProcessFormatVersion(); + + Status ResumeImpl(DBRecoverContext context); + + void MaybeIgnoreError(Status* s) const; + + const Status CreateArchivalDirectory(); + + Status CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options, + const std::string& cf_name, + ColumnFamilyHandle** handle); + + Status DropColumnFamilyImpl(ColumnFamilyHandle* column_family); + + // Delete any unneeded files and stale in-memory entries. + void DeleteObsoleteFiles(); + // Delete obsolete files and log status and information of file deletion + void DeleteObsoleteFileImpl(int job_id, const std::string& fname, + const std::string& path_to_sync, FileType type, + uint64_t number); + + // Background process needs to call + // auto x = CaptureCurrentFileNumberInPendingOutputs() + // auto file_num = versions_->NewFileNumber(); + // + // ReleaseFileNumberFromPendingOutputs(x) + // This will protect any file with number `file_num` or greater from being + // deleted while is running. + // ----------- + // This function will capture current file number and append it to + // pending_outputs_. This will prevent any background process to delete any + // file created after this point. + std::list::iterator CaptureCurrentFileNumberInPendingOutputs(); + // This function should be called with the result of + // CaptureCurrentFileNumberInPendingOutputs(). It then marks that any file + // created between the calls CaptureCurrentFileNumberInPendingOutputs() and + // ReleaseFileNumberFromPendingOutputs() can now be deleted (if it's not live + // and blocked by any other pending_outputs_ calls) + void ReleaseFileNumberFromPendingOutputs( + std::unique_ptr::iterator>& v); + + IOStatus SyncClosedLogs(JobContext* job_context, VersionEdit* synced_wals); + + // Flush the in-memory write buffer to storage. Switches to a new + // log-file/memtable and writes a new descriptor iff successful. Then + // installs a new super version for the column family. + Status FlushMemTableToOutputFile( + ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, + bool* madeProgress, JobContext* job_context, + SuperVersionContext* superversion_context, + std::vector& snapshot_seqs, + SequenceNumber earliest_write_conflict_snapshot, + SnapshotChecker* snapshot_checker, LogBuffer* log_buffer, + Env::Priority thread_pri); + + // Flush the memtables of (multiple) column families to multiple files on + // persistent storage. + Status FlushMemTablesToOutputFiles( + const autovector& bg_flush_args, bool* made_progress, + JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri); + + Status AtomicFlushMemTablesToOutputFiles( + const autovector& bg_flush_args, bool* made_progress, + JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri); + + // REQUIRES: log_numbers are sorted in ascending order + // corrupted_log_found is set to true if we recover from a corrupted log file. + Status RecoverLogFiles(const std::vector& log_numbers, + SequenceNumber* next_sequence, bool read_only, + bool* corrupted_log_found, + RecoveryContext* recovery_ctx); + + // The following two methods are used to flush a memtable to + // storage. The first one is used at database RecoveryTime (when the + // database is opened) and is heavyweight because it holds the mutex + // for the entire period. The second method WriteLevel0Table supports + // concurrent flush memtables to storage. + Status WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, + MemTable* mem, VersionEdit* edit); + + // Get the size of a log file and, if truncate is true, truncate the + // log file to its actual size, thereby freeing preallocated space. + // Return success even if truncate fails + Status GetLogSizeAndMaybeTruncate(uint64_t wal_number, bool truncate, + LogFileNumberSize* log); + + // Restore alive_log_files_ and total_log_size_ after recovery. + // It needs to run only when there's no flush during recovery + // (e.g. avoid_flush_during_recovery=true). May also trigger flush + // in case total_log_size > max_total_wal_size. + Status RestoreAliveLogFiles(const std::vector& log_numbers); + + // num_bytes: for slowdown case, delay time is calculated based on + // `num_bytes` going through. + Status DelayWrite(uint64_t num_bytes, const WriteOptions& write_options); + + // Begin stalling of writes when memory usage increases beyond a certain + // threshold. + void WriteBufferManagerStallWrites(); + + Status ThrottleLowPriWritesIfNeeded(const WriteOptions& write_options, + WriteBatch* my_batch); + + // REQUIRES: mutex locked and in write thread. + Status ScheduleFlushes(WriteContext* context); + + void MaybeFlushStatsCF(autovector* cfds); + + Status TrimMemtableHistory(WriteContext* context); + + Status SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context); + + void SelectColumnFamiliesForAtomicFlush(autovector* cfds); + + // Force current memtable contents to be flushed. + Status FlushMemTable(ColumnFamilyData* cfd, const FlushOptions& options, + FlushReason flush_reason, + bool entered_write_thread = false); + + Status AtomicFlushMemTables( + const autovector& column_family_datas, + const FlushOptions& options, FlushReason flush_reason, + bool entered_write_thread = false); + + // Wait until flushing this column family won't stall writes + Status WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd, + bool* flush_needed); + + // Wait for memtable flushed. + // If flush_memtable_id is non-null, wait until the memtable with the ID + // gets flush. Otherwise, wait until the column family don't have any + // memtable pending flush. + // resuming_from_bg_err indicates whether the caller is attempting to resume + // from background error. + Status WaitForFlushMemTable(ColumnFamilyData* cfd, + const uint64_t* flush_memtable_id = nullptr, + bool resuming_from_bg_err = false) { + return WaitForFlushMemTables({cfd}, {flush_memtable_id}, + resuming_from_bg_err); + } + // Wait for memtables to be flushed for multiple column families. + Status WaitForFlushMemTables( + const autovector& cfds, + const autovector& flush_memtable_ids, + bool resuming_from_bg_err); + + inline void WaitForPendingWrites() { + mutex_.AssertHeld(); + TEST_SYNC_POINT("DBImpl::WaitForPendingWrites:BeforeBlock"); + // In case of pipelined write is enabled, wait for all pending memtable + // writers. + if (immutable_db_options_.enable_pipelined_write) { + // Memtable writers may call DB::Get in case max_successive_merges > 0, + // which may lock mutex. Unlocking mutex here to avoid deadlock. + mutex_.Unlock(); + write_thread_.WaitForMemTableWriters(); + mutex_.Lock(); + } + + if (!immutable_db_options_.unordered_write) { + // Then the writes are finished before the next write group starts + return; + } + + // Wait for the ones who already wrote to the WAL to finish their + // memtable write. + if (pending_memtable_writes_.load() != 0) { + std::unique_lock guard(switch_mutex_); + switch_cv_.wait(guard, + [&] { return pending_memtable_writes_.load() == 0; }); + } + } + + // TaskType is used to identify tasks in thread-pool, currently only + // differentiate manual compaction, which could be unscheduled from the + // thread-pool. + enum class TaskType : uint8_t { + kDefault = 0, + kManualCompaction = 1, + kCount = 2, + }; + + // Task tag is used to identity tasks in thread-pool, which is + // dbImpl obj address + type + inline void* GetTaskTag(TaskType type) { + return GetTaskTag(static_cast(type)); + } + + inline void* GetTaskTag(uint8_t type) { + return static_cast(static_cast(this)) + type; + } + + // REQUIRES: mutex locked and in write thread. + void AssignAtomicFlushSeq(const autovector& cfds); + + // REQUIRES: mutex locked and in write thread. + Status SwitchWAL(WriteContext* write_context); + + // REQUIRES: mutex locked and in write thread. + Status HandleWriteBufferManagerFlush(WriteContext* write_context); + + // REQUIRES: mutex locked + Status PreprocessWrite(const WriteOptions& write_options, + LogContext* log_context, WriteContext* write_context); + + // Merge write batches in the write group into merged_batch. + // Returns OK if merge is successful. + // Returns Corruption if corruption in write batch is detected. + Status MergeBatch(const WriteThread::WriteGroup& write_group, + WriteBatch* tmp_batch, WriteBatch** merged_batch, + size_t* write_with_wal, WriteBatch** to_be_cached_state); + + // rate_limiter_priority is used to charge `DBOptions::rate_limiter` + // for automatic WAL flush (`Options::manual_wal_flush` == false) + // associated with this WriteToWAL + IOStatus WriteToWAL(const WriteBatch& merged_batch, log::Writer* log_writer, + uint64_t* log_used, uint64_t* log_size, + Env::IOPriority rate_limiter_priority, + LogFileNumberSize& log_file_number_size); + + IOStatus WriteToWAL(const WriteThread::WriteGroup& write_group, + log::Writer* log_writer, uint64_t* log_used, + bool need_log_sync, bool need_log_dir_sync, + SequenceNumber sequence, + LogFileNumberSize& log_file_number_size); + + IOStatus ConcurrentWriteToWAL(const WriteThread::WriteGroup& write_group, + uint64_t* log_used, + SequenceNumber* last_sequence, size_t seq_inc); + + // Used by WriteImpl to update bg_error_ if paranoid check is enabled. + // Caller must hold mutex_. + void WriteStatusCheckOnLocked(const Status& status); + + // Used by WriteImpl to update bg_error_ if paranoid check is enabled. + void WriteStatusCheck(const Status& status); + + // Used by WriteImpl to update bg_error_ when IO error happens, e.g., write + // WAL, sync WAL fails, if paranoid check is enabled. + void IOStatusCheck(const IOStatus& status); + + // Used by WriteImpl to update bg_error_ in case of memtable insert error. + void MemTableInsertStatusCheck(const Status& memtable_insert_status); + +#ifndef ROCKSDB_LITE + Status CompactFilesImpl(const CompactionOptions& compact_options, + ColumnFamilyData* cfd, Version* version, + const std::vector& input_file_names, + std::vector* const output_file_names, + const int output_level, int output_path_id, + JobContext* job_context, LogBuffer* log_buffer, + CompactionJobInfo* compaction_job_info); + + // Wait for current IngestExternalFile() calls to finish. + // REQUIRES: mutex_ held + void WaitForIngestFile(); +#else + // IngestExternalFile is not supported in ROCKSDB_LITE so this function + // will be no-op + void WaitForIngestFile() {} +#endif // ROCKSDB_LITE + + ColumnFamilyData* GetColumnFamilyDataByName(const std::string& cf_name); + + void MaybeScheduleFlushOrCompaction(); + + // A flush request specifies the column families to flush as well as the + // largest memtable id to persist for each column family. Once all the + // memtables whose IDs are smaller than or equal to this per-column-family + // specified value, this flush request is considered to have completed its + // work of flushing this column family. After completing the work for all + // column families in this request, this flush is considered complete. + using FlushRequest = std::vector>; + + void GenerateFlushRequest(const autovector& cfds, + FlushRequest* req); + + void SchedulePendingFlush(const FlushRequest& req, FlushReason flush_reason); + + void SchedulePendingCompaction(ColumnFamilyData* cfd); + void SchedulePendingPurge(std::string fname, std::string dir_to_sync, + FileType type, uint64_t number, int job_id); + static void BGWorkCompaction(void* arg); + // Runs a pre-chosen universal compaction involving bottom level in a + // separate, bottom-pri thread pool. + static void BGWorkBottomCompaction(void* arg); + static void BGWorkFlush(void* arg); + static void BGWorkPurge(void* arg); + static void UnscheduleCompactionCallback(void* arg); + static void UnscheduleFlushCallback(void* arg); + void BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction, + Env::Priority thread_pri); + void BackgroundCallFlush(Env::Priority thread_pri); + void BackgroundCallPurge(); + Status BackgroundCompaction(bool* madeProgress, JobContext* job_context, + LogBuffer* log_buffer, + PrepickedCompaction* prepicked_compaction, + Env::Priority thread_pri); + Status BackgroundFlush(bool* madeProgress, JobContext* job_context, + LogBuffer* log_buffer, FlushReason* reason, + Env::Priority thread_pri); + + bool EnoughRoomForCompaction(ColumnFamilyData* cfd, + const std::vector& inputs, + bool* sfm_bookkeeping, LogBuffer* log_buffer); + + // Request compaction tasks token from compaction thread limiter. + // It always succeeds if force = true or limiter is disable. + bool RequestCompactionToken(ColumnFamilyData* cfd, bool force, + std::unique_ptr* token, + LogBuffer* log_buffer); + + // Schedule background tasks + Status StartPeriodicTaskScheduler(); + + Status RegisterRecordSeqnoTimeWorker(); + + void PrintStatistics(); + + size_t EstimateInMemoryStatsHistorySize() const; + + // Return the minimum empty level that could hold the total data in the + // input level. Return the input level, if such level could not be found. + int FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd, + const MutableCFOptions& mutable_cf_options, + int level); + + // Move the files in the input level to the target level. + // If target_level < 0, automatically calculate the minimum level that could + // hold the data set. + Status ReFitLevel(ColumnFamilyData* cfd, int level, int target_level = -1); + + // helper functions for adding and removing from flush & compaction queues + void AddToCompactionQueue(ColumnFamilyData* cfd); + ColumnFamilyData* PopFirstFromCompactionQueue(); + FlushRequest PopFirstFromFlushQueue(); + + // Pick the first unthrottled compaction with task token from queue. + ColumnFamilyData* PickCompactionFromQueue( + std::unique_ptr* token, LogBuffer* log_buffer); + + // helper function to call after some of the logs_ were synced + void MarkLogsSynced(uint64_t up_to, bool synced_dir, VersionEdit* edit); + Status ApplyWALToManifest(VersionEdit* edit); + // WALs with log number up to up_to are not synced successfully. + void MarkLogsNotSynced(uint64_t up_to); + + SnapshotImpl* GetSnapshotImpl(bool is_write_conflict_boundary, + bool lock = true); + + // If snapshot_seq != kMaxSequenceNumber, then this function can only be + // called from the write thread that publishes sequence numbers to readers. + // For 1) write-committed, or 2) write-prepared + one-write-queue, this will + // be the write thread performing memtable writes. For write-prepared with + // two write queues, this will be the write thread writing commit marker to + // the WAL. + // If snapshot_seq == kMaxSequenceNumber, this function is called by a caller + // ensuring no writes to the database. + std::pair> + CreateTimestampedSnapshotImpl(SequenceNumber snapshot_seq, uint64_t ts, + bool lock = true); + + uint64_t GetMaxTotalWalSize() const; + + FSDirectory* GetDataDir(ColumnFamilyData* cfd, size_t path_id) const; + + Status MaybeReleaseTimestampedSnapshotsAndCheck(); + + Status CloseHelper(); + + void WaitForBackgroundWork(); + + // Background threads call this function, which is just a wrapper around + // the InstallSuperVersion() function. Background threads carry + // sv_context which can have new_superversion already + // allocated. + // 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); + + bool GetIntPropertyInternal(ColumnFamilyData* cfd, + const DBPropertyInfo& property_info, + bool is_locked, uint64_t* value); + bool GetPropertyHandleOptionsStatistics(std::string* value); + + bool HasPendingManualCompaction(); + bool HasExclusiveManualCompaction(); + void AddManualCompaction(ManualCompactionState* m); + void RemoveManualCompaction(ManualCompactionState* m); + bool ShouldntRunManualCompaction(ManualCompactionState* m); + bool HaveManualCompaction(ColumnFamilyData* cfd); + bool MCOverlap(ManualCompactionState* m, ManualCompactionState* m1); +#ifndef ROCKSDB_LITE + void BuildCompactionJobInfo(const ColumnFamilyData* cfd, Compaction* c, + const Status& st, + const CompactionJobStats& compaction_job_stats, + const int job_id, const Version* current, + CompactionJobInfo* compaction_job_info) const; + // Reserve the next 'num' file numbers for to-be-ingested external SST files, + // and return the current file_number in 'next_file_number'. + // Write a version edit to the MANIFEST. + Status ReserveFileNumbersBeforeIngestion( + ColumnFamilyData* cfd, uint64_t num, + std::unique_ptr::iterator>& pending_output_elem, + uint64_t* next_file_number); +#endif //! ROCKSDB_LITE + + bool ShouldPurge(uint64_t file_number) const; + void MarkAsGrabbedForPurge(uint64_t file_number); + + size_t GetWalPreallocateBlockSize(uint64_t write_buffer_size) const; + Env::WriteLifeTimeHint CalculateWALWriteHint() { return Env::WLTH_SHORT; } + + IOStatus CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number, + size_t preallocate_block_size, log::Writer** new_log); + + // Validate self-consistency of DB options + static Status ValidateOptions(const DBOptions& db_options); + // Validate self-consistency of DB options and its consistency with cf options + static Status ValidateOptions( + const DBOptions& db_options, + const std::vector& column_families); + + // Utility function to do some debug validation and sort the given vector + // of MultiGet keys + void PrepareMultiGetKeys( + const size_t num_keys, bool sorted, + autovector* key_ptrs); + + // A structure to hold the information required to process MultiGet of keys + // belonging to one column family. For a multi column family MultiGet, there + // will be a container of these objects. + struct MultiGetColumnFamilyData { + ColumnFamilyHandle* cf; + ColumnFamilyData* cfd; + + // For the batched MultiGet which relies on sorted keys, start specifies + // the index of first key belonging to this column family in the sorted + // list. + size_t start; + + // For the batched MultiGet case, num_keys specifies the number of keys + // belonging to this column family in the sorted list + size_t num_keys; + + // SuperVersion for the column family obtained in a manner that ensures a + // consistent view across all column families in the DB + SuperVersion* super_version; + MultiGetColumnFamilyData(ColumnFamilyHandle* column_family, + SuperVersion* sv) + : cf(column_family), + cfd(static_cast(cf)->cfd()), + start(0), + num_keys(0), + super_version(sv) {} + + MultiGetColumnFamilyData(ColumnFamilyHandle* column_family, size_t first, + size_t count, SuperVersion* sv) + : cf(column_family), + cfd(static_cast(cf)->cfd()), + start(first), + num_keys(count), + super_version(sv) {} + + MultiGetColumnFamilyData() = default; + }; + + // A common function to obtain a consistent snapshot, which can be implicit + // if the user doesn't specify a snapshot in read_options, across + // multiple column families for MultiGet. It will attempt to get an implicit + // snapshot without acquiring the db_mutes, but will give up after a few + // tries and acquire the mutex if a memtable flush happens. The template + // allows both the batched and non-batched MultiGet to call this with + // either an std::unordered_map or autovector of column families. + // + // If callback is non-null, the callback is refreshed with the snapshot + // sequence number + // + // A return value of true indicates that the SuperVersions were obtained + // from the ColumnFamilyData, whereas false indicates they are thread + // local + template + bool MultiCFSnapshot( + const ReadOptions& read_options, ReadCallback* callback, + std::function& + iter_deref_func, + T* cf_list, SequenceNumber* snapshot); + + // The actual implementation of the batching MultiGet. The caller is expected + // to have acquired the SuperVersion and pass in a snapshot sequence number + // in order to construct the LookupKeys. The start_key and num_keys specify + // the range of keys in the sorted_keys vector for a single column family. + Status MultiGetImpl( + const ReadOptions& read_options, size_t start_key, size_t num_keys, + autovector* sorted_keys, + SuperVersion* sv, SequenceNumber snap_seqnum, ReadCallback* callback); + + Status DisableFileDeletionsWithLock(); + + Status IncreaseFullHistoryTsLowImpl(ColumnFamilyData* cfd, + std::string ts_low); + + bool ShouldReferenceSuperVersion(const MergeContext& merge_context); + + // Lock over the persistent DB state. Non-nullptr iff successfully acquired. + FileLock* db_lock_; + + // In addition to mutex_, log_write_mutex_ protected writes to stats_history_ + InstrumentedMutex stats_history_mutex_; + // In addition to mutex_, log_write_mutex_ protected writes to logs_ and + // logfile_number_. With two_write_queues it also protects alive_log_files_, + // and log_empty_. Refer to the definition of each variable below for more + // details. + // Note: to avoid dealock, if needed to acquire both log_write_mutex_ and + // mutex_, the order should be first mutex_ and then log_write_mutex_. + InstrumentedMutex log_write_mutex_; + + // If zero, manual compactions are allowed to proceed. If non-zero, manual + // compactions may still be running, but will quickly fail with + // `Status::Incomplete`. The value indicates how many threads have paused + // manual compactions. It is accessed in read mode outside the DB mutex in + // compaction code paths. + std::atomic manual_compaction_paused_; + + // This condition variable is signaled on these conditions: + // * whenever bg_compaction_scheduled_ goes down to 0 + // * if AnyManualCompaction, whenever a compaction finishes, even if it hasn't + // made any progress + // * whenever a compaction made any progress + // * whenever bg_flush_scheduled_ or bg_purge_scheduled_ value decreases + // (i.e. whenever a flush is done, even if it didn't make any progress) + // * whenever there is an error in background purge, flush or compaction + // * whenever num_running_ingest_file_ goes to 0. + // * whenever pending_purge_obsolete_files_ goes to 0. + // * whenever disable_delete_obsolete_files_ goes to 0. + // * whenever SetOptions successfully updates options. + // * whenever a column family is dropped. + InstrumentedCondVar bg_cv_; + // Writes are protected by locking both mutex_ and log_write_mutex_, and reads + // must be under either mutex_ or log_write_mutex_. Since after ::Open, + // logfile_number_ is currently updated only in write_thread_, it can be read + // from the same write_thread_ without any locks. + uint64_t logfile_number_; + // Log files that we can recycle. Must be protected by db mutex_. + std::deque log_recycle_files_; + // Protected by log_write_mutex_. + bool log_dir_synced_; + // Without two_write_queues, read and writes to log_empty_ are protected by + // mutex_. Since it is currently updated/read only in write_thread_, it can be + // accessed from the same write_thread_ without any locks. With + // two_write_queues writes, where it can be updated in different threads, + // read and writes are protected by log_write_mutex_ instead. This is to avoid + // expensive mutex_ lock during WAL write, which update log_empty_. + bool log_empty_; + + ColumnFamilyHandleImpl* persist_stats_cf_handle_; + + bool persistent_stats_cfd_exists_ = true; + + // alive_log_files_ is protected by mutex_ and log_write_mutex_ with details + // as follows: + // 1. read by FindObsoleteFiles() which can be called in either application + // thread or RocksDB bg threads, both mutex_ and log_write_mutex_ are + // held. + // 2. pop_front() by FindObsoleteFiles(), both mutex_ and log_write_mutex_ + // are held. + // 3. push_back() by DBImpl::Open() and DBImpl::RestoreAliveLogFiles() + // (actually called by Open()), only mutex_ is held because at this point, + // the DB::Open() call has not returned success to application, and the + // only other thread(s) that can conflict are bg threads calling + // FindObsoleteFiles() which ensure that both mutex_ and log_write_mutex_ + // are held when accessing alive_log_files_. + // 4. read by DBImpl::Open() is protected by mutex_. + // 5. push_back() by SwitchMemtable(). Both mutex_ and log_write_mutex_ are + // held. This is done by the write group leader. Note that in the case of + // two-write-queues, another WAL-only write thread can be writing to the + // WAL concurrently. See 9. + // 6. read by SwitchWAL() with both mutex_ and log_write_mutex_ held. This is + // done by write group leader. + // 7. read by ConcurrentWriteToWAL() by the write group leader in the case of + // two-write-queues. Only log_write_mutex_ is held to protect concurrent + // pop_front() by FindObsoleteFiles(). + // 8. read by PreprocessWrite() by the write group leader. log_write_mutex_ + // is held to protect the data structure from concurrent pop_front() by + // FindObsoleteFiles(). + // 9. read by ConcurrentWriteToWAL() by a WAL-only write thread in the case + // of two-write-queues. Only log_write_mutex_ is held. This suffices to + // protect the data structure from concurrent push_back() by current + // write group leader as well as pop_front() by FindObsoleteFiles(). + std::deque alive_log_files_; + + // Log files that aren't fully synced, and the current log file. + // Synchronization: + // 1. read by FindObsoleteFiles() which can be called either in application + // thread or RocksDB bg threads. log_write_mutex_ is always held, while + // some reads are performed without mutex_. + // 2. pop_front() by FindObsoleteFiles() with only log_write_mutex_ held. + // 3. read by DBImpl::Open() with both mutex_ and log_write_mutex_. + // 4. emplace_back() by DBImpl::Open() with both mutex_ and log_write_mutex. + // Note that at this point, DB::Open() has not returned success to + // application, thus the only other thread(s) that can conflict are bg + // threads calling FindObsoleteFiles(). See 1. + // 5. iteration and clear() from CloseHelper() always hold log_write_mutex + // and mutex_. + // 6. back() called by APIs FlushWAL() and LockWAL() are protected by only + // log_write_mutex_. These two can be called by application threads after + // DB::Open() returns success to applications. + // 7. read by SyncWAL(), another API, protected by only log_write_mutex_. + // 8. read by MarkLogsNotSynced() and MarkLogsSynced() are protected by + // log_write_mutex_. + // 9. erase() by MarkLogsSynced() protected by log_write_mutex_. + // 10. read by SyncClosedLogs() protected by only log_write_mutex_. This can + // happen in bg flush threads after DB::Open() returns success to + // applications. + // 11. reads, e.g. front(), iteration, and back() called by PreprocessWrite() + // holds only the log_write_mutex_. This is done by the write group + // leader. A bg thread calling FindObsoleteFiles() or MarkLogsSynced() + // can happen concurrently. This is fine because log_write_mutex_ is used + // by all parties. See 2, 5, 9. + // 12. reads, empty(), back() called by SwitchMemtable() hold both mutex_ and + // log_write_mutex_. This happens in the write group leader. + // 13. emplace_back() by SwitchMemtable() hold both mutex_ and + // log_write_mutex_. This happens in the write group leader. Can conflict + // with bg threads calling FindObsoleteFiles(), MarkLogsSynced(), + // SyncClosedLogs(), etc. as well as application threads calling + // FlushWAL(), SyncWAL(), LockWAL(). This is fine because all parties + // require at least log_write_mutex_. + // 14. iteration called in WriteToWAL(write_group) protected by + // log_write_mutex_. This is done by write group leader when + // two-write-queues is disabled and write needs to sync logs. + // 15. back() called in ConcurrentWriteToWAL() protected by log_write_mutex_. + // This can be done by the write group leader if two-write-queues is + // enabled. It can also be done by another WAL-only write thread. + // + // Other observations: + // - back() and items with getting_synced=true are not popped, + // - The same thread that sets getting_synced=true will reset it. + // - it follows that the object referred by back() can be safely read from + // the write_thread_ without using mutex. Note that calling back() without + // mutex may be unsafe because different implementations of deque::back() may + // access other member variables of deque, causing undefined behaviors. + // Generally, do not access stl containers without proper synchronization. + // - it follows that the items with getting_synced=true can be safely read + // from the same thread that has set getting_synced=true + std::deque logs_; + + // Signaled when getting_synced becomes false for some of the logs_. + InstrumentedCondVar log_sync_cv_; + // This is the app-level state that is written to the WAL but will be used + // only during recovery. Using this feature enables not writing the state to + // memtable on normal writes and hence improving the throughput. Each new + // write of the state will replace the previous state entirely even if the + // keys in the two consecutive states do not overlap. + // It is protected by log_write_mutex_ when two_write_queues_ is enabled. + // Otherwise only the heaad of write_thread_ can access it. + WriteBatch cached_recoverable_state_; + std::atomic cached_recoverable_state_empty_ = {true}; + std::atomic total_log_size_; + + // If this is non-empty, we need to delete these log files in background + // threads. Protected by log_write_mutex_. + autovector logs_to_free_; + + bool is_snapshot_supported_; + + std::map> stats_history_; + + std::map stats_slice_; + + bool stats_slice_initialized_ = false; + + Directories directories_; + + WriteBufferManager* write_buffer_manager_; + + WriteThread write_thread_; + WriteBatch tmp_batch_; + // The write thread when the writers have no memtable write. This will be used + // in 2PC to batch the prepares separately from the serial commit. + WriteThread nonmem_write_thread_; + + WriteController write_controller_; + + // Size of the last batch group. In slowdown mode, next write needs to + // sleep if it uses up the quota. + // Note: This is to protect memtable and compaction. If the batch only writes + // to the WAL its size need not to be included in this. + uint64_t last_batch_group_size_; + + FlushScheduler flush_scheduler_; + + TrimHistoryScheduler trim_history_scheduler_; + + SnapshotList snapshots_; + + TimestampedSnapshotList timestamped_snapshots_; + + // For each background job, pending_outputs_ keeps the current file number at + // the time that background job started. + // FindObsoleteFiles()/PurgeObsoleteFiles() never deletes any file that has + // number bigger than any of the file number in pending_outputs_. Since file + // numbers grow monotonically, this also means that pending_outputs_ is always + // sorted. After a background job is done executing, its file number is + // deleted from pending_outputs_, which allows PurgeObsoleteFiles() to clean + // it up. + // State is protected with db mutex. + std::list pending_outputs_; + + // flush_queue_ and compaction_queue_ hold column families that we need to + // flush and compact, respectively. + // A column family is inserted into flush_queue_ when it satisfies condition + // cfd->imm()->IsFlushPending() + // A column family is inserted into compaction_queue_ when it satisfied + // condition cfd->NeedsCompaction() + // Column families in this list are all Ref()-erenced + // TODO(icanadi) Provide some kind of ReferencedColumnFamily class that will + // do RAII on ColumnFamilyData + // Column families are in this queue when they need to be flushed or + // compacted. Consumers of these queues are flush and compaction threads. When + // column family is put on this queue, we increase unscheduled_flushes_ and + // unscheduled_compactions_. When these variables are bigger than zero, that + // means we need to schedule background threads for flush and compaction. + // Once the background threads are scheduled, we decrease unscheduled_flushes_ + // and unscheduled_compactions_. That way we keep track of number of + // compaction and flush threads we need to schedule. This scheduling is done + // in MaybeScheduleFlushOrCompaction() + // invariant(column family present in flush_queue_ <==> + // ColumnFamilyData::pending_flush_ == true) + std::deque flush_queue_; + // invariant(column family present in compaction_queue_ <==> + // ColumnFamilyData::pending_compaction_ == true) + std::deque compaction_queue_; + + // A map to store file numbers and filenames of the files to be purged + std::unordered_map purge_files_; + + // A vector to store the file numbers that have been assigned to certain + // JobContext. Current implementation tracks table and blob files only. + std::unordered_set files_grabbed_for_purge_; + + // A queue to store log writers to close. Protected by db mutex_. + std::deque logs_to_free_queue_; + + std::deque superversions_to_free_queue_; + + int unscheduled_flushes_; + + int unscheduled_compactions_; + + // count how many background compactions are running or have been scheduled in + // the BOTTOM pool + int bg_bottom_compaction_scheduled_; + + // count how many background compactions are running or have been scheduled + int bg_compaction_scheduled_; + + // stores the number of compactions are currently running + int num_running_compactions_; + + // number of background memtable flush jobs, submitted to the HIGH pool + int bg_flush_scheduled_; + + // stores the number of flushes are currently running + int num_running_flushes_; + + // number of background obsolete file purge jobs, submitted to the HIGH pool + int bg_purge_scheduled_; + + std::deque manual_compaction_dequeue_; + + // shall we disable deletion of obsolete files + // if 0 the deletion is enabled. + // if non-zero, files will not be getting deleted + // This enables two different threads to call + // EnableFileDeletions() and DisableFileDeletions() + // without any synchronization + int disable_delete_obsolete_files_; + + // Number of times FindObsoleteFiles has found deletable files and the + // corresponding call to PurgeObsoleteFiles has not yet finished. + int pending_purge_obsolete_files_; + + // last time when DeleteObsoleteFiles with full scan was executed. Originally + // initialized with startup time. + uint64_t delete_obsolete_files_last_run_; + + // last time stats were dumped to LOG + std::atomic last_stats_dump_time_microsec_; + + // The thread that wants to switch memtable, can wait on this cv until the + // pending writes to memtable finishes. + std::condition_variable switch_cv_; + // The mutex used by switch_cv_. mutex_ should be acquired beforehand. + std::mutex switch_mutex_; + // Number of threads intending to write to memtable + std::atomic pending_memtable_writes_ = {}; + + // A flag indicating whether the current rocksdb database has any + // data that is not yet persisted into either WAL or SST file. + // Used when disableWAL is true. + std::atomic has_unpersisted_data_; + + // if an attempt was made to flush all column families that + // the oldest log depends on but uncommitted data in the oldest + // log prevents the log from being released. + // We must attempt to free the dependent memtables again + // at a later time after the transaction in the oldest + // log is fully commited. + bool unable_to_release_oldest_log_; + + // Number of running IngestExternalFile() or CreateColumnFamilyWithImport() + // calls. + // REQUIRES: mutex held + int num_running_ingest_file_; + +#ifndef ROCKSDB_LITE + WalManager wal_manager_; +#endif // ROCKSDB_LITE + + // A value of > 0 temporarily disables scheduling of background work + int bg_work_paused_; + + // A value of > 0 temporarily disables scheduling of background compaction + int bg_compaction_paused_; + + // Guard against multiple concurrent refitting + bool refitting_level_; + + // Indicate DB was opened successfully + bool opened_successfully_; + + // The min threshold to triggere bottommost compaction for removing + // garbages, among all column families. + SequenceNumber bottommost_files_mark_threshold_ = kMaxSequenceNumber; + + LogsWithPrepTracker logs_with_prep_tracker_; + + // Callback for compaction to check if a key is visible to a snapshot. + // REQUIRES: mutex held + std::unique_ptr snapshot_checker_; + + // Callback for when the cached_recoverable_state_ is written to memtable + // Only to be set during initialization + std::unique_ptr recoverable_state_pre_release_callback_; + +#ifndef ROCKSDB_LITE + // Scheduler to run DumpStats(), PersistStats(), and FlushInfoLog(). + // Currently, internally it has a global timer instance for running the tasks. + PeriodicTaskScheduler periodic_task_scheduler_; + + // It contains the implementations for each periodic task. + std::map periodic_task_functions_; +#endif + + // When set, we use a separate queue for writes that don't write to memtable. + // In 2PC these are the writes at Prepare phase. + const bool two_write_queues_; + const bool manual_wal_flush_; + + // LastSequence also indicates last published sequence visibile to the + // readers. Otherwise LastPublishedSequence should be used. + const bool last_seq_same_as_publish_seq_; + // It indicates that a customized gc algorithm must be used for + // flush/compaction and if it is not provided vis SnapshotChecker, we should + // disable gc to be safe. + const bool use_custom_gc_; + // Flag to indicate that the DB instance shutdown has been initiated. This + // different from shutting_down_ atomic in that it is set at the beginning + // of shutdown sequence, specifically in order to prevent any background + // error recovery from going on in parallel. The latter, shutting_down_, + // is set a little later during the shutdown after scheduling memtable + // flushes + std::atomic shutdown_initiated_; + // Flag to indicate whether sst_file_manager object was allocated in + // DB::Open() or passed to us + bool own_sfm_; + + // Flag to check whether Close() has been called on this DB + bool closed_; + // save the closing status, for re-calling the close() + Status closing_status_; + // mutex for DB::Close() + InstrumentedMutex closing_mutex_; + + // Conditional variable to coordinate installation of atomic flush results. + // With atomic flush, each bg thread installs the result of flushing multiple + // column families, and different threads can flush different column + // families. It's difficult to rely on one thread to perform batch + // installation for all threads. This is different from the non-atomic flush + // case. + // atomic_flush_install_cv_ makes sure that threads install atomic flush + // results sequentially. Flush results of memtables with lower IDs get + // installed to MANIFEST first. + InstrumentedCondVar atomic_flush_install_cv_; + + bool wal_in_db_path_; + std::atomic max_total_wal_size_; + + BlobFileCompletionCallback blob_callback_; + + // Pointer to WriteBufferManager stalling interface. + std::unique_ptr wbm_stall_; + + // seqno_time_mapping_ stores the sequence number to time mapping, it's not + // thread safe, both read and write need db mutex hold. + SeqnoToTimeMapping seqno_time_mapping_; +}; + +class GetWithTimestampReadCallback : public ReadCallback { + public: + explicit GetWithTimestampReadCallback(SequenceNumber seq) + : ReadCallback(seq) {} + bool IsVisibleFullCheck(SequenceNumber seq) override { + return seq <= max_visible_seq_; + } +}; + +extern Options SanitizeOptions(const std::string& db, const Options& src, + bool read_only = false, + Status* logger_creation_s = nullptr); + +extern DBOptions SanitizeOptions(const std::string& db, const DBOptions& src, + bool read_only = false, + Status* logger_creation_s = nullptr); + +extern CompressionType GetCompressionFlush( + const ImmutableCFOptions& ioptions, + const MutableCFOptions& mutable_cf_options); + +// Return the earliest log file to keep after the memtable flush is +// finalized. +// `cfd_to_flush` is the column family whose memtable (specified in +// `memtables_to_flush`) will be flushed and thus will not depend on any WAL +// file. +// The function is only applicable to 2pc mode. +extern uint64_t PrecomputeMinLogNumberToKeep2PC( + VersionSet* vset, const ColumnFamilyData& cfd_to_flush, + const autovector& edit_list, + const autovector& memtables_to_flush, + LogsWithPrepTracker* prep_tracker); +// For atomic flush. +extern uint64_t PrecomputeMinLogNumberToKeep2PC( + VersionSet* vset, const autovector& cfds_to_flush, + const autovector>& edit_lists, + const autovector*>& memtables_to_flush, + LogsWithPrepTracker* prep_tracker); + +// In non-2PC mode, WALs with log number < the returned number can be +// deleted after the cfd_to_flush column family is flushed successfully. +extern uint64_t PrecomputeMinLogNumberToKeepNon2PC( + VersionSet* vset, const ColumnFamilyData& cfd_to_flush, + const autovector& edit_list); +// For atomic flush. +extern uint64_t PrecomputeMinLogNumberToKeepNon2PC( + VersionSet* vset, const autovector& cfds_to_flush, + const autovector>& edit_lists); + +// `cfd_to_flush` is the column family whose memtable will be flushed and thus +// will not depend on any WAL file. nullptr means no memtable is being flushed. +// The function is only applicable to 2pc mode. +extern uint64_t FindMinPrepLogReferencedByMemTable( + VersionSet* vset, const autovector& memtables_to_flush); +// For atomic flush. +extern uint64_t FindMinPrepLogReferencedByMemTable( + VersionSet* vset, + const autovector*>& memtables_to_flush); + +// Fix user-supplied options to be reasonable +template +static void ClipToRange(T* ptr, V minvalue, V maxvalue) { + if (static_cast(*ptr) > maxvalue) *ptr = maxvalue; + if (static_cast(*ptr) < minvalue) *ptr = minvalue; +} + +inline Status DBImpl::FailIfCfHasTs( + const ColumnFamilyHandle* column_family) const { + column_family = column_family ? column_family : DefaultColumnFamily(); + assert(column_family); + const Comparator* const ucmp = column_family->GetComparator(); + assert(ucmp); + if (ucmp->timestamp_size() > 0) { + std::ostringstream oss; + oss << "cannot call this method on column family " + << column_family->GetName() << " that enables timestamp"; + return Status::InvalidArgument(oss.str()); + } + return Status::OK(); +} + +inline Status DBImpl::FailIfTsMismatchCf(ColumnFamilyHandle* column_family, + const Slice& ts, + bool ts_for_read) const { + if (!column_family) { + return Status::InvalidArgument("column family handle cannot be null"); + } + assert(column_family); + const Comparator* const ucmp = column_family->GetComparator(); + assert(ucmp); + if (0 == ucmp->timestamp_size()) { + std::stringstream oss; + oss << "cannot call this method on column family " + << column_family->GetName() << " that does not enable timestamp"; + return Status::InvalidArgument(oss.str()); + } + const size_t ts_sz = ts.size(); + if (ts_sz != ucmp->timestamp_size()) { + std::stringstream oss; + oss << "Timestamp sizes mismatch: expect " << ucmp->timestamp_size() << ", " + << ts_sz << " given"; + return Status::InvalidArgument(oss.str()); + } + if (ts_for_read) { + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + std::string current_ts_low = cfd->GetFullHistoryTsLow(); + if (!current_ts_low.empty() && + ucmp->CompareTimestamp(ts, current_ts_low) < 0) { + std::stringstream oss; + oss << "Read timestamp: " << ts.ToString(true) + << " is smaller than full_history_ts_low: " + << Slice(current_ts_low).ToString(true) << std::endl; + return Status::InvalidArgument(oss.str()); + } + } + return Status::OK(); +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl_compaction_flush.cc b/src/rocksdb/db/db_impl/db_impl_compaction_flush.cc new file mode 100644 index 000000000..a605fac87 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_compaction_flush.cc @@ -0,0 +1,3857 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +#include +#include + +#include "db/builder.h" +#include "db/db_impl/db_impl.h" +#include "db/error_handler.h" +#include "db/event_helpers.h" +#include "file/sst_file_manager_impl.h" +#include "logging/logging.h" +#include "monitoring/iostats_context_imp.h" +#include "monitoring/perf_context_imp.h" +#include "monitoring/thread_status_updater.h" +#include "monitoring/thread_status_util.h" +#include "test_util/sync_point.h" +#include "util/cast_util.h" +#include "util/concurrent_task_limiter_impl.h" + +namespace ROCKSDB_NAMESPACE { + +bool DBImpl::EnoughRoomForCompaction( + ColumnFamilyData* cfd, const std::vector& inputs, + bool* sfm_reserved_compact_space, LogBuffer* log_buffer) { + // Check if we have enough room to do the compaction + bool enough_room = true; +#ifndef ROCKSDB_LITE + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); + if (sfm) { + // Pass the current bg_error_ to SFM so it can decide what checks to + // perform. If this DB instance hasn't seen any error yet, the SFM can be + // optimistic and not do disk space checks + Status bg_error = error_handler_.GetBGError(); + enough_room = sfm->EnoughRoomForCompaction(cfd, inputs, bg_error); + bg_error.PermitUncheckedError(); // bg_error is just a copy of the Status + // from the error_handler_ + if (enough_room) { + *sfm_reserved_compact_space = true; + } + } +#else + (void)cfd; + (void)inputs; + (void)sfm_reserved_compact_space; +#endif // ROCKSDB_LITE + if (!enough_room) { + // Just in case tests want to change the value of enough_room + TEST_SYNC_POINT_CALLBACK( + "DBImpl::BackgroundCompaction():CancelledCompaction", &enough_room); + ROCKS_LOG_BUFFER(log_buffer, + "Cancelled compaction because not enough room"); + RecordTick(stats_, COMPACTION_CANCELLED, 1); + } + return enough_room; +} + +bool DBImpl::RequestCompactionToken(ColumnFamilyData* cfd, bool force, + std::unique_ptr* token, + LogBuffer* log_buffer) { + assert(*token == nullptr); + auto limiter = static_cast( + cfd->ioptions()->compaction_thread_limiter.get()); + if (limiter == nullptr) { + return true; + } + *token = limiter->GetToken(force); + if (*token != nullptr) { + ROCKS_LOG_BUFFER(log_buffer, + "Thread limiter [%s] increase [%s] compaction task, " + "force: %s, tasks after: %d", + limiter->GetName().c_str(), cfd->GetName().c_str(), + force ? "true" : "false", limiter->GetOutstandingTask()); + return true; + } + return false; +} + +IOStatus DBImpl::SyncClosedLogs(JobContext* job_context, + VersionEdit* synced_wals) { + TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Start"); + InstrumentedMutexLock l(&log_write_mutex_); + autovector logs_to_sync; + uint64_t current_log_number = logfile_number_; + while (logs_.front().number < current_log_number && + logs_.front().IsSyncing()) { + log_sync_cv_.Wait(); + } + for (auto it = logs_.begin(); + it != logs_.end() && it->number < current_log_number; ++it) { + auto& log = *it; + log.PrepareForSync(); + logs_to_sync.push_back(log.writer); + } + + IOStatus io_s; + if (!logs_to_sync.empty()) { + log_write_mutex_.Unlock(); + + assert(job_context); + + for (log::Writer* log : logs_to_sync) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[JOB %d] Syncing log #%" PRIu64, job_context->job_id, + log->get_log_number()); + if (error_handler_.IsRecoveryInProgress()) { + log->file()->reset_seen_error(); + } + io_s = log->file()->Sync(immutable_db_options_.use_fsync); + if (!io_s.ok()) { + break; + } + + if (immutable_db_options_.recycle_log_file_num > 0) { + if (error_handler_.IsRecoveryInProgress()) { + log->file()->reset_seen_error(); + } + io_s = log->Close(); + if (!io_s.ok()) { + break; + } + } + } + if (io_s.ok()) { + io_s = directories_.GetWalDir()->FsyncWithDirOptions( + IOOptions(), nullptr, + DirFsyncOptions(DirFsyncOptions::FsyncReason::kNewFileSynced)); + } + + TEST_SYNC_POINT_CALLBACK("DBImpl::SyncClosedLogs:BeforeReLock", + /*arg=*/nullptr); + log_write_mutex_.Lock(); + + // "number <= current_log_number - 1" is equivalent to + // "number < current_log_number". + if (io_s.ok()) { + MarkLogsSynced(current_log_number - 1, true, synced_wals); + } else { + MarkLogsNotSynced(current_log_number - 1); + } + if (!io_s.ok()) { + TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Failed"); + return io_s; + } + } + TEST_SYNC_POINT("DBImpl::SyncClosedLogs:end"); + return io_s; +} + +Status DBImpl::FlushMemTableToOutputFile( + ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, + bool* made_progress, JobContext* job_context, + SuperVersionContext* superversion_context, + std::vector& snapshot_seqs, + SequenceNumber earliest_write_conflict_snapshot, + SnapshotChecker* snapshot_checker, LogBuffer* log_buffer, + Env::Priority thread_pri) { + mutex_.AssertHeld(); + assert(cfd); + assert(cfd->imm()); + assert(cfd->imm()->NumNotFlushed() != 0); + assert(cfd->imm()->IsFlushPending()); + assert(versions_); + assert(versions_->GetColumnFamilySet()); + // If there are more than one column families, we need to make sure that + // all the log files except the most recent one are synced. Otherwise if + // the host crashes after flushing and before WAL is persistent, the + // flushed SST may contain data from write batches whose updates to + // other (unflushed) column families are missing. + const bool needs_to_sync_closed_wals = + logfile_number_ > 0 && + versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1; + + // If needs_to_sync_closed_wals is true, we need to record the current + // maximum memtable ID of this column family so that a later PickMemtables() + // call will not pick memtables whose IDs are higher. This is due to the fact + // that SyncClosedLogs() may release the db mutex, and memtable switch can + // happen for this column family in the meantime. The newly created memtables + // have their data backed by unsynced WALs, thus they cannot be included in + // this flush job. + // Another reason why we must record the current maximum memtable ID of this + // column family: SyncClosedLogs() may release db mutex, thus it's possible + // for application to continue to insert into memtables increasing db's + // sequence number. The application may take a snapshot, but this snapshot is + // not included in `snapshot_seqs` which will be passed to flush job because + // `snapshot_seqs` has already been computed before this function starts. + // Recording the max memtable ID ensures that the flush job does not flush + // a memtable without knowing such snapshot(s). + uint64_t max_memtable_id = needs_to_sync_closed_wals + ? cfd->imm()->GetLatestMemTableID() + : std::numeric_limits::max(); + + // If needs_to_sync_closed_wals is false, then the flush job will pick ALL + // existing memtables of the column family when PickMemTable() is called + // later. Although we won't call SyncClosedLogs() in this case, we may still + // call the callbacks of the listeners, i.e. NotifyOnFlushBegin() which also + // releases and re-acquires the db mutex. In the meantime, the application + // can still insert into the memtables and increase the db's sequence number. + // The application can take a snapshot, hoping that the latest visible state + // to this snapshto is preserved. This is hard to guarantee since db mutex + // not held. This newly-created snapshot is not included in `snapshot_seqs` + // and the flush job is unaware of its presence. Consequently, the flush job + // may drop certain keys when generating the L0, causing incorrect data to be + // returned for snapshot read using this snapshot. + // To address this, we make sure NotifyOnFlushBegin() executes after memtable + // picking so that no new snapshot can be taken between the two functions. + + FlushJob flush_job( + dbname_, cfd, immutable_db_options_, mutable_cf_options, max_memtable_id, + file_options_for_compaction_, versions_.get(), &mutex_, &shutting_down_, + snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, + job_context, log_buffer, directories_.GetDbDir(), GetDataDir(cfd, 0U), + 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_, seqno_time_mapping_, db_id_, db_session_id_, + cfd->GetFullHistoryTsLow(), &blob_callback_); + FileMetaData file_meta; + + Status s; + bool need_cancel = false; + IOStatus log_io_s = IOStatus::OK(); + if (needs_to_sync_closed_wals) { + // SyncClosedLogs() may unlock and re-lock the log_write_mutex multiple + // times. + VersionEdit synced_wals; + mutex_.Unlock(); + log_io_s = SyncClosedLogs(job_context, &synced_wals); + mutex_.Lock(); + if (log_io_s.ok() && synced_wals.IsWalAddition()) { + log_io_s = status_to_io_status(ApplyWALToManifest(&synced_wals)); + TEST_SYNC_POINT_CALLBACK("DBImpl::FlushMemTableToOutputFile:CommitWal:1", + nullptr); + } + + if (!log_io_s.ok() && !log_io_s.IsShutdownInProgress() && + !log_io_s.IsColumnFamilyDropped()) { + error_handler_.SetBGError(log_io_s, BackgroundErrorReason::kFlush); + } + } else { + TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Skip"); + } + s = log_io_s; + + // If the log sync failed, we do not need to pick memtable. Otherwise, + // num_flush_not_started_ needs to be rollback. + TEST_SYNC_POINT("DBImpl::FlushMemTableToOutputFile:BeforePickMemtables"); + if (s.ok()) { + flush_job.PickMemTable(); + need_cancel = true; + } + TEST_SYNC_POINT_CALLBACK( + "DBImpl::FlushMemTableToOutputFile:AfterPickMemtables", &flush_job); + +#ifndef ROCKSDB_LITE + // may temporarily unlock and lock the mutex. + NotifyOnFlushBegin(cfd, &file_meta, mutable_cf_options, job_context->job_id); +#endif // ROCKSDB_LITE + + bool switched_to_mempurge = false; + // Within flush_job.Run, rocksdb may call event listener to notify + // file creation and deletion. + // + // Note that flush_job.Run will unlock and lock the db_mutex, + // and EventListener callback will be called when the db_mutex + // is unlocked by the current thread. + if (s.ok()) { + s = flush_job.Run(&logs_with_prep_tracker_, &file_meta, + &switched_to_mempurge); + need_cancel = false; + } + + if (!s.ok() && need_cancel) { + flush_job.Cancel(); + } + + if (s.ok()) { + InstallSuperVersionAndScheduleWork(cfd, superversion_context, + mutable_cf_options); + if (made_progress) { + *made_progress = true; + } + + const std::string& column_family_name = cfd->GetName(); + + Version* const current = cfd->current(); + assert(current); + + const VersionStorageInfo* const storage_info = current->storage_info(); + assert(storage_info); + + VersionStorageInfo::LevelSummaryStorage tmp; + ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n", + column_family_name.c_str(), + storage_info->LevelSummary(&tmp)); + + const auto& blob_files = storage_info->GetBlobFiles(); + if (!blob_files.empty()) { + assert(blob_files.front()); + assert(blob_files.back()); + + ROCKS_LOG_BUFFER( + log_buffer, + "[%s] Blob file summary: head=%" PRIu64 ", tail=%" PRIu64 "\n", + column_family_name.c_str(), blob_files.front()->GetBlobFileNumber(), + blob_files.back()->GetBlobFileNumber()); + } + } + + if (!s.ok() && !s.IsShutdownInProgress() && !s.IsColumnFamilyDropped()) { + if (log_io_s.ok()) { + // Error while writing to MANIFEST. + // In fact, versions_->io_status() can also be the result of renaming + // CURRENT file. With current code, it's just difficult to tell. So just + // be pessimistic and try write to a new MANIFEST. + // TODO: distinguish between MANIFEST write and CURRENT renaming + if (!versions_->io_status().ok()) { + // If WAL sync is successful (either WAL size is 0 or there is no IO + // error), all the Manifest write will be map to soft error. + // TODO: kManifestWriteNoWAL and kFlushNoWAL are misleading. Refactor is + // needed. + error_handler_.SetBGError(s, + BackgroundErrorReason::kManifestWriteNoWAL); + } else { + // If WAL sync is successful (either WAL size is 0 or there is no IO + // error), all the other SST file write errors will be set as + // kFlushNoWAL. + error_handler_.SetBGError(s, BackgroundErrorReason::kFlushNoWAL); + } + } else { + assert(s == log_io_s); + Status new_bg_error = s; + error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush); + } + } + // If flush ran smoothly and no mempurge happened + // install new SST file path. + if (s.ok() && (!switched_to_mempurge)) { +#ifndef ROCKSDB_LITE + // may temporarily unlock and lock the mutex. + NotifyOnFlushCompleted(cfd, mutable_cf_options, + flush_job.GetCommittedFlushJobsInfo()); + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); + 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()); + // 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. + sfm->OnAddFile(file_path).PermitUncheckedError(); + if (sfm->IsMaxAllowedSpaceReached()) { + Status new_bg_error = + Status::SpaceLimit("Max allowed space was reached"); + TEST_SYNC_POINT_CALLBACK( + "DBImpl::FlushMemTableToOutputFile:MaxAllowedSpaceReached", + &new_bg_error); + error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush); + } + } +#endif // ROCKSDB_LITE + } + TEST_SYNC_POINT("DBImpl::FlushMemTableToOutputFile:Finish"); + return s; +} + +Status DBImpl::FlushMemTablesToOutputFiles( + const autovector& bg_flush_args, bool* made_progress, + JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri) { + if (immutable_db_options_.atomic_flush) { + return AtomicFlushMemTablesToOutputFiles( + bg_flush_args, made_progress, job_context, log_buffer, thread_pri); + } + assert(bg_flush_args.size() == 1); + std::vector snapshot_seqs; + SequenceNumber earliest_write_conflict_snapshot; + SnapshotChecker* snapshot_checker; + GetSnapshotContext(job_context, &snapshot_seqs, + &earliest_write_conflict_snapshot, &snapshot_checker); + 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(); + SuperVersionContext* superversion_context = + bg_flush_arg.superversion_context_; + Status s = FlushMemTableToOutputFile( + cfd, mutable_cf_options_copy, made_progress, job_context, + superversion_context, snapshot_seqs, earliest_write_conflict_snapshot, + snapshot_checker, log_buffer, thread_pri); + return s; +} + +/* + * Atomically flushes multiple column families. + * + * For each column family, all memtables with ID smaller than or equal to the + * ID specified in bg_flush_args will be flushed. Only after all column + * families finish flush will this function commit to MANIFEST. If any of the + * column families are not flushed successfully, this function does not have + * any side-effect on the state of the database. + */ +Status DBImpl::AtomicFlushMemTablesToOutputFiles( + const autovector& bg_flush_args, bool* made_progress, + JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri) { + mutex_.AssertHeld(); + + autovector cfds; + for (const auto& arg : bg_flush_args) { + cfds.emplace_back(arg.cfd_); + } + +#ifndef NDEBUG + for (const auto cfd : cfds) { + assert(cfd->imm()->NumNotFlushed() != 0); + assert(cfd->imm()->IsFlushPending()); + assert(cfd->GetFlushReason() == cfds[0]->GetFlushReason()); + } +#endif /* !NDEBUG */ + + std::vector snapshot_seqs; + SequenceNumber earliest_write_conflict_snapshot; + SnapshotChecker* snapshot_checker; + GetSnapshotContext(job_context, &snapshot_seqs, + &earliest_write_conflict_snapshot, &snapshot_checker); + + autovector distinct_output_dirs; + autovector distinct_output_dir_paths; + std::vector> jobs; + std::vector all_mutable_cf_options; + int num_cfs = static_cast(cfds.size()); + all_mutable_cf_options.reserve(num_cfs); + 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; + + // Add to distinct output directories if eligible. Use linear search. Since + // the number of elements in the vector is not large, performance should be + // tolerable. + bool found = false; + for (const auto& path : distinct_output_dir_paths) { + if (path == curr_path) { + found = true; + break; + } + } + if (!found) { + distinct_output_dir_paths.emplace_back(curr_path); + distinct_output_dirs.emplace_back(data_dir); + } + + 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_; + jobs.emplace_back(new FlushJob( + dbname_, cfd, immutable_db_options_, mutable_cf_options, + max_memtable_id, file_options_for_compaction_, versions_.get(), &mutex_, + &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot, + snapshot_checker, job_context, log_buffer, directories_.GetDbDir(), + data_dir, GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), + stats_, &event_logger_, mutable_cf_options.report_bg_io_stats, + false /* sync_output_directory */, false /* write_manifest */, + thread_pri, io_tracer_, seqno_time_mapping_, db_id_, db_session_id_, + cfd->GetFullHistoryTsLow(), &blob_callback_)); + } + + std::vector file_meta(num_cfs); + // Use of deque because vector + // is specific and doesn't allow &v[i]. + std::deque switched_to_mempurge(num_cfs, false); + Status s; + IOStatus log_io_s = IOStatus::OK(); + assert(num_cfs == static_cast(jobs.size())); + +#ifndef ROCKSDB_LITE + for (int i = 0; i != num_cfs; ++i) { + const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.at(i); + // may temporarily unlock and lock the mutex. + NotifyOnFlushBegin(cfds[i], &file_meta[i], mutable_cf_options, + job_context->job_id); + } +#endif /* !ROCKSDB_LITE */ + + if (logfile_number_ > 0) { + // TODO (yanqin) investigate whether we should sync the closed logs for + // single column family case. + VersionEdit synced_wals; + mutex_.Unlock(); + log_io_s = SyncClosedLogs(job_context, &synced_wals); + mutex_.Lock(); + if (log_io_s.ok() && synced_wals.IsWalAddition()) { + log_io_s = status_to_io_status(ApplyWALToManifest(&synced_wals)); + } + + if (!log_io_s.ok() && !log_io_s.IsShutdownInProgress() && + !log_io_s.IsColumnFamilyDropped()) { + if (total_log_size_ > 0) { + error_handler_.SetBGError(log_io_s, BackgroundErrorReason::kFlush); + } else { + // If the WAL is empty, we use different error reason + error_handler_.SetBGError(log_io_s, BackgroundErrorReason::kFlushNoWAL); + } + } + } + s = log_io_s; + + // exec_status stores the execution status of flush_jobs as + // + autovector> exec_status; + std::vector pick_status; + for (int i = 0; i != num_cfs; ++i) { + // Initially all jobs are not executed, with status OK. + exec_status.emplace_back(false, Status::OK()); + pick_status.push_back(false); + } + + if (s.ok()) { + for (int i = 0; i != num_cfs; ++i) { + jobs[i]->PickMemTable(); + pick_status[i] = true; + } + } + + if (s.ok()) { + assert(switched_to_mempurge.size() == + static_cast(num_cfs)); + // TODO (yanqin): parallelize jobs with threads. + for (int i = 1; i != num_cfs; ++i) { + exec_status[i].second = + jobs[i]->Run(&logs_with_prep_tracker_, &file_meta[i], + &(switched_to_mempurge.at(i))); + exec_status[i].first = true; + } + if (num_cfs > 1) { + TEST_SYNC_POINT( + "DBImpl::AtomicFlushMemTablesToOutputFiles:SomeFlushJobsComplete:1"); + TEST_SYNC_POINT( + "DBImpl::AtomicFlushMemTablesToOutputFiles:SomeFlushJobsComplete:2"); + } + assert(exec_status.size() > 0); + assert(!file_meta.empty()); + exec_status[0].second = jobs[0]->Run( + &logs_with_prep_tracker_, file_meta.data() /* &file_meta[0] */, + switched_to_mempurge.empty() ? nullptr : &(switched_to_mempurge.at(0))); + exec_status[0].first = true; + + Status error_status; + for (const auto& e : exec_status) { + if (!e.second.ok()) { + s = e.second; + if (!e.second.IsShutdownInProgress() && + !e.second.IsColumnFamilyDropped()) { + // If a flush job did not return OK, and the CF is not dropped, and + // the DB is not shutting down, then we have to return this result to + // caller later. + error_status = e.second; + } + } + } + + s = error_status.ok() ? s : error_status; + } + + if (s.IsColumnFamilyDropped()) { + s = Status::OK(); + } + + if (s.ok() || s.IsShutdownInProgress()) { + // Sync on all distinct output directories. + for (auto dir : distinct_output_dirs) { + if (dir != nullptr) { + Status error_status = dir->FsyncWithDirOptions( + IOOptions(), nullptr, + DirFsyncOptions(DirFsyncOptions::FsyncReason::kNewFileSynced)); + if (!error_status.ok()) { + s = error_status; + break; + } + } + } + } else { + // Need to undo atomic flush if something went wrong, i.e. s is not OK and + // it is not because of CF drop. + // Have to cancel the flush jobs that have NOT executed because we need to + // unref the versions. + for (int i = 0; i != num_cfs; ++i) { + if (pick_status[i] && !exec_status[i].first) { + jobs[i]->Cancel(); + } + } + for (int i = 0; i != num_cfs; ++i) { + if (exec_status[i].second.ok() && exec_status[i].first) { + auto& mems = jobs[i]->GetMemTables(); + cfds[i]->imm()->RollbackMemtableFlush(mems, + file_meta[i].fd.GetNumber()); + } + } + } + + if (s.ok()) { + const auto wait_to_install_func = + [&]() -> std::pair { + if (!versions_->io_status().ok()) { + // Something went wrong elsewhere, we cannot count on waiting for our + // turn to write/sync to MANIFEST or CURRENT. Just return. + return std::make_pair(versions_->io_status(), false); + } else if (shutting_down_.load(std::memory_order_acquire)) { + return std::make_pair(Status::ShutdownInProgress(), false); + } + bool ready = true; + for (size_t i = 0; i != cfds.size(); ++i) { + const auto& mems = jobs[i]->GetMemTables(); + if (cfds[i]->IsDropped()) { + // If the column family is dropped, then do not wait. + continue; + } else if (!mems.empty() && + cfds[i]->imm()->GetEarliestMemTableID() < mems[0]->GetID()) { + // If a flush job needs to install the flush result for mems and + // mems[0] is not the earliest memtable, it means another thread must + // be installing flush results for the same column family, then the + // current thread needs to wait. + ready = false; + break; + } else if (mems.empty() && cfds[i]->imm()->GetEarliestMemTableID() <= + bg_flush_args[i].max_memtable_id_) { + // If a flush job does not need to install flush results, then it has + // to wait until all memtables up to max_memtable_id_ (inclusive) are + // installed. + ready = false; + break; + } + } + return std::make_pair(Status::OK(), !ready); + }; + + bool resuming_from_bg_err = + error_handler_.IsDBStopped() || + (cfds[0]->GetFlushReason() == FlushReason::kErrorRecovery || + cfds[0]->GetFlushReason() == FlushReason::kErrorRecoveryRetryFlush); + while ((!resuming_from_bg_err || error_handler_.GetRecoveryError().ok())) { + std::pair res = wait_to_install_func(); + + TEST_SYNC_POINT_CALLBACK( + "DBImpl::AtomicFlushMemTablesToOutputFiles:WaitToCommit", &res); + + if (!res.first.ok()) { + s = res.first; + break; + } else if (!res.second) { + break; + } + atomic_flush_install_cv_.Wait(); + + resuming_from_bg_err = + error_handler_.IsDBStopped() || + (cfds[0]->GetFlushReason() == FlushReason::kErrorRecovery || + cfds[0]->GetFlushReason() == FlushReason::kErrorRecoveryRetryFlush); + } + + if (!resuming_from_bg_err) { + // If not resuming from bg err, then we determine future action based on + // whether we hit background error. + if (s.ok()) { + s = error_handler_.GetBGError(); + } + } else if (s.ok()) { + // If resuming from bg err, we still rely on wait_to_install_func()'s + // result to determine future action. If wait_to_install_func() returns + // non-ok already, then we should not proceed to flush result + // installation. + s = error_handler_.GetRecoveryError(); + } + } + + if (s.ok()) { + autovector tmp_cfds; + autovector*> mems_list; + autovector mutable_cf_options_list; + autovector tmp_file_meta; + autovector>*> + committed_flush_jobs_info; + for (int i = 0; i != num_cfs; ++i) { + const auto& mems = jobs[i]->GetMemTables(); + if (!cfds[i]->IsDropped() && !mems.empty()) { + tmp_cfds.emplace_back(cfds[i]); + mems_list.emplace_back(&mems); + mutable_cf_options_list.emplace_back(&all_mutable_cf_options[i]); + tmp_file_meta.emplace_back(&file_meta[i]); +#ifndef ROCKSDB_LITE + committed_flush_jobs_info.emplace_back( + jobs[i]->GetCommittedFlushJobsInfo()); +#endif //! ROCKSDB_LITE + } + } + + s = InstallMemtableAtomicFlushResults( + nullptr /* imm_lists */, tmp_cfds, mutable_cf_options_list, mems_list, + versions_.get(), &logs_with_prep_tracker_, &mutex_, tmp_file_meta, + committed_flush_jobs_info, &job_context->memtables_to_free, + directories_.GetDbDir(), log_buffer); + } + + if (s.ok()) { + assert(num_cfs == + static_cast(job_context->superversion_contexts.size())); + for (int i = 0; i != num_cfs; ++i) { + assert(cfds[i]); + + if (cfds[i]->IsDropped()) { + continue; + } + InstallSuperVersionAndScheduleWork(cfds[i], + &job_context->superversion_contexts[i], + all_mutable_cf_options[i]); + + const std::string& column_family_name = cfds[i]->GetName(); + + Version* const current = cfds[i]->current(); + assert(current); + + const VersionStorageInfo* const storage_info = current->storage_info(); + assert(storage_info); + + VersionStorageInfo::LevelSummaryStorage tmp; + ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n", + column_family_name.c_str(), + storage_info->LevelSummary(&tmp)); + + const auto& blob_files = storage_info->GetBlobFiles(); + if (!blob_files.empty()) { + assert(blob_files.front()); + assert(blob_files.back()); + + ROCKS_LOG_BUFFER( + log_buffer, + "[%s] Blob file summary: head=%" PRIu64 ", tail=%" PRIu64 "\n", + column_family_name.c_str(), blob_files.front()->GetBlobFileNumber(), + blob_files.back()->GetBlobFileNumber()); + } + } + if (made_progress) { + *made_progress = true; + } +#ifndef ROCKSDB_LITE + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); + assert(all_mutable_cf_options.size() == static_cast(num_cfs)); + for (int i = 0; s.ok() && i != num_cfs; ++i) { + // If mempurge happened instead of Flush, + // no NotifyOnFlushCompleted call (no SST file created). + if (switched_to_mempurge[i]) { + continue; + } + if (cfds[i]->IsDropped()) { + continue; + } + NotifyOnFlushCompleted(cfds[i], all_mutable_cf_options[i], + jobs[i]->GetCommittedFlushJobsInfo()); + if (sfm) { + std::string file_path = MakeTableFileName( + 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. + sfm->OnAddFile(file_path).PermitUncheckedError(); + if (sfm->IsMaxAllowedSpaceReached() && + error_handler_.GetBGError().ok()) { + Status new_bg_error = + Status::SpaceLimit("Max allowed space was reached"); + error_handler_.SetBGError(new_bg_error, + BackgroundErrorReason::kFlush); + } + } + } +#endif // ROCKSDB_LITE + } + + // Need to undo atomic flush if something went wrong, i.e. s is not OK and + // it is not because of CF drop. + if (!s.ok() && !s.IsColumnFamilyDropped()) { + if (log_io_s.ok()) { + // Error while writing to MANIFEST. + // In fact, versions_->io_status() can also be the result of renaming + // CURRENT file. With current code, it's just difficult to tell. So just + // be pessimistic and try write to a new MANIFEST. + // TODO: distinguish between MANIFEST write and CURRENT renaming + if (!versions_->io_status().ok()) { + // If WAL sync is successful (either WAL size is 0 or there is no IO + // error), all the Manifest write will be map to soft error. + // TODO: kManifestWriteNoWAL and kFlushNoWAL are misleading. Refactor + // is needed. + error_handler_.SetBGError(s, + BackgroundErrorReason::kManifestWriteNoWAL); + } else { + // If WAL sync is successful (either WAL size is 0 or there is no IO + // error), all the other SST file write errors will be set as + // kFlushNoWAL. + error_handler_.SetBGError(s, BackgroundErrorReason::kFlushNoWAL); + } + } else { + assert(s == log_io_s); + Status new_bg_error = s; + error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush); + } + } + + return s; +} + +void DBImpl::NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta, + const MutableCFOptions& mutable_cf_options, + int job_id) { +#ifndef ROCKSDB_LITE + if (immutable_db_options_.listeners.size() == 0U) { + return; + } + mutex_.AssertHeld(); + if (shutting_down_.load(std::memory_order_acquire)) { + return; + } + bool triggered_writes_slowdown = + (cfd->current()->storage_info()->NumLevelFiles(0) >= + mutable_cf_options.level0_slowdown_writes_trigger); + bool triggered_writes_stop = + (cfd->current()->storage_info()->NumLevelFiles(0) >= + mutable_cf_options.level0_stop_writes_trigger); + // release lock while notifying events + mutex_.Unlock(); + { + FlushJobInfo info{}; + info.cf_id = cfd->GetID(); + info.cf_name = cfd->GetName(); + // TODO(yhchiang): make db_paths dynamic in case flush does not + // 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); + info.file_number = file_number; + info.thread_id = env_->GetThreadID(); + info.job_id = job_id; + info.triggered_writes_slowdown = triggered_writes_slowdown; + info.triggered_writes_stop = triggered_writes_stop; + info.smallest_seqno = file_meta->fd.smallest_seqno; + info.largest_seqno = file_meta->fd.largest_seqno; + info.flush_reason = cfd->GetFlushReason(); + for (auto listener : immutable_db_options_.listeners) { + listener->OnFlushBegin(this, info); + } + } + mutex_.Lock(); +// no need to signal bg_cv_ as it will be signaled at the end of the +// flush process. +#else + (void)cfd; + (void)file_meta; + (void)mutable_cf_options; + (void)job_id; +#endif // ROCKSDB_LITE +} + +void DBImpl::NotifyOnFlushCompleted( + ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, + std::list>* flush_jobs_info) { +#ifndef ROCKSDB_LITE + assert(flush_jobs_info != nullptr); + if (immutable_db_options_.listeners.size() == 0U) { + return; + } + mutex_.AssertHeld(); + if (shutting_down_.load(std::memory_order_acquire)) { + return; + } + bool triggered_writes_slowdown = + (cfd->current()->storage_info()->NumLevelFiles(0) >= + mutable_cf_options.level0_slowdown_writes_trigger); + bool triggered_writes_stop = + (cfd->current()->storage_info()->NumLevelFiles(0) >= + mutable_cf_options.level0_stop_writes_trigger); + // release lock while notifying events + mutex_.Unlock(); + { + for (auto& info : *flush_jobs_info) { + info->triggered_writes_slowdown = triggered_writes_slowdown; + info->triggered_writes_stop = triggered_writes_stop; + for (auto listener : immutable_db_options_.listeners) { + listener->OnFlushCompleted(this, *info); + } + TEST_SYNC_POINT( + "DBImpl::NotifyOnFlushCompleted::PostAllOnFlushCompleted"); + } + flush_jobs_info->clear(); + } + mutex_.Lock(); + // no need to signal bg_cv_ as it will be signaled at the end of the + // flush process. +#else + (void)cfd; + (void)mutable_cf_options; + (void)flush_jobs_info; +#endif // ROCKSDB_LITE +} + +Status DBImpl::CompactRange(const CompactRangeOptions& options, + ColumnFamilyHandle* column_family, + const Slice* begin_without_ts, + const Slice* end_without_ts) { + if (manual_compaction_paused_.load(std::memory_order_acquire) > 0) { + return Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + + if (options.canceled && options.canceled->load(std::memory_order_acquire)) { + return Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + + const Comparator* const ucmp = column_family->GetComparator(); + assert(ucmp); + size_t ts_sz = ucmp->timestamp_size(); + if (ts_sz == 0) { + return CompactRangeInternal(options, column_family, begin_without_ts, + end_without_ts, "" /*trim_ts*/); + } + + std::string begin_str; + std::string end_str; + + // CompactRange compact all keys: [begin, end] inclusively. Add maximum + // timestamp to include all `begin` keys, and add minimal timestamp to include + // all `end` keys. + if (begin_without_ts != nullptr) { + AppendKeyWithMaxTimestamp(&begin_str, *begin_without_ts, ts_sz); + } + if (end_without_ts != nullptr) { + AppendKeyWithMinTimestamp(&end_str, *end_without_ts, ts_sz); + } + Slice begin(begin_str); + Slice end(end_str); + + Slice* begin_with_ts = begin_without_ts ? &begin : nullptr; + Slice* end_with_ts = end_without_ts ? &end : nullptr; + + return CompactRangeInternal(options, column_family, begin_with_ts, + end_with_ts, "" /*trim_ts*/); +} + +Status DBImpl::IncreaseFullHistoryTsLow(ColumnFamilyHandle* column_family, + std::string ts_low) { + ColumnFamilyData* cfd = nullptr; + if (column_family == nullptr) { + cfd = default_cf_handle_->cfd(); + } else { + auto cfh = static_cast_with_check(column_family); + assert(cfh != nullptr); + cfd = cfh->cfd(); + } + assert(cfd != nullptr && cfd->user_comparator() != nullptr); + if (cfd->user_comparator()->timestamp_size() == 0) { + return Status::InvalidArgument( + "Timestamp is not enabled in this column family"); + } + if (cfd->user_comparator()->timestamp_size() != ts_low.size()) { + return Status::InvalidArgument("ts_low size mismatch"); + } + return IncreaseFullHistoryTsLowImpl(cfd, ts_low); +} + +Status DBImpl::IncreaseFullHistoryTsLowImpl(ColumnFamilyData* cfd, + std::string ts_low) { + VersionEdit edit; + edit.SetColumnFamily(cfd->GetID()); + edit.SetFullHistoryTsLow(ts_low); + TEST_SYNC_POINT_CALLBACK("DBImpl::IncreaseFullHistoryTsLowImpl:BeforeEdit", + &edit); + + InstrumentedMutexLock l(&mutex_); + std::string current_ts_low = cfd->GetFullHistoryTsLow(); + const Comparator* ucmp = cfd->user_comparator(); + assert(ucmp->timestamp_size() == ts_low.size() && !ts_low.empty()); + if (!current_ts_low.empty() && + ucmp->CompareTimestamp(ts_low, current_ts_low) < 0) { + return Status::InvalidArgument("Cannot decrease full_history_ts_low"); + } + + Status s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + &edit, &mutex_, directories_.GetDbDir()); + if (!s.ok()) { + return s; + } + current_ts_low = cfd->GetFullHistoryTsLow(); + if (!current_ts_low.empty() && + ucmp->CompareTimestamp(current_ts_low, ts_low) > 0) { + std::stringstream oss; + oss << "full_history_ts_low: " << Slice(current_ts_low).ToString(true) + << " is set to be higher than the requested " + "timestamp: " + << Slice(ts_low).ToString(true) << std::endl; + return Status::TryAgain(oss.str()); + } + return Status::OK(); +} + +Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, + ColumnFamilyHandle* column_family, + const Slice* begin, const Slice* end, + const std::string& trim_ts) { + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + + if (options.target_path_id >= cfd->ioptions()->cf_paths.size()) { + return Status::InvalidArgument("Invalid target path ID"); + } + + bool flush_needed = true; + + // Update full_history_ts_low if it's set + if (options.full_history_ts_low != nullptr && + !options.full_history_ts_low->empty()) { + std::string ts_low = options.full_history_ts_low->ToString(); + if (begin != nullptr || end != nullptr) { + return Status::InvalidArgument( + "Cannot specify compaction range with full_history_ts_low"); + } + Status s = IncreaseFullHistoryTsLowImpl(cfd, ts_low); + if (!s.ok()) { + LogFlush(immutable_db_options_.info_log); + return s; + } + } + + Status s; + if (begin != nullptr && end != nullptr) { + // TODO(ajkr): We could also optimize away the flush in certain cases where + // one/both sides of the interval are unbounded. But it requires more + // changes to RangesOverlapWithMemtables. + Range range(*begin, *end); + SuperVersion* super_version = cfd->GetReferencedSuperVersion(this); + s = cfd->RangesOverlapWithMemtables( + {range}, super_version, immutable_db_options_.allow_data_in_errors, + &flush_needed); + CleanupSuperVersion(super_version); + } + + if (s.ok() && flush_needed) { + FlushOptions fo; + fo.allow_write_stall = options.allow_write_stall; + if (immutable_db_options_.atomic_flush) { + autovector cfds; + mutex_.Lock(); + SelectColumnFamiliesForAtomicFlush(&cfds); + mutex_.Unlock(); + s = AtomicFlushMemTables(cfds, fo, FlushReason::kManualCompaction, + false /* entered_write_thread */); + } else { + s = FlushMemTable(cfd, fo, FlushReason::kManualCompaction, + false /* entered_write_thread */); + } + if (!s.ok()) { + LogFlush(immutable_db_options_.info_log); + return s; + } + } + + constexpr int kInvalidLevel = -1; + int final_output_level = kInvalidLevel; + bool exclusive = options.exclusive_manual_compaction; + if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal && + cfd->NumberLevels() > 1) { + // Always compact all files together. + final_output_level = cfd->NumberLevels() - 1; + // if bottom most level is reserved + if (immutable_db_options_.allow_ingest_behind) { + final_output_level--; + } + s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels, + final_output_level, options, begin, end, exclusive, + false, std::numeric_limits::max(), + trim_ts); + } else { + int first_overlapped_level = kInvalidLevel; + int max_overlapped_level = kInvalidLevel; + { + SuperVersion* super_version = cfd->GetReferencedSuperVersion(this); + Version* current_version = super_version->current; + ReadOptions ro; + ro.total_order_seek = true; + bool overlap; + for (int level = 0; + level < current_version->storage_info()->num_non_empty_levels(); + level++) { + overlap = true; + if (begin != nullptr && end != nullptr) { + Status status = current_version->OverlapWithLevelIterator( + ro, file_options_, *begin, *end, level, &overlap); + if (!status.ok()) { + overlap = current_version->storage_info()->OverlapInLevel( + level, begin, end); + } + } else { + overlap = current_version->storage_info()->OverlapInLevel(level, + begin, end); + } + if (overlap) { + if (first_overlapped_level == kInvalidLevel) { + first_overlapped_level = level; + } + max_overlapped_level = level; + } + } + CleanupSuperVersion(super_version); + } + if (s.ok() && first_overlapped_level != kInvalidLevel) { + // max_file_num_to_ignore can be used to filter out newly created SST + // files, useful for bottom level compaction in a manual compaction + uint64_t max_file_num_to_ignore = std::numeric_limits::max(); + uint64_t next_file_number = versions_->current_next_file_number(); + final_output_level = max_overlapped_level; + int output_level; + for (int level = first_overlapped_level; level <= max_overlapped_level; + level++) { + bool disallow_trivial_move = false; + // in case the compaction is universal or if we're compacting the + // bottom-most level, the output level will be the same as input one. + // level 0 can never be the bottommost level (i.e. if all files are in + // level 0, we will compact to level 1) + if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal || + cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + output_level = level; + } else if (level == max_overlapped_level && level > 0) { + if (options.bottommost_level_compaction == + BottommostLevelCompaction::kSkip) { + // Skip bottommost level compaction + continue; + } else if (options.bottommost_level_compaction == + BottommostLevelCompaction::kIfHaveCompactionFilter && + cfd->ioptions()->compaction_filter == nullptr && + cfd->ioptions()->compaction_filter_factory == nullptr) { + // Skip bottommost level compaction since we don't have a compaction + // filter + continue; + } + output_level = level; + // update max_file_num_to_ignore only for bottom level compaction + // because data in newly compacted files in middle levels may still + // need to be pushed down + max_file_num_to_ignore = next_file_number; + } else { + output_level = level + 1; + if (cfd->ioptions()->compaction_style == kCompactionStyleLevel && + cfd->ioptions()->level_compaction_dynamic_level_bytes && + level == 0) { + output_level = ColumnFamilyData::kCompactToBaseLevel; + } + // if it's a BottommostLevel compaction and `kForce*` compaction is + // set, disallow trivial move + if (level == max_overlapped_level && + (options.bottommost_level_compaction == + BottommostLevelCompaction::kForce || + options.bottommost_level_compaction == + BottommostLevelCompaction::kForceOptimized)) { + disallow_trivial_move = true; + } + } + // trim_ts need real compaction to remove latest record + if (!trim_ts.empty()) { + disallow_trivial_move = true; + } + s = RunManualCompaction(cfd, level, output_level, options, begin, end, + exclusive, disallow_trivial_move, + max_file_num_to_ignore, trim_ts); + if (!s.ok()) { + break; + } + if (output_level == ColumnFamilyData::kCompactToBaseLevel) { + final_output_level = cfd->NumberLevels() - 1; + } else if (output_level > final_output_level) { + final_output_level = output_level; + } + TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1"); + TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2"); + } + } + } + if (!s.ok() || final_output_level == kInvalidLevel) { + LogFlush(immutable_db_options_.info_log); + return s; + } + + if (options.change_level) { + TEST_SYNC_POINT("DBImpl::CompactRange:BeforeRefit:1"); + TEST_SYNC_POINT("DBImpl::CompactRange:BeforeRefit:2"); + + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[RefitLevel] waiting for background threads to stop"); + DisableManualCompaction(); + s = PauseBackgroundWork(); + if (s.ok()) { + TEST_SYNC_POINT("DBImpl::CompactRange:PreRefitLevel"); + s = ReFitLevel(cfd, final_output_level, options.target_level); + TEST_SYNC_POINT("DBImpl::CompactRange:PostRefitLevel"); + // ContinueBackgroundWork always return Status::OK(). + Status temp_s = ContinueBackgroundWork(); + assert(temp_s.ok()); + } + EnableManualCompaction(); + TEST_SYNC_POINT( + "DBImpl::CompactRange:PostRefitLevel:ManualCompactionEnabled"); + } + LogFlush(immutable_db_options_.info_log); + + { + InstrumentedMutexLock l(&mutex_); + // an automatic compaction that has been scheduled might have been + // preempted by the manual compactions. Need to schedule it back. + MaybeScheduleFlushOrCompaction(); + } + + return s; +} + +Status DBImpl::CompactFiles(const CompactionOptions& compact_options, + ColumnFamilyHandle* column_family, + const std::vector& input_file_names, + const int output_level, const int output_path_id, + std::vector* const output_file_names, + CompactionJobInfo* compaction_job_info) { +#ifdef ROCKSDB_LITE + (void)compact_options; + (void)column_family; + (void)input_file_names; + (void)output_level; + (void)output_path_id; + (void)output_file_names; + (void)compaction_job_info; + // not supported in lite version + return Status::NotSupported("Not supported in ROCKSDB LITE"); +#else + if (column_family == nullptr) { + return Status::InvalidArgument("ColumnFamilyHandle must be non-null."); + } + + auto cfd = + static_cast_with_check(column_family)->cfd(); + assert(cfd); + + Status s; + JobContext job_context(next_job_id_.fetch_add(1), true); + LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, + immutable_db_options_.info_log.get()); + + // Perform CompactFiles + TEST_SYNC_POINT("TestCompactFiles::IngestExternalFile2"); + TEST_SYNC_POINT_CALLBACK( + "TestCompactFiles:PausingManualCompaction:3", + reinterpret_cast( + const_cast*>(&manual_compaction_paused_))); + { + InstrumentedMutexLock l(&mutex_); + + // This call will unlock/lock the mutex to wait for current running + // IngestExternalFile() calls to finish. + WaitForIngestFile(); + + // We need to get current after `WaitForIngestFile`, because + // `IngestExternalFile` may add files that overlap with `input_file_names` + auto* current = cfd->current(); + current->Ref(); + + s = CompactFilesImpl(compact_options, cfd, current, input_file_names, + output_file_names, output_level, output_path_id, + &job_context, &log_buffer, compaction_job_info); + + current->Unref(); + } + + // Find and delete obsolete files + { + InstrumentedMutexLock l(&mutex_); + // If !s.ok(), this means that Compaction failed. In that case, we want + // to delete all obsolete files we might have created and we force + // FindObsoleteFiles(). This is because job_context does not + // catch all created files if compaction failed. + FindObsoleteFiles(&job_context, !s.ok()); + } // release the mutex + + // delete unnecessary files if any, this is done outside the mutex + if (job_context.HaveSomethingToClean() || + job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) { + // Have to flush the info logs before bg_compaction_scheduled_-- + // because if bg_flush_scheduled_ becomes 0 and the lock is + // released, the deconstructor of DB can kick in and destroy all the + // states of DB so info_log might not be available after that point. + // It also applies to access other states that DB owns. + log_buffer.FlushBufferToLog(); + if (job_context.HaveSomethingToDelete()) { + // no mutex is locked here. No need to Unlock() and Lock() here. + PurgeObsoleteFiles(job_context); + } + job_context.Clean(); + } + + return s; +#endif // ROCKSDB_LITE +} + +#ifndef ROCKSDB_LITE +Status DBImpl::CompactFilesImpl( + const CompactionOptions& compact_options, ColumnFamilyData* cfd, + Version* version, const std::vector& input_file_names, + std::vector* const output_file_names, const int output_level, + int output_path_id, JobContext* job_context, LogBuffer* log_buffer, + CompactionJobInfo* compaction_job_info) { + mutex_.AssertHeld(); + + if (shutting_down_.load(std::memory_order_acquire)) { + return Status::ShutdownInProgress(); + } + if (manual_compaction_paused_.load(std::memory_order_acquire) > 0) { + return Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + + std::unordered_set input_set; + for (const auto& file_name : input_file_names) { + input_set.insert(TableFileNameToNumber(file_name)); + } + + ColumnFamilyMetaData cf_meta; + // TODO(yhchiang): can directly use version here if none of the + // following functions call is pluggable to external developers. + version->GetColumnFamilyMetaData(&cf_meta); + + if (output_path_id < 0) { + if (cfd->ioptions()->cf_paths.size() == 1U) { + output_path_id = 0; + } else { + return Status::NotSupported( + "Automatic output path selection is not " + "yet supported in CompactFiles()"); + } + } + + Status s = cfd->compaction_picker()->SanitizeCompactionInputFiles( + &input_set, cf_meta, output_level); + if (!s.ok()) { + return s; + } + + std::vector input_files; + s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers( + &input_files, &input_set, version->storage_info(), compact_options); + if (!s.ok()) { + return s; + } + + for (const auto& inputs : input_files) { + if (cfd->compaction_picker()->AreFilesInCompaction(inputs.files)) { + return Status::Aborted( + "Some of the necessary compaction input " + "files are already being compacted"); + } + } + bool sfm_reserved_compact_space = false; + // First check if we have enough room to do the compaction + bool enough_room = EnoughRoomForCompaction( + cfd, input_files, &sfm_reserved_compact_space, log_buffer); + + if (!enough_room) { + // m's vars will get set properly at the end of this function, + // as long as status == CompactionTooLarge + return Status::CompactionTooLarge(); + } + + // At this point, CompactFiles will be run. + bg_compaction_scheduled_++; + + std::unique_ptr c; + 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)); + // 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); + + c->SetInputVersion(version); + // deletion compaction currently not allowed in CompactFiles. + assert(!c->deletion_compaction()); + + std::vector snapshot_seqs; + SequenceNumber earliest_write_conflict_snapshot; + SnapshotChecker* snapshot_checker; + GetSnapshotContext(job_context, &snapshot_seqs, + &earliest_write_conflict_snapshot, &snapshot_checker); + + std::unique_ptr::iterator> pending_outputs_inserted_elem( + new std::list::iterator( + CaptureCurrentFileNumberInPendingOutputs())); + + assert(is_snapshot_supported_ || snapshots_.empty()); + CompactionJobStats compaction_job_stats; + CompactionJob compaction_job( + job_context->job_id, c.get(), immutable_db_options_, mutable_db_options_, + file_options_for_compaction_, versions_.get(), &shutting_down_, + log_buffer, directories_.GetDbDir(), + GetDataDir(c->column_family_data(), c->output_path_id()), + 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_, + &compaction_job_stats, Env::Priority::USER, io_tracer_, + kManualCompactionCanceledFalse_, db_id_, db_session_id_, + c->column_family_data()->GetFullHistoryTsLow(), c->trim_ts(), + &blob_callback_, &bg_compaction_scheduled_, + &bg_bottom_compaction_scheduled_); + + // Creating a compaction influences the compaction score because the score + // 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()); + + compaction_job.Prepare(); + + mutex_.Unlock(); + TEST_SYNC_POINT("CompactFilesImpl:0"); + TEST_SYNC_POINT("CompactFilesImpl:1"); + // Ignore the status here, as it will be checked in the Install down below... + compaction_job.Run().PermitUncheckedError(); + TEST_SYNC_POINT("CompactFilesImpl:2"); + TEST_SYNC_POINT("CompactFilesImpl:3"); + mutex_.Lock(); + + Status status = compaction_job.Install(*c->mutable_cf_options()); + if (status.ok()) { + assert(compaction_job.io_status().ok()); + InstallSuperVersionAndScheduleWork(c->column_family_data(), + &job_context->superversion_contexts[0], + *c->mutable_cf_options()); + } + // status above captures any error during compaction_job.Install, so its ok + // not check compaction_job.io_status() explicitly if we're not calling + // SetBGError + compaction_job.io_status().PermitUncheckedError(); + c->ReleaseCompactionFiles(s); +#ifndef ROCKSDB_LITE + // Need to make sure SstFileManager does its bookkeeping + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); + if (sfm && sfm_reserved_compact_space) { + sfm->OnCompactionCompletion(c.get()); + } +#endif // ROCKSDB_LITE + + ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem); + + if (compaction_job_info != nullptr) { + BuildCompactionJobInfo(cfd, c.get(), s, compaction_job_stats, + job_context->job_id, version, compaction_job_info); + } + + if (status.ok()) { + // Done + } else if (status.IsColumnFamilyDropped() || status.IsShutdownInProgress()) { + // Ignore compaction errors found during shutting down + } else if (status.IsManualCompactionPaused()) { + // Don't report stopping manual compaction as error + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[%s] [JOB %d] Stopping manual compaction", + c->column_family_data()->GetName().c_str(), + job_context->job_id); + } else { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "[%s] [JOB %d] Compaction error: %s", + c->column_family_data()->GetName().c_str(), + job_context->job_id, status.ToString().c_str()); + IOStatus io_s = compaction_job.io_status(); + if (!io_s.ok()) { + error_handler_.SetBGError(io_s, BackgroundErrorReason::kCompaction); + } else { + error_handler_.SetBGError(status, BackgroundErrorReason::kCompaction); + } + } + + 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(), + 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, + blob_file.GetBlobFileNumber())); + } + } + + c.reset(); + + bg_compaction_scheduled_--; + if (bg_compaction_scheduled_ == 0) { + bg_cv_.SignalAll(); + } + MaybeScheduleFlushOrCompaction(); + TEST_SYNC_POINT("CompactFilesImpl:End"); + + return status; +} +#endif // ROCKSDB_LITE + +Status DBImpl::PauseBackgroundWork() { + InstrumentedMutexLock guard_lock(&mutex_); + bg_compaction_paused_++; + while (bg_bottom_compaction_scheduled_ > 0 || bg_compaction_scheduled_ > 0 || + bg_flush_scheduled_ > 0) { + bg_cv_.Wait(); + } + bg_work_paused_++; + return Status::OK(); +} + +Status DBImpl::ContinueBackgroundWork() { + InstrumentedMutexLock guard_lock(&mutex_); + if (bg_work_paused_ == 0) { + return Status::InvalidArgument(); + } + assert(bg_work_paused_ > 0); + assert(bg_compaction_paused_ > 0); + bg_compaction_paused_--; + bg_work_paused_--; + // It's sufficient to check just bg_work_paused_ here since + // bg_work_paused_ is always no greater than bg_compaction_paused_ + if (bg_work_paused_ == 0) { + MaybeScheduleFlushOrCompaction(); + } + return Status::OK(); +} + +void DBImpl::NotifyOnCompactionBegin(ColumnFamilyData* cfd, Compaction* c, + const Status& st, + const CompactionJobStats& job_stats, + int job_id) { +#ifndef ROCKSDB_LITE + if (immutable_db_options_.listeners.empty()) { + return; + } + mutex_.AssertHeld(); + if (shutting_down_.load(std::memory_order_acquire)) { + return; + } + if (c->is_manual_compaction() && + manual_compaction_paused_.load(std::memory_order_acquire) > 0) { + return; + } + + c->SetNotifyOnCompactionCompleted(); + Version* current = cfd->current(); + current->Ref(); + // release lock while notifying events + mutex_.Unlock(); + TEST_SYNC_POINT("DBImpl::NotifyOnCompactionBegin::UnlockMutex"); + { + CompactionJobInfo info{}; + BuildCompactionJobInfo(cfd, c, st, job_stats, job_id, current, &info); + for (auto listener : immutable_db_options_.listeners) { + listener->OnCompactionBegin(this, info); + } + info.status.PermitUncheckedError(); + } + mutex_.Lock(); + current->Unref(); +#else + (void)cfd; + (void)c; + (void)st; + (void)job_stats; + (void)job_id; +#endif // ROCKSDB_LITE +} + +void DBImpl::NotifyOnCompactionCompleted( + ColumnFamilyData* cfd, Compaction* c, const Status& st, + const CompactionJobStats& compaction_job_stats, const int job_id) { +#ifndef ROCKSDB_LITE + if (immutable_db_options_.listeners.size() == 0U) { + return; + } + mutex_.AssertHeld(); + if (shutting_down_.load(std::memory_order_acquire)) { + return; + } + + if (c->ShouldNotifyOnCompactionCompleted() == false) { + return; + } + + Version* current = cfd->current(); + current->Ref(); + // release lock while notifying events + mutex_.Unlock(); + TEST_SYNC_POINT("DBImpl::NotifyOnCompactionCompleted::UnlockMutex"); + { + CompactionJobInfo info{}; + BuildCompactionJobInfo(cfd, c, st, compaction_job_stats, job_id, current, + &info); + for (auto listener : immutable_db_options_.listeners) { + listener->OnCompactionCompleted(this, info); + } + } + mutex_.Lock(); + current->Unref(); + // no need to signal bg_cv_ as it will be signaled at the end of the + // flush process. +#else + (void)cfd; + (void)c; + (void)st; + (void)compaction_job_stats; + (void)job_id; +#endif // ROCKSDB_LITE +} + +// REQUIREMENT: block all background work by calling PauseBackgroundWork() +// before calling this function +Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { + assert(level < cfd->NumberLevels()); + if (target_level >= cfd->NumberLevels()) { + return Status::InvalidArgument("Target level exceeds number of levels"); + } + + SuperVersionContext sv_context(/* create_superversion */ true); + + InstrumentedMutexLock guard_lock(&mutex_); + + // only allow one thread refitting + if (refitting_level_) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[ReFitLevel] another thread is refitting"); + return Status::NotSupported("another thread is refitting"); + } + refitting_level_ = true; + + const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions(); + // move to a smaller level + int to_level = target_level; + if (target_level < 0) { + to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level); + } + + auto* vstorage = cfd->current()->storage_info(); + if (to_level != level) { + if (to_level > level) { + if (level == 0) { + refitting_level_ = false; + return Status::NotSupported( + "Cannot change from level 0 to other levels."); + } + // Check levels are empty for a trivial move + for (int l = level + 1; l <= to_level; l++) { + if (vstorage->NumLevelFiles(l) > 0) { + refitting_level_ = false; + return Status::NotSupported( + "Levels between source and target are not empty for a move."); + } + } + } else { + // to_level < level + // Check levels are empty for a trivial move + for (int l = to_level; l < level; l++) { + if (vstorage->NumLevelFiles(l) > 0) { + refitting_level_ = false; + return Status::NotSupported( + "Levels between source and target are not empty for a move."); + } + } + } + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[%s] Before refitting:\n%s", cfd->GetName().c_str(), + cfd->current()->DebugString().data()); + + VersionEdit edit; + edit.SetColumnFamily(cfd->GetID()); + for (const auto& f : vstorage->LevelFiles(level)) { + edit.DeleteFile(level, f->fd.GetNumber()); + edit.AddFile( + to_level, f->fd.GetNumber(), f->fd.GetPathId(), f->fd.GetFileSize(), + f->smallest, f->largest, f->fd.smallest_seqno, f->fd.largest_seqno, + f->marked_for_compaction, f->temperature, f->oldest_blob_file_number, + f->oldest_ancester_time, f->file_creation_time, f->file_checksum, + f->file_checksum_func_name, f->unique_id); + } + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[%s] Apply version edit:\n%s", cfd->GetName().c_str(), + edit.DebugString().data()); + + Status status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, + &mutex_, directories_.GetDbDir()); + + InstallSuperVersionAndScheduleWork(cfd, &sv_context, mutable_cf_options); + + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] LogAndApply: %s\n", + cfd->GetName().c_str(), status.ToString().data()); + + if (status.ok()) { + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[%s] After refitting:\n%s", cfd->GetName().c_str(), + cfd->current()->DebugString().data()); + } + sv_context.Clean(); + refitting_level_ = false; + + return status; + } + + refitting_level_ = false; + return Status::OK(); +} + +int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) { + auto cfh = static_cast_with_check(column_family); + return cfh->cfd()->NumberLevels(); +} + +int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* /*column_family*/) { + return 0; +} + +int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) { + auto cfh = static_cast_with_check(column_family); + InstrumentedMutexLock l(&mutex_); + return cfh->cfd() + ->GetSuperVersion() + ->mutable_cf_options.level0_stop_writes_trigger; +} + +Status DBImpl::Flush(const FlushOptions& flush_options, + ColumnFamilyHandle* column_family) { + auto cfh = static_cast_with_check(column_family); + ROCKS_LOG_INFO(immutable_db_options_.info_log, "[%s] Manual flush start.", + cfh->GetName().c_str()); + Status s; + if (immutable_db_options_.atomic_flush) { + s = AtomicFlushMemTables({cfh->cfd()}, flush_options, + FlushReason::kManualFlush); + } else { + s = FlushMemTable(cfh->cfd(), flush_options, FlushReason::kManualFlush); + } + + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[%s] Manual flush finished, status: %s\n", + cfh->GetName().c_str(), s.ToString().c_str()); + return s; +} + +Status DBImpl::Flush(const FlushOptions& flush_options, + const std::vector& column_families) { + Status s; + if (!immutable_db_options_.atomic_flush) { + for (auto cfh : column_families) { + s = Flush(flush_options, cfh); + if (!s.ok()) { + break; + } + } + } else { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Manual atomic flush start.\n" + "=====Column families:====="); + for (auto cfh : column_families) { + auto cfhi = static_cast(cfh); + ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s", + cfhi->GetName().c_str()); + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "=====End of column families list====="); + autovector cfds; + std::for_each(column_families.begin(), column_families.end(), + [&cfds](ColumnFamilyHandle* elem) { + auto cfh = static_cast(elem); + cfds.emplace_back(cfh->cfd()); + }); + s = AtomicFlushMemTables(cfds, flush_options, FlushReason::kManualFlush); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Manual atomic flush finished, status: %s\n" + "=====Column families:=====", + s.ToString().c_str()); + for (auto cfh : column_families) { + auto cfhi = static_cast(cfh); + ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s", + cfhi->GetName().c_str()); + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "=====End of column families list====="); + } + return s; +} + +Status DBImpl::RunManualCompaction( + ColumnFamilyData* cfd, int input_level, int output_level, + const CompactRangeOptions& compact_range_options, const Slice* begin, + const Slice* end, bool exclusive, bool disallow_trivial_move, + uint64_t max_file_num_to_ignore, const std::string& trim_ts) { + assert(input_level == ColumnFamilyData::kCompactAllLevels || + input_level >= 0); + + InternalKey begin_storage, end_storage; + CompactionArg* ca = nullptr; + + bool scheduled = false; + bool unscheduled = false; + Env::Priority thread_pool_priority = Env::Priority::TOTAL; + bool manual_conflict = false; + + ManualCompactionState manual( + cfd, input_level, output_level, compact_range_options.target_path_id, + exclusive, disallow_trivial_move, compact_range_options.canceled); + // 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) { + 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) { + manual.end = nullptr; + } else { + end_storage.SetMaxPossibleForUserKey(*end); + manual.end = &end_storage; + } + + TEST_SYNC_POINT("DBImpl::RunManualCompaction:0"); + TEST_SYNC_POINT("DBImpl::RunManualCompaction:1"); + InstrumentedMutexLock l(&mutex_); + + if (manual_compaction_paused_ > 0) { + // Does not make sense to `AddManualCompaction()` in this scenario since + // `DisableManualCompaction()` just waited for the manual compaction queue + // to drain. So return immediately. + TEST_SYNC_POINT("DBImpl::RunManualCompaction:PausedAtStart"); + manual.status = + Status::Incomplete(Status::SubCode::kManualCompactionPaused); + manual.done = true; + return manual.status; + } + + // When a manual compaction arrives, temporarily disable scheduling of + // non-manual compactions and wait until the number of scheduled compaction + // jobs drops to zero. This used to be needed to ensure that this manual + // compaction can compact any range of keys/files. Now it is optional + // (see `CompactRangeOptions::exclusive_manual_compaction`). The use case for + // `exclusive_manual_compaction=true` is unclear beyond not trusting the code. + // + // HasPendingManualCompaction() is true when at least one thread is inside + // RunManualCompaction(), i.e. during that time no other compaction will + // get scheduled (see MaybeScheduleFlushOrCompaction). + // + // Note that the following loop doesn't stop more that one thread calling + // RunManualCompaction() from getting to the second while loop below. + // However, only one of them will actually schedule compaction, while + // others will wait on a condition variable until it completes. + + AddManualCompaction(&manual); + TEST_SYNC_POINT_CALLBACK("DBImpl::RunManualCompaction:NotScheduled", &mutex_); + if (exclusive) { + // Limitation: there's no way to wake up the below loop when user sets + // `*manual.canceled`. So `CompactRangeOptions::exclusive_manual_compaction` + // and `CompactRangeOptions::canceled` might not work well together. + while (bg_bottom_compaction_scheduled_ > 0 || + bg_compaction_scheduled_ > 0) { + if (manual_compaction_paused_ > 0 || manual.canceled == true) { + // Pretend the error came from compaction so the below cleanup/error + // handling code can process it. + manual.done = true; + manual.status = + Status::Incomplete(Status::SubCode::kManualCompactionPaused); + break; + } + TEST_SYNC_POINT("DBImpl::RunManualCompaction:WaitScheduled"); + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "[%s] Manual compaction waiting for all other scheduled background " + "compactions to finish", + cfd->GetName().c_str()); + bg_cv_.Wait(); + } + } + + LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, + immutable_db_options_.info_log.get()); + + ROCKS_LOG_BUFFER(&log_buffer, "[%s] Manual compaction starting", + cfd->GetName().c_str()); + + // We don't check bg_error_ here, because if we get the error in compaction, + // the compaction will set manual.status to bg_error_ and set manual.done to + // true. + while (!manual.done) { + assert(HasPendingManualCompaction()); + manual_conflict = false; + Compaction* compaction = nullptr; + if (ShouldntRunManualCompaction(&manual) || (manual.in_progress == true) || + scheduled || + (((manual.manual_end = &manual.tmp_storage1) != nullptr) && + ((compaction = manual.cfd->CompactRange( + *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 && + manual_conflict))) { + // exclusive manual compactions should not see a conflict during + // CompactRange + assert(!exclusive || !manual_conflict); + // Running either this or some other manual compaction + bg_cv_.Wait(); + if (manual_compaction_paused_ > 0 && scheduled && !unscheduled) { + assert(thread_pool_priority != Env::Priority::TOTAL); + // unschedule all manual compactions + auto unscheduled_task_num = env_->UnSchedule( + GetTaskTag(TaskType::kManualCompaction), thread_pool_priority); + if (unscheduled_task_num > 0) { + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "[%s] Unscheduled %d number of manual compactions from the " + "thread-pool", + cfd->GetName().c_str(), unscheduled_task_num); + // it may unschedule other manual compactions, notify others. + bg_cv_.SignalAll(); + } + unscheduled = true; + TEST_SYNC_POINT("DBImpl::RunManualCompaction:Unscheduled"); + } + if (scheduled && manual.incomplete == true) { + assert(!manual.in_progress); + scheduled = false; + manual.incomplete = false; + } + } else if (!scheduled) { + if (compaction == nullptr) { + manual.done = true; + bg_cv_.SignalAll(); + continue; + } + ca = new CompactionArg; + ca->db = this; + ca->prepicked_compaction = new PrepickedCompaction; + ca->prepicked_compaction->manual_compaction_state = &manual; + ca->prepicked_compaction->compaction = compaction; + if (!RequestCompactionToken( + cfd, true, &ca->prepicked_compaction->task_token, &log_buffer)) { + // Don't throttle manual compaction, only count outstanding tasks. + assert(false); + } + manual.incomplete = false; + if (compaction->bottommost_level() && + env_->GetBackgroundThreads(Env::Priority::BOTTOM) > 0) { + bg_bottom_compaction_scheduled_++; + ca->compaction_pri_ = Env::Priority::BOTTOM; + env_->Schedule(&DBImpl::BGWorkBottomCompaction, ca, + Env::Priority::BOTTOM, + GetTaskTag(TaskType::kManualCompaction), + &DBImpl::UnscheduleCompactionCallback); + thread_pool_priority = Env::Priority::BOTTOM; + } else { + bg_compaction_scheduled_++; + ca->compaction_pri_ = Env::Priority::LOW; + env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, + GetTaskTag(TaskType::kManualCompaction), + &DBImpl::UnscheduleCompactionCallback); + thread_pool_priority = Env::Priority::LOW; + } + scheduled = true; + TEST_SYNC_POINT("DBImpl::RunManualCompaction:Scheduled"); + } + } + + log_buffer.FlushBufferToLog(); + assert(!manual.in_progress); + assert(HasPendingManualCompaction()); + RemoveManualCompaction(&manual); + // if the manual job is unscheduled, try schedule other jobs in case there's + // any unscheduled compaction job which was blocked by exclusive manual + // compaction. + if (manual.status.IsIncomplete() && + manual.status.subcode() == Status::SubCode::kManualCompactionPaused) { + MaybeScheduleFlushOrCompaction(); + } + bg_cv_.SignalAll(); + return manual.status; +} + +void DBImpl::GenerateFlushRequest(const autovector& cfds, + FlushRequest* req) { + assert(req != nullptr); + req->reserve(cfds.size()); + for (const auto cfd : cfds) { + if (nullptr == cfd) { + // cfd may be null, see DBImpl::ScheduleFlushes + continue; + } + uint64_t max_memtable_id = cfd->imm()->GetLatestMemTableID(); + req->emplace_back(cfd, max_memtable_id); + } +} + +Status DBImpl::FlushMemTable(ColumnFamilyData* cfd, + const FlushOptions& flush_options, + FlushReason flush_reason, + bool entered_write_thread) { + // This method should not be called if atomic_flush is true. + assert(!immutable_db_options_.atomic_flush); + if (!flush_options.wait && write_controller_.IsStopped()) { + std::ostringstream oss; + oss << "Writes have been stopped, thus unable to perform manual flush. " + "Please try again later after writes are resumed"; + return Status::TryAgain(oss.str()); + } + Status s; + if (!flush_options.allow_write_stall) { + bool flush_needed = true; + s = WaitUntilFlushWouldNotStallWrites(cfd, &flush_needed); + TEST_SYNC_POINT("DBImpl::FlushMemTable:StallWaitDone"); + if (!s.ok() || !flush_needed) { + return s; + } + } + + const bool needs_to_join_write_thread = !entered_write_thread; + autovector flush_reqs; + autovector memtable_ids_to_wait; + { + WriteContext context; + InstrumentedMutexLock guard_lock(&mutex_); + + WriteThread::Writer w; + WriteThread::Writer nonmem_w; + if (needs_to_join_write_thread) { + write_thread_.EnterUnbatched(&w, &mutex_); + if (two_write_queues_) { + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + } + } + WaitForPendingWrites(); + + if (flush_reason != FlushReason::kErrorRecoveryRetryFlush && + (!cfd->mem()->IsEmpty() || !cached_recoverable_state_empty_.load())) { + // Note that, when flush reason is kErrorRecoveryRetryFlush, during the + // auto retry resume, we want to avoid creating new small memtables. + // Therefore, SwitchMemtable will not be called. Also, since ResumeImpl + // will iterate through all the CFs and call FlushMemtable during auto + // retry resume, it is possible that in some CFs, + // cfd->imm()->NumNotFlushed() = 0. In this case, so no flush request will + // be created and scheduled, status::OK() will be returned. + s = SwitchMemtable(cfd, &context); + } + const uint64_t flush_memtable_id = std::numeric_limits::max(); + if (s.ok()) { + if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() || + !cached_recoverable_state_empty_.load()) { + FlushRequest req{{cfd, flush_memtable_id}}; + flush_reqs.emplace_back(std::move(req)); + memtable_ids_to_wait.emplace_back(cfd->imm()->GetLatestMemTableID()); + } + if (immutable_db_options_.persist_stats_to_disk && + flush_reason != FlushReason::kErrorRecoveryRetryFlush) { + ColumnFamilyData* cfd_stats = + versions_->GetColumnFamilySet()->GetColumnFamily( + kPersistentStatsColumnFamilyName); + if (cfd_stats != nullptr && cfd_stats != cfd && + !cfd_stats->mem()->IsEmpty()) { + // only force flush stats CF when it will be the only CF lagging + // behind after the current flush + bool stats_cf_flush_needed = true; + for (auto* loop_cfd : *versions_->GetColumnFamilySet()) { + if (loop_cfd == cfd_stats || loop_cfd == cfd) { + continue; + } + if (loop_cfd->GetLogNumber() <= cfd_stats->GetLogNumber()) { + stats_cf_flush_needed = false; + } + } + if (stats_cf_flush_needed) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Force flushing stats CF with manual flush of %s " + "to avoid holding old logs", + cfd->GetName().c_str()); + s = SwitchMemtable(cfd_stats, &context); + FlushRequest req{{cfd_stats, flush_memtable_id}}; + flush_reqs.emplace_back(std::move(req)); + memtable_ids_to_wait.emplace_back( + cfd->imm()->GetLatestMemTableID()); + } + } + } + } + + if (s.ok() && !flush_reqs.empty()) { + for (const auto& req : flush_reqs) { + assert(req.size() == 1); + ColumnFamilyData* loop_cfd = req[0].first; + loop_cfd->imm()->FlushRequested(); + } + // If the caller wants to wait for this flush to complete, it indicates + // that the caller expects the ColumnFamilyData not to be free'ed by + // other threads which may drop the column family concurrently. + // Therefore, we increase the cfd's ref count. + if (flush_options.wait) { + for (const auto& req : flush_reqs) { + assert(req.size() == 1); + ColumnFamilyData* loop_cfd = req[0].first; + loop_cfd->Ref(); + } + } + for (const auto& req : flush_reqs) { + SchedulePendingFlush(req, flush_reason); + } + MaybeScheduleFlushOrCompaction(); + } + + if (needs_to_join_write_thread) { + write_thread_.ExitUnbatched(&w); + if (two_write_queues_) { + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } + } + } + TEST_SYNC_POINT("DBImpl::FlushMemTable:AfterScheduleFlush"); + TEST_SYNC_POINT("DBImpl::FlushMemTable:BeforeWaitForBgFlush"); + if (s.ok() && flush_options.wait) { + autovector cfds; + autovector flush_memtable_ids; + assert(flush_reqs.size() == memtable_ids_to_wait.size()); + for (size_t i = 0; i < flush_reqs.size(); ++i) { + assert(flush_reqs[i].size() == 1); + cfds.push_back(flush_reqs[i][0].first); + flush_memtable_ids.push_back(&(memtable_ids_to_wait[i])); + } + s = WaitForFlushMemTables( + cfds, flush_memtable_ids, + (flush_reason == FlushReason::kErrorRecovery || + flush_reason == FlushReason::kErrorRecoveryRetryFlush)); + InstrumentedMutexLock lock_guard(&mutex_); + for (auto* tmp_cfd : cfds) { + tmp_cfd->UnrefAndTryDelete(); + } + } + TEST_SYNC_POINT("DBImpl::FlushMemTable:FlushMemTableFinished"); + return s; +} + +// Flush all elements in 'column_family_datas' +// and atomically record the result to the MANIFEST. +Status DBImpl::AtomicFlushMemTables( + const autovector& column_family_datas, + const FlushOptions& flush_options, FlushReason flush_reason, + bool entered_write_thread) { + assert(immutable_db_options_.atomic_flush); + if (!flush_options.wait && write_controller_.IsStopped()) { + std::ostringstream oss; + oss << "Writes have been stopped, thus unable to perform manual flush. " + "Please try again later after writes are resumed"; + return Status::TryAgain(oss.str()); + } + Status s; + if (!flush_options.allow_write_stall) { + int num_cfs_to_flush = 0; + for (auto cfd : column_family_datas) { + bool flush_needed = true; + s = WaitUntilFlushWouldNotStallWrites(cfd, &flush_needed); + if (!s.ok()) { + return s; + } else if (flush_needed) { + ++num_cfs_to_flush; + } + } + if (0 == num_cfs_to_flush) { + return s; + } + } + const bool needs_to_join_write_thread = !entered_write_thread; + FlushRequest flush_req; + autovector cfds; + { + WriteContext context; + InstrumentedMutexLock guard_lock(&mutex_); + + WriteThread::Writer w; + WriteThread::Writer nonmem_w; + if (needs_to_join_write_thread) { + write_thread_.EnterUnbatched(&w, &mutex_); + if (two_write_queues_) { + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + } + } + WaitForPendingWrites(); + + for (auto cfd : column_family_datas) { + if (cfd->IsDropped()) { + continue; + } + if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() || + !cached_recoverable_state_empty_.load()) { + cfds.emplace_back(cfd); + } + } + for (auto cfd : cfds) { + if ((cfd->mem()->IsEmpty() && cached_recoverable_state_empty_.load()) || + flush_reason == FlushReason::kErrorRecoveryRetryFlush) { + continue; + } + cfd->Ref(); + s = SwitchMemtable(cfd, &context); + cfd->UnrefAndTryDelete(); + if (!s.ok()) { + break; + } + } + if (s.ok()) { + AssignAtomicFlushSeq(cfds); + for (auto cfd : cfds) { + cfd->imm()->FlushRequested(); + } + // If the caller wants to wait for this flush to complete, it indicates + // that the caller expects the ColumnFamilyData not to be free'ed by + // other threads which may drop the column family concurrently. + // Therefore, we increase the cfd's ref count. + if (flush_options.wait) { + for (auto cfd : cfds) { + cfd->Ref(); + } + } + GenerateFlushRequest(cfds, &flush_req); + SchedulePendingFlush(flush_req, flush_reason); + MaybeScheduleFlushOrCompaction(); + } + + if (needs_to_join_write_thread) { + write_thread_.ExitUnbatched(&w); + if (two_write_queues_) { + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } + } + } + TEST_SYNC_POINT("DBImpl::AtomicFlushMemTables:AfterScheduleFlush"); + TEST_SYNC_POINT("DBImpl::AtomicFlushMemTables:BeforeWaitForBgFlush"); + if (s.ok() && flush_options.wait) { + autovector flush_memtable_ids; + for (auto& iter : flush_req) { + flush_memtable_ids.push_back(&(iter.second)); + } + s = WaitForFlushMemTables( + cfds, flush_memtable_ids, + (flush_reason == FlushReason::kErrorRecovery || + flush_reason == FlushReason::kErrorRecoveryRetryFlush)); + InstrumentedMutexLock lock_guard(&mutex_); + for (auto* cfd : cfds) { + cfd->UnrefAndTryDelete(); + } + } + return s; +} + +// Calling FlushMemTable(), whether from DB::Flush() or from Backup Engine, can +// cause write stall, for example if one memtable is being flushed already. +// This method tries to avoid write stall (similar to CompactRange() behavior) +// it emulates how the SuperVersion / LSM would change if flush happens, checks +// it against various constrains and delays flush if it'd cause write stall. +// Caller should check status and flush_needed to see if flush already happened. +Status DBImpl::WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd, + bool* flush_needed) { + { + *flush_needed = true; + InstrumentedMutexLock l(&mutex_); + uint64_t orig_active_memtable_id = cfd->mem()->GetID(); + WriteStallCondition write_stall_condition = WriteStallCondition::kNormal; + do { + if (write_stall_condition != WriteStallCondition::kNormal) { + // Same error handling as user writes: Don't wait if there's a + // background error, even if it's a soft error. We might wait here + // indefinitely as the pending flushes/compactions may never finish + // successfully, resulting in the stall condition lasting indefinitely + if (error_handler_.IsBGWorkStopped()) { + return error_handler_.GetBGError(); + } + + TEST_SYNC_POINT("DBImpl::WaitUntilFlushWouldNotStallWrites:StallWait"); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[%s] WaitUntilFlushWouldNotStallWrites" + " waiting on stall conditions to clear", + cfd->GetName().c_str()); + bg_cv_.Wait(); + } + if (cfd->IsDropped()) { + return Status::ColumnFamilyDropped(); + } + if (shutting_down_.load(std::memory_order_acquire)) { + return Status::ShutdownInProgress(); + } + + uint64_t earliest_memtable_id = + std::min(cfd->mem()->GetID(), cfd->imm()->GetEarliestMemTableID()); + if (earliest_memtable_id > orig_active_memtable_id) { + // We waited so long that the memtable we were originally waiting on was + // flushed. + *flush_needed = false; + return Status::OK(); + } + + 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 + // triggers. If it stalled in these conditions, that'd mean the stall + // 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 && + vstorage->l0_delay_trigger_count() < + mutable_cf_options.level0_file_num_compaction_trigger) { + break; + } + + // check whether one extra immutable memtable or an extra L0 file would + // cause write stalling mode to be entered. It could still enter stall + // mode due to pending compaction bytes, but that's less common + write_stall_condition = ColumnFamilyData::GetWriteStallConditionAndCause( + cfd->imm()->NumNotFlushed() + 1, + vstorage->l0_delay_trigger_count() + 1, + vstorage->estimated_compaction_needed_bytes(), + mutable_cf_options, *cfd->ioptions()) + .first; + } while (write_stall_condition != WriteStallCondition::kNormal); + } + return Status::OK(); +} + +// Wait for memtables to be flushed for multiple column families. +// let N = cfds.size() +// for i in [0, N), +// 1) if flush_memtable_ids[i] is not null, then the memtables with lower IDs +// have to be flushed for THIS column family; +// 2) if flush_memtable_ids[i] is null, then all memtables in THIS column +// family have to be flushed. +// Finish waiting when ALL column families finish flushing memtables. +// resuming_from_bg_err indicates whether the caller is trying to resume from +// background error or in normal processing. +Status DBImpl::WaitForFlushMemTables( + const autovector& cfds, + const autovector& flush_memtable_ids, + bool resuming_from_bg_err) { + int num = static_cast(cfds.size()); + // Wait until the compaction completes + InstrumentedMutexLock l(&mutex_); + Status s; + // If the caller is trying to resume from bg error, then + // error_handler_.IsDBStopped() is true. + while (resuming_from_bg_err || !error_handler_.IsDBStopped()) { + if (shutting_down_.load(std::memory_order_acquire)) { + s = Status::ShutdownInProgress(); + return s; + } + // If an error has occurred during resumption, then no need to wait. + // But flush operation may fail because of this error, so need to + // return the status. + if (!error_handler_.GetRecoveryError().ok()) { + s = error_handler_.GetRecoveryError(); + break; + } + // If BGWorkStopped, which indicate that there is a BG error and + // 1) soft error but requires no BG work, 2) no in auto_recovery_ + if (!resuming_from_bg_err && error_handler_.IsBGWorkStopped() && + error_handler_.GetBGError().severity() < Status::Severity::kHardError) { + s = error_handler_.GetBGError(); + return s; + } + + // Number of column families that have been dropped. + int num_dropped = 0; + // Number of column families that have finished flush. + int num_finished = 0; + for (int i = 0; i < num; ++i) { + if (cfds[i]->IsDropped()) { + ++num_dropped; + } else if (cfds[i]->imm()->NumNotFlushed() == 0 || + (flush_memtable_ids[i] != nullptr && + cfds[i]->imm()->GetEarliestMemTableID() > + *flush_memtable_ids[i])) { + ++num_finished; + } + } + if (1 == num_dropped && 1 == num) { + s = Status::ColumnFamilyDropped(); + return s; + } + // Column families involved in this flush request have either been dropped + // or finished flush. Then it's time to finish waiting. + if (num_dropped + num_finished == num) { + break; + } + bg_cv_.Wait(); + } + // If not resuming from bg error, and an error has caused the DB to stop, + // then report the bg error to caller. + if (!resuming_from_bg_err && error_handler_.IsDBStopped()) { + s = error_handler_.GetBGError(); + } + return s; +} + +Status DBImpl::EnableAutoCompaction( + const std::vector& column_family_handles) { + Status s; + for (auto cf_ptr : column_family_handles) { + Status status = + this->SetOptions(cf_ptr, {{"disable_auto_compactions", "false"}}); + if (!status.ok()) { + s = status; + } + } + + return s; +} + +// NOTE: Calling DisableManualCompaction() may overwrite the +// user-provided canceled variable in CompactRangeOptions +void DBImpl::DisableManualCompaction() { + InstrumentedMutexLock l(&mutex_); + manual_compaction_paused_.fetch_add(1, std::memory_order_release); + + // Mark the canceled as true when the cancellation is triggered by + // manual_compaction_paused (may overwrite user-provided `canceled`) + for (const auto& manual_compaction : manual_compaction_dequeue_) { + manual_compaction->canceled = true; + } + + // Wake up manual compactions waiting to start. + bg_cv_.SignalAll(); + + // Wait for any pending manual compactions to finish (typically through + // failing with `Status::Incomplete`) prior to returning. This way we are + // guaranteed no pending manual compaction will commit while manual + // compactions are "disabled". + while (HasPendingManualCompaction()) { + bg_cv_.Wait(); + } +} + +// NOTE: In contrast to DisableManualCompaction(), calling +// EnableManualCompaction() does NOT overwrite the user-provided *canceled +// variable to be false since there is NO CHANCE a canceled compaction +// is uncanceled. In other words, a canceled compaction must have been +// dropped out of the manual compaction queue, when we disable it. +void DBImpl::EnableManualCompaction() { + InstrumentedMutexLock l(&mutex_); + assert(manual_compaction_paused_ > 0); + manual_compaction_paused_.fetch_sub(1, std::memory_order_release); +} + +void DBImpl::MaybeScheduleFlushOrCompaction() { + mutex_.AssertHeld(); + if (!opened_successfully_) { + // Compaction may introduce data race to DB open + return; + } + if (bg_work_paused_ > 0) { + // we paused the background work + return; + } else if (error_handler_.IsBGWorkStopped() && + !error_handler_.IsRecoveryInProgress()) { + // There has been a hard error and this call is not part of the recovery + // sequence. Bail out here so we don't get into an endless loop of + // scheduling BG work which will again call this function + return; + } else if (shutting_down_.load(std::memory_order_acquire)) { + // DB is being deleted; no more background compactions + return; + } + auto bg_job_limits = GetBGJobLimits(); + bool is_flush_pool_empty = + env_->GetBackgroundThreads(Env::Priority::HIGH) == 0; + while (!is_flush_pool_empty && unscheduled_flushes_ > 0 && + bg_flush_scheduled_ < bg_job_limits.max_flushes) { + bg_flush_scheduled_++; + FlushThreadArg* fta = new FlushThreadArg; + fta->db_ = this; + fta->thread_pri_ = Env::Priority::HIGH; + env_->Schedule(&DBImpl::BGWorkFlush, fta, Env::Priority::HIGH, this, + &DBImpl::UnscheduleFlushCallback); + --unscheduled_flushes_; + TEST_SYNC_POINT_CALLBACK( + "DBImpl::MaybeScheduleFlushOrCompaction:AfterSchedule:0", + &unscheduled_flushes_); + } + + // special case -- if high-pri (flush) thread pool is empty, then schedule + // flushes in low-pri (compaction) thread pool. + if (is_flush_pool_empty) { + while (unscheduled_flushes_ > 0 && + bg_flush_scheduled_ + bg_compaction_scheduled_ < + bg_job_limits.max_flushes) { + bg_flush_scheduled_++; + FlushThreadArg* fta = new FlushThreadArg; + fta->db_ = this; + fta->thread_pri_ = Env::Priority::LOW; + env_->Schedule(&DBImpl::BGWorkFlush, fta, Env::Priority::LOW, this, + &DBImpl::UnscheduleFlushCallback); + --unscheduled_flushes_; + } + } + + if (bg_compaction_paused_ > 0) { + // we paused the background compaction + return; + } else if (error_handler_.IsBGWorkStopped()) { + // Compaction is not part of the recovery sequence from a hard error. We + // might get here because recovery might do a flush and install a new + // super version, which will try to schedule pending compactions. Bail + // out here and let the higher level recovery handle compactions + return; + } + + if (HasExclusiveManualCompaction()) { + // only manual compactions are allowed to run. don't schedule automatic + // compactions + TEST_SYNC_POINT("DBImpl::MaybeScheduleFlushOrCompaction:Conflict"); + return; + } + + while (bg_compaction_scheduled_ + bg_bottom_compaction_scheduled_ < + bg_job_limits.max_compactions && + unscheduled_compactions_ > 0) { + CompactionArg* ca = new CompactionArg; + ca->db = this; + ca->compaction_pri_ = Env::Priority::LOW; + ca->prepicked_compaction = nullptr; + bg_compaction_scheduled_++; + unscheduled_compactions_--; + env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this, + &DBImpl::UnscheduleCompactionCallback); + } +} + +DBImpl::BGJobLimits DBImpl::GetBGJobLimits() const { + mutex_.AssertHeld(); + return GetBGJobLimits(mutable_db_options_.max_background_flushes, + mutable_db_options_.max_background_compactions, + mutable_db_options_.max_background_jobs, + write_controller_.NeedSpeedupCompaction()); +} + +DBImpl::BGJobLimits DBImpl::GetBGJobLimits(int max_background_flushes, + int max_background_compactions, + int max_background_jobs, + bool parallelize_compactions) { + BGJobLimits res; + if (max_background_flushes == -1 && max_background_compactions == -1) { + // for our first stab implementing max_background_jobs, simply allocate a + // quarter of the threads to flushes. + res.max_flushes = std::max(1, max_background_jobs / 4); + res.max_compactions = std::max(1, max_background_jobs - res.max_flushes); + } else { + // compatibility code in case users haven't migrated to max_background_jobs, + // which automatically computes flush/compaction limits + res.max_flushes = std::max(1, max_background_flushes); + res.max_compactions = std::max(1, max_background_compactions); + } + if (!parallelize_compactions) { + // throttle background compactions until we deem necessary + res.max_compactions = 1; + } + return res; +} + +void DBImpl::AddToCompactionQueue(ColumnFamilyData* cfd) { + assert(!cfd->queued_for_compaction()); + cfd->Ref(); + compaction_queue_.push_back(cfd); + cfd->set_queued_for_compaction(true); +} + +ColumnFamilyData* DBImpl::PopFirstFromCompactionQueue() { + assert(!compaction_queue_.empty()); + auto cfd = *compaction_queue_.begin(); + compaction_queue_.pop_front(); + assert(cfd->queued_for_compaction()); + cfd->set_queued_for_compaction(false); + return cfd; +} + +DBImpl::FlushRequest DBImpl::PopFirstFromFlushQueue() { + assert(!flush_queue_.empty()); + FlushRequest flush_req = flush_queue_.front(); + flush_queue_.pop_front(); + if (!immutable_db_options_.atomic_flush) { + assert(flush_req.size() == 1); + } + for (const auto& elem : flush_req) { + if (!immutable_db_options_.atomic_flush) { + ColumnFamilyData* cfd = elem.first; + assert(cfd); + assert(cfd->queued_for_flush()); + cfd->set_queued_for_flush(false); + } + } + // TODO: need to unset flush reason? + return flush_req; +} + +ColumnFamilyData* DBImpl::PickCompactionFromQueue( + std::unique_ptr* token, LogBuffer* log_buffer) { + assert(!compaction_queue_.empty()); + assert(*token == nullptr); + autovector throttled_candidates; + ColumnFamilyData* cfd = nullptr; + while (!compaction_queue_.empty()) { + auto first_cfd = *compaction_queue_.begin(); + compaction_queue_.pop_front(); + assert(first_cfd->queued_for_compaction()); + if (!RequestCompactionToken(first_cfd, false, token, log_buffer)) { + throttled_candidates.push_back(first_cfd); + continue; + } + cfd = first_cfd; + cfd->set_queued_for_compaction(false); + break; + } + // Add throttled compaction candidates back to queue in the original order. + for (auto iter = throttled_candidates.rbegin(); + iter != throttled_candidates.rend(); ++iter) { + compaction_queue_.push_front(*iter); + } + return cfd; +} + +void DBImpl::SchedulePendingFlush(const FlushRequest& flush_req, + FlushReason flush_reason) { + mutex_.AssertHeld(); + if (flush_req.empty()) { + return; + } + if (!immutable_db_options_.atomic_flush) { + // For the non-atomic flush case, we never schedule multiple column + // families in the same flush request. + assert(flush_req.size() == 1); + ColumnFamilyData* cfd = flush_req[0].first; + assert(cfd); + + if (!cfd->queued_for_flush() && cfd->imm()->IsFlushPending()) { + cfd->Ref(); + cfd->set_queued_for_flush(true); + cfd->SetFlushReason(flush_reason); + ++unscheduled_flushes_; + flush_queue_.push_back(flush_req); + } + } else { + for (auto& iter : flush_req) { + ColumnFamilyData* cfd = iter.first; + cfd->Ref(); + cfd->SetFlushReason(flush_reason); + } + ++unscheduled_flushes_; + flush_queue_.push_back(flush_req); + } +} + +void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) { + mutex_.AssertHeld(); + if (!cfd->queued_for_compaction() && cfd->NeedsCompaction()) { + AddToCompactionQueue(cfd); + ++unscheduled_compactions_; + } +} + +void DBImpl::SchedulePendingPurge(std::string fname, std::string dir_to_sync, + FileType type, uint64_t number, int job_id) { + mutex_.AssertHeld(); + PurgeFileInfo file_info(fname, dir_to_sync, type, number, job_id); + purge_files_.insert({{number, std::move(file_info)}}); +} + +void DBImpl::BGWorkFlush(void* arg) { + FlushThreadArg fta = *(reinterpret_cast(arg)); + delete reinterpret_cast(arg); + + IOSTATS_SET_THREAD_POOL_ID(fta.thread_pri_); + TEST_SYNC_POINT("DBImpl::BGWorkFlush"); + static_cast_with_check(fta.db_)->BackgroundCallFlush(fta.thread_pri_); + TEST_SYNC_POINT("DBImpl::BGWorkFlush:done"); +} + +void DBImpl::BGWorkCompaction(void* arg) { + CompactionArg ca = *(reinterpret_cast(arg)); + delete reinterpret_cast(arg); + IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW); + TEST_SYNC_POINT("DBImpl::BGWorkCompaction"); + auto prepicked_compaction = + static_cast(ca.prepicked_compaction); + static_cast_with_check(ca.db)->BackgroundCallCompaction( + prepicked_compaction, Env::Priority::LOW); + delete prepicked_compaction; +} + +void DBImpl::BGWorkBottomCompaction(void* arg) { + CompactionArg ca = *(static_cast(arg)); + delete static_cast(arg); + IOSTATS_SET_THREAD_POOL_ID(Env::Priority::BOTTOM); + TEST_SYNC_POINT("DBImpl::BGWorkBottomCompaction"); + auto* prepicked_compaction = ca.prepicked_compaction; + assert(prepicked_compaction && prepicked_compaction->compaction); + ca.db->BackgroundCallCompaction(prepicked_compaction, Env::Priority::BOTTOM); + delete prepicked_compaction; +} + +void DBImpl::BGWorkPurge(void* db) { + IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH); + TEST_SYNC_POINT("DBImpl::BGWorkPurge:start"); + reinterpret_cast(db)->BackgroundCallPurge(); + TEST_SYNC_POINT("DBImpl::BGWorkPurge:end"); +} + +void DBImpl::UnscheduleCompactionCallback(void* arg) { + CompactionArg* ca_ptr = reinterpret_cast(arg); + Env::Priority compaction_pri = ca_ptr->compaction_pri_; + if (Env::Priority::BOTTOM == compaction_pri) { + // Decrement bg_bottom_compaction_scheduled_ if priority is BOTTOM + ca_ptr->db->bg_bottom_compaction_scheduled_--; + } else if (Env::Priority::LOW == compaction_pri) { + // Decrement bg_compaction_scheduled_ if priority is LOW + ca_ptr->db->bg_compaction_scheduled_--; + } + CompactionArg ca = *(ca_ptr); + delete reinterpret_cast(arg); + if (ca.prepicked_compaction != nullptr) { + // if it's a manual compaction, set status to ManualCompactionPaused + if (ca.prepicked_compaction->manual_compaction_state) { + ca.prepicked_compaction->manual_compaction_state->done = true; + ca.prepicked_compaction->manual_compaction_state->status = + Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + if (ca.prepicked_compaction->compaction != nullptr) { + ca.prepicked_compaction->compaction->ReleaseCompactionFiles( + Status::Incomplete(Status::SubCode::kManualCompactionPaused)); + delete ca.prepicked_compaction->compaction; + } + delete ca.prepicked_compaction; + } + TEST_SYNC_POINT("DBImpl::UnscheduleCompactionCallback"); +} + +void DBImpl::UnscheduleFlushCallback(void* arg) { + // Decrement bg_flush_scheduled_ in flush callback + reinterpret_cast(arg)->db_->bg_flush_scheduled_--; + Env::Priority flush_pri = reinterpret_cast(arg)->thread_pri_; + if (Env::Priority::LOW == flush_pri) { + TEST_SYNC_POINT("DBImpl::UnscheduleLowFlushCallback"); + } else if (Env::Priority::HIGH == flush_pri) { + TEST_SYNC_POINT("DBImpl::UnscheduleHighFlushCallback"); + } + delete reinterpret_cast(arg); + TEST_SYNC_POINT("DBImpl::UnscheduleFlushCallback"); +} + +Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context, + LogBuffer* log_buffer, FlushReason* reason, + Env::Priority thread_pri) { + mutex_.AssertHeld(); + + Status status; + *reason = FlushReason::kOthers; + // If BG work is stopped due to an error, but a recovery is in progress, + // that means this flush is part of the recovery. So allow it to go through + if (!error_handler_.IsBGWorkStopped()) { + if (shutting_down_.load(std::memory_order_acquire)) { + status = Status::ShutdownInProgress(); + } + } else if (!error_handler_.IsRecoveryInProgress()) { + status = error_handler_.GetBGError(); + } + + if (!status.ok()) { + return status; + } + + autovector bg_flush_args; + std::vector& superversion_contexts = + job_context->superversion_contexts; + autovector column_families_not_to_flush; + while (!flush_queue_.empty()) { + // This cfd is already referenced + const FlushRequest& flush_req = PopFirstFromFlushQueue(); + superversion_contexts.clear(); + superversion_contexts.reserve(flush_req.size()); + + for (const auto& iter : flush_req) { + ColumnFamilyData* cfd = iter.first; + if (cfd->GetMempurgeUsed()) { + // If imm() contains silent memtables (e.g.: because + // MemPurge was activated), requesting a flush will + // mark the imm_needed as true. + cfd->imm()->FlushRequested(); + } + + if (cfd->IsDropped() || !cfd->imm()->IsFlushPending()) { + // can't flush this CF, try next one + column_families_not_to_flush.push_back(cfd); + continue; + } + superversion_contexts.emplace_back(SuperVersionContext(true)); + bg_flush_args.emplace_back(cfd, iter.second, + &(superversion_contexts.back())); + } + if (!bg_flush_args.empty()) { + break; + } + } + + if (!bg_flush_args.empty()) { + auto bg_job_limits = GetBGJobLimits(); + for (const auto& arg : bg_flush_args) { + ColumnFamilyData* cfd = arg.cfd_; + ROCKS_LOG_BUFFER( + log_buffer, + "Calling FlushMemTableToOutputFile with column " + "family [%s], flush slots available %d, compaction slots available " + "%d, " + "flush slots scheduled %d, compaction slots scheduled %d", + cfd->GetName().c_str(), bg_job_limits.max_flushes, + bg_job_limits.max_compactions, bg_flush_scheduled_, + bg_compaction_scheduled_); + } + status = FlushMemTablesToOutputFiles(bg_flush_args, made_progress, + job_context, log_buffer, thread_pri); + TEST_SYNC_POINT("DBImpl::BackgroundFlush:BeforeFlush"); + // All the CFDs in the FlushReq must have the same flush reason, so just + // grab the first one + *reason = bg_flush_args[0].cfd_->GetFlushReason(); + for (auto& arg : bg_flush_args) { + ColumnFamilyData* cfd = arg.cfd_; + if (cfd->UnrefAndTryDelete()) { + arg.cfd_ = nullptr; + } + } + } + for (auto cfd : column_families_not_to_flush) { + cfd->UnrefAndTryDelete(); + } + return status; +} + +void DBImpl::BackgroundCallFlush(Env::Priority thread_pri) { + bool made_progress = false; + JobContext job_context(next_job_id_.fetch_add(1), true); + + TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCallFlush:start", nullptr); + + LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, + immutable_db_options_.info_log.get()); + TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:Start:1"); + TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:Start:2"); + { + InstrumentedMutexLock l(&mutex_); + assert(bg_flush_scheduled_); + num_running_flushes_++; + + std::unique_ptr::iterator> + pending_outputs_inserted_elem(new std::list::iterator( + CaptureCurrentFileNumberInPendingOutputs())); + FlushReason reason; + + Status s = BackgroundFlush(&made_progress, &job_context, &log_buffer, + &reason, thread_pri); + if (!s.ok() && !s.IsShutdownInProgress() && !s.IsColumnFamilyDropped() && + reason != FlushReason::kErrorRecovery) { + // Wait a little bit before retrying background flush in + // case this is an environmental problem and we do not want to + // chew up resources for failed flushes for the duration of + // the problem. + uint64_t error_cnt = + default_cf_internal_stats_->BumpAndGetBackgroundErrorCount(); + bg_cv_.SignalAll(); // In case a waiter can proceed despite the error + mutex_.Unlock(); + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Waiting after background flush error: %s" + "Accumulated background error counts: %" PRIu64, + s.ToString().c_str(), error_cnt); + log_buffer.FlushBufferToLog(); + LogFlush(immutable_db_options_.info_log); + immutable_db_options_.clock->SleepForMicroseconds(1000000); + mutex_.Lock(); + } + + TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:FlushFinish:0"); + ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem); + + // If flush failed, we want to delete all temporary files that we might have + // created. Thus, we force full scan in FindObsoleteFiles() + FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress() && + !s.IsColumnFamilyDropped()); + // delete unnecessary files if any, this is done outside the mutex + if (job_context.HaveSomethingToClean() || + job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) { + mutex_.Unlock(); + TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:FilesFound"); + // Have to flush the info logs before bg_flush_scheduled_-- + // because if bg_flush_scheduled_ becomes 0 and the lock is + // released, the deconstructor of DB can kick in and destroy all the + // states of DB so info_log might not be available after that point. + // It also applies to access other states that DB owns. + log_buffer.FlushBufferToLog(); + if (job_context.HaveSomethingToDelete()) { + PurgeObsoleteFiles(job_context); + } + job_context.Clean(); + mutex_.Lock(); + } + TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:ContextCleanedUp"); + + assert(num_running_flushes_ > 0); + num_running_flushes_--; + bg_flush_scheduled_--; + // See if there's more work to be done + MaybeScheduleFlushOrCompaction(); + atomic_flush_install_cv_.SignalAll(); + bg_cv_.SignalAll(); + // IMPORTANT: there should be no code after calling SignalAll. This call may + // signal the DB destructor that it's OK to proceed with destruction. In + // that case, all DB variables will be dealloacated and referencing them + // will cause trouble. + } +} + +void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction, + Env::Priority bg_thread_pri) { + bool made_progress = false; + JobContext job_context(next_job_id_.fetch_add(1), true); + TEST_SYNC_POINT("BackgroundCallCompaction:0"); + LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, + immutable_db_options_.info_log.get()); + { + InstrumentedMutexLock l(&mutex_); + + // This call will unlock/lock the mutex to wait for current running + // IngestExternalFile() calls to finish. + WaitForIngestFile(); + + num_running_compactions_++; + + std::unique_ptr::iterator> + pending_outputs_inserted_elem(new std::list::iterator( + CaptureCurrentFileNumberInPendingOutputs())); + + assert((bg_thread_pri == Env::Priority::BOTTOM && + bg_bottom_compaction_scheduled_) || + (bg_thread_pri == Env::Priority::LOW && bg_compaction_scheduled_)); + Status s = BackgroundCompaction(&made_progress, &job_context, &log_buffer, + prepicked_compaction, bg_thread_pri); + TEST_SYNC_POINT("BackgroundCallCompaction:1"); + if (s.IsBusy()) { + bg_cv_.SignalAll(); // In case a waiter can proceed despite the error + mutex_.Unlock(); + immutable_db_options_.clock->SleepForMicroseconds( + 10000); // prevent hot loop + mutex_.Lock(); + } else if (!s.ok() && !s.IsShutdownInProgress() && + !s.IsManualCompactionPaused() && !s.IsColumnFamilyDropped()) { + // Wait a little bit before retrying background compaction in + // case this is an environmental problem and we do not want to + // chew up resources for failed compactions for the duration of + // the problem. + uint64_t error_cnt = + default_cf_internal_stats_->BumpAndGetBackgroundErrorCount(); + bg_cv_.SignalAll(); // In case a waiter can proceed despite the error + mutex_.Unlock(); + log_buffer.FlushBufferToLog(); + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Waiting after background compaction error: %s, " + "Accumulated background error counts: %" PRIu64, + s.ToString().c_str(), error_cnt); + LogFlush(immutable_db_options_.info_log); + immutable_db_options_.clock->SleepForMicroseconds(1000000); + mutex_.Lock(); + } else if (s.IsManualCompactionPaused()) { + assert(prepicked_compaction); + ManualCompactionState* m = prepicked_compaction->manual_compaction_state; + assert(m); + ROCKS_LOG_BUFFER(&log_buffer, "[%s] [JOB %d] Manual compaction paused", + m->cfd->GetName().c_str(), job_context.job_id); + } + + ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem); + + // If compaction failed, we want to delete all temporary files that we + // might have created (they might not be all recorded in job_context in + // case of a failure). Thus, we force full scan in FindObsoleteFiles() + FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress() && + !s.IsManualCompactionPaused() && + !s.IsColumnFamilyDropped() && + !s.IsBusy()); + TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:FoundObsoleteFiles"); + + // delete unnecessary files if any, this is done outside the mutex + if (job_context.HaveSomethingToClean() || + job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) { + mutex_.Unlock(); + // Have to flush the info logs before bg_compaction_scheduled_-- + // because if bg_flush_scheduled_ becomes 0 and the lock is + // released, the deconstructor of DB can kick in and destroy all the + // states of DB so info_log might not be available after that point. + // It also applies to access other states that DB owns. + log_buffer.FlushBufferToLog(); + if (job_context.HaveSomethingToDelete()) { + PurgeObsoleteFiles(job_context); + TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:PurgedObsoleteFiles"); + } + job_context.Clean(); + mutex_.Lock(); + } + + assert(num_running_compactions_ > 0); + num_running_compactions_--; + + if (bg_thread_pri == Env::Priority::LOW) { + bg_compaction_scheduled_--; + } else { + assert(bg_thread_pri == Env::Priority::BOTTOM); + bg_bottom_compaction_scheduled_--; + } + + // See if there's more work to be done + MaybeScheduleFlushOrCompaction(); + + if (prepicked_compaction != nullptr && + prepicked_compaction->task_token != nullptr) { + // Releasing task tokens affects (and asserts on) the DB state, so + // must be done before we potentially signal the DB close process to + // proceed below. + prepicked_compaction->task_token.reset(); + } + + if (made_progress || + (bg_compaction_scheduled_ == 0 && + bg_bottom_compaction_scheduled_ == 0) || + HasPendingManualCompaction() || unscheduled_compactions_ == 0) { + // signal if + // * made_progress -- need to wakeup DelayWrite + // * bg_{bottom,}_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl + // * HasPendingManualCompaction -- need to wakeup RunManualCompaction + // If none of this is true, there is no need to signal since nobody is + // waiting for it + bg_cv_.SignalAll(); + } + // IMPORTANT: there should be no code after calling SignalAll. This call may + // signal the DB destructor that it's OK to proceed with destruction. In + // that case, all DB variables will be dealloacated and referencing them + // will cause trouble. + } +} + +Status DBImpl::BackgroundCompaction(bool* made_progress, + JobContext* job_context, + LogBuffer* log_buffer, + PrepickedCompaction* prepicked_compaction, + Env::Priority thread_pri) { + ManualCompactionState* manual_compaction = + prepicked_compaction == nullptr + ? nullptr + : prepicked_compaction->manual_compaction_state; + *made_progress = false; + mutex_.AssertHeld(); + TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Start"); + + bool is_manual = (manual_compaction != nullptr); + std::unique_ptr c; + if (prepicked_compaction != nullptr && + prepicked_compaction->compaction != nullptr) { + c.reset(prepicked_compaction->compaction); + } + bool is_prepicked = is_manual || c; + + // (manual_compaction->in_progress == false); + bool trivial_move_disallowed = + is_manual && manual_compaction->disallow_trivial_move; + + CompactionJobStats compaction_job_stats; + Status status; + if (!error_handler_.IsBGWorkStopped()) { + if (shutting_down_.load(std::memory_order_acquire)) { + status = Status::ShutdownInProgress(); + } else if (is_manual && + manual_compaction->canceled.load(std::memory_order_acquire)) { + status = Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + } else { + status = error_handler_.GetBGError(); + // If we get here, it means a hard error happened after this compaction + // was scheduled by MaybeScheduleFlushOrCompaction(), but before it got + // a chance to execute. Since we didn't pop a cfd from the compaction + // queue, increment unscheduled_compactions_ + unscheduled_compactions_++; + } + + if (!status.ok()) { + if (is_manual) { + manual_compaction->status = status; + manual_compaction->done = true; + manual_compaction->in_progress = false; + manual_compaction = nullptr; + } + if (c) { + c->ReleaseCompactionFiles(status); + c.reset(); + } + return status; + } + + if (is_manual) { + // another thread cannot pick up the same work + manual_compaction->in_progress = true; + } + + TEST_SYNC_POINT("DBImpl::BackgroundCompaction:InProgress"); + + std::unique_ptr task_token; + + // InternalKey manual_end_storage; + // InternalKey* manual_end = &manual_end_storage; + bool sfm_reserved_compact_space = false; + if (is_manual) { + ManualCompactionState* m = manual_compaction; + assert(m->in_progress); + if (!c) { + m->done = true; + m->manual_end = nullptr; + ROCKS_LOG_BUFFER( + log_buffer, + "[%s] Manual compaction from level-%d from %s .. " + "%s; nothing to do\n", + m->cfd->GetName().c_str(), m->input_level, + (m->begin ? m->begin->DebugString(true).c_str() : "(begin)"), + (m->end ? m->end->DebugString(true).c_str() : "(end)")); + } else { + // First check if we have enough room to do the compaction + bool enough_room = EnoughRoomForCompaction( + m->cfd, *(c->inputs()), &sfm_reserved_compact_space, log_buffer); + + if (!enough_room) { + // Then don't do the compaction + c->ReleaseCompactionFiles(status); + c.reset(); + // m's vars will get set properly at the end of this function, + // as long as status == CompactionTooLarge + status = Status::CompactionTooLarge(); + } else { + ROCKS_LOG_BUFFER( + log_buffer, + "[%s] Manual compaction from level-%d to level-%d from %s .. " + "%s; will stop at %s\n", + m->cfd->GetName().c_str(), m->input_level, c->output_level(), + (m->begin ? m->begin->DebugString(true).c_str() : "(begin)"), + (m->end ? m->end->DebugString(true).c_str() : "(end)"), + ((m->done || m->manual_end == nullptr) + ? "(end)" + : m->manual_end->DebugString(true).c_str())); + } + } + } else if (!is_prepicked && !compaction_queue_.empty()) { + if (HasExclusiveManualCompaction()) { + // Can't compact right now, but try again later + TEST_SYNC_POINT("DBImpl::BackgroundCompaction()::Conflict"); + + // Stay in the compaction queue. + unscheduled_compactions_++; + + return Status::OK(); + } + + auto cfd = PickCompactionFromQueue(&task_token, log_buffer); + if (cfd == nullptr) { + // Can't find any executable task from the compaction queue. + // All tasks have been throttled by compaction thread limiter. + ++unscheduled_compactions_; + return Status::Busy(); + } + + // We unreference here because the following code will take a Ref() on + // this cfd if it is going to use it (Compaction class holds a + // reference). + // This will all happen under a mutex so we don't have to be afraid of + // somebody else deleting it. + if (cfd->UnrefAndTryDelete()) { + // This was the last reference of the column family, so no need to + // compact. + return Status::OK(); + } + + // Pick up latest mutable CF Options and use it throughout the + // compaction job + // 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()) { + // 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 + TEST_SYNC_POINT("DBImpl::BackgroundCompaction():BeforePickCompaction"); + c.reset(cfd->PickCompaction(*mutable_cf_options, mutable_db_options_, + log_buffer)); + TEST_SYNC_POINT("DBImpl::BackgroundCompaction():AfterPickCompaction"); + + if (c != nullptr) { + bool enough_room = EnoughRoomForCompaction( + cfd, *(c->inputs()), &sfm_reserved_compact_space, log_buffer); + + if (!enough_room) { + // Then don't do the compaction + c->ReleaseCompactionFiles(status); + c->column_family_data() + ->current() + ->storage_info() + ->ComputeCompactionScore(*(c->immutable_options()), + *(c->mutable_cf_options())); + AddToCompactionQueue(cfd); + ++unscheduled_compactions_; + + c.reset(); + // Don't need to sleep here, because BackgroundCallCompaction + // will sleep if !s.ok() + status = Status::CompactionTooLarge(); + } else { + // update statistics + size_t num_files = 0; + for (auto& each_level : *c->inputs()) { + num_files += each_level.files.size(); + } + RecordInHistogram(stats_, NUM_FILES_IN_SINGLE_COMPACTION, num_files); + + // There are three things that can change compaction score: + // 1) When flush or compaction finish. This case is covered by + // InstallSuperVersionAndScheduleWork + // 2) When MutableCFOptions changes. This case is also covered by + // InstallSuperVersionAndScheduleWork, because this is when the new + // options take effect. + // 3) When we Pick a new compaction, we "remove" those files being + // compacted from the calculation, which then influences compaction + // score. Here we check if we need the new compaction even without the + // files that are currently being compacted. If we need another + // compaction, we might be able to execute it in parallel, so we add + // it to the queue and schedule a new thread. + if (cfd->NeedsCompaction()) { + // Yes, we need more compactions! + AddToCompactionQueue(cfd); + ++unscheduled_compactions_; + MaybeScheduleFlushOrCompaction(); + } + } + } + } + } + + IOStatus io_s; + if (!c) { + // Nothing to do + ROCKS_LOG_BUFFER(log_buffer, "Compaction nothing to do"); + } else if (c->deletion_compaction()) { + // TODO(icanadi) Do we want to honor snapshots here? i.e. not delete old + // file if there is alive snapshot pointing to it + 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 == + kCompactionStyleFIFO); + + compaction_job_stats.num_input_files = c->num_input_files(0); + + NotifyOnCompactionBegin(c->column_family_data(), c.get(), status, + compaction_job_stats, job_context->job_id); + + for (const auto& f : *c->inputs(0)) { + c->edit()->DeleteFile(c->level(), f->fd.GetNumber()); + } + status = versions_->LogAndApply(c->column_family_data(), + *c->mutable_cf_options(), c->edit(), + &mutex_, directories_.GetDbDir()); + io_s = versions_->io_status(); + InstallSuperVersionAndScheduleWork(c->column_family_data(), + &job_context->superversion_contexts[0], + *c->mutable_cf_options()); + ROCKS_LOG_BUFFER(log_buffer, "[%s] Deleted %d files\n", + c->column_family_data()->GetName().c_str(), + c->num_input_files(0)); + *made_progress = true; + TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction", + c->column_family_data()); + } else if (!trivial_move_disallowed && c->IsTrivialMove()) { + TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove"); + TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction", + c->column_family_data()); + // Instrument for event update + // TODO(yhchiang): add op details for showing trivial-move. + ThreadStatusUtil::SetColumnFamily( + c->column_family_data(), c->column_family_data()->ioptions()->env, + immutable_db_options_.enable_thread_tracking); + ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION); + + compaction_job_stats.num_input_files = c->num_input_files(0); + + NotifyOnCompactionBegin(c->column_family_data(), c.get(), status, + compaction_job_stats, job_context->job_id); + + // Move files to next level + int32_t moved_files = 0; + int64_t moved_bytes = 0; + for (unsigned int l = 0; l < c->num_input_levels(); l++) { + if (c->level(l) == c->output_level()) { + continue; + } + for (size_t i = 0; i < c->num_input_files(l); i++) { + FileMetaData* f = c->input(l, i); + c->edit()->DeleteFile(c->level(l), f->fd.GetNumber()); + c->edit()->AddFile( + c->output_level(), f->fd.GetNumber(), f->fd.GetPathId(), + f->fd.GetFileSize(), f->smallest, f->largest, f->fd.smallest_seqno, + f->fd.largest_seqno, f->marked_for_compaction, f->temperature, + f->oldest_blob_file_number, f->oldest_ancester_time, + f->file_creation_time, f->file_checksum, f->file_checksum_func_name, + f->unique_id); + + ROCKS_LOG_BUFFER( + log_buffer, + "[%s] Moving #%" PRIu64 " to level-%d %" PRIu64 " bytes\n", + c->column_family_data()->GetName().c_str(), f->fd.GetNumber(), + c->output_level(), f->fd.GetFileSize()); + ++moved_files; + moved_bytes += f->fd.GetFileSize(); + } + } + if (c->compaction_reason() == CompactionReason::kLevelMaxLevelSize && + c->immutable_options()->compaction_pri == kRoundRobin) { + int start_level = c->start_level(); + if (start_level > 0) { + auto vstorage = c->input_version()->storage_info(); + c->edit()->AddCompactCursor( + start_level, + vstorage->GetNextCompactCursor(start_level, c->num_input_files(0))); + } + } + status = versions_->LogAndApply(c->column_family_data(), + *c->mutable_cf_options(), c->edit(), + &mutex_, directories_.GetDbDir()); + io_s = versions_->io_status(); + // Use latest MutableCFOptions + InstallSuperVersionAndScheduleWork(c->column_family_data(), + &job_context->superversion_contexts[0], + *c->mutable_cf_options()); + + VersionStorageInfo::LevelSummaryStorage tmp; + c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(), + moved_bytes); + { + event_logger_.LogToBuffer(log_buffer) + << "job" << job_context->job_id << "event" + << "trivial_move" + << "destination_level" << c->output_level() << "files" << moved_files + << "total_files_size" << moved_bytes; + } + ROCKS_LOG_BUFFER( + log_buffer, + "[%s] Moved #%d files to level-%d %" PRIu64 " bytes %s: %s\n", + c->column_family_data()->GetName().c_str(), moved_files, + c->output_level(), moved_bytes, status.ToString().c_str(), + c->column_family_data()->current()->storage_info()->LevelSummary(&tmp)); + *made_progress = true; + + // Clear Instrument + ThreadStatusUtil::ResetThreadStatus(); + TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction", + c->column_family_data()); + } else if (!is_prepicked && c->output_level() > 0 && + c->output_level() == + c->column_family_data() + ->current() + ->storage_info() + ->MaxOutputLevel( + immutable_db_options_.allow_ingest_behind) && + env_->GetBackgroundThreads(Env::Priority::BOTTOM) > 0) { + // Forward compactions involving last level to the bottom pool if it exists, + // such that compactions unlikely to contribute to write stalls can be + // delayed or deprioritized. + TEST_SYNC_POINT("DBImpl::BackgroundCompaction:ForwardToBottomPriPool"); + CompactionArg* ca = new CompactionArg; + ca->db = this; + ca->compaction_pri_ = Env::Priority::BOTTOM; + ca->prepicked_compaction = new PrepickedCompaction; + ca->prepicked_compaction->compaction = c.release(); + ca->prepicked_compaction->manual_compaction_state = nullptr; + // Transfer requested token, so it doesn't need to do it again. + ca->prepicked_compaction->task_token = std::move(task_token); + ++bg_bottom_compaction_scheduled_; + env_->Schedule(&DBImpl::BGWorkBottomCompaction, ca, Env::Priority::BOTTOM, + this, &DBImpl::UnscheduleCompactionCallback); + } else { + TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction", + c->column_family_data()); + int output_level __attribute__((__unused__)); + output_level = c->output_level(); + TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial", + &output_level); + std::vector snapshot_seqs; + SequenceNumber earliest_write_conflict_snapshot; + SnapshotChecker* snapshot_checker; + GetSnapshotContext(job_context, &snapshot_seqs, + &earliest_write_conflict_snapshot, &snapshot_checker); + assert(is_snapshot_supported_ || snapshots_.empty()); + + CompactionJob compaction_job( + job_context->job_id, c.get(), immutable_db_options_, + mutable_db_options_, file_options_for_compaction_, versions_.get(), + &shutting_down_, log_buffer, directories_.GetDbDir(), + GetDataDir(c->column_family_data(), c->output_path_id()), + 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_, + &compaction_job_stats, thread_pri, io_tracer_, + is_manual ? manual_compaction->canceled + : kManualCompactionCanceledFalse_, + db_id_, db_session_id_, c->column_family_data()->GetFullHistoryTsLow(), + c->trim_ts(), &blob_callback_, &bg_compaction_scheduled_, + &bg_bottom_compaction_scheduled_); + compaction_job.Prepare(); + + NotifyOnCompactionBegin(c->column_family_data(), c.get(), status, + compaction_job_stats, job_context->job_id); + mutex_.Unlock(); + TEST_SYNC_POINT_CALLBACK( + "DBImpl::BackgroundCompaction:NonTrivial:BeforeRun", nullptr); + // Should handle erorr? + compaction_job.Run().PermitUncheckedError(); + TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun"); + mutex_.Lock(); + + status = compaction_job.Install(*c->mutable_cf_options()); + io_s = compaction_job.io_status(); + if (status.ok()) { + InstallSuperVersionAndScheduleWork(c->column_family_data(), + &job_context->superversion_contexts[0], + *c->mutable_cf_options()); + } + *made_progress = true; + TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction", + c->column_family_data()); + } + + if (status.ok() && !io_s.ok()) { + status = io_s; + } else { + io_s.PermitUncheckedError(); + } + + if (c != nullptr) { + c->ReleaseCompactionFiles(status); + *made_progress = true; + +#ifndef ROCKSDB_LITE + // Need to make sure SstFileManager does its bookkeeping + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); + if (sfm && sfm_reserved_compact_space) { + sfm->OnCompactionCompletion(c.get()); + } +#endif // ROCKSDB_LITE + + NotifyOnCompactionCompleted(c->column_family_data(), c.get(), status, + compaction_job_stats, job_context->job_id); + } + + if (status.ok() || status.IsCompactionTooLarge() || + status.IsManualCompactionPaused()) { + // Done + } else if (status.IsColumnFamilyDropped() || status.IsShutdownInProgress()) { + // Ignore compaction errors found during shutting down + } else { + ROCKS_LOG_WARN(immutable_db_options_.info_log, "Compaction error: %s", + status.ToString().c_str()); + if (!io_s.ok()) { + // Error while writing to MANIFEST. + // In fact, versions_->io_status() can also be the result of renaming + // CURRENT file. With current code, it's just difficult to tell. So just + // be pessimistic and try write to a new MANIFEST. + // TODO: distinguish between MANIFEST write and CURRENT renaming + auto err_reason = versions_->io_status().ok() + ? BackgroundErrorReason::kCompaction + : BackgroundErrorReason::kManifestWrite; + error_handler_.SetBGError(io_s, err_reason); + } else { + error_handler_.SetBGError(status, BackgroundErrorReason::kCompaction); + } + if (c != nullptr && !is_manual && !error_handler_.IsBGWorkStopped()) { + // Put this cfd back in the compaction queue so we can retry after some + // time + auto cfd = c->column_family_data(); + assert(cfd != nullptr); + // Since this compaction failed, we need to recompute the score so it + // takes the original input files into account + c->column_family_data() + ->current() + ->storage_info() + ->ComputeCompactionScore(*(c->immutable_options()), + *(c->mutable_cf_options())); + if (!cfd->queued_for_compaction()) { + AddToCompactionQueue(cfd); + ++unscheduled_compactions_; + } + } + } + // this will unref its input_version and column_family_data + c.reset(); + + if (is_manual) { + ManualCompactionState* m = manual_compaction; + if (!status.ok()) { + m->status = status; + m->done = true; + } + // For universal compaction: + // Because universal compaction always happens at level 0, so one + // compaction will pick up all overlapped files. No files will be + // filtered out due to size limit and left for a successive compaction. + // So we can safely conclude the current compaction. + // + // Also note that, if we don't stop here, then the current compaction + // writes a new file back to level 0, which will be used in successive + // compaction. Hence the manual compaction will never finish. + // + // Stop the compaction if manual_end points to nullptr -- this means + // that we compacted the whole range. manual_end should always point + // to nullptr in case of universal compaction + if (m->manual_end == nullptr) { + m->done = true; + } + if (!m->done) { + // 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); + m->tmp_storage = *m->manual_end; + m->begin = &m->tmp_storage; + m->incomplete = true; + } + m->in_progress = false; // not being processed anymore + } + TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Finish"); + return status; +} + +bool DBImpl::HasPendingManualCompaction() { + return (!manual_compaction_dequeue_.empty()); +} + +void DBImpl::AddManualCompaction(DBImpl::ManualCompactionState* m) { + assert(manual_compaction_paused_ == 0); + manual_compaction_dequeue_.push_back(m); +} + +void DBImpl::RemoveManualCompaction(DBImpl::ManualCompactionState* m) { + // Remove from queue + std::deque::iterator it = + manual_compaction_dequeue_.begin(); + while (it != manual_compaction_dequeue_.end()) { + if (m == (*it)) { + it = manual_compaction_dequeue_.erase(it); + return; + } + ++it; + } + assert(false); + return; +} + +bool DBImpl::ShouldntRunManualCompaction(ManualCompactionState* m) { + if (num_running_ingest_file_ > 0) { + // We need to wait for other IngestExternalFile() calls to finish + // before running a manual compaction. + return true; + } + if (m->exclusive) { + return (bg_bottom_compaction_scheduled_ > 0 || + bg_compaction_scheduled_ > 0); + } + std::deque::iterator it = + manual_compaction_dequeue_.begin(); + bool seen = false; + while (it != manual_compaction_dequeue_.end()) { + if (m == (*it)) { + ++it; + seen = true; + continue; + } else if (MCOverlap(m, (*it)) && (!seen && !(*it)->in_progress)) { + // Consider the other manual compaction *it, conflicts if: + // overlaps with m + // and (*it) is ahead in the queue and is not yet in progress + return true; + } + ++it; + } + return false; +} + +bool DBImpl::HaveManualCompaction(ColumnFamilyData* cfd) { + // Remove from priority queue + std::deque::iterator it = + manual_compaction_dequeue_.begin(); + while (it != manual_compaction_dequeue_.end()) { + if ((*it)->exclusive) { + return true; + } + if ((cfd == (*it)->cfd) && (!((*it)->in_progress || (*it)->done))) { + // Allow automatic compaction if manual compaction is + // in progress + return true; + } + ++it; + } + return false; +} + +bool DBImpl::HasExclusiveManualCompaction() { + // Remove from priority queue + std::deque::iterator it = + manual_compaction_dequeue_.begin(); + while (it != manual_compaction_dequeue_.end()) { + if ((*it)->exclusive) { + return true; + } + ++it; + } + return false; +} + +bool DBImpl::MCOverlap(ManualCompactionState* m, ManualCompactionState* m1) { + if ((m->exclusive) || (m1->exclusive)) { + return true; + } + if (m->cfd != m1->cfd) { + return false; + } + return false; +} + +#ifndef ROCKSDB_LITE +void DBImpl::BuildCompactionJobInfo( + const ColumnFamilyData* cfd, Compaction* c, const Status& st, + const CompactionJobStats& compaction_job_stats, const int job_id, + const Version* current, CompactionJobInfo* compaction_job_info) const { + assert(compaction_job_info != nullptr); + compaction_job_info->cf_id = cfd->GetID(); + compaction_job_info->cf_name = cfd->GetName(); + compaction_job_info->status = st; + compaction_job_info->thread_id = env_->GetThreadID(); + compaction_job_info->job_id = job_id; + compaction_job_info->base_input_level = c->start_level(); + compaction_job_info->output_level = c->output_level(); + compaction_job_info->stats = compaction_job_stats; + compaction_job_info->table_properties = c->GetOutputTableProperties(); + compaction_job_info->compaction_reason = c->compaction_reason(); + compaction_job_info->compression = c->output_compression(); + for (size_t i = 0; i < c->num_input_levels(); ++i) { + 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, + desc.GetPathId()); + compaction_job_info->input_files.push_back(fn); + compaction_job_info->input_file_infos.push_back(CompactionFileInfo{ + static_cast(i), file_number, fmd->oldest_blob_file_number}); + if (compaction_job_info->table_properties.count(fn) == 0) { + std::shared_ptr tp; + auto s = current->GetTableProperties(&tp, fmd, &fn); + if (s.ok()) { + compaction_job_info->table_properties[fn] = tp; + } + } + } + } + for (const auto& newf : c->edit()->GetNewFiles()) { + const FileMetaData& meta = newf.second; + 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())); + 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; + + // Update BlobFilesInfo. + for (const auto& blob_file : c->edit()->GetBlobFileAdditions()) { + BlobFileAdditionInfo blob_file_addition_info( + BlobFileName(c->immutable_options()->cf_paths.front().path, + blob_file.GetBlobFileNumber()) /*blob_file_path*/, + blob_file.GetBlobFileNumber(), blob_file.GetTotalBlobCount(), + blob_file.GetTotalBlobBytes()); + compaction_job_info->blob_file_addition_infos.emplace_back( + std::move(blob_file_addition_info)); + } + + // Update BlobFilesGarbageInfo. + for (const auto& blob_file : c->edit()->GetBlobFileGarbages()) { + BlobFileGarbageInfo blob_file_garbage_info( + BlobFileName(c->immutable_options()->cf_paths.front().path, + blob_file.GetBlobFileNumber()) /*blob_file_path*/, + blob_file.GetBlobFileNumber(), blob_file.GetGarbageBlobCount(), + blob_file.GetGarbageBlobBytes()); + compaction_job_info->blob_file_garbage_infos.emplace_back( + std::move(blob_file_garbage_info)); + } +} +#endif + +// SuperVersionContext gets created and destructed outside of the lock -- +// we use this conveniently to: +// * malloc one SuperVersion() outside of the lock -- new_superversion +// * delete SuperVersion()s outside of the lock -- superversions_to_free +// +// However, if InstallSuperVersionAndScheduleWork() gets called twice with the +// same sv_context, we can't reuse the SuperVersion() that got +// malloced because +// first call already used it. In that rare case, we take a hit and create a +// new SuperVersion() inside of the mutex. We do similar thing +// for superversion_to_free + +void DBImpl::InstallSuperVersionAndScheduleWork( + ColumnFamilyData* cfd, SuperVersionContext* sv_context, + const MutableCFOptions& mutable_cf_options) { + mutex_.AssertHeld(); + + // Update max_total_in_memory_state_ + size_t old_memtable_size = 0; + auto* old_sv = cfd->GetSuperVersion(); + if (old_sv) { + old_memtable_size = old_sv->mutable_cf_options.write_buffer_size * + old_sv->mutable_cf_options.max_write_buffer_number; + } + + // this branch is unlikely to step in + if (UNLIKELY(sv_context->new_superversion == nullptr)) { + sv_context->NewSuperVersion(); + } + cfd->InstallSuperVersion(sv_context, mutable_cf_options); + + // There may be a small data race here. The snapshot tricking bottommost + // compaction may already be released here. But assuming there will always be + // newer snapshot created and released frequently, the compaction will be + // triggered soon anyway. + bottommost_files_mark_threshold_ = kMaxSequenceNumber; + for (auto* my_cfd : *versions_->GetColumnFamilySet()) { + 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()); + } + } + + // Whenever we install new SuperVersion, we might need to issue new flushes or + // compactions. + SchedulePendingCompaction(cfd); + MaybeScheduleFlushOrCompaction(); + + // Update max_total_in_memory_state_ + max_total_in_memory_state_ = max_total_in_memory_state_ - old_memtable_size + + mutable_cf_options.write_buffer_size * + mutable_cf_options.max_write_buffer_number; +} + +// ShouldPurge is called by FindObsoleteFiles when doing a full scan, +// and db mutex (mutex_) should already be held. +// Actually, the current implementation of FindObsoleteFiles with +// full_scan=true can issue I/O requests to obtain list of files in +// directories, e.g. env_->getChildren while holding db mutex. +bool DBImpl::ShouldPurge(uint64_t file_number) const { + return files_grabbed_for_purge_.find(file_number) == + files_grabbed_for_purge_.end() && + purge_files_.find(file_number) == purge_files_.end(); +} + +// MarkAsGrabbedForPurge is called by FindObsoleteFiles, and db mutex +// (mutex_) should already be held. +void DBImpl::MarkAsGrabbedForPurge(uint64_t file_number) { + files_grabbed_for_purge_.insert(file_number); +} + +void DBImpl::SetSnapshotChecker(SnapshotChecker* snapshot_checker) { + InstrumentedMutexLock l(&mutex_); + // snapshot_checker_ should only set once. If we need to set it multiple + // times, we need to make sure the old one is not deleted while it is still + // using by a compaction job. + assert(!snapshot_checker_); + snapshot_checker_.reset(snapshot_checker); +} + +void DBImpl::GetSnapshotContext( + JobContext* job_context, std::vector* snapshot_seqs, + SequenceNumber* earliest_write_conflict_snapshot, + SnapshotChecker** snapshot_checker_ptr) { + mutex_.AssertHeld(); + assert(job_context != nullptr); + assert(snapshot_seqs != nullptr); + assert(earliest_write_conflict_snapshot != nullptr); + assert(snapshot_checker_ptr != nullptr); + + *snapshot_checker_ptr = snapshot_checker_.get(); + if (use_custom_gc_ && *snapshot_checker_ptr == nullptr) { + *snapshot_checker_ptr = DisableGCSnapshotChecker::Instance(); + } + if (*snapshot_checker_ptr != nullptr) { + // If snapshot_checker is used, that means the flush/compaction may + // contain values not visible to snapshot taken after + // flush/compaction job starts. Take a snapshot and it will appear + // in snapshot_seqs and force compaction iterator to consider such + // snapshots. + const Snapshot* job_snapshot = + GetSnapshotImpl(false /*write_conflict_boundary*/, false /*lock*/); + job_context->job_snapshot.reset(new ManagedSnapshot(this, job_snapshot)); + } + *snapshot_seqs = snapshots_.GetAll(earliest_write_conflict_snapshot); +} + +Status DBImpl::WaitForCompact(bool wait_unscheduled) { + // Wait until the compaction completes + InstrumentedMutexLock l(&mutex_); + while ((bg_bottom_compaction_scheduled_ || bg_compaction_scheduled_ || + bg_flush_scheduled_ || + (wait_unscheduled && unscheduled_compactions_)) && + (error_handler_.GetBGError().ok())) { + bg_cv_.Wait(); + } + return error_handler_.GetBGError(); +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl_debug.cc b/src/rocksdb/db/db_impl/db_impl_debug.cc new file mode 100644 index 000000000..7054b0669 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_debug.cc @@ -0,0 +1,312 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#ifndef NDEBUG + +#include "db/column_family.h" +#include "db/db_impl/db_impl.h" +#include "db/error_handler.h" +#include "db/periodic_task_scheduler.h" +#include "monitoring/thread_status_updater.h" +#include "util/cast_util.h" + +namespace ROCKSDB_NAMESPACE { +uint64_t DBImpl::TEST_GetLevel0TotalSize() { + InstrumentedMutexLock l(&mutex_); + return default_cf_handle_->cfd()->current()->storage_info()->NumLevelBytes(0); +} + +Status DBImpl::TEST_SwitchWAL() { + WriteContext write_context; + InstrumentedMutexLock l(&mutex_); + void* writer = TEST_BeginWrite(); + auto s = SwitchWAL(&write_context); + TEST_EndWrite(writer); + return s; +} + +uint64_t DBImpl::TEST_MaxNextLevelOverlappingBytes( + ColumnFamilyHandle* column_family) { + ColumnFamilyData* cfd; + if (column_family == nullptr) { + cfd = default_cf_handle_->cfd(); + } else { + auto cfh = static_cast_with_check(column_family); + cfd = cfh->cfd(); + } + InstrumentedMutexLock l(&mutex_); + return cfd->current()->storage_info()->MaxNextLevelOverlappingBytes(); +} + +void DBImpl::TEST_GetFilesMetaData( + ColumnFamilyHandle* column_family, + std::vector>* metadata, + std::vector>* blob_metadata) { + assert(metadata); + + auto cfh = static_cast_with_check(column_family); + assert(cfh); + + auto cfd = cfh->cfd(); + assert(cfd); + + InstrumentedMutexLock l(&mutex_); + + const auto* current = cfd->current(); + assert(current); + + const auto* vstorage = current->storage_info(); + assert(vstorage); + + metadata->resize(NumberLevels()); + + for (int level = 0; level < NumberLevels(); ++level) { + const std::vector& files = vstorage->LevelFiles(level); + + (*metadata)[level].clear(); + (*metadata)[level].reserve(files.size()); + + for (const auto& f : files) { + (*metadata)[level].push_back(*f); + } + } + + if (blob_metadata) { + *blob_metadata = vstorage->GetBlobFiles(); + } +} + +uint64_t DBImpl::TEST_Current_Manifest_FileNo() { + return versions_->manifest_file_number(); +} + +uint64_t DBImpl::TEST_Current_Next_FileNo() { + return versions_->current_next_file_number(); +} + +Status DBImpl::TEST_CompactRange(int level, const Slice* begin, + const Slice* end, + ColumnFamilyHandle* column_family, + bool disallow_trivial_move) { + ColumnFamilyData* cfd; + if (column_family == nullptr) { + cfd = default_cf_handle_->cfd(); + } else { + auto cfh = static_cast_with_check(column_family); + cfd = cfh->cfd(); + } + int output_level = + (cfd->ioptions()->compaction_style == kCompactionStyleUniversal || + cfd->ioptions()->compaction_style == kCompactionStyleFIFO) + ? level + : level + 1; + return RunManualCompaction( + cfd, level, output_level, CompactRangeOptions(), begin, end, true, + disallow_trivial_move, + std::numeric_limits::max() /*max_file_num_to_ignore*/, + "" /*trim_ts*/); +} + +Status DBImpl::TEST_SwitchMemtable(ColumnFamilyData* cfd) { + WriteContext write_context; + InstrumentedMutexLock l(&mutex_); + if (cfd == nullptr) { + cfd = default_cf_handle_->cfd(); + } + + Status s; + void* writer = TEST_BeginWrite(); + if (two_write_queues_) { + WriteThread::Writer nonmem_w; + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + s = SwitchMemtable(cfd, &write_context); + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } else { + s = SwitchMemtable(cfd, &write_context); + } + TEST_EndWrite(writer); + return s; +} + +Status DBImpl::TEST_FlushMemTable(bool wait, bool allow_write_stall, + ColumnFamilyHandle* cfh) { + FlushOptions fo; + fo.wait = wait; + fo.allow_write_stall = allow_write_stall; + ColumnFamilyData* cfd; + if (cfh == nullptr) { + cfd = default_cf_handle_->cfd(); + } else { + auto cfhi = static_cast_with_check(cfh); + cfd = cfhi->cfd(); + } + return FlushMemTable(cfd, fo, FlushReason::kTest); +} + +Status DBImpl::TEST_FlushMemTable(ColumnFamilyData* cfd, + const FlushOptions& flush_opts) { + return FlushMemTable(cfd, flush_opts, FlushReason::kTest); +} + +Status DBImpl::TEST_AtomicFlushMemTables( + const autovector& cfds, const FlushOptions& flush_opts) { + return AtomicFlushMemTables(cfds, flush_opts, FlushReason::kTest); +} + +Status DBImpl::TEST_WaitForBackgroundWork() { + InstrumentedMutexLock l(&mutex_); + WaitForBackgroundWork(); + return error_handler_.GetBGError(); +} + +Status DBImpl::TEST_WaitForFlushMemTable(ColumnFamilyHandle* column_family) { + ColumnFamilyData* cfd; + if (column_family == nullptr) { + cfd = default_cf_handle_->cfd(); + } else { + auto cfh = static_cast_with_check(column_family); + cfd = cfh->cfd(); + } + return WaitForFlushMemTable(cfd, nullptr, false); +} + +Status DBImpl::TEST_WaitForCompact(bool wait_unscheduled) { + // Wait until the compaction completes + return WaitForCompact(wait_unscheduled); +} + +Status DBImpl::TEST_WaitForPurge() { + InstrumentedMutexLock l(&mutex_); + while (bg_purge_scheduled_ && error_handler_.GetBGError().ok()) { + bg_cv_.Wait(); + } + return error_handler_.GetBGError(); +} + +Status DBImpl::TEST_GetBGError() { + InstrumentedMutexLock l(&mutex_); + return error_handler_.GetBGError(); +} + +void DBImpl::TEST_LockMutex() { mutex_.Lock(); } + +void DBImpl::TEST_UnlockMutex() { mutex_.Unlock(); } + +void* DBImpl::TEST_BeginWrite() { + auto w = new WriteThread::Writer(); + write_thread_.EnterUnbatched(w, &mutex_); + return reinterpret_cast(w); +} + +void DBImpl::TEST_EndWrite(void* w) { + auto writer = reinterpret_cast(w); + write_thread_.ExitUnbatched(writer); + delete writer; +} + +size_t DBImpl::TEST_LogsToFreeSize() { + InstrumentedMutexLock l(&log_write_mutex_); + return logs_to_free_.size(); +} + +uint64_t DBImpl::TEST_LogfileNumber() { + InstrumentedMutexLock l(&mutex_); + return logfile_number_; +} + +Status DBImpl::TEST_GetAllImmutableCFOptions( + std::unordered_map* iopts_map) { + std::vector cf_names; + std::vector iopts; + { + InstrumentedMutexLock l(&mutex_); + for (auto cfd : *versions_->GetColumnFamilySet()) { + cf_names.push_back(cfd->GetName()); + iopts.push_back(cfd->ioptions()); + } + } + iopts_map->clear(); + for (size_t i = 0; i < cf_names.size(); ++i) { + iopts_map->insert({cf_names[i], iopts[i]}); + } + + return Status::OK(); +} + +uint64_t DBImpl::TEST_FindMinLogContainingOutstandingPrep() { + return logs_with_prep_tracker_.FindMinLogContainingOutstandingPrep(); +} + +size_t DBImpl::TEST_PreparedSectionCompletedSize() { + return logs_with_prep_tracker_.TEST_PreparedSectionCompletedSize(); +} + +size_t DBImpl::TEST_LogsWithPrepSize() { + return logs_with_prep_tracker_.TEST_LogsWithPrepSize(); +} + +uint64_t DBImpl::TEST_FindMinPrepLogReferencedByMemTable() { + autovector empty_list; + return FindMinPrepLogReferencedByMemTable(versions_.get(), empty_list); +} + +Status DBImpl::TEST_GetLatestMutableCFOptions( + ColumnFamilyHandle* column_family, MutableCFOptions* mutable_cf_options) { + InstrumentedMutexLock l(&mutex_); + + auto cfh = static_cast_with_check(column_family); + *mutable_cf_options = *cfh->cfd()->GetLatestMutableCFOptions(); + return Status::OK(); +} + +int DBImpl::TEST_BGCompactionsAllowed() const { + InstrumentedMutexLock l(&mutex_); + return GetBGJobLimits().max_compactions; +} + +int DBImpl::TEST_BGFlushesAllowed() const { + InstrumentedMutexLock l(&mutex_); + return GetBGJobLimits().max_flushes; +} + +SequenceNumber DBImpl::TEST_GetLastVisibleSequence() const { + if (last_seq_same_as_publish_seq_) { + return versions_->LastSequence(); + } else { + return versions_->LastAllocatedSequence(); + } +} + +size_t DBImpl::TEST_GetWalPreallocateBlockSize( + uint64_t write_buffer_size) const { + InstrumentedMutexLock l(&mutex_); + return GetWalPreallocateBlockSize(write_buffer_size); +} + +#ifndef ROCKSDB_LITE +void DBImpl::TEST_WaitForPeridicTaskRun(std::function callback) const { + periodic_task_scheduler_.TEST_WaitForRun(callback); +} + +const PeriodicTaskScheduler& DBImpl::TEST_GetPeriodicTaskScheduler() const { + return periodic_task_scheduler_; +} + +SeqnoToTimeMapping DBImpl::TEST_GetSeqnoToTimeMapping() const { + InstrumentedMutexLock l(&mutex_); + return seqno_time_mapping_; +} + +#endif // !ROCKSDB_LITE + +size_t DBImpl::TEST_EstimateInMemoryStatsHistorySize() const { + return EstimateInMemoryStatsHistorySize(); +} +} // namespace ROCKSDB_NAMESPACE +#endif // NDEBUG diff --git a/src/rocksdb/db/db_impl/db_impl_experimental.cc b/src/rocksdb/db/db_impl/db_impl_experimental.cc new file mode 100644 index 000000000..c1b1e4137 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_experimental.cc @@ -0,0 +1,158 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#include +#include + +#include "db/column_family.h" +#include "db/db_impl/db_impl.h" +#include "db/job_context.h" +#include "db/version_set.h" +#include "logging/logging.h" +#include "rocksdb/status.h" +#include "util/cast_util.h" + +namespace ROCKSDB_NAMESPACE { + +#ifndef ROCKSDB_LITE +Status DBImpl::SuggestCompactRange(ColumnFamilyHandle* column_family, + const Slice* begin, const Slice* end) { + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + InternalKey start_key, end_key; + if (begin != nullptr) { + start_key.SetMinPossibleForUserKey(*begin); + } + if (end != nullptr) { + end_key.SetMaxPossibleForUserKey(*end); + } + { + InstrumentedMutexLock l(&mutex_); + auto vstorage = cfd->current()->storage_info(); + for (int level = 0; level < vstorage->num_non_empty_levels() - 1; ++level) { + std::vector inputs; + vstorage->GetOverlappingInputs( + level, begin == nullptr ? nullptr : &start_key, + end == nullptr ? nullptr : &end_key, &inputs); + for (auto f : inputs) { + f->marked_for_compaction = true; + } + } + // Since we have some more files to compact, we should also recompute + // compaction score + vstorage->ComputeCompactionScore(*cfd->ioptions(), + *cfd->GetLatestMutableCFOptions()); + SchedulePendingCompaction(cfd); + MaybeScheduleFlushOrCompaction(); + } + return Status::OK(); +} + +Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) { + assert(column_family); + + if (target_level < 1) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "PromoteL0 FAILED. Invalid target level %d\n", target_level); + return Status::InvalidArgument("Invalid target level"); + } + + Status status; + VersionEdit edit; + JobContext job_context(next_job_id_.fetch_add(1), true); + { + InstrumentedMutexLock l(&mutex_); + auto* cfd = static_cast(column_family)->cfd(); + const auto* vstorage = cfd->current()->storage_info(); + + if (target_level >= vstorage->num_levels()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "PromoteL0 FAILED. Target level %d does not exist\n", + target_level); + job_context.Clean(); + status = Status::InvalidArgument("Target level does not exist"); + return status; + } + + // Sort L0 files by range. + const InternalKeyComparator* icmp = &cfd->internal_comparator(); + auto l0_files = vstorage->LevelFiles(0); + std::sort(l0_files.begin(), l0_files.end(), + [icmp](FileMetaData* f1, FileMetaData* f2) { + return icmp->Compare(f1->largest, f2->largest) < 0; + }); + + // Check that no L0 file is being compacted and that they have + // non-overlapping ranges. + for (size_t i = 0; i < l0_files.size(); ++i) { + auto f = l0_files[i]; + if (f->being_compacted) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "PromoteL0 FAILED. File %" PRIu64 " being compacted\n", + f->fd.GetNumber()); + job_context.Clean(); + status = + Status::InvalidArgument("PromoteL0 called during L0 compaction"); + return status; + } + + if (i == 0) continue; + auto prev_f = l0_files[i - 1]; + if (icmp->Compare(prev_f->largest, f->smallest) >= 0) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "PromoteL0 FAILED. Files %" PRIu64 " and %" PRIu64 + " have overlapping ranges\n", + prev_f->fd.GetNumber(), f->fd.GetNumber()); + job_context.Clean(); + status = Status::InvalidArgument("L0 has overlapping files"); + return status; + } + } + + // Check that all levels up to target_level are empty. + for (int level = 1; level <= target_level; ++level) { + if (vstorage->NumLevelFiles(level) > 0) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "PromoteL0 FAILED. Level %d not empty\n", level); + job_context.Clean(); + status = Status::InvalidArgument( + "All levels up to target_level " + "must be empty"); + return status; + } + } + + edit.SetColumnFamily(cfd->GetID()); + for (const auto& f : l0_files) { + edit.DeleteFile(0, f->fd.GetNumber()); + edit.AddFile(target_level, f->fd.GetNumber(), f->fd.GetPathId(), + f->fd.GetFileSize(), f->smallest, f->largest, + f->fd.smallest_seqno, f->fd.largest_seqno, + f->marked_for_compaction, f->temperature, + f->oldest_blob_file_number, f->oldest_ancester_time, + f->file_creation_time, f->file_checksum, + f->file_checksum_func_name, f->unique_id); + } + + status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), + &edit, &mutex_, directories_.GetDbDir()); + if (status.ok()) { + InstallSuperVersionAndScheduleWork(cfd, + &job_context.superversion_contexts[0], + *cfd->GetLatestMutableCFOptions()); + } + } // lock released here + LogFlush(immutable_db_options_.info_log); + job_context.Clean(); + + return status; +} +#endif // ROCKSDB_LITE + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl_files.cc b/src/rocksdb/db/db_impl/db_impl_files.cc new file mode 100644 index 000000000..058df4da7 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_files.cc @@ -0,0 +1,1013 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +#include +#include +#include + +#include "db/db_impl/db_impl.h" +#include "db/event_helpers.h" +#include "db/memtable_list.h" +#include "file/file_util.h" +#include "file/filename.h" +#include "file/sst_file_manager_impl.h" +#include "logging/logging.h" +#include "port/port.h" +#include "util/autovector.h" +#include "util/defer.h" + +namespace ROCKSDB_NAMESPACE { + +uint64_t DBImpl::MinLogNumberToKeep() { + return versions_->min_log_number_to_keep(); +} + +uint64_t DBImpl::MinObsoleteSstNumberToKeep() { + mutex_.AssertHeld(); + if (!pending_outputs_.empty()) { + return *pending_outputs_.begin(); + } + return std::numeric_limits::max(); +} + +Status DBImpl::DisableFileDeletions() { + Status s; + int my_disable_delete_obsolete_files; + { + InstrumentedMutexLock l(&mutex_); + s = DisableFileDeletionsWithLock(); + my_disable_delete_obsolete_files = disable_delete_obsolete_files_; + } + if (my_disable_delete_obsolete_files == 1) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "File Deletions Disabled"); + } else { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "File Deletions Disabled, but already disabled. Counter: %d", + my_disable_delete_obsolete_files); + } + return s; +} + +// FIXME: can be inconsistent with DisableFileDeletions in cases like +// DBImplReadOnly +Status DBImpl::DisableFileDeletionsWithLock() { + mutex_.AssertHeld(); + ++disable_delete_obsolete_files_; + return Status::OK(); +} + +Status DBImpl::EnableFileDeletions(bool force) { + // Job id == 0 means that this is not our background process, but rather + // user thread + JobContext job_context(0); + int saved_counter; // initialize on all paths + { + InstrumentedMutexLock l(&mutex_); + if (force) { + // if force, we need to enable file deletions right away + disable_delete_obsolete_files_ = 0; + } else if (disable_delete_obsolete_files_ > 0) { + --disable_delete_obsolete_files_; + } + saved_counter = disable_delete_obsolete_files_; + if (saved_counter == 0) { + FindObsoleteFiles(&job_context, true); + bg_cv_.SignalAll(); + } + } + if (saved_counter == 0) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "File Deletions Enabled"); + if (job_context.HaveSomethingToDelete()) { + PurgeObsoleteFiles(job_context); + } + } else { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "File Deletions Enable, but not really enabled. Counter: %d", + saved_counter); + } + job_context.Clean(); + LogFlush(immutable_db_options_.info_log); + return Status::OK(); +} + +bool DBImpl::IsFileDeletionsEnabled() const { + return 0 == disable_delete_obsolete_files_; +} + +// * Returns the list of live files in 'sst_live' and 'blob_live'. +// If it's doing full scan: +// * Returns the list of all files in the filesystem in +// 'full_scan_candidate_files'. +// Otherwise, gets obsolete files from VersionSet. +// no_full_scan = true -- never do the full scan using GetChildren() +// force = false -- don't force the full scan, except every +// mutable_db_options_.delete_obsolete_files_period_micros +// force = true -- force the full scan +void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, + bool no_full_scan) { + mutex_.AssertHeld(); + + // if deletion is disabled, do nothing + if (disable_delete_obsolete_files_ > 0) { + return; + } + + bool doing_the_full_scan = false; + + // logic for figuring out if we're doing the full scan + if (no_full_scan) { + doing_the_full_scan = false; + } else if (force || + mutable_db_options_.delete_obsolete_files_period_micros == 0) { + doing_the_full_scan = true; + } else { + const uint64_t now_micros = immutable_db_options_.clock->NowMicros(); + if ((delete_obsolete_files_last_run_ + + mutable_db_options_.delete_obsolete_files_period_micros) < + now_micros) { + doing_the_full_scan = true; + delete_obsolete_files_last_run_ = now_micros; + } + } + + // don't delete files that might be currently written to from compaction + // threads + // Since job_context->min_pending_output is set, until file scan finishes, + // mutex_ cannot be released. Otherwise, we might see no min_pending_output + // here but later find newer generated unfinalized files while scanning. + job_context->min_pending_output = MinObsoleteSstNumberToKeep(); + + // Get obsolete files. This function will also update the list of + // pending files in VersionSet(). + versions_->GetObsoleteFiles( + &job_context->sst_delete_files, &job_context->blob_delete_files, + &job_context->manifest_delete_files, job_context->min_pending_output); + + // Mark the elements in job_context->sst_delete_files and + // job_context->blob_delete_files as "grabbed for purge" so that other threads + // calling FindObsoleteFiles with full_scan=true will not add these files to + // candidate list for purge. + for (const auto& sst_to_del : job_context->sst_delete_files) { + MarkAsGrabbedForPurge(sst_to_del.metadata->fd.GetNumber()); + } + + for (const auto& blob_file : job_context->blob_delete_files) { + MarkAsGrabbedForPurge(blob_file.GetBlobFileNumber()); + } + + // store the current filenum, lognum, etc + job_context->manifest_file_number = versions_->manifest_file_number(); + job_context->pending_manifest_file_number = + versions_->pending_manifest_file_number(); + job_context->log_number = MinLogNumberToKeep(); + job_context->prev_log_number = versions_->prev_log_number(); + + if (doing_the_full_scan) { + versions_->AddLiveFiles(&job_context->sst_live, &job_context->blob_live); + InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(), + dbname_); + std::set paths; + for (size_t path_id = 0; path_id < immutable_db_options_.db_paths.size(); + path_id++) { + paths.insert(immutable_db_options_.db_paths[path_id].path); + } + + // Note that if cf_paths is not specified in the ColumnFamilyOptions + // of a particular column family, we use db_paths as the cf_paths + // setting. Hence, there can be multiple duplicates of files from db_paths + // in the following code. The duplicate are removed while identifying + // unique files in PurgeObsoleteFiles. + for (auto cfd : *versions_->GetColumnFamilySet()) { + for (size_t path_id = 0; path_id < cfd->ioptions()->cf_paths.size(); + path_id++) { + auto& path = cfd->ioptions()->cf_paths[path_id].path; + + if (paths.find(path) == paths.end()) { + paths.insert(path); + } + } + } + + IOOptions io_opts; + io_opts.do_not_recurse = true; + for (auto& path : paths) { + // set of all files in the directory. We'll exclude files that are still + // alive in the subsequent processings. + std::vector files; + Status s = immutable_db_options_.fs->GetChildren( + path, io_opts, &files, /*IODebugContext*=*/nullptr); + s.PermitUncheckedError(); // TODO: What should we do on error? + for (const std::string& file : files) { + uint64_t number; + FileType type; + // 1. If we cannot parse the file name, we skip; + // 2. If the file with file_number equals number has already been + // grabbed for purge by another compaction job, or it has already been + // schedule for purge, we also skip it if we + // are doing full scan in order to avoid double deletion of the same + // file under race conditions. See + // https://github.com/facebook/rocksdb/issues/3573 + if (!ParseFileName(file, &number, info_log_prefix.prefix, &type) || + !ShouldPurge(number)) { + continue; + } + + // TODO(icanadi) clean up this mess to avoid having one-off "/" + // prefixes + job_context->full_scan_candidate_files.emplace_back("/" + file, path); + } + } + + // Add log files in wal_dir + if (!immutable_db_options_.IsWalDirSameAsDBPath(dbname_)) { + std::vector log_files; + Status s = immutable_db_options_.fs->GetChildren( + immutable_db_options_.wal_dir, io_opts, &log_files, + /*IODebugContext*=*/nullptr); + s.PermitUncheckedError(); // TODO: What should we do on error? + for (const std::string& log_file : log_files) { + job_context->full_scan_candidate_files.emplace_back( + log_file, immutable_db_options_.wal_dir); + } + } + + // Add info log files in db_log_dir + if (!immutable_db_options_.db_log_dir.empty() && + immutable_db_options_.db_log_dir != dbname_) { + std::vector info_log_files; + Status s = immutable_db_options_.fs->GetChildren( + immutable_db_options_.db_log_dir, io_opts, &info_log_files, + /*IODebugContext*=*/nullptr); + s.PermitUncheckedError(); // TODO: What should we do on error? + for (std::string& log_file : info_log_files) { + job_context->full_scan_candidate_files.emplace_back( + log_file, immutable_db_options_.db_log_dir); + } + } + } else { + // Instead of filling ob_context->sst_live and job_context->blob_live, + // directly remove files that show up in any Version. This is because + // candidate files tend to be a small percentage of all files, so it is + // usually cheaper to check them against every version, compared to + // building a map for all files. + versions_->RemoveLiveFiles(job_context->sst_delete_files, + job_context->blob_delete_files); + } + + // Before potentially releasing mutex and waiting on condvar, increment + // pending_purge_obsolete_files_ so that another thread executing + // `GetSortedWals` will wait until this thread finishes execution since the + // other thread will be waiting for `pending_purge_obsolete_files_`. + // pending_purge_obsolete_files_ MUST be decremented if there is nothing to + // delete. + ++pending_purge_obsolete_files_; + + Defer cleanup([job_context, this]() { + assert(job_context != nullptr); + if (!job_context->HaveSomethingToDelete()) { + mutex_.AssertHeld(); + --pending_purge_obsolete_files_; + } + }); + + // logs_ is empty when called during recovery, in which case there can't yet + // be any tracked obsolete logs + log_write_mutex_.Lock(); + + if (alive_log_files_.empty() || logs_.empty()) { + mutex_.AssertHeld(); + // We may reach here if the db is DBImplSecondary + log_write_mutex_.Unlock(); + return; + } + + if (!alive_log_files_.empty() && !logs_.empty()) { + uint64_t min_log_number = job_context->log_number; + size_t num_alive_log_files = alive_log_files_.size(); + // find newly obsoleted log files + while (alive_log_files_.begin()->number < min_log_number) { + auto& earliest = *alive_log_files_.begin(); + if (immutable_db_options_.recycle_log_file_num > + log_recycle_files_.size()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "adding log %" PRIu64 " to recycle list\n", + earliest.number); + log_recycle_files_.push_back(earliest.number); + } else { + job_context->log_delete_files.push_back(earliest.number); + } + if (job_context->size_log_to_delete == 0) { + job_context->prev_total_log_size = total_log_size_; + job_context->num_alive_log_files = num_alive_log_files; + } + job_context->size_log_to_delete += earliest.size; + total_log_size_ -= earliest.size; + alive_log_files_.pop_front(); + + // Current log should always stay alive since it can't have + // number < MinLogNumber(). + assert(alive_log_files_.size()); + } + log_write_mutex_.Unlock(); + mutex_.Unlock(); + log_write_mutex_.Lock(); + while (!logs_.empty() && logs_.front().number < min_log_number) { + auto& log = logs_.front(); + if (log.IsSyncing()) { + log_sync_cv_.Wait(); + // logs_ could have changed while we were waiting. + continue; + } + logs_to_free_.push_back(log.ReleaseWriter()); + logs_.pop_front(); + } + // Current log cannot be obsolete. + assert(!logs_.empty()); + } + + // We're just cleaning up for DB::Write(). + assert(job_context->logs_to_free.empty()); + job_context->logs_to_free = logs_to_free_; + + logs_to_free_.clear(); + log_write_mutex_.Unlock(); + mutex_.Lock(); + job_context->log_recycle_files.assign(log_recycle_files_.begin(), + log_recycle_files_.end()); +} + +// Delete obsolete files and log status and information of file deletion +void DBImpl::DeleteObsoleteFileImpl(int job_id, const std::string& fname, + const std::string& path_to_sync, + FileType type, uint64_t number) { + TEST_SYNC_POINT_CALLBACK("DBImpl::DeleteObsoleteFileImpl::BeforeDeletion", + const_cast(&fname)); + + Status file_deletion_status; + if (type == kTableFile || type == kBlobFile || type == kWalFile) { + // Rate limit WAL deletion only if its in the DB dir + file_deletion_status = DeleteDBFile( + &immutable_db_options_, fname, path_to_sync, + /*force_bg=*/false, + /*force_fg=*/(type == kWalFile) ? !wal_in_db_path_ : false); + } else { + file_deletion_status = env_->DeleteFile(fname); + } + TEST_SYNC_POINT_CALLBACK("DBImpl::DeleteObsoleteFileImpl:AfterDeletion", + &file_deletion_status); + if (file_deletion_status.ok()) { + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", job_id, + fname.c_str(), type, number, + file_deletion_status.ToString().c_str()); + } else if (env_->FileExists(fname).IsNotFound()) { + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "[JOB %d] Tried to delete a non-existing file %s type=%d #%" PRIu64 + " -- %s\n", + job_id, fname.c_str(), type, number, + file_deletion_status.ToString().c_str()); + } else { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "[JOB %d] Failed to delete %s type=%d #%" PRIu64 " -- %s\n", + job_id, fname.c_str(), type, number, + file_deletion_status.ToString().c_str()); + } + if (type == kTableFile) { + EventHelpers::LogAndNotifyTableFileDeletion( + &event_logger_, job_id, number, fname, file_deletion_status, GetName(), + immutable_db_options_.listeners); + } + if (type == kBlobFile) { + EventHelpers::LogAndNotifyBlobFileDeletion( + &event_logger_, immutable_db_options_.listeners, job_id, number, fname, + file_deletion_status, GetName()); + } +} + +// Diffs the files listed in filenames and those that do not +// belong to live files are possibly removed. Also, removes all the +// files in sst_delete_files and log_delete_files. +// It is not necessary to hold the mutex when invoking this method. +void DBImpl::PurgeObsoleteFiles(JobContext& state, bool schedule_only) { + TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:Begin"); + // we'd better have sth to delete + assert(state.HaveSomethingToDelete()); + + // FindObsoleteFiles() should've populated this so nonzero + assert(state.manifest_file_number != 0); + + // Now, convert lists to unordered sets, WITHOUT mutex held; set is slow. + std::unordered_set sst_live_set(state.sst_live.begin(), + state.sst_live.end()); + std::unordered_set blob_live_set(state.blob_live.begin(), + state.blob_live.end()); + std::unordered_set log_recycle_files_set( + state.log_recycle_files.begin(), state.log_recycle_files.end()); + + auto candidate_files = state.full_scan_candidate_files; + candidate_files.reserve( + candidate_files.size() + state.sst_delete_files.size() + + state.blob_delete_files.size() + state.log_delete_files.size() + + state.manifest_delete_files.size()); + // We may ignore the dbname when generating the file names. + for (auto& file : state.sst_delete_files) { + if (!file.only_delete_metadata) { + candidate_files.emplace_back( + MakeTableFileName(file.metadata->fd.GetNumber()), file.path); + } + if (file.metadata->table_reader_handle) { + table_cache_->Release(file.metadata->table_reader_handle); + } + file.DeleteMetadata(); + } + + for (const auto& blob_file : state.blob_delete_files) { + candidate_files.emplace_back(BlobFileName(blob_file.GetBlobFileNumber()), + blob_file.GetPath()); + } + + auto wal_dir = immutable_db_options_.GetWalDir(); + for (auto file_num : state.log_delete_files) { + if (file_num > 0) { + candidate_files.emplace_back(LogFileName(file_num), wal_dir); + } + } + for (const auto& filename : state.manifest_delete_files) { + candidate_files.emplace_back(filename, dbname_); + } + + // dedup state.candidate_files so we don't try to delete the same + // file twice + std::sort(candidate_files.begin(), candidate_files.end(), + [](const JobContext::CandidateFileInfo& lhs, + const JobContext::CandidateFileInfo& rhs) { + if (lhs.file_name > rhs.file_name) { + return true; + } else if (lhs.file_name < rhs.file_name) { + return false; + } else { + return (lhs.file_path > rhs.file_path); + } + }); + candidate_files.erase( + std::unique(candidate_files.begin(), candidate_files.end()), + candidate_files.end()); + + if (state.prev_total_log_size > 0) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[JOB %d] Try to delete WAL files size %" PRIu64 + ", prev total WAL file size %" PRIu64 + ", number of live WAL files %" ROCKSDB_PRIszt ".\n", + state.job_id, state.size_log_to_delete, + state.prev_total_log_size, state.num_alive_log_files); + } + + std::vector old_info_log_files; + InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(), + dbname_); + + // File numbers of most recent two OPTIONS file in candidate_files (found in + // previos FindObsoleteFiles(full_scan=true)) + // At this point, there must not be any duplicate file numbers in + // candidate_files. + uint64_t optsfile_num1 = std::numeric_limits::min(); + uint64_t optsfile_num2 = std::numeric_limits::min(); + for (const auto& candidate_file : candidate_files) { + const std::string& fname = candidate_file.file_name; + uint64_t number; + FileType type; + if (!ParseFileName(fname, &number, info_log_prefix.prefix, &type) || + type != kOptionsFile) { + continue; + } + if (number > optsfile_num1) { + optsfile_num2 = optsfile_num1; + optsfile_num1 = number; + } else if (number > optsfile_num2) { + optsfile_num2 = number; + } + } + + // Close WALs before trying to delete them. + for (const auto w : state.logs_to_free) { + // TODO: maybe check the return value of Close. + auto s = w->Close(); + s.PermitUncheckedError(); + } + + bool own_files = OwnTablesAndLogs(); + std::unordered_set files_to_del; + for (const auto& candidate_file : candidate_files) { + const std::string& to_delete = candidate_file.file_name; + uint64_t number; + FileType type; + // Ignore file if we cannot recognize it. + if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) { + continue; + } + + bool keep = true; + switch (type) { + case kWalFile: + keep = ((number >= state.log_number) || + (number == state.prev_log_number) || + (log_recycle_files_set.find(number) != + log_recycle_files_set.end())); + break; + case kDescriptorFile: + // Keep my manifest file, and any newer incarnations' + // (can happen during manifest roll) + keep = (number >= state.manifest_file_number); + break; + case kTableFile: + // If the second condition is not there, this makes + // DontDeletePendingOutputs fail + keep = (sst_live_set.find(number) != sst_live_set.end()) || + number >= state.min_pending_output; + if (!keep) { + files_to_del.insert(number); + } + break; + case kBlobFile: + keep = number >= state.min_pending_output || + (blob_live_set.find(number) != blob_live_set.end()); + if (!keep) { + files_to_del.insert(number); + } + break; + case kTempFile: + // Any temp files that are currently being written to must + // be recorded in pending_outputs_, which is inserted into "live". + // Also, SetCurrentFile creates a temp file when writing out new + // manifest, which is equal to state.pending_manifest_file_number. We + // should not delete that file + // + // TODO(yhchiang): carefully modify the third condition to safely + // remove the temp options files. + keep = (sst_live_set.find(number) != sst_live_set.end()) || + (blob_live_set.find(number) != blob_live_set.end()) || + (number == state.pending_manifest_file_number) || + (to_delete.find(kOptionsFileNamePrefix) != std::string::npos); + break; + case kInfoLogFile: + keep = true; + if (number != 0) { + old_info_log_files.push_back(to_delete); + } + break; + case kOptionsFile: + keep = (number >= optsfile_num2); + break; + case kCurrentFile: + case kDBLockFile: + case kIdentityFile: + case kMetaDatabase: + keep = true; + break; + } + + if (keep) { + continue; + } + + std::string fname; + std::string dir_to_sync; + if (type == kTableFile) { + // evict from cache + TableCache::Evict(table_cache_.get(), number); + fname = MakeTableFileName(candidate_file.file_path, number); + dir_to_sync = candidate_file.file_path; + } else if (type == kBlobFile) { + fname = BlobFileName(candidate_file.file_path, number); + dir_to_sync = candidate_file.file_path; + } else { + dir_to_sync = (type == kWalFile) ? wal_dir : dbname_; + fname = dir_to_sync + + ((!dir_to_sync.empty() && dir_to_sync.back() == '/') || + (!to_delete.empty() && to_delete.front() == '/') + ? "" + : "/") + + to_delete; + } + +#ifndef ROCKSDB_LITE + if (type == kWalFile && (immutable_db_options_.WAL_ttl_seconds > 0 || + immutable_db_options_.WAL_size_limit_MB > 0)) { + wal_manager_.ArchiveWALFile(fname, number); + continue; + } +#endif // !ROCKSDB_LITE + + // If I do not own these files, e.g. secondary instance with max_open_files + // = -1, then no need to delete or schedule delete these files since they + // will be removed by their owner, e.g. the primary instance. + if (!own_files) { + continue; + } + if (schedule_only) { + InstrumentedMutexLock guard_lock(&mutex_); + SchedulePendingPurge(fname, dir_to_sync, type, number, state.job_id); + } else { + DeleteObsoleteFileImpl(state.job_id, fname, dir_to_sync, type, number); + } + } + + { + // After purging obsolete files, remove them from files_grabbed_for_purge_. + InstrumentedMutexLock guard_lock(&mutex_); + autovector to_be_removed; + for (auto fn : files_grabbed_for_purge_) { + if (files_to_del.count(fn) != 0) { + to_be_removed.emplace_back(fn); + } + } + for (auto fn : to_be_removed) { + files_grabbed_for_purge_.erase(fn); + } + } + + // Delete old info log files. + size_t old_info_log_file_count = old_info_log_files.size(); + if (old_info_log_file_count != 0 && + old_info_log_file_count >= immutable_db_options_.keep_log_file_num) { + std::sort(old_info_log_files.begin(), old_info_log_files.end()); + size_t end = + old_info_log_file_count - immutable_db_options_.keep_log_file_num; + for (unsigned int i = 0; i <= end; i++) { + std::string& to_delete = old_info_log_files.at(i); + std::string full_path_to_delete = + (immutable_db_options_.db_log_dir.empty() + ? dbname_ + : immutable_db_options_.db_log_dir) + + "/" + to_delete; + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[JOB %d] Delete info log file %s\n", state.job_id, + full_path_to_delete.c_str()); + Status s = env_->DeleteFile(full_path_to_delete); + if (!s.ok()) { + if (env_->FileExists(full_path_to_delete).IsNotFound()) { + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "[JOB %d] Tried to delete non-existing info log file %s FAILED " + "-- %s\n", + state.job_id, to_delete.c_str(), s.ToString().c_str()); + } else { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "[JOB %d] Delete info log file %s FAILED -- %s\n", + state.job_id, to_delete.c_str(), + s.ToString().c_str()); + } + } + } + } +#ifndef ROCKSDB_LITE + wal_manager_.PurgeObsoleteWALFiles(); +#endif // ROCKSDB_LITE + LogFlush(immutable_db_options_.info_log); + InstrumentedMutexLock l(&mutex_); + --pending_purge_obsolete_files_; + assert(pending_purge_obsolete_files_ >= 0); + if (schedule_only) { + // Must change from pending_purge_obsolete_files_ to bg_purge_scheduled_ + // while holding mutex (for GetSortedWalFiles() etc.) + SchedulePurge(); + } + if (pending_purge_obsolete_files_ == 0) { + bg_cv_.SignalAll(); + } + TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:End"); +} + +void DBImpl::DeleteObsoleteFiles() { + mutex_.AssertHeld(); + JobContext job_context(next_job_id_.fetch_add(1)); + FindObsoleteFiles(&job_context, true); + + mutex_.Unlock(); + if (job_context.HaveSomethingToDelete()) { + bool defer_purge = immutable_db_options_.avoid_unnecessary_blocking_io; + PurgeObsoleteFiles(job_context, defer_purge); + } + job_context.Clean(); + mutex_.Lock(); +} + +uint64_t FindMinPrepLogReferencedByMemTable( + VersionSet* vset, const autovector& memtables_to_flush) { + uint64_t min_log = 0; + + // we must look through the memtables for two phase transactions + // that have been committed but not yet flushed + std::unordered_set memtables_to_flush_set( + memtables_to_flush.begin(), memtables_to_flush.end()); + for (auto loop_cfd : *vset->GetColumnFamilySet()) { + if (loop_cfd->IsDropped()) { + continue; + } + + auto log = loop_cfd->imm()->PrecomputeMinLogContainingPrepSection( + &memtables_to_flush_set); + + if (log > 0 && (min_log == 0 || log < min_log)) { + min_log = log; + } + + log = loop_cfd->mem()->GetMinLogContainingPrepSection(); + + if (log > 0 && (min_log == 0 || log < min_log)) { + min_log = log; + } + } + + return min_log; +} + +uint64_t FindMinPrepLogReferencedByMemTable( + VersionSet* vset, + const autovector*>& memtables_to_flush) { + uint64_t min_log = 0; + + std::unordered_set memtables_to_flush_set; + for (const autovector* memtables : memtables_to_flush) { + memtables_to_flush_set.insert(memtables->begin(), memtables->end()); + } + for (auto loop_cfd : *vset->GetColumnFamilySet()) { + if (loop_cfd->IsDropped()) { + continue; + } + + auto log = loop_cfd->imm()->PrecomputeMinLogContainingPrepSection( + &memtables_to_flush_set); + if (log > 0 && (min_log == 0 || log < min_log)) { + min_log = log; + } + + log = loop_cfd->mem()->GetMinLogContainingPrepSection(); + if (log > 0 && (min_log == 0 || log < min_log)) { + min_log = log; + } + } + + return min_log; +} + +uint64_t PrecomputeMinLogNumberToKeepNon2PC( + VersionSet* vset, const ColumnFamilyData& cfd_to_flush, + const autovector& edit_list) { + assert(vset != nullptr); + + // Precompute the min log number containing unflushed data for the column + // family being flushed (`cfd_to_flush`). + uint64_t cf_min_log_number_to_keep = 0; + for (auto& e : edit_list) { + if (e->HasLogNumber()) { + cf_min_log_number_to_keep = + std::max(cf_min_log_number_to_keep, e->GetLogNumber()); + } + } + if (cf_min_log_number_to_keep == 0) { + // No version edit contains information on log number. The log number + // for this column family should stay the same as it is. + cf_min_log_number_to_keep = cfd_to_flush.GetLogNumber(); + } + + // Get min log number containing unflushed data for other column families. + uint64_t min_log_number_to_keep = + vset->PreComputeMinLogNumberWithUnflushedData(&cfd_to_flush); + if (cf_min_log_number_to_keep != 0) { + min_log_number_to_keep = + std::min(cf_min_log_number_to_keep, min_log_number_to_keep); + } + return min_log_number_to_keep; +} + +uint64_t PrecomputeMinLogNumberToKeepNon2PC( + VersionSet* vset, const autovector& cfds_to_flush, + const autovector>& edit_lists) { + assert(vset != nullptr); + assert(!cfds_to_flush.empty()); + assert(cfds_to_flush.size() == edit_lists.size()); + + uint64_t min_log_number_to_keep = std::numeric_limits::max(); + for (const auto& edit_list : edit_lists) { + uint64_t log = 0; + for (const auto& e : edit_list) { + if (e->HasLogNumber()) { + log = std::max(log, e->GetLogNumber()); + } + } + if (log != 0) { + min_log_number_to_keep = std::min(min_log_number_to_keep, log); + } + } + if (min_log_number_to_keep == std::numeric_limits::max()) { + min_log_number_to_keep = cfds_to_flush[0]->GetLogNumber(); + for (size_t i = 1; i < cfds_to_flush.size(); i++) { + min_log_number_to_keep = + std::min(min_log_number_to_keep, cfds_to_flush[i]->GetLogNumber()); + } + } + + std::unordered_set flushed_cfds( + cfds_to_flush.begin(), cfds_to_flush.end()); + min_log_number_to_keep = + std::min(min_log_number_to_keep, + vset->PreComputeMinLogNumberWithUnflushedData(flushed_cfds)); + + return min_log_number_to_keep; +} + +uint64_t PrecomputeMinLogNumberToKeep2PC( + VersionSet* vset, const ColumnFamilyData& cfd_to_flush, + const autovector& edit_list, + const autovector& memtables_to_flush, + LogsWithPrepTracker* prep_tracker) { + assert(vset != nullptr); + assert(prep_tracker != nullptr); + // Calculate updated min_log_number_to_keep + // Since the function should only be called in 2pc mode, log number in + // the version edit should be sufficient. + + uint64_t min_log_number_to_keep = + PrecomputeMinLogNumberToKeepNon2PC(vset, cfd_to_flush, edit_list); + + // if are 2pc we must consider logs containing prepared + // sections of outstanding transactions. + // + // We must check min logs with outstanding prep before we check + // logs references by memtables because a log referenced by the + // first data structure could transition to the second under us. + // + // TODO: iterating over all column families under db mutex. + // should find more optimal solution + auto min_log_in_prep_heap = + prep_tracker->FindMinLogContainingOutstandingPrep(); + + if (min_log_in_prep_heap != 0 && + min_log_in_prep_heap < min_log_number_to_keep) { + min_log_number_to_keep = min_log_in_prep_heap; + } + + uint64_t min_log_refed_by_mem = + FindMinPrepLogReferencedByMemTable(vset, memtables_to_flush); + + if (min_log_refed_by_mem != 0 && + min_log_refed_by_mem < min_log_number_to_keep) { + min_log_number_to_keep = min_log_refed_by_mem; + } + return min_log_number_to_keep; +} + +uint64_t PrecomputeMinLogNumberToKeep2PC( + VersionSet* vset, const autovector& cfds_to_flush, + const autovector>& edit_lists, + const autovector*>& memtables_to_flush, + LogsWithPrepTracker* prep_tracker) { + assert(vset != nullptr); + assert(prep_tracker != nullptr); + assert(cfds_to_flush.size() == edit_lists.size()); + assert(cfds_to_flush.size() == memtables_to_flush.size()); + + uint64_t min_log_number_to_keep = + PrecomputeMinLogNumberToKeepNon2PC(vset, cfds_to_flush, edit_lists); + + uint64_t min_log_in_prep_heap = + prep_tracker->FindMinLogContainingOutstandingPrep(); + + if (min_log_in_prep_heap != 0 && + min_log_in_prep_heap < min_log_number_to_keep) { + min_log_number_to_keep = min_log_in_prep_heap; + } + + uint64_t min_log_refed_by_mem = + FindMinPrepLogReferencedByMemTable(vset, memtables_to_flush); + + if (min_log_refed_by_mem != 0 && + min_log_refed_by_mem < min_log_number_to_keep) { + min_log_number_to_keep = min_log_refed_by_mem; + } + + return min_log_number_to_keep; +} + +void DBImpl::SetDBId(std::string&& id, bool read_only, + RecoveryContext* recovery_ctx) { + assert(db_id_.empty()); + assert(!id.empty()); + db_id_ = std::move(id); + if (!read_only && immutable_db_options_.write_dbid_to_manifest) { + assert(recovery_ctx != nullptr); + assert(versions_->GetColumnFamilySet() != nullptr); + VersionEdit edit; + edit.SetDBId(db_id_); + versions_->db_id_ = db_id_; + recovery_ctx->UpdateVersionEdits( + versions_->GetColumnFamilySet()->GetDefault(), edit); + } +} + +Status DBImpl::SetupDBId(bool read_only, RecoveryContext* recovery_ctx) { + Status s; + // Check for the IDENTITY file and create it if not there or + // broken or not matching manifest + std::string db_id_in_file; + s = fs_->FileExists(IdentityFileName(dbname_), IOOptions(), nullptr); + if (s.ok()) { + s = GetDbIdentityFromIdentityFile(&db_id_in_file); + if (s.ok() && !db_id_in_file.empty()) { + if (db_id_.empty()) { + // Loaded from file and wasn't already known from manifest + SetDBId(std::move(db_id_in_file), read_only, recovery_ctx); + return s; + } else if (db_id_ == db_id_in_file) { + // Loaded from file and matches manifest + return s; + } + } + } + if (s.IsNotFound()) { + s = Status::OK(); + } + if (!s.ok()) { + assert(s.IsIOError()); + return s; + } + // Otherwise IDENTITY file is missing or no good. + // Generate new id if needed + if (db_id_.empty()) { + SetDBId(env_->GenerateUniqueId(), read_only, recovery_ctx); + } + // Persist it to IDENTITY file if allowed + if (!read_only) { + s = SetIdentityFile(env_, dbname_, db_id_); + } + return s; +} + +Status DBImpl::DeleteUnreferencedSstFiles(RecoveryContext* recovery_ctx) { + mutex_.AssertHeld(); + std::vector paths; + paths.push_back(NormalizePath(dbname_ + std::string(1, kFilePathSeparator))); + for (const auto& db_path : immutable_db_options_.db_paths) { + paths.push_back( + NormalizePath(db_path.path + std::string(1, kFilePathSeparator))); + } + for (const auto* cfd : *versions_->GetColumnFamilySet()) { + for (const auto& cf_path : cfd->ioptions()->cf_paths) { + paths.push_back( + NormalizePath(cf_path.path + std::string(1, kFilePathSeparator))); + } + } + // Dedup paths + std::sort(paths.begin(), paths.end()); + paths.erase(std::unique(paths.begin(), paths.end()), paths.end()); + + uint64_t next_file_number = versions_->current_next_file_number(); + uint64_t largest_file_number = next_file_number; + Status s; + for (const auto& path : paths) { + std::vector files; + s = env_->GetChildren(path, &files); + if (!s.ok()) { + break; + } + for (const auto& fname : files) { + uint64_t number = 0; + FileType type; + if (!ParseFileName(fname, &number, &type)) { + continue; + } + // path ends with '/' or '\\' + const std::string normalized_fpath = path + fname; + largest_file_number = std::max(largest_file_number, number); + if (type == kTableFile && number >= next_file_number && + recovery_ctx->files_to_delete_.find(normalized_fpath) == + recovery_ctx->files_to_delete_.end()) { + recovery_ctx->files_to_delete_.emplace(normalized_fpath); + } + } + } + if (!s.ok()) { + return s; + } + + if (largest_file_number >= next_file_number) { + versions_->next_file_number_.store(largest_file_number + 1); + } + + VersionEdit edit; + edit.SetNextFile(versions_->next_file_number_.load()); + assert(versions_->GetColumnFamilySet()); + ColumnFamilyData* default_cfd = versions_->GetColumnFamilySet()->GetDefault(); + assert(default_cfd); + recovery_ctx->UpdateVersionEdits(default_cfd, edit); + return s; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl_open.cc b/src/rocksdb/db/db_impl/db_impl_open.cc new file mode 100644 index 000000000..40ffa2e85 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_open.cc @@ -0,0 +1,2106 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +#include + +#include "db/builder.h" +#include "db/db_impl/db_impl.h" +#include "db/error_handler.h" +#include "db/periodic_task_scheduler.h" +#include "env/composite_env_wrapper.h" +#include "file/filename.h" +#include "file/read_write_util.h" +#include "file/sst_file_manager_impl.h" +#include "file/writable_file_writer.h" +#include "logging/logging.h" +#include "monitoring/persistent_stats_history.h" +#include "options/options_helper.h" +#include "rocksdb/table.h" +#include "rocksdb/wal_filter.h" +#include "test_util/sync_point.h" +#include "util/rate_limiter.h" + +namespace ROCKSDB_NAMESPACE { +Options SanitizeOptions(const std::string& dbname, const Options& src, + bool read_only, Status* logger_creation_s) { + auto db_options = + SanitizeOptions(dbname, DBOptions(src), read_only, logger_creation_s); + ImmutableDBOptions immutable_db_options(db_options); + auto cf_options = + SanitizeOptions(immutable_db_options, ColumnFamilyOptions(src)); + return Options(db_options, cf_options); +} + +DBOptions SanitizeOptions(const std::string& dbname, const DBOptions& src, + bool read_only, Status* logger_creation_s) { + DBOptions result(src); + + if (result.env == nullptr) { + result.env = Env::Default(); + } + + // result.max_open_files means an "infinite" open files. + if (result.max_open_files != -1) { + int max_max_open_files = port::GetMaxOpenFiles(); + if (max_max_open_files == -1) { + max_max_open_files = 0x400000; + } + ClipToRange(&result.max_open_files, 20, max_max_open_files); + TEST_SYNC_POINT_CALLBACK("SanitizeOptions::AfterChangeMaxOpenFiles", + &result.max_open_files); + } + + if (result.info_log == nullptr && !read_only) { + Status s = CreateLoggerFromOptions(dbname, result, &result.info_log); + if (!s.ok()) { + // No place suitable for logging + result.info_log = nullptr; + if (logger_creation_s) { + *logger_creation_s = s; + } + } + } + + if (!result.write_buffer_manager) { + result.write_buffer_manager.reset( + new WriteBufferManager(result.db_write_buffer_size)); + } + auto bg_job_limits = DBImpl::GetBGJobLimits( + result.max_background_flushes, result.max_background_compactions, + result.max_background_jobs, true /* parallelize_compactions */); + result.env->IncBackgroundThreadsIfNeeded(bg_job_limits.max_compactions, + Env::Priority::LOW); + result.env->IncBackgroundThreadsIfNeeded(bg_job_limits.max_flushes, + Env::Priority::HIGH); + + if (result.rate_limiter.get() != nullptr) { + if (result.bytes_per_sync == 0) { + result.bytes_per_sync = 1024 * 1024; + } + } + + if (result.delayed_write_rate == 0) { + if (result.rate_limiter.get() != nullptr) { + result.delayed_write_rate = result.rate_limiter->GetBytesPerSecond(); + } + if (result.delayed_write_rate == 0) { + result.delayed_write_rate = 16 * 1024 * 1024; + } + } + + if (result.WAL_ttl_seconds > 0 || result.WAL_size_limit_MB > 0) { + result.recycle_log_file_num = false; + } + + if (result.recycle_log_file_num && + (result.wal_recovery_mode == + WALRecoveryMode::kTolerateCorruptedTailRecords || + result.wal_recovery_mode == WALRecoveryMode::kPointInTimeRecovery || + result.wal_recovery_mode == WALRecoveryMode::kAbsoluteConsistency)) { + // - kTolerateCorruptedTailRecords is inconsistent with recycle log file + // feature. WAL recycling expects recovery success upon encountering a + // corrupt record at the point where new data ends and recycled data + // remains at the tail. However, `kTolerateCorruptedTailRecords` must fail + // upon encountering any such corrupt record, as it cannot differentiate + // between this and a real corruption, which would cause committed updates + // to be truncated -- a violation of the recovery guarantee. + // - kPointInTimeRecovery and kAbsoluteConsistency are incompatible with + // recycle log file feature temporarily due to a bug found introducing a + // hole in the recovered data + // (https://github.com/facebook/rocksdb/pull/7252#issuecomment-673766236). + // Besides this bug, we believe the features are fundamentally compatible. + result.recycle_log_file_num = 0; + } + + if (result.db_paths.size() == 0) { + result.db_paths.emplace_back(dbname, std::numeric_limits::max()); + } else if (result.wal_dir.empty()) { + // Use dbname as default + result.wal_dir = dbname; + } + if (!result.wal_dir.empty()) { + // If there is a wal_dir already set, check to see if the wal_dir is the + // same as the dbname AND the same as the db_path[0] (which must exist from + // a few lines ago). If the wal_dir matches both of these values, then clear + // the wal_dir value, which will make wal_dir == dbname. Most likely this + // condition was the result of reading an old options file where we forced + // wal_dir to be set (to dbname). + auto npath = NormalizePath(dbname + "/"); + if (npath == NormalizePath(result.wal_dir + "/") && + npath == NormalizePath(result.db_paths[0].path + "/")) { + result.wal_dir.clear(); + } + } + + if (!result.wal_dir.empty() && result.wal_dir.back() == '/') { + result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1); + } + + if (result.use_direct_reads && result.compaction_readahead_size == 0) { + TEST_SYNC_POINT_CALLBACK("SanitizeOptions:direct_io", nullptr); + result.compaction_readahead_size = 1024 * 1024 * 2; + } + + // Force flush on DB open if 2PC is enabled, since with 2PC we have no + // guarantee that consecutive log files have consecutive sequence id, which + // make recovery complicated. + if (result.allow_2pc) { + result.avoid_flush_during_recovery = false; + } + +#ifndef ROCKSDB_LITE + ImmutableDBOptions immutable_db_options(result); + if (!immutable_db_options.IsWalDirSameAsDBPath()) { + // Either the WAL dir and db_paths[0]/db_name are not the same, or we + // cannot tell for sure. In either case, assume they're different and + // explicitly cleanup the trash log files (bypass DeleteScheduler) + // Do this first so even if we end up calling + // DeleteScheduler::CleanupDirectory on the same dir later, it will be + // safe + std::vector filenames; + IOOptions io_opts; + io_opts.do_not_recurse = true; + auto wal_dir = immutable_db_options.GetWalDir(); + Status s = immutable_db_options.fs->GetChildren( + wal_dir, io_opts, &filenames, /*IODebugContext*=*/nullptr); + s.PermitUncheckedError(); //**TODO: What to do on error? + for (std::string& filename : filenames) { + if (filename.find(".log.trash", filename.length() - + std::string(".log.trash").length()) != + std::string::npos) { + std::string trash_file = wal_dir + "/" + filename; + result.env->DeleteFile(trash_file).PermitUncheckedError(); + } + } + } + // When the DB is stopped, it's possible that there are some .trash files that + // were not deleted yet, when we open the DB we will find these .trash files + // and schedule them to be deleted (or delete immediately if SstFileManager + // was not used) + auto sfm = static_cast(result.sst_file_manager.get()); + for (size_t i = 0; i < result.db_paths.size(); i++) { + DeleteScheduler::CleanupDirectory(result.env, sfm, result.db_paths[i].path) + .PermitUncheckedError(); + } + + // Create a default SstFileManager for purposes of tracking compaction size + // and facilitating recovery from out of space errors. + if (result.sst_file_manager.get() == nullptr) { + std::shared_ptr sst_file_manager( + NewSstFileManager(result.env, result.info_log)); + result.sst_file_manager = sst_file_manager; + } +#endif // !ROCKSDB_LITE + + // Supported wal compression types + if (!StreamingCompressionTypeSupported(result.wal_compression)) { + result.wal_compression = kNoCompression; + ROCKS_LOG_WARN(result.info_log, + "wal_compression is disabled since only zstd is supported"); + } + + if (!result.paranoid_checks) { + result.skip_checking_sst_file_sizes_on_db_open = true; + ROCKS_LOG_INFO(result.info_log, + "file size check will be skipped during open."); + } + + return result; +} + +namespace { +Status ValidateOptionsByTable( + const DBOptions& db_opts, + const std::vector& column_families) { + Status s; + for (auto cf : column_families) { + s = ValidateOptions(db_opts, cf.options); + if (!s.ok()) { + return s; + } + } + return Status::OK(); +} +} // namespace + +Status DBImpl::ValidateOptions( + const DBOptions& db_options, + const std::vector& column_families) { + Status s; + for (auto& cfd : column_families) { + s = ColumnFamilyData::ValidateOptions(db_options, cfd.options); + if (!s.ok()) { + return s; + } + } + s = ValidateOptions(db_options); + return s; +} + +Status DBImpl::ValidateOptions(const DBOptions& db_options) { + if (db_options.db_paths.size() > 4) { + return Status::NotSupported( + "More than four DB paths are not supported yet. "); + } + + if (db_options.allow_mmap_reads && db_options.use_direct_reads) { + // Protect against assert in PosixMMapReadableFile constructor + return Status::NotSupported( + "If memory mapped reads (allow_mmap_reads) are enabled " + "then direct I/O reads (use_direct_reads) must be disabled. "); + } + + if (db_options.allow_mmap_writes && + db_options.use_direct_io_for_flush_and_compaction) { + return Status::NotSupported( + "If memory mapped writes (allow_mmap_writes) are enabled " + "then direct I/O writes (use_direct_io_for_flush_and_compaction) must " + "be disabled. "); + } + + if (db_options.keep_log_file_num == 0) { + return Status::InvalidArgument("keep_log_file_num must be greater than 0"); + } + + if (db_options.unordered_write && + !db_options.allow_concurrent_memtable_write) { + return Status::InvalidArgument( + "unordered_write is incompatible with " + "!allow_concurrent_memtable_write"); + } + + if (db_options.unordered_write && db_options.enable_pipelined_write) { + return Status::InvalidArgument( + "unordered_write is incompatible with enable_pipelined_write"); + } + + if (db_options.atomic_flush && db_options.enable_pipelined_write) { + return Status::InvalidArgument( + "atomic_flush is incompatible with enable_pipelined_write"); + } + + // TODO remove this restriction + if (db_options.atomic_flush && db_options.best_efforts_recovery) { + return Status::InvalidArgument( + "atomic_flush is currently incompatible with best-efforts recovery"); + } + + if (db_options.use_direct_io_for_flush_and_compaction && + 0 == db_options.writable_file_max_buffer_size) { + return Status::InvalidArgument( + "writes in direct IO require writable_file_max_buffer_size > 0"); + } + + return Status::OK(); +} + +Status DBImpl::NewDB(std::vector* new_filenames) { + VersionEdit new_db; + Status s = SetIdentityFile(env_, dbname_); + if (!s.ok()) { + return s; + } + if (immutable_db_options_.write_dbid_to_manifest) { + std::string temp_db_id; + GetDbIdentityFromIdentityFile(&temp_db_id); + new_db.SetDBId(temp_db_id); + } + new_db.SetLogNumber(0); + new_db.SetNextFile(2); + new_db.SetLastSequence(0); + + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Creating manifest 1 \n"); + const std::string manifest = DescriptorFileName(dbname_, 1); + { + if (fs_->FileExists(manifest, IOOptions(), nullptr).ok()) { + fs_->DeleteFile(manifest, IOOptions(), nullptr).PermitUncheckedError(); + } + std::unique_ptr file; + FileOptions file_options = fs_->OptimizeForManifestWrite(file_options_); + s = NewWritableFile(fs_.get(), manifest, &file, file_options); + if (!s.ok()) { + return s; + } + FileTypeSet tmp_set = immutable_db_options_.checksum_handoff_file_types; + file->SetPreallocationBlockSize( + immutable_db_options_.manifest_preallocation_size); + std::unique_ptr file_writer(new WritableFileWriter( + std::move(file), manifest, file_options, immutable_db_options_.clock, + io_tracer_, nullptr /* stats */, immutable_db_options_.listeners, + nullptr, tmp_set.Contains(FileType::kDescriptorFile), + tmp_set.Contains(FileType::kDescriptorFile))); + log::Writer log(std::move(file_writer), 0, false); + std::string record; + new_db.EncodeTo(&record); + s = log.AddRecord(record); + if (s.ok()) { + s = SyncManifest(&immutable_db_options_, log.file()); + } + } + if (s.ok()) { + // Make "CURRENT" file that points to the new manifest file. + s = SetCurrentFile(fs_.get(), dbname_, 1, directories_.GetDbDir()); + if (new_filenames) { + new_filenames->emplace_back( + manifest.substr(manifest.find_last_of("/\\") + 1)); + } + } else { + fs_->DeleteFile(manifest, IOOptions(), nullptr).PermitUncheckedError(); + } + return s; +} + +IOStatus DBImpl::CreateAndNewDirectory( + FileSystem* fs, const std::string& dirname, + std::unique_ptr* directory) { + // We call CreateDirIfMissing() as the directory may already exist (if we + // are reopening a DB), when this happens we don't want creating the + // directory to cause an error. However, we need to check if creating the + // directory fails or else we may get an obscure message about the lock + // file not existing. One real-world example of this occurring is if + // env->CreateDirIfMissing() doesn't create intermediate directories, e.g. + // when dbname_ is "dir/db" but when "dir" doesn't exist. + IOStatus io_s = fs->CreateDirIfMissing(dirname, IOOptions(), nullptr); + if (!io_s.ok()) { + return io_s; + } + return fs->NewDirectory(dirname, IOOptions(), directory, nullptr); +} + +IOStatus Directories::SetDirectories(FileSystem* fs, const std::string& dbname, + const std::string& wal_dir, + const std::vector& data_paths) { + IOStatus io_s = DBImpl::CreateAndNewDirectory(fs, dbname, &db_dir_); + if (!io_s.ok()) { + return io_s; + } + if (!wal_dir.empty() && dbname != wal_dir) { + io_s = DBImpl::CreateAndNewDirectory(fs, wal_dir, &wal_dir_); + if (!io_s.ok()) { + return io_s; + } + } + + data_dirs_.clear(); + for (auto& p : data_paths) { + const std::string db_path = p.path; + if (db_path == dbname) { + data_dirs_.emplace_back(nullptr); + } else { + std::unique_ptr path_directory; + io_s = DBImpl::CreateAndNewDirectory(fs, db_path, &path_directory); + if (!io_s.ok()) { + return io_s; + } + data_dirs_.emplace_back(path_directory.release()); + } + } + assert(data_dirs_.size() == data_paths.size()); + return IOStatus::OK(); +} + +Status DBImpl::Recover( + const std::vector& column_families, bool read_only, + bool error_if_wal_file_exists, bool error_if_data_exists_in_wals, + uint64_t* recovered_seq, RecoveryContext* recovery_ctx) { + mutex_.AssertHeld(); + + bool is_new_db = false; + assert(db_lock_ == nullptr); + std::vector files_in_dbname; + if (!read_only) { + Status s = directories_.SetDirectories(fs_.get(), dbname_, + immutable_db_options_.wal_dir, + immutable_db_options_.db_paths); + if (!s.ok()) { + return s; + } + + s = env_->LockFile(LockFileName(dbname_), &db_lock_); + if (!s.ok()) { + return s; + } + + std::string current_fname = CurrentFileName(dbname_); + // Path to any MANIFEST file in the db dir. It does not matter which one. + // Since best-efforts recovery ignores CURRENT file, existence of a + // MANIFEST indicates the recovery to recover existing db. If no MANIFEST + // can be found, a new db will be created. + std::string manifest_path; + if (!immutable_db_options_.best_efforts_recovery) { + s = env_->FileExists(current_fname); + } else { + s = Status::NotFound(); + IOOptions io_opts; + io_opts.do_not_recurse = true; + Status io_s = immutable_db_options_.fs->GetChildren( + dbname_, io_opts, &files_in_dbname, /*IODebugContext*=*/nullptr); + if (!io_s.ok()) { + s = io_s; + files_in_dbname.clear(); + } + for (const std::string& file : files_in_dbname) { + uint64_t number = 0; + FileType type = kWalFile; // initialize + if (ParseFileName(file, &number, &type) && type == kDescriptorFile) { + uint64_t bytes; + s = env_->GetFileSize(DescriptorFileName(dbname_, number), &bytes); + if (s.ok() && bytes != 0) { + // Found non-empty MANIFEST (descriptor log), thus best-efforts + // recovery does not have to treat the db as empty. + manifest_path = dbname_ + "/" + file; + break; + } + } + } + } + if (s.IsNotFound()) { + if (immutable_db_options_.create_if_missing) { + s = NewDB(&files_in_dbname); + is_new_db = true; + if (!s.ok()) { + return s; + } + } else { + return Status::InvalidArgument( + current_fname, "does not exist (create_if_missing is false)"); + } + } else if (s.ok()) { + if (immutable_db_options_.error_if_exists) { + return Status::InvalidArgument(dbname_, + "exists (error_if_exists is true)"); + } + } else { + // Unexpected error reading file + assert(s.IsIOError()); + return s; + } + // Verify compatibility of file_options_ and filesystem + { + std::unique_ptr idfile; + FileOptions customized_fs(file_options_); + customized_fs.use_direct_reads |= + immutable_db_options_.use_direct_io_for_flush_and_compaction; + const std::string& fname = + manifest_path.empty() ? current_fname : manifest_path; + s = fs_->NewRandomAccessFile(fname, customized_fs, &idfile, nullptr); + if (!s.ok()) { + std::string error_str = s.ToString(); + // Check if unsupported Direct I/O is the root cause + customized_fs.use_direct_reads = false; + s = fs_->NewRandomAccessFile(fname, customized_fs, &idfile, nullptr); + if (s.ok()) { + return Status::InvalidArgument( + "Direct I/O is not supported by the specified DB."); + } else { + return Status::InvalidArgument( + "Found options incompatible with filesystem", error_str.c_str()); + } + } + } + } else if (immutable_db_options_.best_efforts_recovery) { + assert(files_in_dbname.empty()); + IOOptions io_opts; + io_opts.do_not_recurse = true; + Status s = immutable_db_options_.fs->GetChildren( + dbname_, io_opts, &files_in_dbname, /*IODebugContext*=*/nullptr); + if (s.IsNotFound()) { + return Status::InvalidArgument(dbname_, + "does not exist (open for read only)"); + } else if (s.IsIOError()) { + return s; + } + assert(s.ok()); + } + assert(db_id_.empty()); + Status s; + bool missing_table_file = false; + if (!immutable_db_options_.best_efforts_recovery) { + s = versions_->Recover(column_families, read_only, &db_id_); + } else { + assert(!files_in_dbname.empty()); + s = versions_->TryRecover(column_families, read_only, files_in_dbname, + &db_id_, &missing_table_file); + if (s.ok()) { + // TryRecover may delete previous column_family_set_. + column_family_memtables_.reset( + new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet())); + } + } + if (!s.ok()) { + return s; + } + s = SetupDBId(read_only, recovery_ctx); + ROCKS_LOG_INFO(immutable_db_options_.info_log, "DB ID: %s\n", db_id_.c_str()); + if (s.ok() && !read_only) { + s = DeleteUnreferencedSstFiles(recovery_ctx); + } + + if (immutable_db_options_.paranoid_checks && s.ok()) { + s = CheckConsistency(); + } + if (s.ok() && !read_only) { + // TODO: share file descriptors (FSDirectory) with SetDirectories above + std::map> created_dirs; + for (auto cfd : *versions_->GetColumnFamilySet()) { + s = cfd->AddDirectories(&created_dirs); + if (!s.ok()) { + return s; + } + } + } + + std::vector files_in_wal_dir; + if (s.ok()) { + // Initial max_total_in_memory_state_ before recovery wals. Log recovery + // 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; + } + + SequenceNumber next_sequence(kMaxSequenceNumber); + default_cf_handle_ = new ColumnFamilyHandleImpl( + versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_); + default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats(); + + // Recover from all newer log files than the ones named in the + // descriptor (new log files may have been added by the previous + // incarnation without registering them in the descriptor). + // + // Note that prev_log_number() is no longer used, but we pay + // attention to it in case we are recovering a database + // produced by an older version of rocksdb. + auto wal_dir = immutable_db_options_.GetWalDir(); + if (!immutable_db_options_.best_efforts_recovery) { + IOOptions io_opts; + io_opts.do_not_recurse = true; + s = immutable_db_options_.fs->GetChildren( + wal_dir, io_opts, &files_in_wal_dir, /*IODebugContext*=*/nullptr); + } + if (s.IsNotFound()) { + return Status::InvalidArgument("wal_dir not found", wal_dir); + } else if (!s.ok()) { + return s; + } + + std::unordered_map wal_files; + for (const auto& file : files_in_wal_dir) { + uint64_t number; + FileType type; + if (ParseFileName(file, &number, &type) && type == kWalFile) { + if (is_new_db) { + return Status::Corruption( + "While creating a new Db, wal_dir contains " + "existing log file: ", + file); + } else { + wal_files[number] = LogFileName(wal_dir, number); + } + } + } + + if (immutable_db_options_.track_and_verify_wals_in_manifest) { + if (!immutable_db_options_.best_efforts_recovery) { + // Verify WALs in MANIFEST. + s = versions_->GetWalSet().CheckWals(env_, wal_files); + } // else since best effort recovery does not recover from WALs, no need + // to check WALs. + } else if (!versions_->GetWalSet().GetWals().empty()) { + // Tracking is disabled, clear previously tracked WALs from MANIFEST, + // otherwise, in the future, if WAL tracking is enabled again, + // since the WALs deleted when WAL tracking is disabled are not persisted + // into MANIFEST, WAL check may fail. + VersionEdit edit; + WalNumber max_wal_number = + versions_->GetWalSet().GetWals().rbegin()->first; + edit.DeleteWalsBefore(max_wal_number + 1); + assert(recovery_ctx != nullptr); + assert(versions_->GetColumnFamilySet() != nullptr); + recovery_ctx->UpdateVersionEdits( + versions_->GetColumnFamilySet()->GetDefault(), edit); + } + if (!s.ok()) { + return s; + } + + if (!wal_files.empty()) { + if (error_if_wal_file_exists) { + return Status::Corruption( + "The db was opened in readonly mode with error_if_wal_file_exists" + "flag but a WAL file already exists"); + } else if (error_if_data_exists_in_wals) { + for (auto& wal_file : wal_files) { + uint64_t bytes; + s = env_->GetFileSize(wal_file.second, &bytes); + if (s.ok()) { + if (bytes > 0) { + return Status::Corruption( + "error_if_data_exists_in_wals is set but there are data " + " in WAL files."); + } + } + } + } + } + + if (!wal_files.empty()) { + // Recover in the order in which the wals were generated + std::vector wals; + wals.reserve(wal_files.size()); + for (const auto& wal_file : wal_files) { + wals.push_back(wal_file.first); + } + std::sort(wals.begin(), wals.end()); + + bool corrupted_wal_found = false; + s = RecoverLogFiles(wals, &next_sequence, read_only, &corrupted_wal_found, + recovery_ctx); + if (corrupted_wal_found && recovered_seq != nullptr) { + *recovered_seq = next_sequence; + } + if (!s.ok()) { + // Clear memtables if recovery failed + for (auto cfd : *versions_->GetColumnFamilySet()) { + cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(), + kMaxSequenceNumber); + } + } + } + } + + if (read_only) { + // If we are opening as read-only, we need to update options_file_number_ + // to reflect the most recent OPTIONS file. It does not matter for regular + // read-write db instance because options_file_number_ will later be + // updated to versions_->NewFileNumber() in RenameTempFileToOptionsFile. + std::vector filenames; + if (s.ok()) { + const std::string normalized_dbname = NormalizePath(dbname_); + const std::string normalized_wal_dir = + NormalizePath(immutable_db_options_.GetWalDir()); + if (immutable_db_options_.best_efforts_recovery) { + filenames = std::move(files_in_dbname); + } else if (normalized_dbname == normalized_wal_dir) { + filenames = std::move(files_in_wal_dir); + } else { + IOOptions io_opts; + io_opts.do_not_recurse = true; + s = immutable_db_options_.fs->GetChildren( + GetName(), io_opts, &filenames, /*IODebugContext*=*/nullptr); + } + } + if (s.ok()) { + uint64_t number = 0; + uint64_t options_file_number = 0; + FileType type; + for (const auto& fname : filenames) { + if (ParseFileName(fname, &number, &type) && type == kOptionsFile) { + options_file_number = std::max(number, options_file_number); + } + } + versions_->options_file_number_ = options_file_number; + uint64_t options_file_size = 0; + if (options_file_number > 0) { + s = env_->GetFileSize(OptionsFileName(GetName(), options_file_number), + &options_file_size); + } + versions_->options_file_size_ = options_file_size; + } + } + return s; +} + +Status DBImpl::PersistentStatsProcessFormatVersion() { + mutex_.AssertHeld(); + Status s; + // persist version when stats CF doesn't exist + bool should_persist_format_version = !persistent_stats_cfd_exists_; + mutex_.Unlock(); + if (persistent_stats_cfd_exists_) { + // Check persistent stats format version compatibility. Drop and recreate + // persistent stats CF if format version is incompatible + uint64_t format_version_recovered = 0; + Status s_format = DecodePersistentStatsVersionNumber( + this, StatsVersionKeyType::kFormatVersion, &format_version_recovered); + uint64_t compatible_version_recovered = 0; + Status s_compatible = DecodePersistentStatsVersionNumber( + this, StatsVersionKeyType::kCompatibleVersion, + &compatible_version_recovered); + // abort reading from existing stats CF if any of following is true: + // 1. failed to read format version or compatible version from disk + // 2. sst's format version is greater than current format version, meaning + // this sst is encoded with a newer RocksDB release, and current compatible + // version is below the sst's compatible version + if (!s_format.ok() || !s_compatible.ok() || + (kStatsCFCurrentFormatVersion < format_version_recovered && + kStatsCFCompatibleFormatVersion < compatible_version_recovered)) { + if (!s_format.ok() || !s_compatible.ok()) { + ROCKS_LOG_WARN( + immutable_db_options_.info_log, + "Recreating persistent stats column family since reading " + "persistent stats version key failed. Format key: %s, compatible " + "key: %s", + s_format.ToString().c_str(), s_compatible.ToString().c_str()); + } else { + ROCKS_LOG_WARN( + immutable_db_options_.info_log, + "Recreating persistent stats column family due to corrupted or " + "incompatible format version. Recovered format: %" PRIu64 + "; recovered format compatible since: %" PRIu64 "\n", + format_version_recovered, compatible_version_recovered); + } + s = DropColumnFamily(persist_stats_cf_handle_); + if (s.ok()) { + s = DestroyColumnFamilyHandle(persist_stats_cf_handle_); + } + ColumnFamilyHandle* handle = nullptr; + if (s.ok()) { + ColumnFamilyOptions cfo; + OptimizeForPersistentStats(&cfo); + s = CreateColumnFamily(cfo, kPersistentStatsColumnFamilyName, &handle); + } + if (s.ok()) { + persist_stats_cf_handle_ = static_cast(handle); + // should also persist version here because old stats CF is discarded + should_persist_format_version = true; + } + } + } + if (should_persist_format_version) { + // Persistent stats CF being created for the first time, need to write + // format version key + WriteBatch batch; + if (s.ok()) { + s = batch.Put(persist_stats_cf_handle_, kFormatVersionKeyString, + std::to_string(kStatsCFCurrentFormatVersion)); + } + if (s.ok()) { + s = batch.Put(persist_stats_cf_handle_, kCompatibleVersionKeyString, + std::to_string(kStatsCFCompatibleFormatVersion)); + } + if (s.ok()) { + WriteOptions wo; + wo.low_pri = true; + wo.no_slowdown = true; + wo.sync = false; + s = Write(wo, &batch); + } + } + mutex_.Lock(); + return s; +} + +Status DBImpl::InitPersistStatsColumnFamily() { + mutex_.AssertHeld(); + assert(!persist_stats_cf_handle_); + ColumnFamilyData* persistent_stats_cfd = + versions_->GetColumnFamilySet()->GetColumnFamily( + kPersistentStatsColumnFamilyName); + persistent_stats_cfd_exists_ = persistent_stats_cfd != nullptr; + + Status s; + if (persistent_stats_cfd != nullptr) { + // We are recovering from a DB which already contains persistent stats CF, + // the CF is already created in VersionSet::ApplyOneVersionEdit, but + // column family handle was not. Need to explicitly create handle here. + persist_stats_cf_handle_ = + new ColumnFamilyHandleImpl(persistent_stats_cfd, this, &mutex_); + } else { + mutex_.Unlock(); + ColumnFamilyHandle* handle = nullptr; + ColumnFamilyOptions cfo; + OptimizeForPersistentStats(&cfo); + s = CreateColumnFamily(cfo, kPersistentStatsColumnFamilyName, &handle); + persist_stats_cf_handle_ = static_cast(handle); + mutex_.Lock(); + } + return s; +} + +Status DBImpl::LogAndApplyForRecovery(const RecoveryContext& recovery_ctx) { + mutex_.AssertHeld(); + assert(versions_->descriptor_log_ == nullptr); + Status s = versions_->LogAndApply( + recovery_ctx.cfds_, recovery_ctx.mutable_cf_opts_, + recovery_ctx.edit_lists_, &mutex_, directories_.GetDbDir()); + if (s.ok() && !(recovery_ctx.files_to_delete_.empty())) { + mutex_.Unlock(); + for (const auto& fname : recovery_ctx.files_to_delete_) { + s = env_->DeleteFile(fname); + if (!s.ok()) { + break; + } + } + mutex_.Lock(); + } + return s; +} + +void DBImpl::InvokeWalFilterIfNeededOnColumnFamilyToWalNumberMap() { +#ifndef ROCKSDB_LITE + if (immutable_db_options_.wal_filter == nullptr) { + return; + } + assert(immutable_db_options_.wal_filter != nullptr); + WalFilter& wal_filter = *(immutable_db_options_.wal_filter); + + std::map cf_name_id_map; + std::map cf_lognumber_map; + assert(versions_); + assert(versions_->GetColumnFamilySet()); + for (auto cfd : *versions_->GetColumnFamilySet()) { + assert(cfd); + cf_name_id_map.insert(std::make_pair(cfd->GetName(), cfd->GetID())); + cf_lognumber_map.insert(std::make_pair(cfd->GetID(), cfd->GetLogNumber())); + } + + wal_filter.ColumnFamilyLogNumberMap(cf_lognumber_map, cf_name_id_map); +#endif // !ROCKSDB_LITE +} + +bool DBImpl::InvokeWalFilterIfNeededOnWalRecord(uint64_t wal_number, + const std::string& wal_fname, + log::Reader::Reporter& reporter, + Status& status, + bool& stop_replay, + WriteBatch& batch) { +#ifndef ROCKSDB_LITE + if (immutable_db_options_.wal_filter == nullptr) { + return true; + } + assert(immutable_db_options_.wal_filter != nullptr); + WalFilter& wal_filter = *(immutable_db_options_.wal_filter); + + WriteBatch new_batch; + bool batch_changed = false; + + bool process_current_record = true; + + WalFilter::WalProcessingOption wal_processing_option = + wal_filter.LogRecordFound(wal_number, wal_fname, batch, &new_batch, + &batch_changed); + + switch (wal_processing_option) { + case WalFilter::WalProcessingOption::kContinueProcessing: + // do nothing, proceeed normally + break; + case WalFilter::WalProcessingOption::kIgnoreCurrentRecord: + // skip current record + process_current_record = false; + break; + case WalFilter::WalProcessingOption::kStopReplay: + // skip current record and stop replay + process_current_record = false; + stop_replay = true; + break; + case WalFilter::WalProcessingOption::kCorruptedRecord: { + status = Status::Corruption("Corruption reported by Wal Filter ", + wal_filter.Name()); + MaybeIgnoreError(&status); + if (!status.ok()) { + process_current_record = false; + reporter.Corruption(batch.GetDataSize(), status); + } + break; + } + default: { + // logical error which should not happen. If RocksDB throws, we would + // just do `throw std::logic_error`. + assert(false); + status = Status::NotSupported( + "Unknown WalProcessingOption returned by Wal Filter ", + wal_filter.Name()); + MaybeIgnoreError(&status); + if (!status.ok()) { + // Ignore the error with current record processing. + stop_replay = true; + } + break; + } + } + + if (!process_current_record) { + return false; + } + + if (batch_changed) { + // Make sure that the count in the new batch is + // within the orignal count. + int new_count = WriteBatchInternal::Count(&new_batch); + int original_count = WriteBatchInternal::Count(&batch); + if (new_count > original_count) { + ROCKS_LOG_FATAL( + immutable_db_options_.info_log, + "Recovering log #%" PRIu64 + " mode %d log filter %s returned " + "more records (%d) than original (%d) which is not allowed. " + "Aborting recovery.", + wal_number, static_cast(immutable_db_options_.wal_recovery_mode), + wal_filter.Name(), new_count, original_count); + status = Status::NotSupported( + "More than original # of records " + "returned by Wal Filter ", + wal_filter.Name()); + return false; + } + // Set the same sequence number in the new_batch + // as the original batch. + WriteBatchInternal::SetSequence(&new_batch, + WriteBatchInternal::Sequence(&batch)); + batch = new_batch; + } + return true; +#else // !ROCKSDB_LITE + (void)wal_number; + (void)wal_fname; + (void)reporter; + (void)status; + (void)stop_replay; + (void)batch; + return true; +#endif // ROCKSDB_LITE +} + +// REQUIRES: wal_numbers are sorted in ascending order +Status DBImpl::RecoverLogFiles(const std::vector& wal_numbers, + SequenceNumber* next_sequence, bool read_only, + bool* corrupted_wal_found, + RecoveryContext* recovery_ctx) { + struct LogReporter : public log::Reader::Reporter { + Env* env; + Logger* info_log; + const char* fname; + Status* status; // nullptr if immutable_db_options_.paranoid_checks==false + void Corruption(size_t bytes, const Status& s) override { + ROCKS_LOG_WARN(info_log, "%s%s: dropping %d bytes; %s", + (status == nullptr ? "(ignoring error) " : ""), fname, + static_cast(bytes), s.ToString().c_str()); + if (status != nullptr && status->ok()) { + *status = s; + } + } + }; + + mutex_.AssertHeld(); + Status status; + std::unordered_map version_edits; + // no need to refcount because iteration is under mutex + for (auto cfd : *versions_->GetColumnFamilySet()) { + VersionEdit edit; + edit.SetColumnFamily(cfd->GetID()); + version_edits.insert({cfd->GetID(), edit}); + } + int job_id = next_job_id_.fetch_add(1); + { + auto stream = event_logger_.Log(); + stream << "job" << job_id << "event" + << "recovery_started"; + stream << "wal_files"; + stream.StartArray(); + for (auto wal_number : wal_numbers) { + stream << wal_number; + } + stream.EndArray(); + } + + // No-op for immutable_db_options_.wal_filter == nullptr. + InvokeWalFilterIfNeededOnColumnFamilyToWalNumberMap(); + + bool stop_replay_by_wal_filter = false; + bool stop_replay_for_corruption = false; + bool flushed = false; + uint64_t corrupted_wal_number = kMaxSequenceNumber; + uint64_t min_wal_number = MinLogNumberToKeep(); + if (!allow_2pc()) { + // In non-2pc mode, we skip WALs that do not back unflushed data. + min_wal_number = + std::max(min_wal_number, versions_->MinLogNumberWithUnflushedData()); + } + for (auto wal_number : wal_numbers) { + if (wal_number < min_wal_number) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Skipping log #%" PRIu64 + " since it is older than min log to keep #%" PRIu64, + wal_number, min_wal_number); + continue; + } + // The previous incarnation may not have written any MANIFEST + // records after allocating this log number. So we manually + // update the file number allocation counter in VersionSet. + versions_->MarkFileNumberUsed(wal_number); + // Open the log file + std::string fname = + LogFileName(immutable_db_options_.GetWalDir(), wal_number); + + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Recovering log #%" PRIu64 " mode %d", wal_number, + static_cast(immutable_db_options_.wal_recovery_mode)); + auto logFileDropped = [this, &fname]() { + uint64_t bytes; + if (env_->GetFileSize(fname, &bytes).ok()) { + auto info_log = immutable_db_options_.info_log.get(); + ROCKS_LOG_WARN(info_log, "%s: dropping %d bytes", fname.c_str(), + static_cast(bytes)); + } + }; + if (stop_replay_by_wal_filter) { + logFileDropped(); + continue; + } + + std::unique_ptr file_reader; + { + std::unique_ptr file; + status = fs_->NewSequentialFile( + fname, fs_->OptimizeForLogRead(file_options_), &file, nullptr); + if (!status.ok()) { + MaybeIgnoreError(&status); + if (!status.ok()) { + return status; + } else { + // Fail with one log file, but that's ok. + // Try next one. + continue; + } + } + file_reader.reset(new SequentialFileReader( + std::move(file), fname, immutable_db_options_.log_readahead_size, + io_tracer_)); + } + + // Create the log reader. + LogReporter reporter; + reporter.env = env_; + reporter.info_log = immutable_db_options_.info_log.get(); + reporter.fname = fname.c_str(); + if (!immutable_db_options_.paranoid_checks || + immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kSkipAnyCorruptedRecords) { + reporter.status = nullptr; + } else { + reporter.status = &status; + } + // We intentially make log::Reader do checksumming even if + // paranoid_checks==false so that corruptions cause entire commits + // to be skipped instead of propagating bad information (like overly + // large sequence numbers). + log::Reader reader(immutable_db_options_.info_log, std::move(file_reader), + &reporter, true /*checksum*/, wal_number); + + // Determine if we should tolerate incomplete records at the tail end of the + // Read all the records and add to a memtable + std::string scratch; + Slice record; + + TEST_SYNC_POINT_CALLBACK("DBImpl::RecoverLogFiles:BeforeReadWal", + /*arg=*/nullptr); + uint64_t record_checksum; + while (!stop_replay_by_wal_filter && + reader.ReadRecord(&record, &scratch, + immutable_db_options_.wal_recovery_mode, + &record_checksum) && + status.ok()) { + if (record.size() < WriteBatchInternal::kHeader) { + reporter.Corruption(record.size(), + Status::Corruption("log record too small")); + continue; + } + + // We create a new batch and initialize with a valid prot_info_ to store + // the data checksums + WriteBatch batch; + + status = WriteBatchInternal::SetContents(&batch, record); + if (!status.ok()) { + return status; + } + TEST_SYNC_POINT_CALLBACK( + "DBImpl::RecoverLogFiles:BeforeUpdateProtectionInfo:batch", &batch); + TEST_SYNC_POINT_CALLBACK( + "DBImpl::RecoverLogFiles:BeforeUpdateProtectionInfo:checksum", + &record_checksum); + status = WriteBatchInternal::UpdateProtectionInfo( + &batch, 8 /* bytes_per_key */, &record_checksum); + if (!status.ok()) { + return status; + } + + SequenceNumber sequence = WriteBatchInternal::Sequence(&batch); + + if (immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kPointInTimeRecovery) { + // In point-in-time recovery mode, if sequence id of log files are + // consecutive, we continue recovery despite corruption. This could + // happen when we open and write to a corrupted DB, where sequence id + // will start from the last sequence id we recovered. + if (sequence == *next_sequence) { + stop_replay_for_corruption = false; + } + if (stop_replay_for_corruption) { + logFileDropped(); + break; + } + } + + // For the default case of wal_filter == nullptr, always performs no-op + // and returns true. + if (!InvokeWalFilterIfNeededOnWalRecord(wal_number, fname, reporter, + status, stop_replay_by_wal_filter, + batch)) { + continue; + } + + // If column family was not found, it might mean that the WAL write + // batch references to the column family that was dropped after the + // insert. We don't want to fail the whole write batch in that case -- + // we just ignore the update. + // That's why we set ignore missing column families to true + bool has_valid_writes = false; + status = WriteBatchInternal::InsertInto( + &batch, column_family_memtables_.get(), &flush_scheduler_, + &trim_history_scheduler_, true, wal_number, this, + false /* concurrent_memtable_writes */, next_sequence, + &has_valid_writes, seq_per_batch_, batch_per_txn_); + MaybeIgnoreError(&status); + if (!status.ok()) { + // We are treating this as a failure while reading since we read valid + // blocks that do not form coherent data + reporter.Corruption(record.size(), status); + continue; + } + + if (has_valid_writes && !read_only) { + // we can do this because this is called before client has access to the + // DB and there is only a single thread operating on DB + ColumnFamilyData* cfd; + + while ((cfd = flush_scheduler_.TakeNextColumnFamily()) != nullptr) { + cfd->UnrefAndTryDelete(); + // If this asserts, it means that InsertInto failed in + // filtering updates to already-flushed column families + assert(cfd->GetLogNumber() <= wal_number); + auto iter = version_edits.find(cfd->GetID()); + assert(iter != version_edits.end()); + VersionEdit* edit = &iter->second; + status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit); + if (!status.ok()) { + // Reflect errors immediately so that conditions like full + // file-systems cause the DB::Open() to fail. + return status; + } + flushed = true; + + cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(), + *next_sequence); + } + } + } + + if (!status.ok()) { + if (status.IsNotSupported()) { + // We should not treat NotSupported as corruption. It is rather a clear + // sign that we are processing a WAL that is produced by an incompatible + // version of the code. + return status; + } + if (immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kSkipAnyCorruptedRecords) { + // We should ignore all errors unconditionally + status = Status::OK(); + } else if (immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kPointInTimeRecovery) { + if (status.IsIOError()) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "IOError during point-in-time reading log #%" PRIu64 + " seq #%" PRIu64 + ". %s. This likely mean loss of synced WAL, " + "thus recovery fails.", + wal_number, *next_sequence, + status.ToString().c_str()); + return status; + } + // We should ignore the error but not continue replaying + status = Status::OK(); + stop_replay_for_corruption = true; + corrupted_wal_number = wal_number; + if (corrupted_wal_found != nullptr) { + *corrupted_wal_found = true; + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Point in time recovered to log #%" PRIu64 + " seq #%" PRIu64, + wal_number, *next_sequence); + } else { + assert(immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kTolerateCorruptedTailRecords || + immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kAbsoluteConsistency); + return status; + } + } + + flush_scheduler_.Clear(); + trim_history_scheduler_.Clear(); + auto last_sequence = *next_sequence - 1; + if ((*next_sequence != kMaxSequenceNumber) && + (versions_->LastSequence() <= last_sequence)) { + versions_->SetLastAllocatedSequence(last_sequence); + versions_->SetLastPublishedSequence(last_sequence); + versions_->SetLastSequence(last_sequence); + } + } + // Compare the corrupted log number to all columnfamily's current log number. + // Abort Open() if any column family's log number is greater than + // the corrupted log number, which means CF contains data beyond the point of + // corruption. This could during PIT recovery when the WAL is corrupted and + // some (but not all) CFs are flushed + // Exclude the PIT case where no log is dropped after the corruption point. + // This is to cover the case for empty wals after corrupted log, in which we + // don't reset stop_replay_for_corruption. + if (stop_replay_for_corruption == true && + (immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kPointInTimeRecovery || + immutable_db_options_.wal_recovery_mode == + WALRecoveryMode::kTolerateCorruptedTailRecords)) { + for (auto cfd : *versions_->GetColumnFamilySet()) { + // One special case cause cfd->GetLogNumber() > corrupted_wal_number but + // the CF is still consistent: If a new column family is created during + // the flush and the WAL sync fails at the same time, the new CF points to + // the new WAL but the old WAL is curropted. Since the new CF is empty, it + // is still consistent. We add the check of CF sst file size to avoid the + // false positive alert. + + // Note that, the check of (cfd->GetLiveSstFilesSize() > 0) may leads to + // the ignorance of a very rare inconsistency case caused in data + // canclation. One CF is empty due to KV deletion. But those operations + // are in the WAL. If the WAL is corrupted, the status of this CF might + // not be consistent with others. However, the consistency check will be + // bypassed due to empty CF. + // TODO: a better and complete implementation is needed to ensure strict + // consistency check in WAL recovery including hanlding the tailing + // issues. + if (cfd->GetLogNumber() > corrupted_wal_number && + cfd->GetLiveSstFilesSize() > 0) { + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Column family inconsistency: SST file contains data" + " beyond the point of corruption."); + return Status::Corruption("SST file is ahead of WALs in CF " + + cfd->GetName()); + } + } + } + + // True if there's any data in the WALs; if not, we can skip re-processing + // them later + bool data_seen = false; + if (!read_only) { + // no need to refcount since client still doesn't have access + // to the DB and can not drop column families while we iterate + const WalNumber max_wal_number = wal_numbers.back(); + for (auto cfd : *versions_->GetColumnFamilySet()) { + auto iter = version_edits.find(cfd->GetID()); + assert(iter != version_edits.end()); + VersionEdit* edit = &iter->second; + + if (cfd->GetLogNumber() > max_wal_number) { + // Column family cfd has already flushed the data + // from all wals. Memtable has to be empty because + // we filter the updates based on wal_number + // (in WriteBatch::InsertInto) + assert(cfd->mem()->GetFirstSequenceNumber() == 0); + assert(edit->NumEntries() == 0); + continue; + } + + TEST_SYNC_POINT_CALLBACK( + "DBImpl::RecoverLogFiles:BeforeFlushFinalMemtable", /*arg=*/nullptr); + + // flush the final memtable (if non-empty) + if (cfd->mem()->GetFirstSequenceNumber() != 0) { + // If flush happened in the middle of recovery (e.g. due to memtable + // being full), we flush at the end. Otherwise we'll need to record + // where we were on last flush, which make the logic complicated. + if (flushed || !immutable_db_options_.avoid_flush_during_recovery) { + status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit); + if (!status.ok()) { + // Recovery failed + break; + } + flushed = true; + + cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(), + versions_->LastSequence()); + } + data_seen = true; + } + + // Update the log number info in the version edit corresponding to this + // column family. Note that the version edits will be written to MANIFEST + // together later. + // writing wal_number in the manifest means that any log file + // with number strongly less than (wal_number + 1) is already + // recovered and should be ignored on next reincarnation. + // Since we already recovered max_wal_number, we want all wals + // with numbers `<= max_wal_number` (includes this one) to be ignored + if (flushed || cfd->mem()->GetFirstSequenceNumber() == 0) { + edit->SetLogNumber(max_wal_number + 1); + } + } + if (status.ok()) { + // we must mark the next log number as used, even though it's + // not actually used. that is because VersionSet assumes + // VersionSet::next_file_number_ always to be strictly greater than any + // log number + versions_->MarkFileNumberUsed(max_wal_number + 1); + assert(recovery_ctx != nullptr); + + for (auto* cfd : *versions_->GetColumnFamilySet()) { + auto iter = version_edits.find(cfd->GetID()); + assert(iter != version_edits.end()); + recovery_ctx->UpdateVersionEdits(cfd, iter->second); + } + + if (flushed) { + VersionEdit wal_deletion; + if (immutable_db_options_.track_and_verify_wals_in_manifest) { + wal_deletion.DeleteWalsBefore(max_wal_number + 1); + } + if (!allow_2pc()) { + // In non-2pc mode, flushing the memtables of the column families + // means we can advance min_log_number_to_keep. + wal_deletion.SetMinLogNumberToKeep(max_wal_number + 1); + } + assert(versions_->GetColumnFamilySet() != nullptr); + recovery_ctx->UpdateVersionEdits( + versions_->GetColumnFamilySet()->GetDefault(), wal_deletion); + } + } + } + + if (status.ok()) { + if (data_seen && !flushed) { + status = RestoreAliveLogFiles(wal_numbers); + } else if (!wal_numbers.empty()) { // If there's no data in the WAL, or we + // flushed all the data, still + // truncate the log file. If the process goes into a crash loop before + // the file is deleted, the preallocated space will never get freed. + const bool truncate = !read_only; + GetLogSizeAndMaybeTruncate(wal_numbers.back(), truncate, nullptr) + .PermitUncheckedError(); + } + } + + event_logger_.Log() << "job" << job_id << "event" + << "recovery_finished"; + + return status; +} + +Status DBImpl::GetLogSizeAndMaybeTruncate(uint64_t wal_number, bool truncate, + LogFileNumberSize* log_ptr) { + LogFileNumberSize log(wal_number); + std::string fname = + LogFileName(immutable_db_options_.GetWalDir(), wal_number); + Status s; + // This gets the appear size of the wals, not including preallocated space. + s = env_->GetFileSize(fname, &log.size); + TEST_SYNC_POINT_CALLBACK("DBImpl::GetLogSizeAndMaybeTruncate:0", /*arg=*/&s); + if (s.ok() && truncate) { + std::unique_ptr last_log; + Status truncate_status = fs_->ReopenWritableFile( + fname, + fs_->OptimizeForLogWrite( + file_options_, + BuildDBOptions(immutable_db_options_, mutable_db_options_)), + &last_log, nullptr); + if (truncate_status.ok()) { + truncate_status = last_log->Truncate(log.size, IOOptions(), nullptr); + } + if (truncate_status.ok()) { + truncate_status = last_log->Close(IOOptions(), nullptr); + } + // Not a critical error if fail to truncate. + if (!truncate_status.ok() && !truncate_status.IsNotSupported()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "Failed to truncate log #%" PRIu64 ": %s", wal_number, + truncate_status.ToString().c_str()); + } + } + if (log_ptr) { + *log_ptr = log; + } + return s; +} + +Status DBImpl::RestoreAliveLogFiles(const std::vector& wal_numbers) { + if (wal_numbers.empty()) { + return Status::OK(); + } + Status s; + mutex_.AssertHeld(); + assert(immutable_db_options_.avoid_flush_during_recovery); + // Mark these as alive so they'll be considered for deletion later by + // FindObsoleteFiles() + total_log_size_ = 0; + log_empty_ = false; + uint64_t min_wal_with_unflushed_data = + versions_->MinLogNumberWithUnflushedData(); + for (auto wal_number : wal_numbers) { + if (!allow_2pc() && wal_number < min_wal_with_unflushed_data) { + // In non-2pc mode, the WAL files not backing unflushed data are not + // alive, thus should not be added to the alive_log_files_. + continue; + } + // We preallocate space for wals, but then after a crash and restart, those + // preallocated space are not needed anymore. It is likely only the last + // log has such preallocated space, so we only truncate for the last log. + LogFileNumberSize log; + s = GetLogSizeAndMaybeTruncate( + wal_number, /*truncate=*/(wal_number == wal_numbers.back()), &log); + if (!s.ok()) { + break; + } + total_log_size_ += log.size; + alive_log_files_.push_back(log); + } + return s; +} + +Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd, + MemTable* mem, VersionEdit* edit) { + mutex_.AssertHeld(); + assert(cfd); + assert(cfd->imm()); + // The immutable memtable list must be empty. + assert(std::numeric_limits::max() == + cfd->imm()->GetEarliestMemTableID()); + + const uint64_t start_micros = immutable_db_options_.clock->NowMicros(); + + FileMetaData meta; + std::vector blob_file_additions; + + std::unique_ptr::iterator> pending_outputs_inserted_elem( + new std::list::iterator( + CaptureCurrentFileNumberInPendingOutputs())); + meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0); + ReadOptions ro; + ro.total_order_seek = true; + Arena arena; + Status s; + TableProperties table_properties; + { + ScopedArenaIterator iter(mem->NewIterator(ro, &arena)); + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[%s] [WriteLevel0TableForRecovery]" + " Level-0 table #%" PRIu64 ": started", + 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(); + bool paranoid_file_checks = + cfd->GetLatestMutableCFOptions()->paranoid_file_checks; + + int64_t _current_time = 0; + immutable_db_options_.clock->GetCurrentTime(&_current_time) + .PermitUncheckedError(); // ignore error + const uint64_t current_time = static_cast(_current_time); + meta.oldest_ancester_time = current_time; + + { + auto write_hint = cfd->CalculateSSTWriteHint(0); + mutex_.Unlock(); + + SequenceNumber earliest_write_conflict_snapshot; + std::vector snapshot_seqs = + snapshots_.GetAll(&earliest_write_conflict_snapshot); + auto snapshot_checker = snapshot_checker_.get(); + if (use_custom_gc_ && snapshot_checker == nullptr) { + snapshot_checker = DisableGCSnapshotChecker::Instance(); + } + std::vector> + range_del_iters; + auto range_del_iter = + // This is called during recovery, where a live memtable is flushed + // directly. In this case, no fragmented tombstone list is cached in + // this memtable yet. + mem->NewRangeTombstoneIterator(ro, kMaxSequenceNumber, + false /* immutable_memtable */); + if (range_del_iter != nullptr) { + range_del_iters.emplace_back(range_del_iter); + } + + IOStatus io_s; + TableBuilderOptions tboptions( + *cfd->ioptions(), mutable_cf_options, cfd->internal_comparator(), + cfd->int_tbl_prop_collector_factories(), + GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), + mutable_cf_options.compression_opts, cfd->GetID(), cfd->GetName(), + 0 /* level */, 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()); + SeqnoToTimeMapping empty_seqno_time_mapping; + s = BuildTable( + dbname_, versions_.get(), immutable_db_options_, tboptions, + file_options_for_compaction_, cfd->table_cache(), iter.get(), + std::move(range_del_iters), &meta, &blob_file_additions, + snapshot_seqs, earliest_write_conflict_snapshot, kMaxSequenceNumber, + snapshot_checker, paranoid_file_checks, cfd->internal_stats(), &io_s, + io_tracer_, BlobFileCreationReason::kRecovery, + empty_seqno_time_mapping, &event_logger_, job_id, Env::IO_HIGH, + nullptr /* table_properties */, write_hint, + nullptr /*full_history_ts_low*/, &blob_callback_); + LogFlush(immutable_db_options_.info_log); + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[%s] [WriteLevel0TableForRecovery]" + " Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s", + cfd->GetName().c_str(), meta.fd.GetNumber(), + meta.fd.GetFileSize(), s.ToString().c_str()); + mutex_.Lock(); + + // TODO(AR) is this ok? + if (!io_s.ok() && s.ok()) { + s = io_s; + } + } + } + ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem); + + // Note that if file_size is zero, the file has been deleted and + // should not be added to the manifest. + const bool has_output = meta.fd.GetFileSize() > 0; + + constexpr int level = 0; + + if (s.ok() && has_output) { + edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(), + meta.fd.GetFileSize(), meta.smallest, meta.largest, + meta.fd.smallest_seqno, meta.fd.largest_seqno, + meta.marked_for_compaction, meta.temperature, + meta.oldest_blob_file_number, meta.oldest_ancester_time, + meta.file_creation_time, meta.file_checksum, + meta.file_checksum_func_name, meta.unique_id); + + for (const auto& blob : blob_file_additions) { + edit->AddBlobFile(blob); + } + } + + InternalStats::CompactionStats stats(CompactionReason::kFlush, 1); + stats.micros = immutable_db_options_.clock->NowMicros() - start_micros; + + if (has_output) { + stats.bytes_written = meta.fd.GetFileSize(); + stats.num_output_files = 1; + } + + const auto& blobs = edit->GetBlobFileAdditions(); + for (const auto& blob : blobs) { + stats.bytes_written_blob += blob.GetTotalBlobBytes(); + } + + stats.num_output_files_blob = static_cast(blobs.size()); + + cfd->internal_stats()->AddCompactionStats(level, Env::Priority::USER, stats); + cfd->internal_stats()->AddCFStats( + InternalStats::BYTES_FLUSHED, + stats.bytes_written + stats.bytes_written_blob); + RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize()); + return s; +} + +Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) { + DBOptions db_options(options); + ColumnFamilyOptions cf_options(options); + std::vector column_families; + column_families.push_back( + ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options)); + if (db_options.persist_stats_to_disk) { + column_families.push_back( + ColumnFamilyDescriptor(kPersistentStatsColumnFamilyName, cf_options)); + } + std::vector handles; + Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr); + if (s.ok()) { + if (db_options.persist_stats_to_disk) { + assert(handles.size() == 2); + } else { + assert(handles.size() == 1); + } + // i can delete the handle since DBImpl is always holding a reference to + // default column family + if (db_options.persist_stats_to_disk && handles[1] != nullptr) { + delete handles[1]; + } + delete handles[0]; + } + return s; +} + +Status DB::Open(const DBOptions& db_options, const std::string& dbname, + const std::vector& column_families, + std::vector* handles, DB** dbptr) { + const bool kSeqPerBatch = true; + const bool kBatchPerTxn = true; + return DBImpl::Open(db_options, dbname, column_families, handles, dbptr, + !kSeqPerBatch, kBatchPerTxn); +} + +// TODO: Implement the trimming in flush code path. +// TODO: Perform trimming before inserting into memtable during recovery. +// TODO: Pick files with max_timestamp > trim_ts by each file's timestamp meta +// info, and handle only these files to reduce io. +Status DB::OpenAndTrimHistory( + const DBOptions& db_options, const std::string& dbname, + const std::vector& column_families, + std::vector* handles, DB** dbptr, + std::string trim_ts) { + assert(dbptr != nullptr); + assert(handles != nullptr); + auto validate_options = [&db_options] { + if (db_options.avoid_flush_during_recovery) { + return Status::InvalidArgument( + "avoid_flush_during_recovery incompatible with " + "OpenAndTrimHistory"); + } + return Status::OK(); + }; + auto s = validate_options(); + if (!s.ok()) { + return s; + } + + DB* db = nullptr; + s = DB::Open(db_options, dbname, column_families, handles, &db); + if (!s.ok()) { + return s; + } + assert(db); + CompactRangeOptions options; + options.bottommost_level_compaction = + BottommostLevelCompaction::kForceOptimized; + auto db_impl = static_cast_with_check(db); + for (auto handle : *handles) { + assert(handle != nullptr); + auto cfh = static_cast_with_check(handle); + auto cfd = cfh->cfd(); + assert(cfd != nullptr); + // Only compact column families with timestamp enabled + if (cfd->user_comparator() != nullptr && + cfd->user_comparator()->timestamp_size() > 0) { + s = db_impl->CompactRangeInternal(options, handle, nullptr, nullptr, + trim_ts); + if (!s.ok()) { + break; + } + } + } + auto clean_op = [&handles, &db] { + for (auto handle : *handles) { + auto temp_s = db->DestroyColumnFamilyHandle(handle); + assert(temp_s.ok()); + } + handles->clear(); + delete db; + }; + if (!s.ok()) { + clean_op(); + return s; + } + + *dbptr = db; + return s; +} + +IOStatus DBImpl::CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number, + size_t preallocate_block_size, + log::Writer** new_log) { + IOStatus io_s; + std::unique_ptr lfile; + + DBOptions db_options = + BuildDBOptions(immutable_db_options_, mutable_db_options_); + FileOptions opt_file_options = + fs_->OptimizeForLogWrite(file_options_, db_options); + std::string wal_dir = immutable_db_options_.GetWalDir(); + std::string log_fname = LogFileName(wal_dir, log_file_num); + + if (recycle_log_number) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "reusing log %" PRIu64 " from recycle list\n", + recycle_log_number); + std::string old_log_fname = LogFileName(wal_dir, recycle_log_number); + TEST_SYNC_POINT("DBImpl::CreateWAL:BeforeReuseWritableFile1"); + TEST_SYNC_POINT("DBImpl::CreateWAL:BeforeReuseWritableFile2"); + io_s = fs_->ReuseWritableFile(log_fname, old_log_fname, opt_file_options, + &lfile, /*dbg=*/nullptr); + } else { + io_s = NewWritableFile(fs_.get(), log_fname, &lfile, opt_file_options); + } + + if (io_s.ok()) { + lfile->SetWriteLifeTimeHint(CalculateWALWriteHint()); + lfile->SetPreallocationBlockSize(preallocate_block_size); + + const auto& listeners = immutable_db_options_.listeners; + FileTypeSet tmp_set = immutable_db_options_.checksum_handoff_file_types; + std::unique_ptr file_writer(new WritableFileWriter( + std::move(lfile), log_fname, opt_file_options, + immutable_db_options_.clock, io_tracer_, nullptr /* stats */, listeners, + nullptr, tmp_set.Contains(FileType::kWalFile), + tmp_set.Contains(FileType::kWalFile))); + *new_log = new log::Writer(std::move(file_writer), log_file_num, + immutable_db_options_.recycle_log_file_num > 0, + immutable_db_options_.manual_wal_flush, + immutable_db_options_.wal_compression); + io_s = (*new_log)->AddCompressionTypeRecord(); + } + return io_s; +} + +Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, + const std::vector& column_families, + std::vector* handles, DB** dbptr, + const bool seq_per_batch, const bool batch_per_txn) { + Status s = ValidateOptionsByTable(db_options, column_families); + if (!s.ok()) { + return s; + } + + s = ValidateOptions(db_options, column_families); + if (!s.ok()) { + return s; + } + + *dbptr = nullptr; + assert(handles); + handles->clear(); + + size_t max_write_buffer_size = 0; + for (auto cf : column_families) { + max_write_buffer_size = + std::max(max_write_buffer_size, cf.options.write_buffer_size); + } + + DBImpl* impl = new DBImpl(db_options, dbname, seq_per_batch, batch_per_txn); + if (!impl->immutable_db_options_.info_log) { + s = impl->init_logger_creation_s_; + delete impl; + return s; + } else { + assert(impl->init_logger_creation_s_.ok()); + } + s = impl->env_->CreateDirIfMissing(impl->immutable_db_options_.GetWalDir()); + if (s.ok()) { + std::vector paths; + for (auto& db_path : impl->immutable_db_options_.db_paths) { + paths.emplace_back(db_path.path); + } + for (auto& cf : column_families) { + for (auto& cf_path : cf.options.cf_paths) { + paths.emplace_back(cf_path.path); + } + } + for (auto& path : paths) { + s = impl->env_->CreateDirIfMissing(path); + if (!s.ok()) { + break; + } + } + + // For recovery from NoSpace() error, we can only handle + // the case where the database is stored in a single path + if (paths.size() <= 1) { + impl->error_handler_.EnableAutoRecovery(); + } + } + if (s.ok()) { + s = impl->CreateArchivalDirectory(); + } + if (!s.ok()) { + delete impl; + return s; + } + + impl->wal_in_db_path_ = impl->immutable_db_options_.IsWalDirSameAsDBPath(); + RecoveryContext recovery_ctx; + impl->mutex_.Lock(); + + // Handles create_if_missing, error_if_exists + uint64_t recovered_seq(kMaxSequenceNumber); + s = impl->Recover(column_families, false, false, false, &recovered_seq, + &recovery_ctx); + if (s.ok()) { + uint64_t new_log_number = impl->versions_->NewFileNumber(); + log::Writer* new_log = nullptr; + const size_t preallocate_block_size = + impl->GetWalPreallocateBlockSize(max_write_buffer_size); + s = impl->CreateWAL(new_log_number, 0 /*recycle_log_number*/, + preallocate_block_size, &new_log); + if (s.ok()) { + InstrumentedMutexLock wl(&impl->log_write_mutex_); + impl->logfile_number_ = new_log_number; + assert(new_log != nullptr); + assert(impl->logs_.empty()); + impl->logs_.emplace_back(new_log_number, new_log); + } + + if (s.ok()) { + impl->alive_log_files_.push_back( + DBImpl::LogFileNumberSize(impl->logfile_number_)); + // In WritePrepared there could be gap in sequence numbers. This breaks + // the trick we use in kPointInTimeRecovery which assumes the first seq in + // the log right after the corrupted log is one larger than the last seq + // we read from the wals. To let this trick keep working, we add a dummy + // entry with the expected sequence to the first log right after recovery. + // In non-WritePrepared case also the new log after recovery could be + // empty, and thus missing the consecutive seq hint to distinguish + // middle-log corruption to corrupted-log-remained-after-recovery. This + // case also will be addressed by a dummy write. + if (recovered_seq != kMaxSequenceNumber) { + WriteBatch empty_batch; + WriteBatchInternal::SetSequence(&empty_batch, recovered_seq); + WriteOptions write_options; + uint64_t log_used, log_size; + log::Writer* log_writer = impl->logs_.back().writer; + LogFileNumberSize& log_file_number_size = impl->alive_log_files_.back(); + + assert(log_writer->get_log_number() == log_file_number_size.number); + impl->mutex_.AssertHeld(); + s = impl->WriteToWAL(empty_batch, log_writer, &log_used, &log_size, + Env::IO_TOTAL, log_file_number_size); + if (s.ok()) { + // Need to fsync, otherwise it might get lost after a power reset. + s = impl->FlushWAL(false); + TEST_SYNC_POINT_CALLBACK("DBImpl::Open::BeforeSyncWAL", /*arg=*/&s); + if (s.ok()) { + s = log_writer->file()->Sync(impl->immutable_db_options_.use_fsync); + } + } + } + } + } + if (s.ok()) { + s = impl->LogAndApplyForRecovery(recovery_ctx); + } + + if (s.ok() && impl->immutable_db_options_.persist_stats_to_disk) { + impl->mutex_.AssertHeld(); + s = impl->InitPersistStatsColumnFamily(); + } + + if (s.ok()) { + // set column family handles + for (auto cf : column_families) { + auto cfd = + impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name); + if (cfd != nullptr) { + handles->push_back( + new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_)); + impl->NewThreadStatusCfInfo(cfd); + } else { + if (db_options.create_missing_column_families) { + // missing column family, create it + ColumnFamilyHandle* handle = nullptr; + impl->mutex_.Unlock(); + s = impl->CreateColumnFamily(cf.options, cf.name, &handle); + impl->mutex_.Lock(); + if (s.ok()) { + handles->push_back(handle); + } else { + break; + } + } else { + s = Status::InvalidArgument("Column family not found", cf.name); + break; + } + } + } + } + + if (s.ok()) { + SuperVersionContext sv_context(/* create_superversion */ true); + for (auto cfd : *impl->versions_->GetColumnFamilySet()) { + impl->InstallSuperVersionAndScheduleWork( + cfd, &sv_context, *cfd->GetLatestMutableCFOptions()); + } + sv_context.Clean(); + } + + if (s.ok() && impl->immutable_db_options_.persist_stats_to_disk) { + // try to read format version + s = impl->PersistentStatsProcessFormatVersion(); + } + + if (s.ok()) { + for (auto cfd : *impl->versions_->GetColumnFamilySet()) { + if (!cfd->mem()->IsSnapshotSupported()) { + impl->is_snapshot_supported_ = false; + } + if (cfd->ioptions()->merge_operator != nullptr && + !cfd->mem()->IsMergeOperatorSupported()) { + s = Status::InvalidArgument( + "The memtable of column family %s does not support merge operator " + "its options.merge_operator is non-null", + cfd->GetName().c_str()); + } + if (!s.ok()) { + break; + } + } + } + TEST_SYNC_POINT("DBImpl::Open:Opened"); + Status persist_options_status; + if (s.ok()) { + // Persist RocksDB Options before scheduling the compaction. + // The WriteOptionsFile() will release and lock the mutex internally. + persist_options_status = impl->WriteOptionsFile( + false /*need_mutex_lock*/, false /*need_enter_write_thread*/); + + *dbptr = impl; + impl->opened_successfully_ = true; + impl->DeleteObsoleteFiles(); + TEST_SYNC_POINT("DBImpl::Open:AfterDeleteFiles"); + impl->MaybeScheduleFlushOrCompaction(); + } else { + persist_options_status.PermitUncheckedError(); + } + impl->mutex_.Unlock(); + +#ifndef ROCKSDB_LITE + auto sfm = static_cast( + impl->immutable_db_options_.sst_file_manager.get()); + if (s.ok() && sfm) { + // Set Statistics ptr for SstFileManager to dump the stats of + // DeleteScheduler. + sfm->SetStatisticsPtr(impl->immutable_db_options_.statistics); + ROCKS_LOG_INFO(impl->immutable_db_options_.info_log, + "SstFileManager instance %p", sfm); + + // Notify SstFileManager about all sst files that already exist in + // db_paths[0] and cf_paths[0] when the DB is opened. + + // SstFileManagerImpl needs to know sizes of the files. For files whose size + // we already know (sst files that appear in manifest - typically that's the + // vast majority of all files), we'll pass the size to SstFileManager. + // For all other files SstFileManager will query the size from filesystem. + + std::vector metadata; + impl->GetAllColumnFamilyMetaData(&metadata); + + std::unordered_map known_file_sizes; + for (const auto& md : metadata) { + for (const auto& lmd : md.levels) { + for (const auto& fmd : lmd.files) { + known_file_sizes[fmd.relative_filename] = fmd.size; + } + } + for (const auto& bmd : md.blob_files) { + std::string name = bmd.blob_file_name; + // The BlobMetaData.blob_file_name may start with "/". + if (!name.empty() && name[0] == '/') { + name = name.substr(1); + } + known_file_sizes[name] = bmd.blob_file_size; + } + } + + std::vector paths; + paths.emplace_back(impl->immutable_db_options_.db_paths[0].path); + for (auto& cf : column_families) { + if (!cf.options.cf_paths.empty()) { + paths.emplace_back(cf.options.cf_paths[0].path); + } + } + // Remove duplicate paths. + std::sort(paths.begin(), paths.end()); + paths.erase(std::unique(paths.begin(), paths.end()), paths.end()); + IOOptions io_opts; + io_opts.do_not_recurse = true; + for (auto& path : paths) { + std::vector existing_files; + impl->immutable_db_options_.fs + ->GetChildren(path, io_opts, &existing_files, + /*IODebugContext*=*/nullptr) + .PermitUncheckedError(); //**TODO: What do to on error? + for (auto& file_name : existing_files) { + uint64_t file_number; + FileType file_type; + std::string file_path = path + "/" + file_name; + if (ParseFileName(file_name, &file_number, &file_type) && + (file_type == kTableFile || file_type == kBlobFile)) { + // TODO: Check for errors from OnAddFile? + if (known_file_sizes.count(file_name)) { + // We're assuming that each sst file name exists in at most one of + // the paths. + sfm->OnAddFile(file_path, known_file_sizes.at(file_name)) + .PermitUncheckedError(); + } else { + sfm->OnAddFile(file_path).PermitUncheckedError(); + } + } + } + } + + // Reserve some disk buffer space. This is a heuristic - when we run out + // of disk space, this ensures that there is atleast write_buffer_size + // amount of free space before we resume DB writes. In low disk space + // conditions, we want to avoid a lot of small L0 files due to frequent + // WAL write failures and resultant forced flushes + sfm->ReserveDiskBuffer(max_write_buffer_size, + impl->immutable_db_options_.db_paths[0].path); + } + +#endif // !ROCKSDB_LITE + + if (s.ok()) { + ROCKS_LOG_HEADER(impl->immutable_db_options_.info_log, "DB pointer %p", + impl); + LogFlush(impl->immutable_db_options_.info_log); + if (!impl->WALBufferIsEmpty()) { + s = impl->FlushWAL(false); + if (s.ok()) { + // Sync is needed otherwise WAL buffered data might get lost after a + // power reset. + log::Writer* log_writer = impl->logs_.back().writer; + s = log_writer->file()->Sync(impl->immutable_db_options_.use_fsync); + } + } + if (s.ok() && !persist_options_status.ok()) { + s = Status::IOError( + "DB::Open() failed --- Unable to persist Options file", + persist_options_status.ToString()); + } + } + if (!s.ok()) { + ROCKS_LOG_WARN(impl->immutable_db_options_.info_log, + "DB::Open() failed: %s", s.ToString().c_str()); + } + if (s.ok()) { + s = impl->StartPeriodicTaskScheduler(); + } + + if (s.ok()) { + s = impl->RegisterRecordSeqnoTimeWorker(); + } + if (!s.ok()) { + for (auto* h : *handles) { + delete h; + } + handles->clear(); + delete impl; + *dbptr = nullptr; + } + return s; +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl_readonly.cc b/src/rocksdb/db/db_impl/db_impl_readonly.cc new file mode 100644 index 000000000..0f10baf24 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_readonly.cc @@ -0,0 +1,341 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "db/db_impl/db_impl_readonly.h" + +#include "db/arena_wrapped_db_iter.h" +#include "db/db_impl/compacted_db_impl.h" +#include "db/db_impl/db_impl.h" +#include "db/db_iter.h" +#include "db/merge_context.h" +#include "logging/logging.h" +#include "monitoring/perf_context_imp.h" +#include "util/cast_util.h" + +namespace ROCKSDB_NAMESPACE { + +#ifndef ROCKSDB_LITE + +DBImplReadOnly::DBImplReadOnly(const DBOptions& db_options, + const std::string& dbname) + : DBImpl(db_options, dbname, /*seq_per_batch*/ false, + /*batch_per_txn*/ true, /*read_only*/ true) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Opening the db in read only mode"); + LogFlush(immutable_db_options_.info_log); +} + +DBImplReadOnly::~DBImplReadOnly() {} + +// Implementations of the DB interface +Status DBImplReadOnly::Get(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* pinnable_val) { + return Get(read_options, column_family, key, pinnable_val, + /*timestamp*/ nullptr); +} + +Status DBImplReadOnly::Get(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* pinnable_val, + std::string* timestamp) { + assert(pinnable_val != nullptr); + // TODO: stopwatch DB_GET needed?, perf timer needed? + PERF_TIMER_GUARD(get_snapshot_time); + + assert(column_family); + if (read_options.timestamp) { + const Status s = FailIfTsMismatchCf( + column_family, *(read_options.timestamp), /*ts_for_read=*/true); + if (!s.ok()) { + return s; + } + } else { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + } + + // Clear the timestamps for returning results so that we can distinguish + // between tombstone or key that has never been written + if (timestamp) { + timestamp->clear(); + } + + const Comparator* ucmp = column_family->GetComparator(); + assert(ucmp); + std::string* ts = ucmp->timestamp_size() > 0 ? timestamp : nullptr; + + Status s; + SequenceNumber snapshot = versions_->LastSequence(); + GetWithTimestampReadCallback read_cb(snapshot); + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + tracer_->Get(column_family, key); + } + } + SuperVersion* super_version = cfd->GetSuperVersion(); + MergeContext merge_context; + SequenceNumber max_covering_tombstone_seq = 0; + LookupKey lkey(key, snapshot, read_options.timestamp); + PERF_TIMER_STOP(get_snapshot_time); + if (super_version->mem->Get(lkey, pinnable_val->GetSelf(), + /*columns=*/nullptr, ts, &s, &merge_context, + &max_covering_tombstone_seq, read_options, + false /* immutable_memtable */, &read_cb)) { + pinnable_val->PinSelf(); + RecordTick(stats_, MEMTABLE_HIT); + } else { + PERF_TIMER_GUARD(get_from_output_files_time); + PinnedIteratorsManager pinned_iters_mgr; + super_version->current->Get( + read_options, lkey, pinnable_val, /*columns=*/nullptr, ts, &s, + &merge_context, &max_covering_tombstone_seq, &pinned_iters_mgr, + /*value_found*/ nullptr, + /*key_exists*/ nullptr, /*seq*/ nullptr, &read_cb, + /*is_blob*/ nullptr, + /*do_merge*/ true); + RecordTick(stats_, MEMTABLE_MISS); + } + RecordTick(stats_, NUMBER_KEYS_READ); + size_t size = pinnable_val->size(); + RecordTick(stats_, BYTES_READ, size); + RecordInHistogram(stats_, BYTES_PER_READ, size); + PERF_COUNTER_ADD(get_read_bytes, size); + return s; +} + +Iterator* DBImplReadOnly::NewIterator(const ReadOptions& read_options, + ColumnFamilyHandle* column_family) { + assert(column_family); + if (read_options.timestamp) { + const Status s = FailIfTsMismatchCf( + column_family, *(read_options.timestamp), /*ts_for_read=*/true); + if (!s.ok()) { + return NewErrorIterator(s); + } + } else { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return NewErrorIterator(s); + } + } + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + SuperVersion* super_version = cfd->GetSuperVersion()->Ref(); + SequenceNumber latest_snapshot = versions_->LastSequence(); + SequenceNumber read_seq = + read_options.snapshot != nullptr + ? reinterpret_cast(read_options.snapshot) + ->number_ + : latest_snapshot; + ReadCallback* read_callback = nullptr; // No read callback provided. + auto db_iter = NewArenaWrappedDbIterator( + 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); + auto internal_iter = NewInternalIterator( + db_iter->GetReadOptions(), cfd, super_version, db_iter->GetArena(), + read_seq, /* allow_unprepared_value */ true, db_iter); + db_iter->SetIterUnderDBIter(internal_iter); + return db_iter; +} + +Status DBImplReadOnly::NewIterators( + const ReadOptions& read_options, + const std::vector& column_families, + std::vector* iterators) { + if (read_options.timestamp) { + for (auto* cf : column_families) { + assert(cf); + const Status s = FailIfTsMismatchCf(cf, *(read_options.timestamp), + /*ts_for_read=*/true); + if (!s.ok()) { + return s; + } + } + } else { + for (auto* cf : column_families) { + assert(cf); + const Status s = FailIfCfHasTs(cf); + if (!s.ok()) { + return s; + } + } + } + + ReadCallback* read_callback = nullptr; // No read callback provided. + if (iterators == nullptr) { + return Status::InvalidArgument("iterators not allowed to be nullptr"); + } + iterators->clear(); + iterators->reserve(column_families.size()); + SequenceNumber latest_snapshot = versions_->LastSequence(); + SequenceNumber read_seq = + read_options.snapshot != nullptr + ? reinterpret_cast(read_options.snapshot) + ->number_ + : latest_snapshot; + + for (auto cfh : column_families) { + auto* cfd = static_cast_with_check(cfh)->cfd(); + auto* sv = cfd->GetSuperVersion()->Ref(); + auto* db_iter = NewArenaWrappedDbIterator( + 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); + auto* internal_iter = NewInternalIterator( + db_iter->GetReadOptions(), cfd, sv, db_iter->GetArena(), read_seq, + /* allow_unprepared_value */ true, db_iter); + db_iter->SetIterUnderDBIter(internal_iter); + iterators->push_back(db_iter); + } + + return Status::OK(); +} + +namespace { +// Return OK if dbname exists in the file system or create it if +// create_if_missing +Status OpenForReadOnlyCheckExistence(const DBOptions& db_options, + const std::string& dbname) { + Status s; + if (!db_options.create_if_missing) { + // Attempt to read "CURRENT" file + const std::shared_ptr& fs = db_options.env->GetFileSystem(); + std::string manifest_path; + uint64_t manifest_file_number; + s = VersionSet::GetCurrentManifestPath(dbname, fs.get(), &manifest_path, + &manifest_file_number); + } else { + // Historic behavior that doesn't necessarily make sense + s = db_options.env->CreateDirIfMissing(dbname); + } + return s; +} +} // namespace + +Status DB::OpenForReadOnly(const Options& options, const std::string& dbname, + DB** dbptr, bool /*error_if_wal_file_exists*/) { + Status s = OpenForReadOnlyCheckExistence(options, dbname); + if (!s.ok()) { + return s; + } + + *dbptr = nullptr; + + // Try to first open DB as fully compacted DB + s = CompactedDBImpl::Open(options, dbname, dbptr); + if (s.ok()) { + return s; + } + + DBOptions db_options(options); + ColumnFamilyOptions cf_options(options); + std::vector column_families; + column_families.push_back( + ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options)); + std::vector handles; + + s = DBImplReadOnly::OpenForReadOnlyWithoutCheck( + db_options, dbname, column_families, &handles, dbptr); + if (s.ok()) { + assert(handles.size() == 1); + // i can delete the handle since DBImpl is always holding a + // reference to default column family + delete handles[0]; + } + return s; +} + +Status DB::OpenForReadOnly( + const DBOptions& db_options, const std::string& dbname, + const std::vector& column_families, + std::vector* handles, DB** dbptr, + bool error_if_wal_file_exists) { + // If dbname does not exist in the file system, should not do anything + Status s = OpenForReadOnlyCheckExistence(db_options, dbname); + if (!s.ok()) { + return s; + } + + return DBImplReadOnly::OpenForReadOnlyWithoutCheck( + db_options, dbname, column_families, handles, dbptr, + error_if_wal_file_exists); +} + +Status DBImplReadOnly::OpenForReadOnlyWithoutCheck( + const DBOptions& db_options, const std::string& dbname, + const std::vector& column_families, + std::vector* handles, DB** dbptr, + bool error_if_wal_file_exists) { + *dbptr = nullptr; + handles->clear(); + + SuperVersionContext sv_context(/* create_superversion */ true); + DBImplReadOnly* impl = new DBImplReadOnly(db_options, dbname); + impl->mutex_.Lock(); + Status s = impl->Recover(column_families, true /* read only */, + error_if_wal_file_exists); + if (s.ok()) { + // set column family handles + for (auto cf : column_families) { + auto cfd = + impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name); + if (cfd == nullptr) { + s = Status::InvalidArgument("Column family not found", cf.name); + break; + } + handles->push_back(new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_)); + } + } + if (s.ok()) { + for (auto cfd : *impl->versions_->GetColumnFamilySet()) { + sv_context.NewSuperVersion(); + cfd->InstallSuperVersion(&sv_context, &impl->mutex_); + } + } + impl->mutex_.Unlock(); + sv_context.Clean(); + if (s.ok()) { + *dbptr = impl; + for (auto* h : *handles) { + impl->NewThreadStatusCfInfo( + static_cast_with_check(h)->cfd()); + } + } else { + for (auto h : *handles) { + delete h; + } + handles->clear(); + delete impl; + } + return s; +} + +#else // !ROCKSDB_LITE + +Status DB::OpenForReadOnly(const Options& /*options*/, + const std::string& /*dbname*/, DB** /*dbptr*/, + bool /*error_if_wal_file_exists*/) { + return Status::NotSupported("Not supported in ROCKSDB_LITE."); +} + +Status DB::OpenForReadOnly( + const DBOptions& /*db_options*/, const std::string& /*dbname*/, + const std::vector& /*column_families*/, + std::vector* /*handles*/, DB** /*dbptr*/, + bool /*error_if_wal_file_exists*/) { + return Status::NotSupported("Not supported in ROCKSDB_LITE."); +} +#endif // !ROCKSDB_LITE + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl_readonly.h b/src/rocksdb/db/db_impl/db_impl_readonly.h new file mode 100644 index 000000000..b876a0fda --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_readonly.h @@ -0,0 +1,170 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#ifndef ROCKSDB_LITE + +#include +#include + +#include "db/db_impl/db_impl.h" + +namespace ROCKSDB_NAMESPACE { + +// TODO: Share common structure with CompactedDBImpl and DBImplSecondary +class DBImplReadOnly : public DBImpl { + public: + DBImplReadOnly(const DBOptions& options, const std::string& dbname); + // No copying allowed + DBImplReadOnly(const DBImplReadOnly&) = delete; + void operator=(const DBImplReadOnly&) = delete; + + virtual ~DBImplReadOnly(); + + // Implementations of the DB interface + using DB::Get; + virtual Status Get(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value) override; + Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value, + std::string* timestamp) override; + + // TODO: Implement ReadOnly MultiGet? + + using DBImpl::NewIterator; + virtual Iterator* NewIterator(const ReadOptions&, + ColumnFamilyHandle* column_family) override; + + virtual Status NewIterators( + const ReadOptions& options, + const std::vector& column_families, + std::vector* iterators) override; + + using DBImpl::Put; + virtual Status Put(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/, const Slice& /*value*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + using DBImpl::PutEntity; + Status PutEntity(const WriteOptions& /* options */, + ColumnFamilyHandle* /* column_family */, + const Slice& /* key */, + const WideColumns& /* columns */) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + using DBImpl::Merge; + virtual Status Merge(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/, const Slice& /*value*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + using DBImpl::Delete; + virtual Status Delete(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + using DBImpl::SingleDelete; + virtual Status SingleDelete(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + virtual Status Write(const WriteOptions& /*options*/, + WriteBatch* /*updates*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + using DBImpl::CompactRange; + virtual Status CompactRange(const CompactRangeOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice* /*begin*/, + const Slice* /*end*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + using DBImpl::CompactFiles; + virtual Status CompactFiles( + const CompactionOptions& /*compact_options*/, + ColumnFamilyHandle* /*column_family*/, + const std::vector& /*input_file_names*/, + const int /*output_level*/, const int /*output_path_id*/ = -1, + std::vector* const /*output_file_names*/ = nullptr, + CompactionJobInfo* /*compaction_job_info*/ = nullptr) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + virtual Status DisableFileDeletions() override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + virtual Status EnableFileDeletions(bool /*force*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + virtual Status GetLiveFiles(std::vector& ret, + uint64_t* manifest_file_size, + bool /*flush_memtable*/) override { + return DBImpl::GetLiveFiles(ret, manifest_file_size, + false /* flush_memtable */); + } + + using DBImpl::Flush; + virtual Status Flush(const FlushOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + using DBImpl::SyncWAL; + virtual Status SyncWAL() override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + using DB::IngestExternalFile; + virtual Status IngestExternalFile( + ColumnFamilyHandle* /*column_family*/, + const std::vector& /*external_files*/, + const IngestExternalFileOptions& /*ingestion_options*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + using DB::CreateColumnFamilyWithImport; + virtual Status CreateColumnFamilyWithImport( + const ColumnFamilyOptions& /*options*/, + const std::string& /*column_family_name*/, + const ImportColumnFamilyOptions& /*import_options*/, + const ExportImportFilesMetaData& /*metadata*/, + ColumnFamilyHandle** /*handle*/) override { + return Status::NotSupported("Not supported operation in read only mode."); + } + + // FIXME: some missing overrides for more "write" functions + + protected: +#ifndef ROCKSDB_LITE + Status FlushForGetLiveFiles() override { + // No-op for read-only DB + return Status::OK(); + } +#endif // !ROCKSDB_LITE + + private: + // A "helper" function for DB::OpenForReadOnly without column families + // to reduce unnecessary I/O + // It has the same functionality as DB::OpenForReadOnly with column families + // but does not check the existence of dbname in the file system + static Status OpenForReadOnlyWithoutCheck( + const DBOptions& db_options, const std::string& dbname, + const std::vector& column_families, + std::vector* handles, DB** dbptr, + bool error_if_wal_file_exists = false); + friend class DB; +}; +} // namespace ROCKSDB_NAMESPACE + +#endif // !ROCKSDB_LITE diff --git a/src/rocksdb/db/db_impl/db_impl_secondary.cc b/src/rocksdb/db/db_impl/db_impl_secondary.cc new file mode 100644 index 000000000..5189d17d9 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_secondary.cc @@ -0,0 +1,967 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "db/db_impl/db_impl_secondary.h" + +#include + +#include "db/arena_wrapped_db_iter.h" +#include "db/merge_context.h" +#include "logging/auto_roll_logger.h" +#include "logging/logging.h" +#include "monitoring/perf_context_imp.h" +#include "rocksdb/configurable.h" +#include "util/cast_util.h" + +namespace ROCKSDB_NAMESPACE { + +#ifndef ROCKSDB_LITE +DBImplSecondary::DBImplSecondary(const DBOptions& db_options, + const std::string& dbname, + std::string secondary_path) + : DBImpl(db_options, dbname, false, true, true), + secondary_path_(std::move(secondary_path)) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Opening the db in secondary mode"); + LogFlush(immutable_db_options_.info_log); +} + +DBImplSecondary::~DBImplSecondary() {} + +Status DBImplSecondary::Recover( + const std::vector& column_families, + bool /*readonly*/, bool /*error_if_wal_file_exists*/, + bool /*error_if_data_exists_in_wals*/, uint64_t*, + RecoveryContext* /*recovery_ctx*/) { + mutex_.AssertHeld(); + + JobContext job_context(0); + Status s; + s = static_cast(versions_.get()) + ->Recover(column_families, &manifest_reader_, &manifest_reporter_, + &manifest_reader_status_); + if (!s.ok()) { + if (manifest_reader_status_) { + manifest_reader_status_->PermitUncheckedError(); + } + return s; + } + if (immutable_db_options_.paranoid_checks && s.ok()) { + s = CheckConsistency(); + } + // 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; + } + if (s.ok()) { + default_cf_handle_ = new ColumnFamilyHandleImpl( + versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_); + default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats(); + + std::unordered_set cfds_changed; + s = FindAndRecoverLogFiles(&cfds_changed, &job_context); + } + + if (s.IsPathNotFound()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Secondary tries to read WAL, but WAL file(s) have already " + "been purged by primary."); + s = Status::OK(); + } + // TODO: update options_file_number_ needed? + + job_context.Clean(); + return s; +} + +// find new WAL and apply them in order to the secondary instance +Status DBImplSecondary::FindAndRecoverLogFiles( + std::unordered_set* cfds_changed, + JobContext* job_context) { + assert(nullptr != cfds_changed); + assert(nullptr != job_context); + Status s; + std::vector logs; + s = FindNewLogNumbers(&logs); + if (s.ok() && !logs.empty()) { + SequenceNumber next_sequence(kMaxSequenceNumber); + s = RecoverLogFiles(logs, &next_sequence, cfds_changed, job_context); + } + return s; +} + +// List wal_dir and find all new WALs, return these log numbers +Status DBImplSecondary::FindNewLogNumbers(std::vector* logs) { + assert(logs != nullptr); + std::vector filenames; + Status s; + IOOptions io_opts; + io_opts.do_not_recurse = true; + s = immutable_db_options_.fs->GetChildren(immutable_db_options_.GetWalDir(), + io_opts, &filenames, + /*IODebugContext*=*/nullptr); + if (s.IsNotFound()) { + return Status::InvalidArgument("Failed to open wal_dir", + immutable_db_options_.GetWalDir()); + } else if (!s.ok()) { + return s; + } + + // if log_readers_ is non-empty, it means we have applied all logs with log + // numbers smaller than the smallest log in log_readers_, so there is no + // need to pass these logs to RecoverLogFiles + uint64_t log_number_min = 0; + if (!log_readers_.empty()) { + log_number_min = log_readers_.begin()->first; + } + for (size_t i = 0; i < filenames.size(); i++) { + uint64_t number; + FileType type; + if (ParseFileName(filenames[i], &number, &type) && type == kWalFile && + number >= log_number_min) { + logs->push_back(number); + } + } + // Recover logs in the order that they were generated + if (!logs->empty()) { + std::sort(logs->begin(), logs->end()); + } + return s; +} + +Status DBImplSecondary::MaybeInitLogReader( + uint64_t log_number, log::FragmentBufferedReader** log_reader) { + auto iter = log_readers_.find(log_number); + // make sure the log file is still present + if (iter == log_readers_.end() || + iter->second->reader_->GetLogNumber() != log_number) { + // delete the obsolete log reader if log number mismatch + if (iter != log_readers_.end()) { + log_readers_.erase(iter); + } + // initialize log reader from log_number + // TODO: min_log_number_to_keep_2pc check needed? + // Open the log file + std::string fname = + LogFileName(immutable_db_options_.GetWalDir(), log_number); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Recovering log #%" PRIu64 " mode %d", log_number, + static_cast(immutable_db_options_.wal_recovery_mode)); + + std::unique_ptr file_reader; + { + std::unique_ptr file; + Status status = fs_->NewSequentialFile( + fname, fs_->OptimizeForLogRead(file_options_), &file, nullptr); + if (!status.ok()) { + *log_reader = nullptr; + return status; + } + file_reader.reset(new SequentialFileReader( + std::move(file), fname, immutable_db_options_.log_readahead_size, + io_tracer_)); + } + + // Create the log reader. + LogReaderContainer* log_reader_container = new LogReaderContainer( + env_, immutable_db_options_.info_log, std::move(fname), + std::move(file_reader), log_number); + log_readers_.insert(std::make_pair( + log_number, std::unique_ptr(log_reader_container))); + } + iter = log_readers_.find(log_number); + assert(iter != log_readers_.end()); + *log_reader = iter->second->reader_; + return Status::OK(); +} + +// After manifest recovery, replay WALs and refresh log_readers_ if necessary +// REQUIRES: log_numbers are sorted in ascending order +Status DBImplSecondary::RecoverLogFiles( + const std::vector& log_numbers, SequenceNumber* next_sequence, + std::unordered_set* cfds_changed, + JobContext* job_context) { + assert(nullptr != cfds_changed); + assert(nullptr != job_context); + mutex_.AssertHeld(); + Status status; + for (auto log_number : log_numbers) { + log::FragmentBufferedReader* reader = nullptr; + status = MaybeInitLogReader(log_number, &reader); + if (!status.ok()) { + return status; + } + assert(reader != nullptr); + } + for (auto log_number : log_numbers) { + auto it = log_readers_.find(log_number); + assert(it != log_readers_.end()); + log::FragmentBufferedReader* reader = it->second->reader_; + Status* wal_read_status = it->second->status_; + assert(wal_read_status); + // Manually update the file number allocation counter in VersionSet. + versions_->MarkFileNumberUsed(log_number); + + // Determine if we should tolerate incomplete records at the tail end of the + // Read all the records and add to a memtable + std::string scratch; + Slice record; + WriteBatch batch; + + while (reader->ReadRecord(&record, &scratch, + immutable_db_options_.wal_recovery_mode) && + wal_read_status->ok() && status.ok()) { + if (record.size() < WriteBatchInternal::kHeader) { + reader->GetReporter()->Corruption( + record.size(), Status::Corruption("log record too small")); + continue; + } + status = WriteBatchInternal::SetContents(&batch, record); + if (!status.ok()) { + break; + } + SequenceNumber seq_of_batch = WriteBatchInternal::Sequence(&batch); + std::vector column_family_ids; + status = CollectColumnFamilyIdsFromWriteBatch(batch, &column_family_ids); + if (status.ok()) { + for (const auto id : column_family_ids) { + ColumnFamilyData* cfd = + versions_->GetColumnFamilySet()->GetColumnFamily(id); + if (cfd == nullptr) { + continue; + } + if (cfds_changed->count(cfd) == 0) { + cfds_changed->insert(cfd); + } + const std::vector& l0_files = + cfd->current()->storage_info()->LevelFiles(0); + SequenceNumber seq = + l0_files.empty() ? 0 : l0_files.back()->fd.largest_seqno; + // If the write batch's sequence number is smaller than the last + // sequence number of the largest sequence persisted for this column + // family, then its data must reside in an SST that has already been + // added in the prior MANIFEST replay. + if (seq_of_batch <= seq) { + continue; + } + auto curr_log_num = std::numeric_limits::max(); + if (cfd_to_current_log_.count(cfd) > 0) { + curr_log_num = cfd_to_current_log_[cfd]; + } + // If the active memtable contains records added by replaying an + // earlier WAL, then we need to seal the memtable, add it to the + // immutable memtable list and create a new active memtable. + 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); + cfd->mem()->SetNextLogNumber(log_number); + cfd->mem()->ConstructFragmentedRangeTombstones(); + cfd->imm()->Add(cfd->mem(), &job_context->memtables_to_free); + new_mem->Ref(); + cfd->SetMemtable(new_mem); + } + } + bool has_valid_writes = false; + status = WriteBatchInternal::InsertInto( + &batch, column_family_memtables_.get(), + nullptr /* flush_scheduler */, nullptr /* trim_history_scheduler*/, + true, log_number, this, false /* concurrent_memtable_writes */, + next_sequence, &has_valid_writes, seq_per_batch_, batch_per_txn_); + } + // If column family was not found, it might mean that the WAL write + // batch references to the column family that was dropped after the + // insert. We don't want to fail the whole write batch in that case -- + // we just ignore the update. + // That's why we set ignore missing column families to true + // passing null flush_scheduler will disable memtable flushing which is + // needed for secondary instances + if (status.ok()) { + for (const auto id : column_family_ids) { + ColumnFamilyData* cfd = + versions_->GetColumnFamilySet()->GetColumnFamily(id); + if (cfd == nullptr) { + continue; + } + std::unordered_map::iterator iter = + cfd_to_current_log_.find(cfd); + if (iter == cfd_to_current_log_.end()) { + cfd_to_current_log_.insert({cfd, log_number}); + } else if (log_number > iter->second) { + iter->second = log_number; + } + } + auto last_sequence = *next_sequence - 1; + if ((*next_sequence != kMaxSequenceNumber) && + (versions_->LastSequence() <= last_sequence)) { + versions_->SetLastAllocatedSequence(last_sequence); + versions_->SetLastPublishedSequence(last_sequence); + versions_->SetLastSequence(last_sequence); + } + } else { + // We are treating this as a failure while reading since we read valid + // blocks that do not form coherent data + reader->GetReporter()->Corruption(record.size(), status); + } + } + if (status.ok() && !wal_read_status->ok()) { + status = *wal_read_status; + } + if (!status.ok()) { + return status; + } + } + // remove logreaders from map after successfully recovering the WAL + if (log_readers_.size() > 1) { + auto erase_iter = log_readers_.begin(); + std::advance(erase_iter, log_readers_.size() - 1); + log_readers_.erase(log_readers_.begin(), erase_iter); + } + return status; +} + +// Implementation of the DB interface +Status DBImplSecondary::Get(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value) { + return GetImpl(read_options, column_family, key, value, + /*timestamp*/ nullptr); +} + +Status DBImplSecondary::Get(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableSlice* value, std::string* timestamp) { + return GetImpl(read_options, column_family, key, value, timestamp); +} + +Status DBImplSecondary::GetImpl(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* pinnable_val, + std::string* timestamp) { + assert(pinnable_val != nullptr); + PERF_CPU_TIMER_GUARD(get_cpu_nanos, immutable_db_options_.clock); + StopWatch sw(immutable_db_options_.clock, stats_, DB_GET); + PERF_TIMER_GUARD(get_snapshot_time); + + assert(column_family); + if (read_options.timestamp) { + const Status s = FailIfTsMismatchCf( + column_family, *(read_options.timestamp), /*ts_for_read=*/true); + if (!s.ok()) { + return s; + } + } else { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + } + + // Clear the timestamp for returning results so that we can distinguish + // between tombstone or key that has never been written later. + if (timestamp) { + timestamp->clear(); + } + + auto cfh = static_cast(column_family); + ColumnFamilyData* cfd = cfh->cfd(); + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_) { + tracer_->Get(column_family, key); + } + } + // Acquire SuperVersion + SuperVersion* super_version = GetAndRefSuperVersion(cfd); + SequenceNumber snapshot = versions_->LastSequence(); + GetWithTimestampReadCallback read_cb(snapshot); + MergeContext merge_context; + SequenceNumber max_covering_tombstone_seq = 0; + Status s; + LookupKey lkey(key, snapshot, read_options.timestamp); + PERF_TIMER_STOP(get_snapshot_time); + + bool done = false; + const Comparator* ucmp = column_family->GetComparator(); + assert(ucmp); + std::string* ts = ucmp->timestamp_size() > 0 ? timestamp : nullptr; + if (super_version->mem->Get(lkey, pinnable_val->GetSelf(), + /*columns=*/nullptr, ts, &s, &merge_context, + &max_covering_tombstone_seq, read_options, + false /* immutable_memtable */, &read_cb)) { + done = true; + pinnable_val->PinSelf(); + RecordTick(stats_, MEMTABLE_HIT); + } else if ((s.ok() || s.IsMergeInProgress()) && + super_version->imm->Get( + lkey, pinnable_val->GetSelf(), /*columns=*/nullptr, ts, &s, + &merge_context, &max_covering_tombstone_seq, read_options, + &read_cb)) { + done = true; + pinnable_val->PinSelf(); + RecordTick(stats_, MEMTABLE_HIT); + } + if (!done && !s.ok() && !s.IsMergeInProgress()) { + ReturnAndCleanupSuperVersion(cfd, super_version); + return s; + } + if (!done) { + PERF_TIMER_GUARD(get_from_output_files_time); + PinnedIteratorsManager pinned_iters_mgr; + super_version->current->Get( + read_options, lkey, pinnable_val, /*columns=*/nullptr, ts, &s, + &merge_context, &max_covering_tombstone_seq, &pinned_iters_mgr, + /*value_found*/ nullptr, + /*key_exists*/ nullptr, /*seq*/ nullptr, &read_cb, /*is_blob*/ nullptr, + /*do_merge*/ true); + RecordTick(stats_, MEMTABLE_MISS); + } + { + PERF_TIMER_GUARD(get_post_process_time); + ReturnAndCleanupSuperVersion(cfd, super_version); + RecordTick(stats_, NUMBER_KEYS_READ); + size_t size = pinnable_val->size(); + RecordTick(stats_, BYTES_READ, size); + RecordTimeToHistogram(stats_, BYTES_PER_READ, size); + PERF_COUNTER_ADD(get_read_bytes, size); + } + return s; +} + +Iterator* DBImplSecondary::NewIterator(const ReadOptions& read_options, + ColumnFamilyHandle* column_family) { + if (read_options.managed) { + return NewErrorIterator( + Status::NotSupported("Managed iterator is not supported anymore.")); + } + if (read_options.read_tier == kPersistedTier) { + return NewErrorIterator(Status::NotSupported( + "ReadTier::kPersistedData is not yet supported in iterators.")); + } + + assert(column_family); + if (read_options.timestamp) { + const Status s = FailIfTsMismatchCf( + column_family, *(read_options.timestamp), /*ts_for_read=*/true); + if (!s.ok()) { + return NewErrorIterator(s); + } + } else { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return NewErrorIterator(s); + } + } + + Iterator* result = nullptr; + auto cfh = static_cast_with_check(column_family); + auto cfd = cfh->cfd(); + ReadCallback* read_callback = nullptr; // No read callback provided. + if (read_options.tailing) { + return NewErrorIterator(Status::NotSupported( + "tailing iterator not supported in secondary mode")); + } else if (read_options.snapshot != nullptr) { + // TODO (yanqin) support snapshot. + return NewErrorIterator( + Status::NotSupported("snapshot not supported in secondary mode")); + } else { + SequenceNumber snapshot(kMaxSequenceNumber); + result = NewIteratorImpl(read_options, cfd, snapshot, read_callback); + } + return result; +} + +ArenaWrappedDBIter* DBImplSecondary::NewIteratorImpl( + const ReadOptions& read_options, ColumnFamilyData* cfd, + SequenceNumber snapshot, ReadCallback* read_callback, + bool expose_blob_index, bool allow_refresh) { + assert(nullptr != cfd); + SuperVersion* super_version = cfd->GetReferencedSuperVersion(this); + assert(snapshot == kMaxSequenceNumber); + snapshot = versions_->LastSequence(); + assert(snapshot != kMaxSequenceNumber); + auto db_iter = NewArenaWrappedDbIterator( + env_, read_options, *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, this, cfd, + expose_blob_index, read_options.snapshot ? false : allow_refresh); + auto internal_iter = NewInternalIterator( + db_iter->GetReadOptions(), cfd, super_version, db_iter->GetArena(), + snapshot, /* allow_unprepared_value */ true, db_iter); + db_iter->SetIterUnderDBIter(internal_iter); + return db_iter; +} + +Status DBImplSecondary::NewIterators( + const ReadOptions& read_options, + const std::vector& column_families, + std::vector* iterators) { + if (read_options.managed) { + return Status::NotSupported("Managed iterator is not supported anymore."); + } + if (read_options.read_tier == kPersistedTier) { + return Status::NotSupported( + "ReadTier::kPersistedData is not yet supported in iterators."); + } + ReadCallback* read_callback = nullptr; // No read callback provided. + if (iterators == nullptr) { + return Status::InvalidArgument("iterators not allowed to be nullptr"); + } + + if (read_options.timestamp) { + for (auto* cf : column_families) { + assert(cf); + const Status s = FailIfTsMismatchCf(cf, *(read_options.timestamp), + /*ts_for_read=*/true); + if (!s.ok()) { + return s; + } + } + } else { + for (auto* cf : column_families) { + assert(cf); + const Status s = FailIfCfHasTs(cf); + if (!s.ok()) { + return s; + } + } + } + iterators->clear(); + iterators->reserve(column_families.size()); + if (read_options.tailing) { + return Status::NotSupported( + "tailing iterator not supported in secondary mode"); + } else if (read_options.snapshot != nullptr) { + // TODO (yanqin) support snapshot. + return Status::NotSupported("snapshot not supported in secondary mode"); + } else { + SequenceNumber read_seq(kMaxSequenceNumber); + for (auto cfh : column_families) { + ColumnFamilyData* cfd = static_cast(cfh)->cfd(); + iterators->push_back( + NewIteratorImpl(read_options, cfd, read_seq, read_callback)); + } + } + return Status::OK(); +} + +Status DBImplSecondary::CheckConsistency() { + mutex_.AssertHeld(); + Status s = DBImpl::CheckConsistency(); + // If DBImpl::CheckConsistency() which is stricter returns success, then we + // do not need to give a second chance. + if (s.ok()) { + return s; + } + // It's possible that DBImpl::CheckConssitency() can fail because the primary + // may have removed certain files, causing the GetFileSize(name) call to + // fail and returning a PathNotFound. In this case, we take a best-effort + // approach and just proceed. + TEST_SYNC_POINT_CALLBACK( + "DBImplSecondary::CheckConsistency:AfterFirstAttempt", &s); + + if (immutable_db_options_.skip_checking_sst_file_sizes_on_db_open) { + return Status::OK(); + } + + std::vector metadata; + versions_->GetLiveFilesMetaData(&metadata); + + std::string corruption_messages; + for (const auto& md : metadata) { + // md.name has a leading "/". + std::string file_path = md.db_path + md.name; + + uint64_t fsize = 0; + s = env_->GetFileSize(file_path, &fsize); + if (!s.ok() && + (env_->GetFileSize(Rocks2LevelTableFileName(file_path), &fsize).ok() || + s.IsPathNotFound())) { + s = Status::OK(); + } + if (!s.ok()) { + corruption_messages += + "Can't access " + md.name + ": " + s.ToString() + "\n"; + } + } + return corruption_messages.empty() ? Status::OK() + : Status::Corruption(corruption_messages); +} + +Status DBImplSecondary::TryCatchUpWithPrimary() { + assert(versions_.get() != nullptr); + assert(manifest_reader_.get() != nullptr); + Status s; + // read the manifest and apply new changes to the secondary instance + std::unordered_set cfds_changed; + JobContext job_context(0, true /*create_superversion*/); + { + InstrumentedMutexLock lock_guard(&mutex_); + s = static_cast_with_check(versions_.get()) + ->ReadAndApply(&mutex_, &manifest_reader_, + manifest_reader_status_.get(), &cfds_changed); + + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Last sequence is %" PRIu64, + static_cast(versions_->LastSequence())); + for (ColumnFamilyData* cfd : cfds_changed) { + if (cfd->IsDropped()) { + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] is dropped\n", + cfd->GetName().c_str()); + continue; + } + VersionStorageInfo::LevelSummaryStorage tmp; + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[%s] Level summary: %s\n", cfd->GetName().c_str(), + cfd->current()->storage_info()->LevelSummary(&tmp)); + } + + // list wal_dir to discover new WALs and apply new changes to the secondary + // instance + if (s.ok()) { + s = FindAndRecoverLogFiles(&cfds_changed, &job_context); + } + if (s.IsPathNotFound()) { + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "Secondary tries to read WAL, but WAL file(s) have already " + "been purged by primary."); + s = Status::OK(); + } + if (s.ok()) { + for (auto cfd : cfds_changed) { + cfd->imm()->RemoveOldMemTables(cfd->GetLogNumber(), + &job_context.memtables_to_free); + auto& sv_context = job_context.superversion_contexts.back(); + cfd->InstallSuperVersion(&sv_context, &mutex_); + sv_context.NewSuperVersion(); + } + } + } + job_context.Clean(); + + // Cleanup unused, obsolete files. + JobContext purge_files_job_context(0); + { + InstrumentedMutexLock lock_guard(&mutex_); + // Currently, secondary instance does not own the database files, thus it + // is unnecessary for the secondary to force full scan. + FindObsoleteFiles(&purge_files_job_context, /*force=*/false); + } + if (purge_files_job_context.HaveSomethingToDelete()) { + PurgeObsoleteFiles(purge_files_job_context); + } + purge_files_job_context.Clean(); + return s; +} + +Status DB::OpenAsSecondary(const Options& options, const std::string& dbname, + const std::string& secondary_path, DB** dbptr) { + *dbptr = nullptr; + + DBOptions db_options(options); + ColumnFamilyOptions cf_options(options); + std::vector column_families; + column_families.emplace_back(kDefaultColumnFamilyName, cf_options); + std::vector handles; + + Status s = DB::OpenAsSecondary(db_options, dbname, secondary_path, + column_families, &handles, dbptr); + if (s.ok()) { + assert(handles.size() == 1); + delete handles[0]; + } + return s; +} + +Status DB::OpenAsSecondary( + const DBOptions& db_options, const std::string& dbname, + const std::string& secondary_path, + const std::vector& column_families, + std::vector* handles, DB** dbptr) { + *dbptr = nullptr; + + DBOptions tmp_opts(db_options); + Status s; + if (nullptr == tmp_opts.info_log) { + s = CreateLoggerFromOptions(secondary_path, tmp_opts, &tmp_opts.info_log); + if (!s.ok()) { + tmp_opts.info_log = nullptr; + return s; + } + } + + assert(tmp_opts.info_log != nullptr); + if (db_options.max_open_files != -1) { + std::ostringstream oss; + oss << "The primary instance may delete all types of files after they " + "become obsolete. The application can coordinate the primary and " + "secondary so that primary does not delete/rename files that are " + "currently being used by the secondary. Alternatively, a custom " + "Env/FS can be provided such that files become inaccessible only " + "after all primary and secondaries indicate that they are obsolete " + "and deleted. If the above two are not possible, you can open the " + "secondary instance with `max_open_files==-1` so that secondary " + "will eagerly keep all table files open. Even if a file is deleted, " + "its content can still be accessed via a prior open file " + "descriptor. This is a hacky workaround for only table files. If " + "none of the above is done, then point lookup or " + "range scan via the secondary instance can result in IOError: file " + "not found. This can be resolved by retrying " + "TryCatchUpWithPrimary()."; + ROCKS_LOG_WARN(tmp_opts.info_log, "%s", oss.str().c_str()); + } + + handles->clear(); + DBImplSecondary* impl = new DBImplSecondary(tmp_opts, dbname, secondary_path); + impl->versions_.reset(new ReactiveVersionSet( + dbname, &impl->immutable_db_options_, impl->file_options_, + impl->table_cache_.get(), impl->write_buffer_manager_, + &impl->write_controller_, impl->io_tracer_)); + impl->column_family_memtables_.reset( + new ColumnFamilyMemTablesImpl(impl->versions_->GetColumnFamilySet())); + impl->wal_in_db_path_ = impl->immutable_db_options_.IsWalDirSameAsDBPath(); + + impl->mutex_.Lock(); + s = impl->Recover(column_families, true, false, false); + if (s.ok()) { + for (auto cf : column_families) { + auto cfd = + impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name); + if (nullptr == cfd) { + s = Status::InvalidArgument("Column family not found", cf.name); + break; + } + handles->push_back(new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_)); + } + } + SuperVersionContext sv_context(true /* create_superversion */); + if (s.ok()) { + for (auto cfd : *impl->versions_->GetColumnFamilySet()) { + sv_context.NewSuperVersion(); + cfd->InstallSuperVersion(&sv_context, &impl->mutex_); + } + } + impl->mutex_.Unlock(); + sv_context.Clean(); + if (s.ok()) { + *dbptr = impl; + for (auto h : *handles) { + impl->NewThreadStatusCfInfo( + static_cast_with_check(h)->cfd()); + } + } else { + for (auto h : *handles) { + delete h; + } + handles->clear(); + delete impl; + } + return s; +} + +Status DBImplSecondary::CompactWithoutInstallation( + const OpenAndCompactOptions& options, ColumnFamilyHandle* cfh, + const CompactionServiceInput& input, CompactionServiceResult* result) { + if (options.canceled && options.canceled->load(std::memory_order_acquire)) { + return Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + InstrumentedMutexLock l(&mutex_); + auto cfd = static_cast_with_check(cfh)->cfd(); + if (!cfd) { + return Status::InvalidArgument("Cannot find column family" + + cfh->GetName()); + } + + std::unordered_set input_set; + for (const auto& file_name : input.input_files) { + input_set.insert(TableFileNameToNumber(file_name)); + } + + auto* version = cfd->current(); + + 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); + + std::vector input_files; + Status s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers( + &input_files, &input_set, vstorage, comp_options); + if (!s.ok()) { + return s; + } + + std::unique_ptr c; + 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)); + assert(c != nullptr); + + c->SetInputVersion(version); + + // Create output directory if it's not existed yet + std::unique_ptr output_dir; + s = CreateAndNewDirectory(fs_.get(), secondary_path_, &output_dir); + if (!s.ok()) { + return s; + } + + LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, + immutable_db_options_.info_log.get()); + + const int job_id = next_job_id_.fetch_add(1); + + // use primary host's db_id for running the compaction, but db_session_id is + // using the local one, which is to make sure the unique id is unique from + // the remote compactors. Because the id is generated from db_id, + // db_session_id and orig_file_number, unlike the local compaction, remote + // compaction cannot guarantee the uniqueness of orig_file_number, the file + // number is only assigned when compaction is done. + CompactionServiceCompactionJob compaction_job( + job_id, c.get(), immutable_db_options_, mutable_db_options_, + file_options_for_compaction_, versions_.get(), &shutting_down_, + &log_buffer, output_dir.get(), stats_, &mutex_, &error_handler_, + input.snapshots, table_cache_, &event_logger_, dbname_, io_tracer_, + options.canceled ? *options.canceled : kManualCompactionCanceledFalse_, + input.db_id, db_session_id_, secondary_path_, input, result); + + mutex_.Unlock(); + s = compaction_job.Run(); + mutex_.Lock(); + + // clean up + compaction_job.io_status().PermitUncheckedError(); + compaction_job.CleanupCompaction(); + c->ReleaseCompactionFiles(s); + c.reset(); + + TEST_SYNC_POINT_CALLBACK("DBImplSecondary::CompactWithoutInstallation::End", + &s); + result->status = s; + return s; +} + +Status DB::OpenAndCompact( + const OpenAndCompactOptions& options, const std::string& name, + const std::string& output_directory, const std::string& input, + std::string* output, + const CompactionServiceOptionsOverride& override_options) { + if (options.canceled && options.canceled->load(std::memory_order_acquire)) { + return Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + CompactionServiceInput compaction_input; + Status s = CompactionServiceInput::Read(input, &compaction_input); + if (!s.ok()) { + return s; + } + + compaction_input.db_options.max_open_files = -1; + compaction_input.db_options.compaction_service = nullptr; + if (compaction_input.db_options.statistics) { + compaction_input.db_options.statistics.reset(); + } + compaction_input.db_options.env = override_options.env; + compaction_input.db_options.file_checksum_gen_factory = + override_options.file_checksum_gen_factory; + compaction_input.db_options.statistics = override_options.statistics; + compaction_input.column_family.options.comparator = + override_options.comparator; + compaction_input.column_family.options.merge_operator = + override_options.merge_operator; + compaction_input.column_family.options.compaction_filter = + override_options.compaction_filter; + compaction_input.column_family.options.compaction_filter_factory = + override_options.compaction_filter_factory; + compaction_input.column_family.options.prefix_extractor = + override_options.prefix_extractor; + compaction_input.column_family.options.table_factory = + override_options.table_factory; + compaction_input.column_family.options.sst_partitioner_factory = + override_options.sst_partitioner_factory; + compaction_input.column_family.options.table_properties_collector_factories = + override_options.table_properties_collector_factories; + compaction_input.db_options.listeners = override_options.listeners; + + std::vector column_families; + column_families.push_back(compaction_input.column_family); + // TODO: we have to open default CF, because of an implementation limitation, + // currently we just use the same CF option from input, which is not collect + // and open may fail. + if (compaction_input.column_family.name != kDefaultColumnFamilyName) { + column_families.emplace_back(kDefaultColumnFamilyName, + compaction_input.column_family.options); + } + + DB* db; + std::vector handles; + + s = DB::OpenAsSecondary(compaction_input.db_options, name, output_directory, + column_families, &handles, &db); + if (!s.ok()) { + return s; + } + + CompactionServiceResult compaction_result; + DBImplSecondary* db_secondary = static_cast_with_check(db); + assert(handles.size() > 0); + s = db_secondary->CompactWithoutInstallation( + options, handles[0], compaction_input, &compaction_result); + + Status serialization_status = compaction_result.Write(output); + + for (auto& handle : handles) { + delete handle; + } + delete db; + if (s.ok()) { + return serialization_status; + } + return s; +} + +Status DB::OpenAndCompact( + const std::string& name, const std::string& output_directory, + const std::string& input, std::string* output, + const CompactionServiceOptionsOverride& override_options) { + return OpenAndCompact(OpenAndCompactOptions(), name, output_directory, input, + output, override_options); +} + +#else // !ROCKSDB_LITE + +Status DB::OpenAsSecondary(const Options& /*options*/, + const std::string& /*name*/, + const std::string& /*secondary_path*/, + DB** /*dbptr*/) { + return Status::NotSupported("Not supported in ROCKSDB_LITE."); +} + +Status DB::OpenAsSecondary( + const DBOptions& /*db_options*/, const std::string& /*dbname*/, + const std::string& /*secondary_path*/, + const std::vector& /*column_families*/, + std::vector* /*handles*/, DB** /*dbptr*/) { + return Status::NotSupported("Not supported in ROCKSDB_LITE."); +} +#endif // !ROCKSDB_LITE + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/db/db_impl/db_impl_secondary.h b/src/rocksdb/db/db_impl/db_impl_secondary.h new file mode 100644 index 000000000..eb9361875 --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_secondary.h @@ -0,0 +1,410 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#ifndef ROCKSDB_LITE + +#include +#include + +#include "db/db_impl/db_impl.h" +#include "logging/logging.h" + +namespace ROCKSDB_NAMESPACE { + +// A wrapper class to hold log reader, log reporter, log status. +class LogReaderContainer { + public: + LogReaderContainer() + : reader_(nullptr), reporter_(nullptr), status_(nullptr) {} + LogReaderContainer(Env* env, std::shared_ptr info_log, + std::string fname, + std::unique_ptr&& file_reader, + uint64_t log_number) { + LogReporter* reporter = new LogReporter(); + status_ = new Status(); + reporter->env = env; + reporter->info_log = info_log.get(); + reporter->fname = std::move(fname); + reporter->status = status_; + reporter_ = reporter; + // We intentially make log::Reader do checksumming even if + // paranoid_checks==false so that corruptions cause entire commits + // to be skipped instead of propagating bad information (like overly + // large sequence numbers). + reader_ = new log::FragmentBufferedReader(info_log, std::move(file_reader), + reporter, true /*checksum*/, + log_number); + } + log::FragmentBufferedReader* reader_; + log::Reader::Reporter* reporter_; + Status* status_; + ~LogReaderContainer() { + delete reader_; + delete reporter_; + delete status_; + } + + private: + struct LogReporter : public log::Reader::Reporter { + Env* env; + Logger* info_log; + std::string fname; + Status* status; // nullptr if immutable_db_options_.paranoid_checks==false + void Corruption(size_t bytes, const Status& s) override { + ROCKS_LOG_WARN(info_log, "%s%s: dropping %d bytes; %s", + (this->status == nullptr ? "(ignoring error) " : ""), + fname.c_str(), static_cast(bytes), + s.ToString().c_str()); + if (this->status != nullptr && this->status->ok()) { + *this->status = s; + } + } + }; +}; + +// The secondary instance shares access to the storage as the primary. +// The secondary is able to read and replay changes described in both the +// MANIFEST and the WAL files without coordination with the primary. +// The secondary instance can be opened using `DB::OpenAsSecondary`. After +// that, it can call `DBImplSecondary::TryCatchUpWithPrimary` to make best +// effort attempts to catch up with the primary. +// TODO: Share common structure with CompactedDBImpl and DBImplReadOnly +class DBImplSecondary : public DBImpl { + public: + DBImplSecondary(const DBOptions& options, const std::string& dbname, + std::string secondary_path); + ~DBImplSecondary() override; + + // Recover by replaying MANIFEST and WAL. Also initialize manifest_reader_ + // and log_readers_ to facilitate future operations. + Status Recover(const std::vector& column_families, + bool read_only, bool error_if_wal_file_exists, + bool error_if_data_exists_in_wals, uint64_t* = nullptr, + RecoveryContext* recovery_ctx = nullptr) override; + + // Implementations of the DB interface. + using DB::Get; + // Can return IOError due to files being deleted by the primary. To avoid + // IOError in this case, application can coordinate between primary and + // secondaries so that primary will not delete files that are currently being + // used by the secondaries. The application can also provide a custom FS/Env + // implementation so that files will remain present until all primary and + // secondaries indicate that they can be deleted. As a partial hacky + // workaround, the secondaries can be opened with `max_open_files=-1` so that + // it eagerly keeps all talbe files open and is able to access the contents of + // deleted files via prior open fd. + Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value) override; + + Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value, + std::string* timestamp) override; + + Status GetImpl(const ReadOptions& options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableSlice* value, + std::string* timestamp); + + using DBImpl::NewIterator; + // Operations on the created iterators can return IOError due to files being + // deleted by the primary. To avoid IOError in this case, application can + // coordinate between primary and secondaries so that primary will not delete + // files that are currently being used by the secondaries. The application can + // also provide a custom FS/Env implementation so that files will remain + // present until all primary and secondaries indicate that they can be + // deleted. As a partial hacky workaround, the secondaries can be opened with + // `max_open_files=-1` so that it eagerly keeps all talbe files open and is + // able to access the contents of deleted files via prior open fd. + Iterator* NewIterator(const ReadOptions&, + ColumnFamilyHandle* column_family) override; + + ArenaWrappedDBIter* NewIteratorImpl(const ReadOptions& read_options, + ColumnFamilyData* cfd, + SequenceNumber snapshot, + ReadCallback* read_callback, + bool expose_blob_index = false, + bool allow_refresh = true); + + Status NewIterators(const ReadOptions& options, + const std::vector& column_families, + std::vector* iterators) override; + + using DBImpl::Put; + Status Put(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, const Slice& /*key*/, + const Slice& /*value*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::PutEntity; + Status PutEntity(const WriteOptions& /* options */, + ColumnFamilyHandle* /* column_family */, + const Slice& /* key */, + const WideColumns& /* columns */) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::Merge; + Status Merge(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, const Slice& /*key*/, + const Slice& /*value*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::Delete; + Status Delete(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::SingleDelete; + Status SingleDelete(const WriteOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice& /*key*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + Status Write(const WriteOptions& /*options*/, + WriteBatch* /*updates*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::CompactRange; + Status CompactRange(const CompactRangeOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/, + const Slice* /*begin*/, const Slice* /*end*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::CompactFiles; + Status CompactFiles( + const CompactionOptions& /*compact_options*/, + ColumnFamilyHandle* /*column_family*/, + const std::vector& /*input_file_names*/, + const int /*output_level*/, const int /*output_path_id*/ = -1, + std::vector* const /*output_file_names*/ = nullptr, + CompactionJobInfo* /*compaction_job_info*/ = nullptr) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + Status DisableFileDeletions() override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + Status EnableFileDeletions(bool /*force*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + Status GetLiveFiles(std::vector&, + uint64_t* /*manifest_file_size*/, + bool /*flush_memtable*/ = true) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::Flush; + Status Flush(const FlushOptions& /*options*/, + ColumnFamilyHandle* /*column_family*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::SetDBOptions; + Status SetDBOptions(const std::unordered_map& + /*options_map*/) override { + // Currently not supported because changing certain options may cause + // flush/compaction. + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::SetOptions; + Status SetOptions( + ColumnFamilyHandle* /*cfd*/, + const std::unordered_map& /*options_map*/) + override { + // Currently not supported because changing certain options may cause + // flush/compaction and/or write to MANIFEST. + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DBImpl::SyncWAL; + Status SyncWAL() override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + using DB::IngestExternalFile; + Status IngestExternalFile( + ColumnFamilyHandle* /*column_family*/, + const std::vector& /*external_files*/, + const IngestExternalFileOptions& /*ingestion_options*/) override { + return Status::NotSupported("Not supported operation in secondary mode."); + } + + // Try to catch up with the primary by reading as much as possible from the + // log files until there is nothing more to read or encounters an error. If + // the amount of information in the log files to process is huge, this + // method can take long time due to all the I/O and CPU costs. + Status TryCatchUpWithPrimary() override; + + // Try to find log reader using log_number from log_readers_ map, initialize + // if it doesn't exist + Status MaybeInitLogReader(uint64_t log_number, + log::FragmentBufferedReader** log_reader); + + // Check if all live files exist on file system and that their file sizes + // matche to the in-memory records. It is possible that some live files may + // have been deleted by the primary. In this case, CheckConsistency() does + // not flag the missing file as inconsistency. + Status CheckConsistency() override; + +#ifndef NDEBUG + Status TEST_CompactWithoutInstallation(const OpenAndCompactOptions& options, + ColumnFamilyHandle* cfh, + const CompactionServiceInput& input, + CompactionServiceResult* result) { + return CompactWithoutInstallation(options, cfh, input, result); + } +#endif // NDEBUG + + protected: +#ifndef ROCKSDB_LITE + Status FlushForGetLiveFiles() override { + // No-op for read-only DB + return Status::OK(); + } +#endif // !ROCKSDB_LITE + + // ColumnFamilyCollector is a write batch handler which does nothing + // except recording unique column family IDs + class ColumnFamilyCollector : public WriteBatch::Handler { + std::unordered_set column_family_ids_; + + Status AddColumnFamilyId(uint32_t column_family_id) { + if (column_family_ids_.find(column_family_id) == + column_family_ids_.end()) { + column_family_ids_.insert(column_family_id); + } + return Status::OK(); + } + + public: + explicit ColumnFamilyCollector() {} + + ~ColumnFamilyCollector() override {} + + Status PutCF(uint32_t column_family_id, const Slice&, + const Slice&) override { + return AddColumnFamilyId(column_family_id); + } + + Status DeleteCF(uint32_t column_family_id, const Slice&) override { + return AddColumnFamilyId(column_family_id); + } + + Status SingleDeleteCF(uint32_t column_family_id, const Slice&) override { + return AddColumnFamilyId(column_family_id); + } + + Status DeleteRangeCF(uint32_t column_family_id, const Slice&, + const Slice&) override { + return AddColumnFamilyId(column_family_id); + } + + Status MergeCF(uint32_t column_family_id, const Slice&, + const Slice&) override { + return AddColumnFamilyId(column_family_id); + } + + Status PutBlobIndexCF(uint32_t column_family_id, const Slice&, + const Slice&) override { + return AddColumnFamilyId(column_family_id); + } + + Status MarkBeginPrepare(bool) override { return Status::OK(); } + + Status MarkEndPrepare(const Slice&) override { return Status::OK(); } + + Status MarkRollback(const Slice&) override { return Status::OK(); } + + Status MarkCommit(const Slice&) override { return Status::OK(); } + + Status MarkCommitWithTimestamp(const Slice&, const Slice&) override { + return Status::OK(); + } + + Status MarkNoop(bool) override { return Status::OK(); } + + const std::unordered_set& column_families() const { + return column_family_ids_; + } + }; + + Status CollectColumnFamilyIdsFromWriteBatch( + const WriteBatch& batch, std::vector* column_family_ids) { + assert(column_family_ids != nullptr); + column_family_ids->clear(); + ColumnFamilyCollector handler; + Status s = batch.Iterate(&handler); + if (s.ok()) { + for (const auto& cf : handler.column_families()) { + column_family_ids->push_back(cf); + } + } + return s; + } + + bool OwnTablesAndLogs() const override { + // Currently, the secondary instance does not own the database files. It + // simply opens the files of the primary instance and tracks their file + // descriptors until they become obsolete. In the future, the secondary may + // create links to database files. OwnTablesAndLogs will return true then. + return false; + } + + private: + friend class DB; + + // No copying allowed + DBImplSecondary(const DBImplSecondary&); + void operator=(const DBImplSecondary&); + + using DBImpl::Recover; + + Status FindAndRecoverLogFiles( + std::unordered_set* cfds_changed, + JobContext* job_context); + Status FindNewLogNumbers(std::vector* logs); + // After manifest recovery, replay WALs and refresh log_readers_ if necessary + // REQUIRES: log_numbers are sorted in ascending order + Status RecoverLogFiles(const std::vector& log_numbers, + SequenceNumber* next_sequence, + std::unordered_set* cfds_changed, + JobContext* job_context); + + // Run compaction without installation, the output files will be placed in the + // secondary DB path. The LSM tree won't be changed, the secondary DB is still + // in read-only mode. + Status CompactWithoutInstallation(const OpenAndCompactOptions& options, + ColumnFamilyHandle* cfh, + const CompactionServiceInput& input, + CompactionServiceResult* result); + + std::unique_ptr manifest_reader_; + std::unique_ptr manifest_reporter_; + std::unique_ptr manifest_reader_status_; + + // Cache log readers for each log number, used for continue WAL replay + // after recovery + std::map> log_readers_; + + // Current WAL number replayed for each column family. + std::unordered_map cfd_to_current_log_; + + const std::string secondary_path_; +}; + +} // namespace ROCKSDB_NAMESPACE + +#endif // !ROCKSDB_LITE diff --git a/src/rocksdb/db/db_impl/db_impl_write.cc b/src/rocksdb/db/db_impl/db_impl_write.cc new file mode 100644 index 000000000..a597c168d --- /dev/null +++ b/src/rocksdb/db/db_impl/db_impl_write.cc @@ -0,0 +1,2435 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +#include + +#include "db/db_impl/db_impl.h" +#include "db/error_handler.h" +#include "db/event_helpers.h" +#include "logging/logging.h" +#include "monitoring/perf_context_imp.h" +#include "options/options_helper.h" +#include "test_util/sync_point.h" +#include "util/cast_util.h" + +namespace ROCKSDB_NAMESPACE { +// Convenience methods +Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& val) { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + return DB::Put(o, column_family, key, val); +} + +Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts, const Slice& val) { + const Status s = FailIfTsMismatchCf(column_family, ts, /*ts_for_read=*/false); + if (!s.ok()) { + return s; + } + return DB::Put(o, column_family, key, ts, val); +} + +Status DBImpl::PutEntity(const WriteOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns) { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + + return DB::PutEntity(options, column_family, key, columns); +} + +Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& val) { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + auto cfh = static_cast_with_check(column_family); + 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); + } +} + +Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts, const Slice& val) { + const Status s = FailIfTsMismatchCf(column_family, ts, /*ts_for_read=*/false); + if (!s.ok()) { + return s; + } + return DB::Merge(o, column_family, key, ts, val); +} + +Status DBImpl::Delete(const WriteOptions& write_options, + ColumnFamilyHandle* column_family, const Slice& key) { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + return DB::Delete(write_options, column_family, key); +} + +Status DBImpl::Delete(const WriteOptions& write_options, + ColumnFamilyHandle* column_family, const Slice& key, + const Slice& ts) { + const Status s = FailIfTsMismatchCf(column_family, ts, /*ts_for_read=*/false); + if (!s.ok()) { + return s; + } + return DB::Delete(write_options, column_family, key, ts); +} + +Status DBImpl::SingleDelete(const WriteOptions& write_options, + ColumnFamilyHandle* column_family, + const Slice& key) { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + return DB::SingleDelete(write_options, column_family, key); +} + +Status DBImpl::SingleDelete(const WriteOptions& write_options, + ColumnFamilyHandle* column_family, const Slice& key, + const Slice& ts) { + const Status s = FailIfTsMismatchCf(column_family, ts, /*ts_for_read=*/false); + if (!s.ok()) { + return s; + } + return DB::SingleDelete(write_options, column_family, key, ts); +} + +Status DBImpl::DeleteRange(const WriteOptions& write_options, + ColumnFamilyHandle* column_family, + const Slice& begin_key, const Slice& end_key) { + const Status s = FailIfCfHasTs(column_family); + if (!s.ok()) { + return s; + } + return DB::DeleteRange(write_options, column_family, begin_key, end_key); +} + +Status DBImpl::DeleteRange(const WriteOptions& write_options, + ColumnFamilyHandle* column_family, + const Slice& begin_key, const Slice& end_key, + const Slice& ts) { + const Status s = FailIfTsMismatchCf(column_family, ts, /*ts_for_read=*/false); + if (!s.ok()) { + return s; + } + return DB::DeleteRange(write_options, column_family, begin_key, end_key, ts); +} + +void DBImpl::SetRecoverableStatePreReleaseCallback( + PreReleaseCallback* callback) { + recoverable_state_pre_release_callback_.reset(callback); +} + +Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) { + Status s; + if (write_options.protection_bytes_per_key > 0) { + s = WriteBatchInternal::UpdateProtectionInfo( + my_batch, write_options.protection_bytes_per_key); + } + if (s.ok()) { + s = WriteImpl(write_options, my_batch, /*callback=*/nullptr, + /*log_used=*/nullptr); + } + return s; +} + +#ifndef ROCKSDB_LITE +Status DBImpl::WriteWithCallback(const WriteOptions& write_options, + WriteBatch* my_batch, + WriteCallback* callback) { + Status s; + if (write_options.protection_bytes_per_key > 0) { + s = WriteBatchInternal::UpdateProtectionInfo( + my_batch, write_options.protection_bytes_per_key); + } + if (s.ok()) { + s = WriteImpl(write_options, my_batch, callback, nullptr); + } + return s; +} +#endif // ROCKSDB_LITE + +// The main write queue. This is the only write queue that updates LastSequence. +// When using one write queue, the same sequence also indicates the last +// published sequence. +Status DBImpl::WriteImpl(const WriteOptions& write_options, + WriteBatch* my_batch, WriteCallback* callback, + uint64_t* log_used, uint64_t log_ref, + bool disable_memtable, uint64_t* seq_used, + size_t batch_cnt, + PreReleaseCallback* pre_release_callback, + PostMemTableCallback* post_memtable_callback) { + assert(!seq_per_batch_ || batch_cnt != 0); + assert(my_batch == nullptr || my_batch->Count() == 0 || + write_options.protection_bytes_per_key == 0 || + write_options.protection_bytes_per_key == + my_batch->GetProtectionBytesPerKey()); + if (my_batch == nullptr) { + return Status::InvalidArgument("Batch is nullptr!"); + } else if (!disable_memtable && + WriteBatchInternal::TimestampsUpdateNeeded(*my_batch)) { + // If writing to memtable, then we require the caller to set/update the + // timestamps for the keys in the write batch. + // Otherwise, it means we are just writing to the WAL, and we allow + // timestamps unset for the keys in the write batch. This can happen if we + // use TransactionDB with write-committed policy, and we currently do not + // support user-defined timestamp with other policies. + // In the prepare phase, a transaction can write the batch to the WAL + // without inserting to memtable. The keys in the batch do not have to be + // assigned timestamps because they will be used only during recovery if + // there is a commit marker which includes their commit timestamp. + return Status::InvalidArgument("write batch must have timestamp(s) set"); + } else if (write_options.rate_limiter_priority != Env::IO_TOTAL && + write_options.rate_limiter_priority != Env::IO_USER) { + return Status::InvalidArgument( + "WriteOptions::rate_limiter_priority only allows " + "Env::IO_TOTAL and Env::IO_USER due to implementation constraints"); + } else if (write_options.rate_limiter_priority != Env::IO_TOTAL && + (write_options.disableWAL || manual_wal_flush_)) { + return Status::InvalidArgument( + "WriteOptions::rate_limiter_priority currently only supports " + "rate-limiting automatic WAL flush, which requires " + "`WriteOptions::disableWAL` and " + "`DBOptions::manual_wal_flush` both set to false"); + } else if (write_options.protection_bytes_per_key != 0 && + write_options.protection_bytes_per_key != 8) { + return Status::InvalidArgument( + "`WriteOptions::protection_bytes_per_key` must be zero or eight"); + } + // TODO: this use of operator bool on `tracer_` can avoid unnecessary lock + // grabs but does not seem thread-safe. + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_ && !tracer_->IsWriteOrderPreserved()) { + // We don't have to preserve write order so can trace anywhere. It's more + // efficient to trace here than to add latency to a phase of the log/apply + // pipeline. + // TODO: maybe handle the tracing status? + tracer_->Write(my_batch).PermitUncheckedError(); + } + } + if (write_options.sync && write_options.disableWAL) { + return Status::InvalidArgument("Sync writes has to enable WAL."); + } + if (two_write_queues_ && immutable_db_options_.enable_pipelined_write) { + return Status::NotSupported( + "pipelined_writes is not compatible with concurrent prepares"); + } + if (seq_per_batch_ && immutable_db_options_.enable_pipelined_write) { + // TODO(yiwu): update pipeline write with seq_per_batch and batch_cnt + return Status::NotSupported( + "pipelined_writes is not compatible with seq_per_batch"); + } + if (immutable_db_options_.unordered_write && + immutable_db_options_.enable_pipelined_write) { + return Status::NotSupported( + "pipelined_writes is not compatible with unordered_write"); + } + if (immutable_db_options_.enable_pipelined_write && + post_memtable_callback != nullptr) { + return Status::NotSupported( + "pipelined write currently does not honor post_memtable_callback"); + } + if (seq_per_batch_ && post_memtable_callback != nullptr) { + return Status::NotSupported( + "seq_per_batch currently does not honor post_memtable_callback"); + } + // Otherwise IsLatestPersistentState optimization does not make sense + assert(!WriteBatchInternal::IsLatestPersistentState(my_batch) || + disable_memtable); + + if (write_options.low_pri) { + Status s = ThrottleLowPriWritesIfNeeded(write_options, my_batch); + if (!s.ok()) { + return s; + } + } + + if (two_write_queues_ && disable_memtable) { + AssignOrder assign_order = + seq_per_batch_ ? kDoAssignOrder : kDontAssignOrder; + // Otherwise it is WAL-only Prepare batches in WriteCommitted policy and + // they don't consume sequence. + return WriteImplWALOnly(&nonmem_write_thread_, write_options, my_batch, + callback, log_used, log_ref, seq_used, batch_cnt, + pre_release_callback, assign_order, + kDontPublishLastSeq, disable_memtable); + } + + if (immutable_db_options_.unordered_write) { + const size_t sub_batch_cnt = batch_cnt != 0 + ? batch_cnt + // every key is a sub-batch consuming a seq + : WriteBatchInternal::Count(my_batch); + uint64_t seq = 0; + // Use a write thread to i) optimize for WAL write, ii) publish last + // sequence in in increasing order, iii) call pre_release_callback serially + Status status = WriteImplWALOnly( + &write_thread_, write_options, my_batch, callback, log_used, log_ref, + &seq, sub_batch_cnt, pre_release_callback, kDoAssignOrder, + kDoPublishLastSeq, disable_memtable); + TEST_SYNC_POINT("DBImpl::WriteImpl:UnorderedWriteAfterWriteWAL"); + if (!status.ok()) { + return status; + } + if (seq_used) { + *seq_used = seq; + } + if (!disable_memtable) { + TEST_SYNC_POINT("DBImpl::WriteImpl:BeforeUnorderedWriteMemtable"); + status = UnorderedWriteMemtable(write_options, my_batch, callback, + log_ref, seq, sub_batch_cnt); + } + return status; + } + + if (immutable_db_options_.enable_pipelined_write) { + return PipelinedWriteImpl(write_options, my_batch, callback, log_used, + log_ref, disable_memtable, seq_used); + } + + PERF_TIMER_GUARD(write_pre_and_post_process_time); + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + disable_memtable, batch_cnt, pre_release_callback, + post_memtable_callback); + StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); + + write_thread_.JoinBatchGroup(&w); + if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { + // we are a non-leader in a parallel group + + if (w.ShouldWriteToMemtable()) { + PERF_TIMER_STOP(write_pre_and_post_process_time); + PERF_TIMER_GUARD(write_memtable_time); + + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, + write_options.ignore_missing_column_families, 0 /*log_number*/, this, + true /*concurrent_memtable_writes*/, seq_per_batch_, w.batch_cnt, + batch_per_txn_, write_options.memtable_insert_hint_per_batch); + + PERF_TIMER_START(write_pre_and_post_process_time); + } + + if (write_thread_.CompleteParallelMemTableWriter(&w)) { + // we're responsible for exit batch group + // TODO(myabandeh): propagate status to write_group + auto last_sequence = w.write_group->last_sequence; + for (auto* tmp_w : *(w.write_group)) { + assert(tmp_w); + if (tmp_w->post_memtable_callback) { + Status tmp_s = + (*tmp_w->post_memtable_callback)(last_sequence, disable_memtable); + // TODO: propagate the execution status of post_memtable_callback to + // caller. + assert(tmp_s.ok()); + } + } + versions_->SetLastSequence(last_sequence); + MemTableInsertStatusCheck(w.status); + write_thread_.ExitAsBatchGroupFollower(&w); + } + assert(w.state == WriteThread::STATE_COMPLETED); + // STATE_COMPLETED conditional below handles exit + } + if (w.state == WriteThread::STATE_COMPLETED) { + if (log_used != nullptr) { + *log_used = w.log_used; + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + // write is complete and leader has updated sequence + return w.FinalStatus(); + } + // else we are the leader of the write batch group + assert(w.state == WriteThread::STATE_GROUP_LEADER); + Status status; + // Once reaches this point, the current writer "w" will try to do its write + // job. It may also pick up some of the remaining writers in the "writers_" + // when it finds suitable, and finish them in the same write batch. + // This is how a write job could be done by the other writer. + WriteContext write_context; + LogContext log_context(write_options.sync); + WriteThread::WriteGroup write_group; + bool in_parallel_group = false; + uint64_t last_sequence = kMaxSequenceNumber; + + assert(!two_write_queues_ || !disable_memtable); + { + // With concurrent writes we do preprocess only in the write thread that + // also does write to memtable to avoid sync issue on shared data structure + // with the other thread + + // PreprocessWrite does its own perf timing. + PERF_TIMER_STOP(write_pre_and_post_process_time); + + status = PreprocessWrite(write_options, &log_context, &write_context); + if (!two_write_queues_) { + // Assign it after ::PreprocessWrite since the sequence might advance + // inside it by WriteRecoverableState + last_sequence = versions_->LastSequence(); + } + + PERF_TIMER_START(write_pre_and_post_process_time); + } + + // Add to log and apply to memtable. We can release the lock + // during this phase since &w is currently responsible for logging + // and protects against concurrent loggers and concurrent writes + // into memtables + + TEST_SYNC_POINT("DBImpl::WriteImpl:BeforeLeaderEnters"); + last_batch_group_size_ = + write_thread_.EnterAsBatchGroupLeader(&w, &write_group); + + IOStatus io_s; + Status pre_release_cb_status; + if (status.ok()) { + // TODO: this use of operator bool on `tracer_` can avoid unnecessary lock + // grabs but does not seem thread-safe. + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_ && tracer_->IsWriteOrderPreserved()) { + for (auto* writer : write_group) { + // TODO: maybe handle the tracing status? + tracer_->Write(writer->batch).PermitUncheckedError(); + } + } + } + // Rules for when we can update the memtable concurrently + // 1. supported by memtable + // 2. Puts are not okay if inplace_update_support + // 3. Merges are not okay + // + // Rules 1..2 are enforced by checking the options + // during startup (CheckConcurrentWritesSupported), so if + // options.allow_concurrent_memtable_write is true then they can be + // assumed to be true. Rule 3 is checked for each batch. We could + // relax rules 2 if we could prevent write batches from referring + // more than once to a particular key. + bool parallel = immutable_db_options_.allow_concurrent_memtable_write && + write_group.size > 1; + size_t total_count = 0; + size_t valid_batches = 0; + size_t total_byte_size = 0; + size_t pre_release_callback_cnt = 0; + for (auto* writer : write_group) { + assert(writer); + if (writer->CheckCallback(this)) { + valid_batches += writer->batch_cnt; + if (writer->ShouldWriteToMemtable()) { + total_count += WriteBatchInternal::Count(writer->batch); + parallel = parallel && !writer->batch->HasMerge(); + } + total_byte_size = WriteBatchInternal::AppendedByteSize( + total_byte_size, WriteBatchInternal::ByteSize(writer->batch)); + if (writer->pre_release_callback) { + pre_release_callback_cnt++; + } + } + } + // Note about seq_per_batch_: either disableWAL is set for the entire write + // group or not. In either case we inc seq for each write batch with no + // failed callback. This means that there could be a batch with + // disalbe_memtable in between; although we do not write this batch to + // memtable it still consumes a seq. Otherwise, if !seq_per_batch_, we inc + // the seq per valid written key to mem. + size_t seq_inc = seq_per_batch_ ? valid_batches : total_count; + + const bool concurrent_update = two_write_queues_; + // Update stats while we are an exclusive group leader, so we know + // that nobody else can be writing to these particular stats. + // We're optimistic, updating the stats before we successfully + // commit. That lets us release our leader status early. + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count, + concurrent_update); + RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); + stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size, + concurrent_update); + RecordTick(stats_, BYTES_WRITTEN, total_byte_size); + stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1, + concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_SELF); + auto write_done_by_other = write_group.size - 1; + if (write_done_by_other > 0) { + stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther, + write_done_by_other, concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_OTHER, write_done_by_other); + } + RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); + + if (write_options.disableWAL) { + has_unpersisted_data_.store(true, std::memory_order_relaxed); + } + + PERF_TIMER_STOP(write_pre_and_post_process_time); + + if (!two_write_queues_) { + if (status.ok() && !write_options.disableWAL) { + assert(log_context.log_file_number_size); + LogFileNumberSize& log_file_number_size = + *(log_context.log_file_number_size); + PERF_TIMER_GUARD(write_wal_time); + io_s = + WriteToWAL(write_group, log_context.writer, log_used, + log_context.need_log_sync, log_context.need_log_dir_sync, + last_sequence + 1, log_file_number_size); + } + } else { + if (status.ok() && !write_options.disableWAL) { + PERF_TIMER_GUARD(write_wal_time); + // LastAllocatedSequence is increased inside WriteToWAL under + // wal_write_mutex_ to ensure ordered events in WAL + io_s = ConcurrentWriteToWAL(write_group, log_used, &last_sequence, + seq_inc); + } else { + // Otherwise we inc seq number for memtable writes + last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc); + } + } + status = io_s; + assert(last_sequence != kMaxSequenceNumber); + const SequenceNumber current_sequence = last_sequence + 1; + last_sequence += seq_inc; + + // PreReleaseCallback is called after WAL write and before memtable write + if (status.ok()) { + SequenceNumber next_sequence = current_sequence; + size_t index = 0; + // Note: the logic for advancing seq here must be consistent with the + // logic in WriteBatchInternal::InsertInto(write_group...) as well as + // with WriteBatchInternal::InsertInto(write_batch...) that is called on + // the merged batch during recovery from the WAL. + for (auto* writer : write_group) { + if (writer->CallbackFailed()) { + continue; + } + writer->sequence = next_sequence; + if (writer->pre_release_callback) { + Status ws = writer->pre_release_callback->Callback( + writer->sequence, disable_memtable, writer->log_used, index++, + pre_release_callback_cnt); + if (!ws.ok()) { + status = pre_release_cb_status = ws; + break; + } + } + if (seq_per_batch_) { + assert(writer->batch_cnt); + next_sequence += writer->batch_cnt; + } else if (writer->ShouldWriteToMemtable()) { + next_sequence += WriteBatchInternal::Count(writer->batch); + } + } + } + + if (status.ok()) { + PERF_TIMER_GUARD(write_memtable_time); + + if (!parallel) { + // w.sequence will be set inside InsertInto + w.status = WriteBatchInternal::InsertInto( + write_group, current_sequence, column_family_memtables_.get(), + &flush_scheduler_, &trim_history_scheduler_, + write_options.ignore_missing_column_families, + 0 /*recovery_log_number*/, this, parallel, seq_per_batch_, + batch_per_txn_); + } else { + write_group.last_sequence = last_sequence; + write_thread_.LaunchParallelMemTableWriters(&write_group); + in_parallel_group = true; + + // Each parallel follower is doing each own writes. The leader should + // also do its own. + if (w.ShouldWriteToMemtable()) { + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + assert(w.sequence == current_sequence); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, + write_options.ignore_missing_column_families, 0 /*log_number*/, + this, true /*concurrent_memtable_writes*/, seq_per_batch_, + w.batch_cnt, batch_per_txn_, + write_options.memtable_insert_hint_per_batch); + } + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + } + } + PERF_TIMER_START(write_pre_and_post_process_time); + + if (!io_s.ok()) { + // Check WriteToWAL status + IOStatusCheck(io_s); + } + if (!w.CallbackFailed()) { + if (!io_s.ok()) { + assert(pre_release_cb_status.ok()); + } else { + WriteStatusCheck(pre_release_cb_status); + } + } else { + assert(pre_release_cb_status.ok()); + } + + if (log_context.need_log_sync) { + VersionEdit synced_wals; + log_write_mutex_.Lock(); + if (status.ok()) { + MarkLogsSynced(logfile_number_, log_context.need_log_dir_sync, + &synced_wals); + } else { + MarkLogsNotSynced(logfile_number_); + } + log_write_mutex_.Unlock(); + if (status.ok() && synced_wals.IsWalAddition()) { + InstrumentedMutexLock l(&mutex_); + status = ApplyWALToManifest(&synced_wals); + } + + // Requesting sync with two_write_queues_ is expected to be very rare. We + // hence provide a simple implementation that is not necessarily efficient. + if (two_write_queues_) { + if (manual_wal_flush_) { + status = FlushWAL(true); + } else { + status = SyncWAL(); + } + } + } + + bool should_exit_batch_group = true; + if (in_parallel_group) { + // CompleteParallelWorker returns true if this thread should + // handle exit, false means somebody else did + should_exit_batch_group = write_thread_.CompleteParallelMemTableWriter(&w); + } + if (should_exit_batch_group) { + if (status.ok()) { + for (auto* tmp_w : write_group) { + assert(tmp_w); + if (tmp_w->post_memtable_callback) { + Status tmp_s = + (*tmp_w->post_memtable_callback)(last_sequence, disable_memtable); + // TODO: propagate the execution status of post_memtable_callback to + // caller. + assert(tmp_s.ok()); + } + } + // Note: if we are to resume after non-OK statuses we need to revisit how + // we reacts to non-OK statuses here. + versions_->SetLastSequence(last_sequence); + } + MemTableInsertStatusCheck(w.status); + write_thread_.ExitAsBatchGroupLeader(write_group, status); + } + + if (status.ok()) { + status = w.FinalStatus(); + } + return status; +} + +Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, + WriteBatch* my_batch, WriteCallback* callback, + uint64_t* log_used, uint64_t log_ref, + bool disable_memtable, uint64_t* seq_used) { + PERF_TIMER_GUARD(write_pre_and_post_process_time); + StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); + + WriteContext write_context; + + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + disable_memtable, /*_batch_cnt=*/0, + /*_pre_release_callback=*/nullptr); + write_thread_.JoinBatchGroup(&w); + TEST_SYNC_POINT("DBImplWrite::PipelinedWriteImpl:AfterJoinBatchGroup"); + if (w.state == WriteThread::STATE_GROUP_LEADER) { + WriteThread::WriteGroup wal_write_group; + if (w.callback && !w.callback->AllowWriteBatching()) { + write_thread_.WaitForMemTableWriters(); + } + LogContext log_context(!write_options.disableWAL && write_options.sync); + // PreprocessWrite does its own perf timing. + PERF_TIMER_STOP(write_pre_and_post_process_time); + w.status = PreprocessWrite(write_options, &log_context, &write_context); + PERF_TIMER_START(write_pre_and_post_process_time); + + // This can set non-OK status if callback fail. + last_batch_group_size_ = + write_thread_.EnterAsBatchGroupLeader(&w, &wal_write_group); + const SequenceNumber current_sequence = + write_thread_.UpdateLastSequence(versions_->LastSequence()) + 1; + size_t total_count = 0; + size_t total_byte_size = 0; + + if (w.status.ok()) { + // TODO: this use of operator bool on `tracer_` can avoid unnecessary lock + // grabs but does not seem thread-safe. + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_ != nullptr && tracer_->IsWriteOrderPreserved()) { + for (auto* writer : wal_write_group) { + // TODO: maybe handle the tracing status? + tracer_->Write(writer->batch).PermitUncheckedError(); + } + } + } + SequenceNumber next_sequence = current_sequence; + for (auto* writer : wal_write_group) { + assert(writer); + if (writer->CheckCallback(this)) { + if (writer->ShouldWriteToMemtable()) { + writer->sequence = next_sequence; + size_t count = WriteBatchInternal::Count(writer->batch); + next_sequence += count; + total_count += count; + } + total_byte_size = WriteBatchInternal::AppendedByteSize( + total_byte_size, WriteBatchInternal::ByteSize(writer->batch)); + } + } + if (w.disable_wal) { + has_unpersisted_data_.store(true, std::memory_order_relaxed); + } + write_thread_.UpdateLastSequence(current_sequence + total_count - 1); + } + + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count); + RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); + stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size); + RecordTick(stats_, BYTES_WRITTEN, total_byte_size); + RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); + + PERF_TIMER_STOP(write_pre_and_post_process_time); + + IOStatus io_s; + io_s.PermitUncheckedError(); // Allow io_s to be uninitialized + + if (w.status.ok() && !write_options.disableWAL) { + PERF_TIMER_GUARD(write_wal_time); + stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1); + RecordTick(stats_, WRITE_DONE_BY_SELF, 1); + if (wal_write_group.size > 1) { + stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther, + wal_write_group.size - 1); + RecordTick(stats_, WRITE_DONE_BY_OTHER, wal_write_group.size - 1); + } + assert(log_context.log_file_number_size); + LogFileNumberSize& log_file_number_size = + *(log_context.log_file_number_size); + io_s = + WriteToWAL(wal_write_group, log_context.writer, log_used, + log_context.need_log_sync, log_context.need_log_dir_sync, + current_sequence, log_file_number_size); + w.status = io_s; + } + + if (!io_s.ok()) { + // Check WriteToWAL status + IOStatusCheck(io_s); + } else if (!w.CallbackFailed()) { + WriteStatusCheck(w.status); + } + + VersionEdit synced_wals; + if (log_context.need_log_sync) { + InstrumentedMutexLock l(&log_write_mutex_); + if (w.status.ok()) { + MarkLogsSynced(logfile_number_, log_context.need_log_dir_sync, + &synced_wals); + } else { + MarkLogsNotSynced(logfile_number_); + } + } + if (w.status.ok() && synced_wals.IsWalAddition()) { + InstrumentedMutexLock l(&mutex_); + w.status = ApplyWALToManifest(&synced_wals); + } + write_thread_.ExitAsBatchGroupLeader(wal_write_group, w.status); + } + + // NOTE: the memtable_write_group is declared before the following + // `if` statement because its lifetime needs to be longer + // that the inner context of the `if` as a reference to it + // may be used further below within the outer _write_thread + WriteThread::WriteGroup memtable_write_group; + + if (w.state == WriteThread::STATE_MEMTABLE_WRITER_LEADER) { + PERF_TIMER_GUARD(write_memtable_time); + assert(w.ShouldWriteToMemtable()); + write_thread_.EnterAsMemTableWriter(&w, &memtable_write_group); + if (memtable_write_group.size > 1 && + immutable_db_options_.allow_concurrent_memtable_write) { + write_thread_.LaunchParallelMemTableWriters(&memtable_write_group); + } else { + memtable_write_group.status = WriteBatchInternal::InsertInto( + memtable_write_group, w.sequence, column_family_memtables_.get(), + &flush_scheduler_, &trim_history_scheduler_, + write_options.ignore_missing_column_families, 0 /*log_number*/, this, + false /*concurrent_memtable_writes*/, seq_per_batch_, batch_per_txn_); + versions_->SetLastSequence(memtable_write_group.last_sequence); + write_thread_.ExitAsMemTableWriter(&w, memtable_write_group); + } + } else { + // NOTE: the memtable_write_group is never really used, + // so we need to set its status to pass ASSERT_STATUS_CHECKED + memtable_write_group.status.PermitUncheckedError(); + } + + if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { + assert(w.ShouldWriteToMemtable()); + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, write_options.ignore_missing_column_families, + 0 /*log_number*/, this, true /*concurrent_memtable_writes*/, + false /*seq_per_batch*/, 0 /*batch_cnt*/, true /*batch_per_txn*/, + write_options.memtable_insert_hint_per_batch); + if (write_thread_.CompleteParallelMemTableWriter(&w)) { + MemTableInsertStatusCheck(w.status); + versions_->SetLastSequence(w.write_group->last_sequence); + write_thread_.ExitAsMemTableWriter(&w, *w.write_group); + } + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + + assert(w.state == WriteThread::STATE_COMPLETED); + return w.FinalStatus(); +} + +Status DBImpl::UnorderedWriteMemtable(const WriteOptions& write_options, + WriteBatch* my_batch, + WriteCallback* callback, uint64_t log_ref, + SequenceNumber seq, + const size_t sub_batch_cnt) { + PERF_TIMER_GUARD(write_pre_and_post_process_time); + StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); + + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + false /*disable_memtable*/); + + if (w.CheckCallback(this) && w.ShouldWriteToMemtable()) { + w.sequence = seq; + size_t total_count = WriteBatchInternal::Count(my_batch); + InternalStats* stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsNumKeysWritten, total_count); + RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count); + + ColumnFamilyMemTablesImpl column_family_memtables( + versions_->GetColumnFamilySet()); + w.status = WriteBatchInternal::InsertInto( + &w, w.sequence, &column_family_memtables, &flush_scheduler_, + &trim_history_scheduler_, write_options.ignore_missing_column_families, + 0 /*log_number*/, this, true /*concurrent_memtable_writes*/, + seq_per_batch_, sub_batch_cnt, true /*batch_per_txn*/, + write_options.memtable_insert_hint_per_batch); + if (write_options.disableWAL) { + has_unpersisted_data_.store(true, std::memory_order_relaxed); + } + } + + size_t pending_cnt = pending_memtable_writes_.fetch_sub(1) - 1; + if (pending_cnt == 0) { + // switch_cv_ waits until pending_memtable_writes_ = 0. Locking its mutex + // before notify ensures that cv is in waiting state when it is notified + // thus not missing the update to pending_memtable_writes_ even though it is + // not modified under the mutex. + std::lock_guard lck(switch_mutex_); + switch_cv_.notify_all(); + } + WriteStatusCheck(w.status); + + if (!w.FinalStatus().ok()) { + return w.FinalStatus(); + } + return Status::OK(); +} + +// The 2nd write queue. If enabled it will be used only for WAL-only writes. +// This is the only queue that updates LastPublishedSequence which is only +// applicable in a two-queue setting. +Status DBImpl::WriteImplWALOnly( + WriteThread* write_thread, const WriteOptions& write_options, + WriteBatch* my_batch, WriteCallback* callback, uint64_t* log_used, + const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt, + PreReleaseCallback* pre_release_callback, const AssignOrder assign_order, + const PublishLastSeq publish_last_seq, const bool disable_memtable) { + PERF_TIMER_GUARD(write_pre_and_post_process_time); + WriteThread::Writer w(write_options, my_batch, callback, log_ref, + disable_memtable, sub_batch_cnt, pre_release_callback); + StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); + + write_thread->JoinBatchGroup(&w); + assert(w.state != WriteThread::STATE_PARALLEL_MEMTABLE_WRITER); + if (w.state == WriteThread::STATE_COMPLETED) { + if (log_used != nullptr) { + *log_used = w.log_used; + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + return w.FinalStatus(); + } + // else we are the leader of the write batch group + assert(w.state == WriteThread::STATE_GROUP_LEADER); + + if (publish_last_seq == kDoPublishLastSeq) { + Status status; + + // Currently we only use kDoPublishLastSeq in unordered_write + assert(immutable_db_options_.unordered_write); + WriteContext write_context; + if (error_handler_.IsDBStopped()) { + status = error_handler_.GetBGError(); + } + // TODO(myabandeh): Make preliminary checks thread-safe so we could do them + // without paying the cost of obtaining the mutex. + if (status.ok()) { + LogContext log_context; + status = PreprocessWrite(write_options, &log_context, &write_context); + WriteStatusCheckOnLocked(status); + } + if (!status.ok()) { + WriteThread::WriteGroup write_group; + write_thread->EnterAsBatchGroupLeader(&w, &write_group); + write_thread->ExitAsBatchGroupLeader(write_group, status); + return status; + } + } + + WriteThread::WriteGroup write_group; + uint64_t last_sequence; + write_thread->EnterAsBatchGroupLeader(&w, &write_group); + // Note: no need to update last_batch_group_size_ here since the batch writes + // to WAL only + // TODO: this use of operator bool on `tracer_` can avoid unnecessary lock + // grabs but does not seem thread-safe. + if (tracer_) { + InstrumentedMutexLock lock(&trace_mutex_); + if (tracer_ != nullptr && tracer_->IsWriteOrderPreserved()) { + for (auto* writer : write_group) { + // TODO: maybe handle the tracing status? + tracer_->Write(writer->batch).PermitUncheckedError(); + } + } + } + + size_t pre_release_callback_cnt = 0; + size_t total_byte_size = 0; + for (auto* writer : write_group) { + assert(writer); + if (writer->CheckCallback(this)) { + total_byte_size = WriteBatchInternal::AppendedByteSize( + total_byte_size, WriteBatchInternal::ByteSize(writer->batch)); + if (writer->pre_release_callback) { + pre_release_callback_cnt++; + } + } + } + + const bool concurrent_update = true; + // Update stats while we are an exclusive group leader, so we know + // that nobody else can be writing to these particular stats. + // We're optimistic, updating the stats before we successfully + // commit. That lets us release our leader status early. + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsBytesWritten, total_byte_size, + concurrent_update); + RecordTick(stats_, BYTES_WRITTEN, total_byte_size); + stats->AddDBStats(InternalStats::kIntStatsWriteDoneBySelf, 1, + concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_SELF); + auto write_done_by_other = write_group.size - 1; + if (write_done_by_other > 0) { + stats->AddDBStats(InternalStats::kIntStatsWriteDoneByOther, + write_done_by_other, concurrent_update); + RecordTick(stats_, WRITE_DONE_BY_OTHER, write_done_by_other); + } + RecordInHistogram(stats_, BYTES_PER_WRITE, total_byte_size); + + PERF_TIMER_STOP(write_pre_and_post_process_time); + + PERF_TIMER_GUARD(write_wal_time); + // LastAllocatedSequence is increased inside WriteToWAL under + // wal_write_mutex_ to ensure ordered events in WAL + size_t seq_inc = 0 /* total_count */; + if (assign_order == kDoAssignOrder) { + size_t total_batch_cnt = 0; + for (auto* writer : write_group) { + assert(writer->batch_cnt || !seq_per_batch_); + if (!writer->CallbackFailed()) { + total_batch_cnt += writer->batch_cnt; + } + } + seq_inc = total_batch_cnt; + } + Status status; + if (!write_options.disableWAL) { + IOStatus io_s = + ConcurrentWriteToWAL(write_group, log_used, &last_sequence, seq_inc); + status = io_s; + // last_sequence may not be set if there is an error + // This error checking and return is moved up to avoid using uninitialized + // last_sequence. + if (!io_s.ok()) { + IOStatusCheck(io_s); + write_thread->ExitAsBatchGroupLeader(write_group, status); + return status; + } + } else { + // Otherwise we inc seq number to do solely the seq allocation + last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc); + } + + size_t memtable_write_cnt = 0; + auto curr_seq = last_sequence + 1; + for (auto* writer : write_group) { + if (writer->CallbackFailed()) { + continue; + } + writer->sequence = curr_seq; + if (assign_order == kDoAssignOrder) { + assert(writer->batch_cnt || !seq_per_batch_); + curr_seq += writer->batch_cnt; + } + if (!writer->disable_memtable) { + memtable_write_cnt++; + } + // else seq advances only by memtable writes + } + if (status.ok() && write_options.sync) { + assert(!write_options.disableWAL); + // Requesting sync with two_write_queues_ is expected to be very rare. We + // hance provide a simple implementation that is not necessarily efficient. + if (manual_wal_flush_) { + status = FlushWAL(true); + } else { + status = SyncWAL(); + } + } + PERF_TIMER_START(write_pre_and_post_process_time); + + if (!w.CallbackFailed()) { + WriteStatusCheck(status); + } + if (status.ok()) { + size_t index = 0; + for (auto* writer : write_group) { + if (!writer->CallbackFailed() && writer->pre_release_callback) { + assert(writer->sequence != kMaxSequenceNumber); + Status ws = writer->pre_release_callback->Callback( + writer->sequence, disable_memtable, writer->log_used, index++, + pre_release_callback_cnt); + if (!ws.ok()) { + status = ws; + break; + } + } + } + } + if (publish_last_seq == kDoPublishLastSeq) { + versions_->SetLastSequence(last_sequence + seq_inc); + // Currently we only use kDoPublishLastSeq in unordered_write + assert(immutable_db_options_.unordered_write); + } + if (immutable_db_options_.unordered_write && status.ok()) { + pending_memtable_writes_ += memtable_write_cnt; + } + write_thread->ExitAsBatchGroupLeader(write_group, status); + if (status.ok()) { + status = w.FinalStatus(); + } + if (seq_used != nullptr) { + *seq_used = w.sequence; + } + return status; +} + +void DBImpl::WriteStatusCheckOnLocked(const Status& status) { + // Is setting bg_error_ enough here? This will at least stop + // compaction and fail any further writes. + InstrumentedMutexLock l(&mutex_); + assert(!status.IsIOFenced() || !error_handler_.GetBGError().ok()); + if (immutable_db_options_.paranoid_checks && !status.ok() && + !status.IsBusy() && !status.IsIncomplete()) { + // Maybe change the return status to void? + error_handler_.SetBGError(status, BackgroundErrorReason::kWriteCallback); + } +} + +void DBImpl::WriteStatusCheck(const Status& status) { + // Is setting bg_error_ enough here? This will at least stop + // compaction and fail any further writes. + assert(!status.IsIOFenced() || !error_handler_.GetBGError().ok()); + if (immutable_db_options_.paranoid_checks && !status.ok() && + !status.IsBusy() && !status.IsIncomplete()) { + mutex_.Lock(); + // Maybe change the return status to void? + error_handler_.SetBGError(status, BackgroundErrorReason::kWriteCallback); + mutex_.Unlock(); + } +} + +void DBImpl::IOStatusCheck(const IOStatus& io_status) { + // Is setting bg_error_ enough here? This will at least stop + // compaction and fail any further writes. + if ((immutable_db_options_.paranoid_checks && !io_status.ok() && + !io_status.IsBusy() && !io_status.IsIncomplete()) || + io_status.IsIOFenced()) { + mutex_.Lock(); + // Maybe change the return status to void? + error_handler_.SetBGError(io_status, BackgroundErrorReason::kWriteCallback); + mutex_.Unlock(); + } else { + // Force writable file to be continue writable. + logs_.back().writer->file()->reset_seen_error(); + } +} + +void DBImpl::MemTableInsertStatusCheck(const Status& status) { + // A non-OK status here indicates that the state implied by the + // WAL has diverged from the in-memory state. This could be + // because of a corrupt write_batch (very bad), or because the + // client specified an invalid column family and didn't specify + // ignore_missing_column_families. + if (!status.ok()) { + mutex_.Lock(); + assert(!error_handler_.IsBGWorkStopped()); + // Maybe change the return status to void? + error_handler_.SetBGError(status, BackgroundErrorReason::kMemTable) + .PermitUncheckedError(); + mutex_.Unlock(); + } +} + +Status DBImpl::PreprocessWrite(const WriteOptions& write_options, + LogContext* log_context, + WriteContext* write_context) { + assert(write_context != nullptr && log_context != nullptr); + Status status; + + if (error_handler_.IsDBStopped()) { + InstrumentedMutexLock l(&mutex_); + status = error_handler_.GetBGError(); + } + + PERF_TIMER_GUARD(write_scheduling_flushes_compactions_time); + + if (UNLIKELY(status.ok() && total_log_size_ > GetMaxTotalWalSize())) { + assert(versions_); + InstrumentedMutexLock l(&mutex_); + const ColumnFamilySet* const column_families = + versions_->GetColumnFamilySet(); + assert(column_families); + size_t num_cfs = column_families->NumberOfColumnFamilies(); + assert(num_cfs >= 1); + if (num_cfs > 1) { + WaitForPendingWrites(); + status = SwitchWAL(write_context); + } + } + + if (UNLIKELY(status.ok() && write_buffer_manager_->ShouldFlush())) { + // Before a new memtable is added in SwitchMemtable(), + // write_buffer_manager_->ShouldFlush() will keep returning true. If another + // thread is writing to another DB with the same write buffer, they may also + // be flushed. We may end up with flushing much more DBs than needed. It's + // suboptimal but still correct. + InstrumentedMutexLock l(&mutex_); + WaitForPendingWrites(); + status = HandleWriteBufferManagerFlush(write_context); + } + + if (UNLIKELY(status.ok() && !trim_history_scheduler_.Empty())) { + InstrumentedMutexLock l(&mutex_); + status = TrimMemtableHistory(write_context); + } + + if (UNLIKELY(status.ok() && !flush_scheduler_.Empty())) { + InstrumentedMutexLock l(&mutex_); + WaitForPendingWrites(); + status = ScheduleFlushes(write_context); + } + + PERF_TIMER_STOP(write_scheduling_flushes_compactions_time); + PERF_TIMER_GUARD(write_pre_and_post_process_time); + + if (UNLIKELY(status.ok() && (write_controller_.IsStopped() || + write_controller_.NeedsDelay()))) { + PERF_TIMER_STOP(write_pre_and_post_process_time); + PERF_TIMER_GUARD(write_delay_time); + // We don't know size of curent batch so that we always use the size + // for previous one. It might create a fairness issue that expiration + // might happen for smaller writes but larger writes can go through. + // Can optimize it if it is an issue. + InstrumentedMutexLock l(&mutex_); + status = DelayWrite(last_batch_group_size_, write_options); + PERF_TIMER_START(write_pre_and_post_process_time); + } + + // If memory usage exceeded beyond a certain threshold, + // write_buffer_manager_->ShouldStall() returns true to all threads writing to + // all DBs and writers will be stalled. + // It does soft checking because WriteBufferManager::buffer_limit_ has already + // exceeded at this point so no new write (including current one) will go + // through until memory usage is decreased. + if (UNLIKELY(status.ok() && write_buffer_manager_->ShouldStall())) { + if (write_options.no_slowdown) { + status = Status::Incomplete("Write stall"); + } else { + InstrumentedMutexLock l(&mutex_); + WriteBufferManagerStallWrites(); + } + } + InstrumentedMutexLock l(&log_write_mutex_); + if (status.ok() && log_context->need_log_sync) { + // Wait until the parallel syncs are finished. Any sync process has to sync + // the front log too so it is enough to check the status of front() + // We do a while loop since log_sync_cv_ is signalled when any sync is + // finished + // Note: there does not seem to be a reason to wait for parallel sync at + // this early step but it is not important since parallel sync (SyncWAL) and + // need_log_sync are usually not used together. + while (logs_.front().IsSyncing()) { + log_sync_cv_.Wait(); + } + for (auto& log : logs_) { + // This is just to prevent the logs to be synced by a parallel SyncWAL + // call. We will do the actual syncing later after we will write to the + // WAL. + // Note: there does not seem to be a reason to set this early before we + // actually write to the WAL + log.PrepareForSync(); + } + } else { + log_context->need_log_sync = false; + } + log_context->writer = logs_.back().writer; + log_context->need_log_dir_sync = + log_context->need_log_dir_sync && !log_dir_synced_; + log_context->log_file_number_size = std::addressof(alive_log_files_.back()); + + return status; +} + +Status DBImpl::MergeBatch(const WriteThread::WriteGroup& write_group, + WriteBatch* tmp_batch, WriteBatch** merged_batch, + size_t* write_with_wal, + WriteBatch** to_be_cached_state) { + assert(write_with_wal != nullptr); + assert(tmp_batch != nullptr); + assert(*to_be_cached_state == nullptr); + *write_with_wal = 0; + auto* leader = write_group.leader; + assert(!leader->disable_wal); // Same holds for all in the batch group + if (write_group.size == 1 && !leader->CallbackFailed() && + leader->batch->GetWalTerminationPoint().is_cleared()) { + // we simply write the first WriteBatch to WAL if the group only + // contains one batch, that batch should be written to the WAL, + // and the batch is not wanting to be truncated + *merged_batch = leader->batch; + if (WriteBatchInternal::IsLatestPersistentState(*merged_batch)) { + *to_be_cached_state = *merged_batch; + } + *write_with_wal = 1; + } else { + // WAL needs all of the batches flattened into a single batch. + // We could avoid copying here with an iov-like AddRecord + // interface + *merged_batch = tmp_batch; + for (auto writer : write_group) { + if (!writer->CallbackFailed()) { + Status s = WriteBatchInternal::Append(*merged_batch, writer->batch, + /*WAL_only*/ true); + if (!s.ok()) { + tmp_batch->Clear(); + return s; + } + if (WriteBatchInternal::IsLatestPersistentState(writer->batch)) { + // We only need to cache the last of such write batch + *to_be_cached_state = writer->batch; + } + (*write_with_wal)++; + } + } + } + // return merged_batch; + return Status::OK(); +} + +// When two_write_queues_ is disabled, this function is called from the only +// write thread. Otherwise this must be called holding log_write_mutex_. +IOStatus DBImpl::WriteToWAL(const WriteBatch& merged_batch, + log::Writer* log_writer, uint64_t* log_used, + uint64_t* log_size, + Env::IOPriority rate_limiter_priority, + LogFileNumberSize& log_file_number_size) { + assert(log_size != nullptr); + + Slice log_entry = WriteBatchInternal::Contents(&merged_batch); + TEST_SYNC_POINT_CALLBACK("DBImpl::WriteToWAL:log_entry", &log_entry); + auto s = merged_batch.VerifyChecksum(); + if (!s.ok()) { + return status_to_io_status(std::move(s)); + } + *log_size = log_entry.size(); + // When two_write_queues_ WriteToWAL has to be protected from concurretn calls + // from the two queues anyway and log_write_mutex_ is already held. Otherwise + // if manual_wal_flush_ is enabled we need to protect log_writer->AddRecord + // from possible concurrent calls via the FlushWAL by the application. + const bool needs_locking = manual_wal_flush_ && !two_write_queues_; + // Due to performance cocerns of missed branch prediction penalize the new + // manual_wal_flush_ feature (by UNLIKELY) instead of the more common case + // when we do not need any locking. + if (UNLIKELY(needs_locking)) { + log_write_mutex_.Lock(); + } + IOStatus io_s = log_writer->AddRecord(log_entry, rate_limiter_priority); + + if (UNLIKELY(needs_locking)) { + log_write_mutex_.Unlock(); + } + if (log_used != nullptr) { + *log_used = logfile_number_; + } + total_log_size_ += log_entry.size(); + log_file_number_size.AddSize(*log_size); + log_empty_ = false; + return io_s; +} + +IOStatus DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group, + log::Writer* log_writer, uint64_t* log_used, + bool need_log_sync, bool need_log_dir_sync, + SequenceNumber sequence, + LogFileNumberSize& log_file_number_size) { + IOStatus io_s; + assert(!two_write_queues_); + assert(!write_group.leader->disable_wal); + // Same holds for all in the batch group + size_t write_with_wal = 0; + WriteBatch* to_be_cached_state = nullptr; + WriteBatch* merged_batch; + io_s = status_to_io_status(MergeBatch(write_group, &tmp_batch_, &merged_batch, + &write_with_wal, &to_be_cached_state)); + if (UNLIKELY(!io_s.ok())) { + return io_s; + } + + if (merged_batch == write_group.leader->batch) { + write_group.leader->log_used = logfile_number_; + } else if (write_with_wal > 1) { + for (auto writer : write_group) { + writer->log_used = logfile_number_; + } + } + + WriteBatchInternal::SetSequence(merged_batch, sequence); + + uint64_t log_size; + io_s = WriteToWAL(*merged_batch, log_writer, log_used, &log_size, + write_group.leader->rate_limiter_priority, + log_file_number_size); + if (to_be_cached_state) { + cached_recoverable_state_ = *to_be_cached_state; + cached_recoverable_state_empty_ = false; + } + + if (io_s.ok() && need_log_sync) { + StopWatch sw(immutable_db_options_.clock, stats_, WAL_FILE_SYNC_MICROS); + // It's safe to access logs_ with unlocked mutex_ here because: + // - we've set getting_synced=true for all logs, + // so other threads won't pop from logs_ while we're here, + // - only writer thread can push to logs_, and we're in + // writer thread, so no one will push to logs_, + // - as long as other threads don't modify it, it's safe to read + // from std::deque from multiple threads concurrently. + // + // Sync operation should work with locked log_write_mutex_, because: + // when DBOptions.manual_wal_flush_ is set, + // FlushWAL function will be invoked by another thread. + // if without locked log_write_mutex_, the log file may get data + // corruption + + const bool needs_locking = manual_wal_flush_ && !two_write_queues_; + if (UNLIKELY(needs_locking)) { + log_write_mutex_.Lock(); + } + + for (auto& log : logs_) { + io_s = log.writer->file()->Sync(immutable_db_options_.use_fsync); + if (!io_s.ok()) { + break; + } + } + + if (UNLIKELY(needs_locking)) { + log_write_mutex_.Unlock(); + } + + if (io_s.ok() && need_log_dir_sync) { + // We only sync WAL directory the first time WAL syncing is + // requested, so that in case users never turn on WAL sync, + // we can avoid the disk I/O in the write code path. + io_s = directories_.GetWalDir()->FsyncWithDirOptions( + IOOptions(), nullptr, + DirFsyncOptions(DirFsyncOptions::FsyncReason::kNewFileSynced)); + } + } + + if (merged_batch == &tmp_batch_) { + tmp_batch_.Clear(); + } + if (io_s.ok()) { + auto stats = default_cf_internal_stats_; + if (need_log_sync) { + stats->AddDBStats(InternalStats::kIntStatsWalFileSynced, 1); + RecordTick(stats_, WAL_FILE_SYNCED); + } + stats->AddDBStats(InternalStats::kIntStatsWalFileBytes, log_size); + RecordTick(stats_, WAL_FILE_BYTES, log_size); + stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal); + RecordTick(stats_, WRITE_WITH_WAL, write_with_wal); + } + return io_s; +} + +IOStatus DBImpl::ConcurrentWriteToWAL( + const WriteThread::WriteGroup& write_group, uint64_t* log_used, + SequenceNumber* last_sequence, size_t seq_inc) { + IOStatus io_s; + + assert(two_write_queues_ || immutable_db_options_.unordered_write); + assert(!write_group.leader->disable_wal); + // Same holds for all in the batch group + WriteBatch tmp_batch; + size_t write_with_wal = 0; + WriteBatch* to_be_cached_state = nullptr; + WriteBatch* merged_batch; + io_s = status_to_io_status(MergeBatch(write_group, &tmp_batch, &merged_batch, + &write_with_wal, &to_be_cached_state)); + if (UNLIKELY(!io_s.ok())) { + return io_s; + } + + // We need to lock log_write_mutex_ since logs_ and alive_log_files might be + // pushed back concurrently + log_write_mutex_.Lock(); + if (merged_batch == write_group.leader->batch) { + write_group.leader->log_used = logfile_number_; + } else if (write_with_wal > 1) { + for (auto writer : write_group) { + writer->log_used = logfile_number_; + } + } + *last_sequence = versions_->FetchAddLastAllocatedSequence(seq_inc); + auto sequence = *last_sequence + 1; + WriteBatchInternal::SetSequence(merged_batch, sequence); + + log::Writer* log_writer = logs_.back().writer; + LogFileNumberSize& log_file_number_size = alive_log_files_.back(); + + assert(log_writer->get_log_number() == log_file_number_size.number); + + uint64_t log_size; + io_s = WriteToWAL(*merged_batch, log_writer, log_used, &log_size, + write_group.leader->rate_limiter_priority, + log_file_number_size); + if (to_be_cached_state) { + cached_recoverable_state_ = *to_be_cached_state; + cached_recoverable_state_empty_ = false; + } + log_write_mutex_.Unlock(); + + if (io_s.ok()) { + const bool concurrent = true; + auto stats = default_cf_internal_stats_; + stats->AddDBStats(InternalStats::kIntStatsWalFileBytes, log_size, + concurrent); + RecordTick(stats_, WAL_FILE_BYTES, log_size); + stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal, + concurrent); + RecordTick(stats_, WRITE_WITH_WAL, write_with_wal); + } + return io_s; +} + +Status DBImpl::WriteRecoverableState() { + mutex_.AssertHeld(); + if (!cached_recoverable_state_empty_) { + bool dont_care_bool; + SequenceNumber next_seq; + if (two_write_queues_) { + log_write_mutex_.Lock(); + } + SequenceNumber seq; + if (two_write_queues_) { + seq = versions_->FetchAddLastAllocatedSequence(0); + } else { + seq = versions_->LastSequence(); + } + WriteBatchInternal::SetSequence(&cached_recoverable_state_, seq + 1); + auto status = WriteBatchInternal::InsertInto( + &cached_recoverable_state_, column_family_memtables_.get(), + &flush_scheduler_, &trim_history_scheduler_, true, + 0 /*recovery_log_number*/, this, false /* concurrent_memtable_writes */, + &next_seq, &dont_care_bool, seq_per_batch_); + auto last_seq = next_seq - 1; + if (two_write_queues_) { + versions_->FetchAddLastAllocatedSequence(last_seq - seq); + versions_->SetLastPublishedSequence(last_seq); + } + versions_->SetLastSequence(last_seq); + if (two_write_queues_) { + log_write_mutex_.Unlock(); + } + if (status.ok() && recoverable_state_pre_release_callback_) { + const bool DISABLE_MEMTABLE = true; + for (uint64_t sub_batch_seq = seq + 1; + sub_batch_seq < next_seq && status.ok(); sub_batch_seq++) { + uint64_t const no_log_num = 0; + // Unlock it since the callback might end up locking mutex. e.g., + // AddCommitted -> AdvanceMaxEvictedSeq -> GetSnapshotListFromDB + mutex_.Unlock(); + status = recoverable_state_pre_release_callback_->Callback( + sub_batch_seq, !DISABLE_MEMTABLE, no_log_num, 0, 1); + mutex_.Lock(); + } + } + if (status.ok()) { + cached_recoverable_state_.Clear(); + cached_recoverable_state_empty_ = true; + } + return status; + } + return Status::OK(); +} + +void DBImpl::SelectColumnFamiliesForAtomicFlush( + autovector* cfds) { + for (ColumnFamilyData* cfd : *versions_->GetColumnFamilySet()) { + if (cfd->IsDropped()) { + continue; + } + if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() || + !cached_recoverable_state_empty_.load()) { + cfds->push_back(cfd); + } + } +} + +// Assign sequence number for atomic flush. +void DBImpl::AssignAtomicFlushSeq(const autovector& cfds) { + assert(immutable_db_options_.atomic_flush); + auto seq = versions_->LastSequence(); + for (auto cfd : cfds) { + cfd->imm()->AssignAtomicFlushSeq(seq); + } +} + +Status DBImpl::SwitchWAL(WriteContext* write_context) { + mutex_.AssertHeld(); + assert(write_context != nullptr); + Status status; + + if (alive_log_files_.begin()->getting_flushed) { + return status; + } + + auto oldest_alive_log = alive_log_files_.begin()->number; + bool flush_wont_release_oldest_log = false; + if (allow_2pc()) { + auto oldest_log_with_uncommitted_prep = + logs_with_prep_tracker_.FindMinLogContainingOutstandingPrep(); + + assert(oldest_log_with_uncommitted_prep == 0 || + oldest_log_with_uncommitted_prep >= oldest_alive_log); + if (oldest_log_with_uncommitted_prep > 0 && + oldest_log_with_uncommitted_prep == oldest_alive_log) { + if (unable_to_release_oldest_log_) { + // we already attempted to flush all column families dependent on + // the oldest alive log but the log still contained uncommitted + // transactions so there is still nothing that we can do. + return status; + } else { + ROCKS_LOG_WARN( + immutable_db_options_.info_log, + "Unable to release oldest log due to uncommitted transaction"); + unable_to_release_oldest_log_ = true; + flush_wont_release_oldest_log = true; + } + } + } + if (!flush_wont_release_oldest_log) { + // we only mark this log as getting flushed if we have successfully + // flushed all data in this log. If this log contains outstanding prepared + // transactions then we cannot flush this log until those transactions are + // commited. + unable_to_release_oldest_log_ = false; + alive_log_files_.begin()->getting_flushed = true; + } + + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "Flushing all column families with data in WAL number %" PRIu64 + ". Total log size is %" PRIu64 " while max_total_wal_size is %" PRIu64, + oldest_alive_log, total_log_size_.load(), GetMaxTotalWalSize()); + // no need to refcount because drop is happening in write thread, so can't + // happen while we're in the write thread + autovector cfds; + if (immutable_db_options_.atomic_flush) { + SelectColumnFamiliesForAtomicFlush(&cfds); + } else { + for (auto cfd : *versions_->GetColumnFamilySet()) { + if (cfd->IsDropped()) { + continue; + } + if (cfd->OldestLogToKeep() <= oldest_alive_log) { + cfds.push_back(cfd); + } + } + MaybeFlushStatsCF(&cfds); + } + WriteThread::Writer nonmem_w; + if (two_write_queues_) { + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + } + + for (const auto cfd : cfds) { + cfd->Ref(); + status = SwitchMemtable(cfd, write_context); + cfd->UnrefAndTryDelete(); + if (!status.ok()) { + break; + } + } + if (two_write_queues_) { + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } + + if (status.ok()) { + if (immutable_db_options_.atomic_flush) { + AssignAtomicFlushSeq(cfds); + } + for (auto cfd : cfds) { + cfd->imm()->FlushRequested(); + if (!immutable_db_options_.atomic_flush) { + FlushRequest flush_req; + GenerateFlushRequest({cfd}, &flush_req); + SchedulePendingFlush(flush_req, FlushReason::kWalFull); + } + } + if (immutable_db_options_.atomic_flush) { + FlushRequest flush_req; + GenerateFlushRequest(cfds, &flush_req); + SchedulePendingFlush(flush_req, FlushReason::kWalFull); + } + MaybeScheduleFlushOrCompaction(); + } + return status; +} + +Status DBImpl::HandleWriteBufferManagerFlush(WriteContext* write_context) { + mutex_.AssertHeld(); + assert(write_context != nullptr); + Status status; + + // Before a new memtable is added in SwitchMemtable(), + // write_buffer_manager_->ShouldFlush() will keep returning true. If another + // thread is writing to another DB with the same write buffer, they may also + // be flushed. We may end up with flushing much more DBs than needed. It's + // suboptimal but still correct. + // no need to refcount because drop is happening in write thread, so can't + // happen while we're in the write thread + autovector cfds; + if (immutable_db_options_.atomic_flush) { + SelectColumnFamiliesForAtomicFlush(&cfds); + } else { + ColumnFamilyData* cfd_picked = nullptr; + SequenceNumber seq_num_for_cf_picked = kMaxSequenceNumber; + + for (auto cfd : *versions_->GetColumnFamilySet()) { + if (cfd->IsDropped()) { + continue; + } + if (!cfd->mem()->IsEmpty() && !cfd->imm()->IsFlushPendingOrRunning()) { + // We only consider flush on CFs with bytes in the mutable memtable, + // and no immutable memtables for which flush has yet to finish. If + // we triggered flush on CFs already trying to flush, we would risk + // creating too many immutable memtables leading to write stalls. + uint64_t seq = cfd->mem()->GetCreationSeq(); + if (cfd_picked == nullptr || seq < seq_num_for_cf_picked) { + cfd_picked = cfd; + seq_num_for_cf_picked = seq; + } + } + } + if (cfd_picked != nullptr) { + cfds.push_back(cfd_picked); + } + MaybeFlushStatsCF(&cfds); + } + if (!cfds.empty()) { + ROCKS_LOG_INFO( + immutable_db_options_.info_log, + "Flushing triggered to alleviate write buffer memory usage. Write " + "buffer is using %" ROCKSDB_PRIszt + " bytes out of a total of %" ROCKSDB_PRIszt ".", + write_buffer_manager_->memory_usage(), + write_buffer_manager_->buffer_size()); + } + + WriteThread::Writer nonmem_w; + if (two_write_queues_) { + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + } + for (const auto cfd : cfds) { + if (cfd->mem()->IsEmpty()) { + continue; + } + cfd->Ref(); + status = SwitchMemtable(cfd, write_context); + cfd->UnrefAndTryDelete(); + if (!status.ok()) { + break; + } + } + if (two_write_queues_) { + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } + + if (status.ok()) { + if (immutable_db_options_.atomic_flush) { + AssignAtomicFlushSeq(cfds); + } + for (const auto cfd : cfds) { + cfd->imm()->FlushRequested(); + if (!immutable_db_options_.atomic_flush) { + FlushRequest flush_req; + GenerateFlushRequest({cfd}, &flush_req); + SchedulePendingFlush(flush_req, FlushReason::kWriteBufferManager); + } + } + if (immutable_db_options_.atomic_flush) { + FlushRequest flush_req; + GenerateFlushRequest(cfds, &flush_req); + SchedulePendingFlush(flush_req, FlushReason::kWriteBufferManager); + } + MaybeScheduleFlushOrCompaction(); + } + return status; +} + +uint64_t DBImpl::GetMaxTotalWalSize() const { + uint64_t max_total_wal_size = + max_total_wal_size_.load(std::memory_order_acquire); + if (max_total_wal_size > 0) { + return max_total_wal_size; + } + return 4 * max_total_in_memory_state_.load(std::memory_order_acquire); +} + +// REQUIRES: mutex_ is held +// REQUIRES: this thread is currently at the front of the writer queue +Status DBImpl::DelayWrite(uint64_t num_bytes, + const WriteOptions& write_options) { + uint64_t time_delayed = 0; + bool delayed = false; + { + StopWatch sw(immutable_db_options_.clock, stats_, WRITE_STALL, + &time_delayed); + uint64_t delay = + write_controller_.GetDelay(immutable_db_options_.clock, num_bytes); + TEST_SYNC_POINT("DBImpl::DelayWrite:Start"); + if (delay > 0) { + if (write_options.no_slowdown) { + return Status::Incomplete("Write stall"); + } + TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep"); + + // Notify write_thread_ about the stall so it can setup a barrier and + // fail any pending writers with no_slowdown + write_thread_.BeginWriteStall(); + mutex_.Unlock(); + TEST_SYNC_POINT("DBImpl::DelayWrite:BeginWriteStallDone"); + // We will delay the write until we have slept for `delay` microseconds + // or we don't need a delay anymore. We check for cancellation every 1ms + // (slightly longer because WriteController minimum delay is 1ms, in + // case of sleep imprecision, rounding, etc.) + const uint64_t kDelayInterval = 1001; + uint64_t stall_end = sw.start_time() + delay; + while (write_controller_.NeedsDelay()) { + if (immutable_db_options_.clock->NowMicros() >= stall_end) { + // We already delayed this write `delay` microseconds + break; + } + + delayed = true; + // Sleep for 0.001 seconds + immutable_db_options_.clock->SleepForMicroseconds(kDelayInterval); + } + mutex_.Lock(); + write_thread_.EndWriteStall(); + } + + // Don't wait if there's a background error, even if its a soft error. We + // might wait here indefinitely as the background compaction may never + // finish successfully, resulting in the stall condition lasting + // indefinitely + while (error_handler_.GetBGError().ok() && write_controller_.IsStopped() && + !shutting_down_.load(std::memory_order_relaxed)) { + if (write_options.no_slowdown) { + return Status::Incomplete("Write stall"); + } + delayed = true; + + // Notify write_thread_ about the stall so it can setup a barrier and + // fail any pending writers with no_slowdown + write_thread_.BeginWriteStall(); + TEST_SYNC_POINT("DBImpl::DelayWrite:Wait"); + bg_cv_.Wait(); + write_thread_.EndWriteStall(); + } + } + assert(!delayed || !write_options.no_slowdown); + if (delayed) { + default_cf_internal_stats_->AddDBStats( + InternalStats::kIntStatsWriteStallMicros, time_delayed); + RecordTick(stats_, STALL_MICROS, time_delayed); + } + + // If DB is not in read-only mode and write_controller is not stopping + // writes, we can ignore any background errors and allow the write to + // proceed + Status s; + if (write_controller_.IsStopped()) { + if (!shutting_down_.load(std::memory_order_relaxed)) { + // If writes are still stopped and db not shutdown, it means we bailed + // due to a background error + s = Status::Incomplete(error_handler_.GetBGError().ToString()); + } else { + s = Status::ShutdownInProgress("stalled writes"); + } + } + if (error_handler_.IsDBStopped()) { + s = error_handler_.GetBGError(); + } + return s; +} + +// REQUIRES: mutex_ is held +// REQUIRES: this thread is currently at the front of the writer queue +void DBImpl::WriteBufferManagerStallWrites() { + mutex_.AssertHeld(); + // First block future writer threads who want to add themselves to the queue + // of WriteThread. + write_thread_.BeginWriteStall(); + mutex_.Unlock(); + + // Change the state to State::Blocked. + static_cast(wbm_stall_.get()) + ->SetState(WBMStallInterface::State::BLOCKED); + // Then WriteBufferManager will add DB instance to its queue + // and block this thread by calling WBMStallInterface::Block(). + write_buffer_manager_->BeginWriteStall(wbm_stall_.get()); + wbm_stall_->Block(); + + mutex_.Lock(); + // Stall has ended. Signal writer threads so that they can add + // themselves to the WriteThread queue for writes. + write_thread_.EndWriteStall(); +} + +Status DBImpl::ThrottleLowPriWritesIfNeeded(const WriteOptions& write_options, + WriteBatch* my_batch) { + assert(write_options.low_pri); + // This is called outside the DB mutex. Although it is safe to make the call, + // the consistency condition is not guaranteed to hold. It's OK to live with + // it in this case. + // If we need to speed compaction, it means the compaction is left behind + // and we start to limit low pri writes to a limit. + if (write_controller_.NeedSpeedupCompaction()) { + if (allow_2pc() && (my_batch->HasCommit() || my_batch->HasRollback())) { + // For 2PC, we only rate limit prepare, not commit. + return Status::OK(); + } + if (write_options.no_slowdown) { + return Status::Incomplete("Low priority write stall"); + } else { + assert(my_batch != nullptr); + // Rate limit those writes. The reason that we don't completely wait + // is that in case the write is heavy, low pri writes may never have + // a chance to run. Now we guarantee we are still slowly making + // progress. + PERF_TIMER_GUARD(write_delay_time); + write_controller_.low_pri_rate_limiter()->Request( + my_batch->GetDataSize(), Env::IO_HIGH, nullptr /* stats */, + RateLimiter::OpType::kWrite); + } + } + return Status::OK(); +} + +void DBImpl::MaybeFlushStatsCF(autovector* cfds) { + assert(cfds != nullptr); + if (!cfds->empty() && immutable_db_options_.persist_stats_to_disk) { + ColumnFamilyData* cfd_stats = + versions_->GetColumnFamilySet()->GetColumnFamily( + kPersistentStatsColumnFamilyName); + if (cfd_stats != nullptr && !cfd_stats->mem()->IsEmpty()) { + for (ColumnFamilyData* cfd : *cfds) { + if (cfd == cfd_stats) { + // stats CF already included in cfds + return; + } + } + // force flush stats CF when its log number is less than all other CF's + // log numbers + bool force_flush_stats_cf = true; + for (auto* loop_cfd : *versions_->GetColumnFamilySet()) { + if (loop_cfd == cfd_stats) { + continue; + } + if (loop_cfd->GetLogNumber() <= cfd_stats->GetLogNumber()) { + force_flush_stats_cf = false; + } + } + if (force_flush_stats_cf) { + cfds->push_back(cfd_stats); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Force flushing stats CF with automated flush " + "to avoid holding old logs"); + } + } + } +} + +Status DBImpl::TrimMemtableHistory(WriteContext* context) { + autovector cfds; + ColumnFamilyData* tmp_cfd; + while ((tmp_cfd = trim_history_scheduler_.TakeNextColumnFamily()) != + nullptr) { + cfds.push_back(tmp_cfd); + } + for (auto& cfd : cfds) { + autovector to_delete; + bool trimmed = cfd->imm()->TrimHistory(&context->memtables_to_free_, + cfd->mem()->MemoryAllocatedBytes()); + if (trimmed) { + context->superversion_context.NewSuperVersion(); + assert(context->superversion_context.new_superversion.get() != nullptr); + cfd->InstallSuperVersion(&context->superversion_context, &mutex_); + } + + if (cfd->UnrefAndTryDelete()) { + cfd = nullptr; + } + } + return Status::OK(); +} + +Status DBImpl::ScheduleFlushes(WriteContext* context) { + autovector cfds; + if (immutable_db_options_.atomic_flush) { + SelectColumnFamiliesForAtomicFlush(&cfds); + for (auto cfd : cfds) { + cfd->Ref(); + } + flush_scheduler_.Clear(); + } else { + ColumnFamilyData* tmp_cfd; + while ((tmp_cfd = flush_scheduler_.TakeNextColumnFamily()) != nullptr) { + cfds.push_back(tmp_cfd); + } + MaybeFlushStatsCF(&cfds); + } + Status status; + WriteThread::Writer nonmem_w; + if (two_write_queues_) { + nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_); + } + + for (auto& cfd : cfds) { + if (!cfd->mem()->IsEmpty()) { + status = SwitchMemtable(cfd, context); + } + if (cfd->UnrefAndTryDelete()) { + cfd = nullptr; + } + if (!status.ok()) { + break; + } + } + + if (two_write_queues_) { + nonmem_write_thread_.ExitUnbatched(&nonmem_w); + } + + if (status.ok()) { + if (immutable_db_options_.atomic_flush) { + AssignAtomicFlushSeq(cfds); + FlushRequest flush_req; + GenerateFlushRequest(cfds, &flush_req); + SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull); + } else { + for (auto* cfd : cfds) { + FlushRequest flush_req; + GenerateFlushRequest({cfd}, &flush_req); + SchedulePendingFlush(flush_req, FlushReason::kWriteBufferFull); + } + } + MaybeScheduleFlushOrCompaction(); + } + return status; +} + +#ifndef ROCKSDB_LITE +void DBImpl::NotifyOnMemTableSealed(ColumnFamilyData* /*cfd*/, + const MemTableInfo& mem_table_info) { + if (immutable_db_options_.listeners.size() == 0U) { + return; + } + if (shutting_down_.load(std::memory_order_acquire)) { + return; + } + + mutex_.Unlock(); + for (auto listener : immutable_db_options_.listeners) { + listener->OnMemTableSealed(mem_table_info); + } + mutex_.Lock(); +} +#endif // ROCKSDB_LITE + +// REQUIRES: mutex_ is held +// REQUIRES: this thread is currently at the front of the writer queue +// REQUIRES: this thread is currently at the front of the 2nd writer queue if +// two_write_queues_ is true (This is to simplify the reasoning.) +Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) { + mutex_.AssertHeld(); + log::Writer* new_log = nullptr; + MemTable* new_mem = nullptr; + IOStatus io_s; + + // Recoverable state is persisted in WAL. After memtable switch, WAL might + // be deleted, so we write the state to memtable to be persisted as well. + Status s = WriteRecoverableState(); + if (!s.ok()) { + return s; + } + + // Attempt to switch to a new memtable and trigger flush of old. + // Do this without holding the dbmutex lock. + assert(versions_->prev_log_number() == 0); + if (two_write_queues_) { + log_write_mutex_.Lock(); + } + bool creating_new_log = !log_empty_; + if (two_write_queues_) { + log_write_mutex_.Unlock(); + } + uint64_t recycle_log_number = 0; + if (creating_new_log && immutable_db_options_.recycle_log_file_num && + !log_recycle_files_.empty()) { + recycle_log_number = log_recycle_files_.front(); + } + uint64_t new_log_number = + creating_new_log ? versions_->NewFileNumber() : logfile_number_; + const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions(); + + // Set memtable_info for memtable sealed callback +#ifndef ROCKSDB_LITE + MemTableInfo memtable_info; + memtable_info.cf_name = cfd->GetName(); + memtable_info.first_seqno = cfd->mem()->GetFirstSequenceNumber(); + memtable_info.earliest_seqno = cfd->mem()->GetEarliestSequenceNumber(); + memtable_info.num_entries = cfd->mem()->num_entries(); + memtable_info.num_deletes = cfd->mem()->num_deletes(); +#endif // ROCKSDB_LITE + // Log this later after lock release. It may be outdated, e.g., if background + // 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); + mutex_.Unlock(); + if (creating_new_log) { + // TODO: Write buffer size passed in should be max of all CF's instead + // of mutable_cf_options.write_buffer_size. + io_s = CreateWAL(new_log_number, recycle_log_number, preallocate_block_size, + &new_log); + if (s.ok()) { + s = io_s; + } + } + if (s.ok()) { + SequenceNumber seq = versions_->LastSequence(); + new_mem = cfd->ConstructNewMemtable(mutable_cf_options, seq); + context->superversion_context.NewSuperVersion(); + } + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[%s] New memtable created with log file: #%" PRIu64 + ". Immutable memtables: %d.\n", + cfd->GetName().c_str(), new_log_number, num_imm_unflushed); + // There should be no concurrent write as the thread is at the front of + // writer queue + cfd->mem()->ConstructFragmentedRangeTombstones(); + + mutex_.Lock(); + if (recycle_log_number != 0) { + // Since renaming the file is done outside DB mutex, we need to ensure + // concurrent full purges don't delete the file while we're recycling it. + // To achieve that we hold the old log number in the recyclable list until + // after it has been renamed. + assert(log_recycle_files_.front() == recycle_log_number); + log_recycle_files_.pop_front(); + } + if (s.ok() && creating_new_log) { + InstrumentedMutexLock l(&log_write_mutex_); + assert(new_log != nullptr); + if (!logs_.empty()) { + // Alway flush the buffer of the last log before switching to a new one + log::Writer* cur_log_writer = logs_.back().writer; + if (error_handler_.IsRecoveryInProgress()) { + // In recovery path, we force another try of writing WAL buffer. + cur_log_writer->file()->reset_seen_error(); + } + io_s = cur_log_writer->WriteBuffer(); + if (s.ok()) { + s = io_s; + } + if (!s.ok()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "[%s] Failed to switch from #%" PRIu64 " to #%" PRIu64 + " WAL file\n", + cfd->GetName().c_str(), cur_log_writer->get_log_number(), + new_log_number); + } + } + if (s.ok()) { + logfile_number_ = new_log_number; + log_empty_ = true; + log_dir_synced_ = false; + logs_.emplace_back(logfile_number_, new_log); + alive_log_files_.push_back(LogFileNumberSize(logfile_number_)); + } + } + + if (!s.ok()) { + // how do we fail if we're not creating new log? + assert(creating_new_log); + delete new_mem; + delete new_log; + context->superversion_context.new_superversion.reset(); + // We may have lost data from the WritableFileBuffer in-memory buffer for + // the current log, so treat it as a fatal error and set bg_error + if (!io_s.ok()) { + error_handler_.SetBGError(io_s, BackgroundErrorReason::kMemTable); + } else { + error_handler_.SetBGError(s, BackgroundErrorReason::kMemTable); + } + // Read back bg_error in order to get the right severity + s = error_handler_.GetBGError(); + return s; + } + + bool empty_cf_updated = false; + if (immutable_db_options_.track_and_verify_wals_in_manifest && + !immutable_db_options_.allow_2pc && creating_new_log) { + // In non-2pc mode, WALs become obsolete if they do not contain unflushed + // data. Updating the empty CF's log number might cause some WALs to become + // obsolete. So we should track the WAL obsoletion event before actually + // updating the empty CF's log number. + uint64_t min_wal_number_to_keep = + versions_->PreComputeMinLogNumberWithUnflushedData(logfile_number_); + if (min_wal_number_to_keep > + versions_->GetWalSet().GetMinWalNumberToKeep()) { + // Get a snapshot of the empty column families. + // LogAndApply may release and reacquire db + // mutex, during that period, column family may become empty (e.g. its + // flush succeeds), then it affects the computed min_log_number_to_keep, + // so we take a snapshot for consistency of column family data + // status. If a column family becomes non-empty afterwards, its active log + // should still be the created new log, so the min_log_number_to_keep is + // not affected. + autovector empty_cfs; + for (auto cf : *versions_->GetColumnFamilySet()) { + if (cf->IsEmpty()) { + empty_cfs.push_back(cf); + } + } + + VersionEdit wal_deletion; + wal_deletion.DeleteWalsBefore(min_wal_number_to_keep); + s = versions_->LogAndApplyToDefaultColumnFamily(&wal_deletion, &mutex_, + directories_.GetDbDir()); + if (!s.ok() && versions_->io_status().IsIOError()) { + s = error_handler_.SetBGError(versions_->io_status(), + BackgroundErrorReason::kManifestWrite); + } + if (!s.ok()) { + return s; + } + + for (auto cf : empty_cfs) { + if (cf->IsEmpty()) { + cf->SetLogNumber(logfile_number_); + // MEMPURGE: No need to change this, because new adds + // should still receive new sequence numbers. + cf->mem()->SetCreationSeq(versions_->LastSequence()); + } // cf may become non-empty. + } + empty_cf_updated = true; + } + } + if (!empty_cf_updated) { + for (auto cf : *versions_->GetColumnFamilySet()) { + // all this is just optimization to delete logs that + // are no longer needed -- if CF is empty, that means it + // doesn't need that particular log to stay alive, so we just + // advance the log number. no need to persist this in the manifest + if (cf->IsEmpty()) { + if (creating_new_log) { + cf->SetLogNumber(logfile_number_); + } + cf->mem()->SetCreationSeq(versions_->LastSequence()); + } + } + } + + cfd->mem()->SetNextLogNumber(logfile_number_); + assert(new_mem != nullptr); + cfd->imm()->Add(cfd->mem(), &context->memtables_to_free_); + new_mem->Ref(); + cfd->SetMemtable(new_mem); + InstallSuperVersionAndScheduleWork(cfd, &context->superversion_context, + mutable_cf_options); + +#ifndef ROCKSDB_LITE + // Notify client that memtable is sealed, now that we have successfully + // installed a new memtable + NotifyOnMemTableSealed(cfd, memtable_info); +#endif // ROCKSDB_LITE + // It is possible that we got here without checking the value of i_os, but + // that is okay. If we did, it most likely means that s was already an error. + // In any case, ignore any unchecked error for i_os here. + io_s.PermitUncheckedError(); + return s; +} + +size_t DBImpl::GetWalPreallocateBlockSize(uint64_t write_buffer_size) const { + mutex_.AssertHeld(); + size_t bsize = + static_cast(write_buffer_size / 10 + write_buffer_size); + // Some users might set very high write_buffer_size and rely on + // max_total_wal_size or other parameters to control the WAL size. + if (mutable_db_options_.max_total_wal_size > 0) { + bsize = std::min( + bsize, static_cast(mutable_db_options_.max_total_wal_size)); + } + if (immutable_db_options_.db_write_buffer_size > 0) { + bsize = std::min(bsize, immutable_db_options_.db_write_buffer_size); + } + if (immutable_db_options_.write_buffer_manager && + immutable_db_options_.write_buffer_manager->enabled()) { + bsize = std::min( + bsize, immutable_db_options_.write_buffer_manager->buffer_size()); + } + + return bsize; +} + +// Default implementations of convenience methods that subclasses of DB +// can call if they wish +Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& value) { + // Pre-allocate size of write batch conservatively. + // 8 bytes are taken by header, 4 bytes for count, 1 byte for type, + // and we allocate 11 extra bytes for key length, as well as value length. + WriteBatch batch(key.size() + value.size() + 24, 0 /* max_bytes */, + opt.protection_bytes_per_key, 0 /* default_cf_ts_sz */); + Status s = batch.Put(column_family, key, value); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts, const Slice& value) { + ColumnFamilyHandle* default_cf = DefaultColumnFamily(); + assert(default_cf); + const Comparator* const default_cf_ucmp = default_cf->GetComparator(); + assert(default_cf_ucmp); + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, + default_cf_ucmp->timestamp_size()); + Status s = batch.Put(column_family, key, ts, value); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::PutEntity(const WriteOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns) { + const ColumnFamilyHandle* const default_cf = DefaultColumnFamily(); + assert(default_cf); + + const Comparator* const default_cf_ucmp = default_cf->GetComparator(); + assert(default_cf_ucmp); + + WriteBatch batch(/* reserved_bytes */ 0, /* max_bytes */ 0, + options.protection_bytes_per_key, + default_cf_ucmp->timestamp_size()); + + const Status s = batch.PutEntity(column_family, key, columns); + if (!s.ok()) { + return s; + } + + return Write(options, &batch); +} + +Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family, + const Slice& key) { + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, 0 /* default_cf_ts_sz */); + Status s = batch.Delete(column_family, key); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts) { + ColumnFamilyHandle* default_cf = DefaultColumnFamily(); + assert(default_cf); + const Comparator* const default_cf_ucmp = default_cf->GetComparator(); + assert(default_cf_ucmp); + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, + default_cf_ucmp->timestamp_size()); + Status s = batch.Delete(column_family, key, ts); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::SingleDelete(const WriteOptions& opt, + ColumnFamilyHandle* column_family, const Slice& key) { + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, 0 /* default_cf_ts_sz */); + Status s = batch.SingleDelete(column_family, key); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::SingleDelete(const WriteOptions& opt, + ColumnFamilyHandle* column_family, const Slice& key, + const Slice& ts) { + ColumnFamilyHandle* default_cf = DefaultColumnFamily(); + assert(default_cf); + const Comparator* const default_cf_ucmp = default_cf->GetComparator(); + assert(default_cf_ucmp); + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, + default_cf_ucmp->timestamp_size()); + Status s = batch.SingleDelete(column_family, key, ts); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::DeleteRange(const WriteOptions& opt, + ColumnFamilyHandle* column_family, + const Slice& begin_key, const Slice& end_key) { + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, 0 /* default_cf_ts_sz */); + Status s = batch.DeleteRange(column_family, begin_key, end_key); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::DeleteRange(const WriteOptions& opt, + ColumnFamilyHandle* column_family, + const Slice& begin_key, const Slice& end_key, + const Slice& ts) { + ColumnFamilyHandle* default_cf = DefaultColumnFamily(); + assert(default_cf); + const Comparator* const default_cf_ucmp = default_cf->GetComparator(); + assert(default_cf_ucmp); + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, + default_cf_ucmp->timestamp_size()); + Status s = batch.DeleteRange(column_family, begin_key, end_key, ts); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& value) { + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, 0 /* default_cf_ts_sz */); + Status s = batch.Merge(column_family, key, value); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family, + const Slice& key, const Slice& ts, const Slice& value) { + ColumnFamilyHandle* default_cf = DefaultColumnFamily(); + assert(default_cf); + const Comparator* const default_cf_ucmp = default_cf->GetComparator(); + assert(default_cf_ucmp); + WriteBatch batch(0 /* reserved_bytes */, 0 /* max_bytes */, + opt.protection_bytes_per_key, + default_cf_ucmp->timestamp_size()); + Status s = batch.Merge(column_family, key, ts, value); + if (!s.ok()) { + return s; + } + return Write(opt, &batch); +} + +} // namespace ROCKSDB_NAMESPACE -- cgit v1.2.3