mirror of
https://github.com/facebook/rocksdb.git
synced 2024-11-26 16:30:56 +00:00
98d5db5c2e
Summary:
**Context:**
Sorting L0 files by `largest_seqno` has at least two inconvenience:
- File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap.
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n")
- insert k1@1 to memtable m1
- ingest file s1 with k2@2, ingest file s2 with k3@3
- insert k4@4 to m1
- compact files s1, s2 and result in new file s3 of seqno range [2, 3]
- flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1
- However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption.
- Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example)
- an existing SST s1 contains only k1@1
- insert k1@2 to memtable m1
- ingest file s2 with k3@3, ingest file s3 with k4@4
- insert single delete k5@5 in m1
- flush m1 and result in new file s4 of seqno range [2, 5]
- compact s1, s2, s3 and result in new file s5 of seqno range [1, 4]
- compact s4 and result in new file s6 of seqno range [2] due to single delete
- By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno`
Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways:
- In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more.
- In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption.
**Summary:**
- Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`.
- `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`)
- Compaction output file is assigned with the minimum `epoch_number` among input files'
- Refit level: reuse refitted file's epoch_number
- Other paths needing `epoch_number` treatment:
- Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`
- Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`.
- Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair).
- Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder.
- Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery
- Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more
- Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag`
- Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above
- Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`.
- Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR.
- Misc:
- update existing tests with `epoch_number` so make check will pass
- update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases
- assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber()
Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922
Test Plan:
- `make check`
- New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc`
- Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930
- [Ongoing] Compatibility test: manually run 36a5686ec0
(with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox`
- [Ongoing] normal db stress test
- [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761
Reviewed By: ajkr
Differential Revision: D41063187
Pulled By: hx235
fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
866 lines
35 KiB
C++
866 lines
35 KiB
C++
// 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 <atomic>
|
|
#include <string>
|
|
#include <unordered_map>
|
|
#include <vector>
|
|
|
|
#include "cache/cache_reservation_manager.h"
|
|
#include "db/memtable_list.h"
|
|
#include "db/table_cache.h"
|
|
#include "db/table_properties_collector.h"
|
|
#include "db/write_batch_internal.h"
|
|
#include "db/write_controller.h"
|
|
#include "options/cf_options.h"
|
|
#include "rocksdb/compaction_job_stats.h"
|
|
#include "rocksdb/db.h"
|
|
#include "rocksdb/env.h"
|
|
#include "rocksdb/options.h"
|
|
#include "trace_replay/block_cache_tracer.h"
|
|
#include "util/hash_containers.h"
|
|
#include "util/thread_local.h"
|
|
|
|
namespace ROCKSDB_NAMESPACE {
|
|
|
|
class Version;
|
|
class VersionSet;
|
|
class VersionStorageInfo;
|
|
class MemTable;
|
|
class MemTableListVersion;
|
|
class CompactionPicker;
|
|
class Compaction;
|
|
class InternalKey;
|
|
class InternalStats;
|
|
class ColumnFamilyData;
|
|
class DBImpl;
|
|
class LogBuffer;
|
|
class InstrumentedMutex;
|
|
class InstrumentedMutexLock;
|
|
struct SuperVersionContext;
|
|
class BlobFileCache;
|
|
class BlobSource;
|
|
|
|
extern const double kIncSlowdownRatio;
|
|
// This file contains a list of data structures for managing column family
|
|
// level metadata.
|
|
//
|
|
// The basic relationships among classes declared here are illustrated as
|
|
// following:
|
|
//
|
|
// +----------------------+ +----------------------+ +--------+
|
|
// +---+ ColumnFamilyHandle 1 | +--+ ColumnFamilyHandle 2 | | DBImpl |
|
|
// | +----------------------+ | +----------------------+ +----+---+
|
|
// | +--------------------------+ |
|
|
// | | +-----------------------------+
|
|
// | | |
|
|
// | | +-----------------------------v-------------------------------+
|
|
// | | | |
|
|
// | | | ColumnFamilySet |
|
|
// | | | |
|
|
// | | +-------------+--------------------------+----------------+---+
|
|
// | | | | |
|
|
// | +-------------------------------------+ | |
|
|
// | | | | v
|
|
// | +-------------v-------------+ +-----v----v---------+
|
|
// | | | | |
|
|
// | | ColumnFamilyData 1 | | ColumnFamilyData 2 | ......
|
|
// | | | | |
|
|
// +---> | | |
|
|
// | +---------+ | |
|
|
// | | MemTable| | |
|
|
// | | List | | |
|
|
// +--------+---+--+-+----+----+ +--------------------++
|
|
// | | | |
|
|
// | | | |
|
|
// | | | +-----------------------+
|
|
// | | +-----------+ |
|
|
// v +--------+ | |
|
|
// +--------+--------+ | | |
|
|
// | | | | +----------v----------+
|
|
// +---> |SuperVersion 1.a +-----------------> |
|
|
// | +------+ | | MemTableListVersion |
|
|
// +---+-------------+ | | | | |
|
|
// | | | | +----+------------+---+
|
|
// | current | | | | |
|
|
// | +-------------+ | |mem | |
|
|
// | | | | | |
|
|
// +-v---v-------+ +---v--v---+ +-----v----+ +----v-----+
|
|
// | | | | | | | |
|
|
// | Version 1.a | | memtable | | memtable | | memtable |
|
|
// | | | 1.a | | 1.b | | 1.c |
|
|
// +-------------+ | | | | | |
|
|
// +----------+ +----------+ +----------+
|
|
//
|
|
// DBImpl keeps a ColumnFamilySet, which references to all column families by
|
|
// pointing to respective ColumnFamilyData object of each column family.
|
|
// This is how DBImpl can list and operate on all the column families.
|
|
// ColumnFamilyHandle also points to ColumnFamilyData directly, so that
|
|
// when a user executes a query, it can directly find memtables and Version
|
|
// as well as SuperVersion to the column family, without going through
|
|
// ColumnFamilySet.
|
|
//
|
|
// ColumnFamilySet points to the latest view of the LSM-tree (list of memtables
|
|
// and SST files) indirectly, while ongoing operations may hold references
|
|
// to a current or an out-of-date SuperVersion, which in turn points to a
|
|
// point-in-time view of the LSM-tree. This guarantees the memtables and SST
|
|
// files being operated on will not go away, until the SuperVersion is
|
|
// unreferenced to 0 and destoryed.
|
|
//
|
|
// The following graph illustrates a possible referencing relationships:
|
|
//
|
|
// Column +--------------+ current +-----------+
|
|
// Family +---->+ +------------------->+ |
|
|
// Data | SuperVersion +----------+ | Version A |
|
|
// | 3 | imm | | |
|
|
// Iter2 +----->+ | +-------v------+ +-----------+
|
|
// +-----+--------+ | MemtableList +----------------> Empty
|
|
// | | Version r | +-----------+
|
|
// | +--------------+ | |
|
|
// +------------------+ current| Version B |
|
|
// +--------------+ | +----->+ |
|
|
// | | | | +-----+-----+
|
|
// Compaction +>+ SuperVersion +-------------+ ^
|
|
// Job | 2 +------+ | |current
|
|
// | +----+ | | mem | +------------+
|
|
// +--------------+ | | +---------------------> |
|
|
// | +------------------------> MemTable a |
|
|
// | mem | | |
|
|
// +--------------+ | | +------------+
|
|
// | +--------------------------+
|
|
// Iter1 +-----> SuperVersion | | +------------+
|
|
// | 1 +------------------------------>+ |
|
|
// | +-+ | mem | MemTable b |
|
|
// +--------------+ | | | |
|
|
// | | +--------------+ +-----^------+
|
|
// | |imm | MemtableList | |
|
|
// | +--->+ Version s +------------+
|
|
// | +--------------+
|
|
// | +--------------+
|
|
// | | MemtableList |
|
|
// +------>+ Version t +--------> Empty
|
|
// imm +--------------+
|
|
//
|
|
// In this example, even if the current LSM-tree consists of Version A and
|
|
// memtable a, which is also referenced by SuperVersion, two older SuperVersion
|
|
// SuperVersion2 and Superversion1 still exist, and are referenced by a
|
|
// compaction job and an old iterator Iter1, respectively. SuperVersion2
|
|
// contains Version B, memtable a and memtable b; SuperVersion1 contains
|
|
// Version B and memtable b (mutable). As a result, Version B and memtable b
|
|
// are prevented from being destroyed or deleted.
|
|
|
|
// ColumnFamilyHandleImpl is the class that clients use to access different
|
|
// column families. It has non-trivial destructor, which gets called when client
|
|
// is done using the column family
|
|
class ColumnFamilyHandleImpl : public ColumnFamilyHandle {
|
|
public:
|
|
// create while holding the mutex
|
|
ColumnFamilyHandleImpl(ColumnFamilyData* cfd, DBImpl* db,
|
|
InstrumentedMutex* mutex);
|
|
// destroy without mutex
|
|
virtual ~ColumnFamilyHandleImpl();
|
|
virtual ColumnFamilyData* cfd() const { return cfd_; }
|
|
|
|
virtual uint32_t GetID() const override;
|
|
virtual const std::string& GetName() const override;
|
|
virtual Status GetDescriptor(ColumnFamilyDescriptor* desc) override;
|
|
virtual const Comparator* GetComparator() const override;
|
|
|
|
private:
|
|
ColumnFamilyData* cfd_;
|
|
DBImpl* db_;
|
|
InstrumentedMutex* mutex_;
|
|
};
|
|
|
|
// Does not ref-count ColumnFamilyData
|
|
// We use this dummy ColumnFamilyHandleImpl because sometimes MemTableInserter
|
|
// calls DBImpl methods. When this happens, MemTableInserter need access to
|
|
// ColumnFamilyHandle (same as the client would need). In that case, we feed
|
|
// MemTableInserter dummy ColumnFamilyHandle and enable it to call DBImpl
|
|
// methods
|
|
class ColumnFamilyHandleInternal : public ColumnFamilyHandleImpl {
|
|
public:
|
|
ColumnFamilyHandleInternal()
|
|
: ColumnFamilyHandleImpl(nullptr, nullptr, nullptr),
|
|
internal_cfd_(nullptr) {}
|
|
|
|
void SetCFD(ColumnFamilyData* _cfd) { internal_cfd_ = _cfd; }
|
|
virtual ColumnFamilyData* cfd() const override { return internal_cfd_; }
|
|
|
|
private:
|
|
ColumnFamilyData* internal_cfd_;
|
|
};
|
|
|
|
// holds references to memtable, all immutable memtables and version
|
|
struct SuperVersion {
|
|
// Accessing members of this class is not thread-safe and requires external
|
|
// synchronization (ie db mutex held or on write thread).
|
|
ColumnFamilyData* cfd;
|
|
MemTable* mem;
|
|
MemTableListVersion* imm;
|
|
Version* current;
|
|
MutableCFOptions mutable_cf_options;
|
|
// Version number of the current SuperVersion
|
|
uint64_t version_number;
|
|
WriteStallCondition write_stall_condition;
|
|
|
|
// should be called outside the mutex
|
|
SuperVersion() = default;
|
|
~SuperVersion();
|
|
SuperVersion* Ref();
|
|
// If Unref() returns true, Cleanup() should be called with mutex held
|
|
// before deleting this SuperVersion.
|
|
bool Unref();
|
|
|
|
// call these two methods with db mutex held
|
|
// Cleanup unrefs mem, imm and current. Also, it stores all memtables
|
|
// that needs to be deleted in to_delete vector. Unrefing those
|
|
// objects needs to be done in the mutex
|
|
void Cleanup();
|
|
void Init(ColumnFamilyData* new_cfd, MemTable* new_mem,
|
|
MemTableListVersion* new_imm, Version* new_current);
|
|
|
|
// The value of dummy is not actually used. kSVInUse takes its address as a
|
|
// mark in the thread local storage to indicate the SuperVersion is in use
|
|
// by thread. This way, the value of kSVInUse is guaranteed to have no
|
|
// conflict with SuperVersion object address and portable on different
|
|
// platform.
|
|
static int dummy;
|
|
static void* const kSVInUse;
|
|
static void* const kSVObsolete;
|
|
|
|
private:
|
|
std::atomic<uint32_t> refs;
|
|
// We need to_delete because during Cleanup(), imm->Unref() returns
|
|
// all memtables that we need to free through this vector. We then
|
|
// delete all those memtables outside of mutex, during destruction
|
|
autovector<MemTable*> to_delete;
|
|
};
|
|
|
|
extern Status CheckCompressionSupported(const ColumnFamilyOptions& cf_options);
|
|
|
|
extern Status CheckConcurrentWritesSupported(
|
|
const ColumnFamilyOptions& cf_options);
|
|
|
|
extern Status CheckCFPathsSupported(const DBOptions& db_options,
|
|
const ColumnFamilyOptions& cf_options);
|
|
|
|
extern ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
|
|
const ColumnFamilyOptions& src);
|
|
// Wrap user defined table properties collector factories `from cf_options`
|
|
// into internal ones in int_tbl_prop_collector_factories. Add a system internal
|
|
// one too.
|
|
extern void GetIntTblPropCollectorFactory(
|
|
const ImmutableCFOptions& ioptions,
|
|
IntTblPropCollectorFactories* int_tbl_prop_collector_factories);
|
|
|
|
class ColumnFamilySet;
|
|
|
|
// This class keeps all the data that a column family needs.
|
|
// Most methods require DB mutex held, unless otherwise noted
|
|
class ColumnFamilyData {
|
|
public:
|
|
~ColumnFamilyData();
|
|
|
|
// thread-safe
|
|
uint32_t GetID() const { return id_; }
|
|
// thread-safe
|
|
const std::string& GetName() const { return name_; }
|
|
|
|
// Ref() can only be called from a context where the caller can guarantee
|
|
// that ColumnFamilyData is alive (while holding a non-zero ref already,
|
|
// holding a DB mutex, or as the leader in a write batch group).
|
|
void Ref() { refs_.fetch_add(1); }
|
|
|
|
// UnrefAndTryDelete() decreases the reference count and do free if needed,
|
|
// return true if this is freed else false, UnrefAndTryDelete() can only
|
|
// be called while holding a DB mutex, or during single-threaded recovery.
|
|
bool UnrefAndTryDelete();
|
|
|
|
// SetDropped() can only be called under following conditions:
|
|
// 1) Holding a DB mutex,
|
|
// 2) from single-threaded write thread, AND
|
|
// 3) from single-threaded VersionSet::LogAndApply()
|
|
// After dropping column family no other operation on that column family
|
|
// will be executed. All the files and memory will be, however, kept around
|
|
// until client drops the column family handle. That way, client can still
|
|
// access data from dropped column family.
|
|
// Column family can be dropped and still alive. In that state:
|
|
// *) Compaction and flush is not executed on the dropped column family.
|
|
// *) Client can continue reading from column family. Writes will fail unless
|
|
// WriteOptions::ignore_missing_column_families is true
|
|
// When the dropped column family is unreferenced, then we:
|
|
// *) Remove column family from the linked list maintained by ColumnFamilySet
|
|
// *) delete all memory associated with that column family
|
|
// *) delete all the files associated with that column family
|
|
void SetDropped();
|
|
bool IsDropped() const { return dropped_.load(std::memory_order_relaxed); }
|
|
|
|
// thread-safe
|
|
int NumberLevels() const { return ioptions_.num_levels; }
|
|
|
|
void SetLogNumber(uint64_t log_number) { log_number_ = log_number; }
|
|
uint64_t GetLogNumber() const { return log_number_; }
|
|
|
|
void SetFlushReason(FlushReason flush_reason) {
|
|
flush_reason_ = flush_reason;
|
|
}
|
|
FlushReason GetFlushReason() const { return flush_reason_; }
|
|
// thread-safe
|
|
const FileOptions* soptions() const;
|
|
const ImmutableOptions* ioptions() const { return &ioptions_; }
|
|
// REQUIRES: DB mutex held
|
|
// This returns the MutableCFOptions used by current SuperVersion
|
|
// You should use this API to reference MutableCFOptions most of the time.
|
|
const MutableCFOptions* GetCurrentMutableCFOptions() const {
|
|
return &(super_version_->mutable_cf_options);
|
|
}
|
|
// REQUIRES: DB mutex held
|
|
// This returns the latest MutableCFOptions, which may be not in effect yet.
|
|
const MutableCFOptions* GetLatestMutableCFOptions() const {
|
|
return &mutable_cf_options_;
|
|
}
|
|
|
|
// REQUIRES: DB mutex held
|
|
// Build ColumnFamiliesOptions with immutable options and latest mutable
|
|
// options.
|
|
ColumnFamilyOptions GetLatestCFOptions() const;
|
|
|
|
bool is_delete_range_supported() { return is_delete_range_supported_; }
|
|
|
|
// Validate CF options against DB options
|
|
static Status ValidateOptions(const DBOptions& db_options,
|
|
const ColumnFamilyOptions& cf_options);
|
|
#ifndef ROCKSDB_LITE
|
|
// REQUIRES: DB mutex held
|
|
Status SetOptions(
|
|
const DBOptions& db_options,
|
|
const std::unordered_map<std::string, std::string>& options_map);
|
|
#endif // ROCKSDB_LITE
|
|
|
|
InternalStats* internal_stats() { return internal_stats_.get(); }
|
|
|
|
MemTableList* imm() { return &imm_; }
|
|
MemTable* mem() { return mem_; }
|
|
|
|
bool IsEmpty() {
|
|
return mem()->GetFirstSequenceNumber() == 0 && imm()->NumNotFlushed() == 0;
|
|
}
|
|
|
|
Version* current() { return current_; }
|
|
Version* dummy_versions() { return dummy_versions_; }
|
|
void SetCurrent(Version* _current);
|
|
uint64_t GetNumLiveVersions() const; // REQUIRE: DB mutex held
|
|
uint64_t GetTotalSstFilesSize() const; // REQUIRE: DB mutex held
|
|
uint64_t GetLiveSstFilesSize() const; // REQUIRE: DB mutex held
|
|
uint64_t GetTotalBlobFileSize() const; // REQUIRE: DB mutex held
|
|
void SetMemtable(MemTable* new_mem) {
|
|
uint64_t memtable_id = last_memtable_id_.fetch_add(1) + 1;
|
|
new_mem->SetID(memtable_id);
|
|
mem_ = new_mem;
|
|
}
|
|
|
|
// calculate the oldest log needed for the durability of this column family
|
|
uint64_t OldestLogToKeep();
|
|
|
|
// See Memtable constructor for explanation of earliest_seq param.
|
|
MemTable* ConstructNewMemtable(const MutableCFOptions& mutable_cf_options,
|
|
SequenceNumber earliest_seq);
|
|
void CreateNewMemtable(const MutableCFOptions& mutable_cf_options,
|
|
SequenceNumber earliest_seq);
|
|
|
|
TableCache* table_cache() const { return table_cache_.get(); }
|
|
BlobSource* blob_source() const { return blob_source_.get(); }
|
|
|
|
// See documentation in compaction_picker.h
|
|
// REQUIRES: DB mutex held
|
|
bool NeedsCompaction() const;
|
|
// REQUIRES: DB mutex held
|
|
Compaction* PickCompaction(const MutableCFOptions& mutable_options,
|
|
const MutableDBOptions& mutable_db_options,
|
|
LogBuffer* log_buffer);
|
|
|
|
// Check if the passed range overlap with any running compactions.
|
|
// REQUIRES: DB mutex held
|
|
bool RangeOverlapWithCompaction(const Slice& smallest_user_key,
|
|
const Slice& largest_user_key,
|
|
int level) const;
|
|
|
|
// Check if the passed ranges overlap with any unflushed memtables
|
|
// (immutable or mutable).
|
|
//
|
|
// @param super_version A referenced SuperVersion that will be held for the
|
|
// duration of this function.
|
|
//
|
|
// Thread-safe
|
|
Status RangesOverlapWithMemtables(const autovector<Range>& ranges,
|
|
SuperVersion* super_version,
|
|
bool allow_data_in_errors, bool* overlap);
|
|
|
|
// A flag to tell a manual compaction is to compact all levels together
|
|
// instead of a specific level.
|
|
static const int kCompactAllLevels;
|
|
// A flag to tell a manual compaction's output is base level.
|
|
static const int kCompactToBaseLevel;
|
|
// REQUIRES: DB mutex held
|
|
Compaction* CompactRange(const MutableCFOptions& mutable_cf_options,
|
|
const MutableDBOptions& mutable_db_options,
|
|
int input_level, int output_level,
|
|
const CompactRangeOptions& compact_range_options,
|
|
const InternalKey* begin, const InternalKey* end,
|
|
InternalKey** compaction_end, bool* manual_conflict,
|
|
uint64_t max_file_num_to_ignore,
|
|
const std::string& trim_ts);
|
|
|
|
CompactionPicker* compaction_picker() { return compaction_picker_.get(); }
|
|
// thread-safe
|
|
const Comparator* user_comparator() const {
|
|
return internal_comparator_.user_comparator();
|
|
}
|
|
// thread-safe
|
|
const InternalKeyComparator& internal_comparator() const {
|
|
return internal_comparator_;
|
|
}
|
|
|
|
const IntTblPropCollectorFactories* int_tbl_prop_collector_factories() const {
|
|
return &int_tbl_prop_collector_factories_;
|
|
}
|
|
|
|
SuperVersion* GetSuperVersion() { return super_version_; }
|
|
// thread-safe
|
|
// Return a already referenced SuperVersion to be used safely.
|
|
SuperVersion* GetReferencedSuperVersion(DBImpl* db);
|
|
// thread-safe
|
|
// Get SuperVersion stored in thread local storage. If it does not exist,
|
|
// get a reference from a current SuperVersion.
|
|
SuperVersion* GetThreadLocalSuperVersion(DBImpl* db);
|
|
// Try to return SuperVersion back to thread local storage. Return true on
|
|
// success and false on failure. It fails when the thread local storage
|
|
// contains anything other than SuperVersion::kSVInUse flag.
|
|
bool ReturnThreadLocalSuperVersion(SuperVersion* sv);
|
|
// thread-safe
|
|
uint64_t GetSuperVersionNumber() const {
|
|
return super_version_number_.load();
|
|
}
|
|
// will return a pointer to SuperVersion* if previous SuperVersion
|
|
// if its reference count is zero and needs deletion or nullptr if not
|
|
// As argument takes a pointer to allocated SuperVersion to enable
|
|
// the clients to allocate SuperVersion outside of mutex.
|
|
// IMPORTANT: Only call this from DBImpl::InstallSuperVersion()
|
|
void InstallSuperVersion(SuperVersionContext* sv_context,
|
|
const MutableCFOptions& mutable_cf_options);
|
|
void InstallSuperVersion(SuperVersionContext* sv_context,
|
|
InstrumentedMutex* db_mutex);
|
|
|
|
void ResetThreadLocalSuperVersions();
|
|
|
|
// Protected by DB mutex
|
|
void set_queued_for_flush(bool value) { queued_for_flush_ = value; }
|
|
void set_queued_for_compaction(bool value) { queued_for_compaction_ = value; }
|
|
bool queued_for_flush() { return queued_for_flush_; }
|
|
bool queued_for_compaction() { return queued_for_compaction_; }
|
|
|
|
enum class WriteStallCause {
|
|
kNone,
|
|
kMemtableLimit,
|
|
kL0FileCountLimit,
|
|
kPendingCompactionBytes,
|
|
};
|
|
static std::pair<WriteStallCondition, WriteStallCause>
|
|
GetWriteStallConditionAndCause(
|
|
int num_unflushed_memtables, int num_l0_files,
|
|
uint64_t num_compaction_needed_bytes,
|
|
const MutableCFOptions& mutable_cf_options,
|
|
const ImmutableCFOptions& immutable_cf_options);
|
|
|
|
// Recalculate some stall conditions, which are changed only during
|
|
// compaction, adding new memtable and/or recalculation of compaction score.
|
|
WriteStallCondition RecalculateWriteStallConditions(
|
|
const MutableCFOptions& mutable_cf_options);
|
|
|
|
void set_initialized() { initialized_.store(true); }
|
|
|
|
bool initialized() const { return initialized_.load(); }
|
|
|
|
const ColumnFamilyOptions& initial_cf_options() {
|
|
return initial_cf_options_;
|
|
}
|
|
|
|
Env::WriteLifeTimeHint CalculateSSTWriteHint(int level);
|
|
|
|
// created_dirs remembers directory created, so that we don't need to call
|
|
// the same data creation operation again.
|
|
Status AddDirectories(
|
|
std::map<std::string, std::shared_ptr<FSDirectory>>* created_dirs);
|
|
|
|
FSDirectory* GetDataDir(size_t path_id) const;
|
|
|
|
// full_history_ts_low_ can only increase.
|
|
void SetFullHistoryTsLow(std::string ts_low) {
|
|
assert(!ts_low.empty());
|
|
const Comparator* ucmp = user_comparator();
|
|
assert(ucmp);
|
|
if (full_history_ts_low_.empty() ||
|
|
ucmp->CompareTimestamp(ts_low, full_history_ts_low_) > 0) {
|
|
full_history_ts_low_ = std::move(ts_low);
|
|
}
|
|
}
|
|
|
|
const std::string& GetFullHistoryTsLow() const {
|
|
return full_history_ts_low_;
|
|
}
|
|
|
|
ThreadLocalPtr* TEST_GetLocalSV() { return local_sv_.get(); }
|
|
WriteBufferManager* write_buffer_mgr() { return write_buffer_manager_; }
|
|
std::shared_ptr<CacheReservationManager>
|
|
GetFileMetadataCacheReservationManager() {
|
|
return file_metadata_cache_res_mgr_;
|
|
}
|
|
|
|
SequenceNumber GetFirstMemtableSequenceNumber() const;
|
|
|
|
static const uint32_t kDummyColumnFamilyDataId;
|
|
|
|
// Keep track of whether the mempurge feature was ever used.
|
|
void SetMempurgeUsed() { mempurge_used_ = true; }
|
|
bool GetMempurgeUsed() { return mempurge_used_; }
|
|
|
|
// Allocate and return a new epoch number
|
|
uint64_t NewEpochNumber() { return next_epoch_number_.fetch_add(1); }
|
|
|
|
// Get the next epoch number to be assigned
|
|
uint64_t GetNextEpochNumber() const { return next_epoch_number_.load(); }
|
|
|
|
// Set the next epoch number to be assigned
|
|
void SetNextEpochNumber(uint64_t next_epoch_number) {
|
|
next_epoch_number_.store(next_epoch_number);
|
|
}
|
|
|
|
// Reset the next epoch number to be assigned
|
|
void ResetNextEpochNumber() { next_epoch_number_.store(1); }
|
|
|
|
// Recover the next epoch number of this CF and epoch number
|
|
// of its files (if missing)
|
|
void RecoverEpochNumbers();
|
|
|
|
private:
|
|
friend class ColumnFamilySet;
|
|
ColumnFamilyData(uint32_t id, const std::string& name,
|
|
Version* dummy_versions, Cache* table_cache,
|
|
WriteBufferManager* write_buffer_manager,
|
|
const ColumnFamilyOptions& options,
|
|
const ImmutableDBOptions& db_options,
|
|
const FileOptions* file_options,
|
|
ColumnFamilySet* column_family_set,
|
|
BlockCacheTracer* const block_cache_tracer,
|
|
const std::shared_ptr<IOTracer>& io_tracer,
|
|
const std::string& db_id, const std::string& db_session_id);
|
|
|
|
std::vector<std::string> GetDbPaths() const;
|
|
|
|
uint32_t id_;
|
|
const std::string name_;
|
|
Version* dummy_versions_; // Head of circular doubly-linked list of versions.
|
|
Version* current_; // == dummy_versions->prev_
|
|
|
|
std::atomic<int> refs_; // outstanding references to ColumnFamilyData
|
|
std::atomic<bool> initialized_;
|
|
std::atomic<bool> dropped_; // true if client dropped it
|
|
|
|
const InternalKeyComparator internal_comparator_;
|
|
IntTblPropCollectorFactories int_tbl_prop_collector_factories_;
|
|
|
|
const ColumnFamilyOptions initial_cf_options_;
|
|
const ImmutableOptions ioptions_;
|
|
MutableCFOptions mutable_cf_options_;
|
|
|
|
const bool is_delete_range_supported_;
|
|
|
|
std::unique_ptr<TableCache> table_cache_;
|
|
std::unique_ptr<BlobFileCache> blob_file_cache_;
|
|
std::unique_ptr<BlobSource> blob_source_;
|
|
|
|
std::unique_ptr<InternalStats> internal_stats_;
|
|
|
|
WriteBufferManager* write_buffer_manager_;
|
|
|
|
MemTable* mem_;
|
|
MemTableList imm_;
|
|
SuperVersion* super_version_;
|
|
|
|
// An ordinal representing the current SuperVersion. Updated by
|
|
// InstallSuperVersion(), i.e. incremented every time super_version_
|
|
// changes.
|
|
std::atomic<uint64_t> super_version_number_;
|
|
|
|
// Thread's local copy of SuperVersion pointer
|
|
// This needs to be destructed before mutex_
|
|
std::unique_ptr<ThreadLocalPtr> local_sv_;
|
|
|
|
// pointers for a circular linked list. we use it to support iterations over
|
|
// all column families that are alive (note: dropped column families can also
|
|
// be alive as long as client holds a reference)
|
|
ColumnFamilyData* next_;
|
|
ColumnFamilyData* prev_;
|
|
|
|
// This is the earliest log file number that contains data from this
|
|
// Column Family. All earlier log files must be ignored and not
|
|
// recovered from
|
|
uint64_t log_number_;
|
|
|
|
std::atomic<FlushReason> flush_reason_;
|
|
|
|
// An object that keeps all the compaction stats
|
|
// and picks the next compaction
|
|
std::unique_ptr<CompactionPicker> compaction_picker_;
|
|
|
|
ColumnFamilySet* column_family_set_;
|
|
|
|
std::unique_ptr<WriteControllerToken> write_controller_token_;
|
|
|
|
// If true --> this ColumnFamily is currently present in DBImpl::flush_queue_
|
|
bool queued_for_flush_;
|
|
|
|
// If true --> this ColumnFamily is currently present in
|
|
// DBImpl::compaction_queue_
|
|
bool queued_for_compaction_;
|
|
|
|
uint64_t prev_compaction_needed_bytes_;
|
|
|
|
// if the database was opened with 2pc enabled
|
|
bool allow_2pc_;
|
|
|
|
// Memtable id to track flush.
|
|
std::atomic<uint64_t> last_memtable_id_;
|
|
|
|
// Directories corresponding to cf_paths.
|
|
std::vector<std::shared_ptr<FSDirectory>> data_dirs_;
|
|
|
|
bool db_paths_registered_;
|
|
|
|
std::string full_history_ts_low_;
|
|
|
|
// For charging memory usage of file metadata created for newly added files to
|
|
// a Version associated with this CFD
|
|
std::shared_ptr<CacheReservationManager> file_metadata_cache_res_mgr_;
|
|
bool mempurge_used_;
|
|
|
|
std::atomic<uint64_t> next_epoch_number_;
|
|
};
|
|
|
|
// ColumnFamilySet has interesting thread-safety requirements
|
|
// * CreateColumnFamily() or RemoveColumnFamily() -- need to be protected by DB
|
|
// mutex AND executed in the write thread.
|
|
// CreateColumnFamily() should ONLY be called from VersionSet::LogAndApply() AND
|
|
// single-threaded write thread. It is also called during Recovery and in
|
|
// DumpManifest().
|
|
// RemoveColumnFamily() is only called from SetDropped(). DB mutex needs to be
|
|
// held and it needs to be executed from the write thread. SetDropped() also
|
|
// guarantees that it will be called only from single-threaded LogAndApply(),
|
|
// but this condition is not that important.
|
|
// * Iteration -- hold DB mutex. If you want to release the DB mutex in the
|
|
// body of the iteration, wrap in a RefedColumnFamilySet.
|
|
// * GetDefault() -- thread safe
|
|
// * GetColumnFamily() -- either inside of DB mutex or from a write thread
|
|
// * GetNextColumnFamilyID(), GetMaxColumnFamily(), UpdateMaxColumnFamily(),
|
|
// NumberOfColumnFamilies -- inside of DB mutex
|
|
class ColumnFamilySet {
|
|
public:
|
|
// ColumnFamilySet supports iteration
|
|
class iterator {
|
|
public:
|
|
explicit iterator(ColumnFamilyData* cfd) : current_(cfd) {}
|
|
// NOTE: minimum operators for for-loop iteration
|
|
iterator& operator++() {
|
|
current_ = current_->next_;
|
|
return *this;
|
|
}
|
|
bool operator!=(const iterator& other) const {
|
|
return this->current_ != other.current_;
|
|
}
|
|
ColumnFamilyData* operator*() { return current_; }
|
|
|
|
private:
|
|
ColumnFamilyData* current_;
|
|
};
|
|
|
|
ColumnFamilySet(const std::string& dbname,
|
|
const ImmutableDBOptions* db_options,
|
|
const FileOptions& file_options, Cache* table_cache,
|
|
WriteBufferManager* _write_buffer_manager,
|
|
WriteController* _write_controller,
|
|
BlockCacheTracer* const block_cache_tracer,
|
|
const std::shared_ptr<IOTracer>& io_tracer,
|
|
const std::string& db_id, const std::string& db_session_id);
|
|
~ColumnFamilySet();
|
|
|
|
ColumnFamilyData* GetDefault() const;
|
|
// GetColumnFamily() calls return nullptr if column family is not found
|
|
ColumnFamilyData* GetColumnFamily(uint32_t id) const;
|
|
ColumnFamilyData* GetColumnFamily(const std::string& name) const;
|
|
// this call will return the next available column family ID. it guarantees
|
|
// that there is no column family with id greater than or equal to the
|
|
// returned value in the current running instance or anytime in RocksDB
|
|
// instance history.
|
|
uint32_t GetNextColumnFamilyID();
|
|
uint32_t GetMaxColumnFamily();
|
|
void UpdateMaxColumnFamily(uint32_t new_max_column_family);
|
|
size_t NumberOfColumnFamilies() const;
|
|
|
|
ColumnFamilyData* CreateColumnFamily(const std::string& name, uint32_t id,
|
|
Version* dummy_version,
|
|
const ColumnFamilyOptions& options);
|
|
|
|
iterator begin() { return iterator(dummy_cfd_->next_); }
|
|
iterator end() { return iterator(dummy_cfd_); }
|
|
|
|
Cache* get_table_cache() { return table_cache_; }
|
|
|
|
WriteBufferManager* write_buffer_manager() { return write_buffer_manager_; }
|
|
|
|
WriteController* write_controller() { return write_controller_; }
|
|
|
|
private:
|
|
friend class ColumnFamilyData;
|
|
// helper function that gets called from cfd destructor
|
|
// REQUIRES: DB mutex held
|
|
void RemoveColumnFamily(ColumnFamilyData* cfd);
|
|
|
|
// column_families_ and column_family_data_ need to be protected:
|
|
// * when mutating both conditions have to be satisfied:
|
|
// 1. DB mutex locked
|
|
// 2. thread currently in single-threaded write thread
|
|
// * when reading, at least one condition needs to be satisfied:
|
|
// 1. DB mutex locked
|
|
// 2. accessed from a single-threaded write thread
|
|
UnorderedMap<std::string, uint32_t> column_families_;
|
|
UnorderedMap<uint32_t, ColumnFamilyData*> column_family_data_;
|
|
|
|
uint32_t max_column_family_;
|
|
const FileOptions file_options_;
|
|
|
|
ColumnFamilyData* dummy_cfd_;
|
|
// We don't hold the refcount here, since default column family always exists
|
|
// We are also not responsible for cleaning up default_cfd_cache_. This is
|
|
// just a cache that makes common case (accessing default column family)
|
|
// faster
|
|
ColumnFamilyData* default_cfd_cache_;
|
|
|
|
const std::string db_name_;
|
|
const ImmutableDBOptions* const db_options_;
|
|
Cache* table_cache_;
|
|
WriteBufferManager* write_buffer_manager_;
|
|
WriteController* write_controller_;
|
|
BlockCacheTracer* const block_cache_tracer_;
|
|
std::shared_ptr<IOTracer> io_tracer_;
|
|
const std::string& db_id_;
|
|
std::string db_session_id_;
|
|
};
|
|
|
|
// A wrapper for ColumnFamilySet that supports releasing DB mutex during each
|
|
// iteration over the iterator, because the cfd is Refed and Unrefed during
|
|
// each iteration to prevent concurrent CF drop from destroying it (until
|
|
// Unref).
|
|
class RefedColumnFamilySet {
|
|
public:
|
|
explicit RefedColumnFamilySet(ColumnFamilySet* cfs) : wrapped_(cfs) {}
|
|
|
|
class iterator {
|
|
public:
|
|
explicit iterator(ColumnFamilySet::iterator wrapped) : wrapped_(wrapped) {
|
|
MaybeRef(*wrapped_);
|
|
}
|
|
~iterator() { MaybeUnref(*wrapped_); }
|
|
inline void MaybeRef(ColumnFamilyData* cfd) {
|
|
if (cfd->GetID() != ColumnFamilyData::kDummyColumnFamilyDataId) {
|
|
cfd->Ref();
|
|
}
|
|
}
|
|
inline void MaybeUnref(ColumnFamilyData* cfd) {
|
|
if (cfd->GetID() != ColumnFamilyData::kDummyColumnFamilyDataId) {
|
|
cfd->UnrefAndTryDelete();
|
|
}
|
|
}
|
|
// NOTE: minimum operators for for-loop iteration
|
|
inline iterator& operator++() {
|
|
ColumnFamilyData* old = *wrapped_;
|
|
++wrapped_;
|
|
// Can only unref & potentially free cfd after accessing its next_
|
|
MaybeUnref(old);
|
|
MaybeRef(*wrapped_);
|
|
return *this;
|
|
}
|
|
inline bool operator!=(const iterator& other) const {
|
|
return this->wrapped_ != other.wrapped_;
|
|
}
|
|
inline ColumnFamilyData* operator*() { return *wrapped_; }
|
|
|
|
private:
|
|
ColumnFamilySet::iterator wrapped_;
|
|
};
|
|
|
|
iterator begin() { return iterator(wrapped_->begin()); }
|
|
iterator end() { return iterator(wrapped_->end()); }
|
|
|
|
private:
|
|
ColumnFamilySet* wrapped_;
|
|
};
|
|
|
|
// We use ColumnFamilyMemTablesImpl to provide WriteBatch a way to access
|
|
// memtables of different column families (specified by ID in the write batch)
|
|
class ColumnFamilyMemTablesImpl : public ColumnFamilyMemTables {
|
|
public:
|
|
explicit ColumnFamilyMemTablesImpl(ColumnFamilySet* column_family_set)
|
|
: column_family_set_(column_family_set), current_(nullptr) {}
|
|
|
|
// Constructs a ColumnFamilyMemTablesImpl equivalent to one constructed
|
|
// with the arguments used to construct *orig.
|
|
explicit ColumnFamilyMemTablesImpl(ColumnFamilyMemTablesImpl* orig)
|
|
: column_family_set_(orig->column_family_set_), current_(nullptr) {}
|
|
|
|
// sets current_ to ColumnFamilyData with column_family_id
|
|
// returns false if column family doesn't exist
|
|
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
|
|
// under a DB mutex OR from a write thread
|
|
bool Seek(uint32_t column_family_id) override;
|
|
|
|
// Returns log number of the selected column family
|
|
// REQUIRES: under a DB mutex OR from a write thread
|
|
uint64_t GetLogNumber() const override;
|
|
|
|
// REQUIRES: Seek() called first
|
|
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
|
|
// under a DB mutex OR from a write thread
|
|
virtual MemTable* GetMemTable() const override;
|
|
|
|
// Returns column family handle for the selected column family
|
|
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
|
|
// under a DB mutex OR from a write thread
|
|
virtual ColumnFamilyHandle* GetColumnFamilyHandle() override;
|
|
|
|
// Cannot be called while another thread is calling Seek().
|
|
// REQUIRES: use this function of DBImpl::column_family_memtables_ should be
|
|
// under a DB mutex OR from a write thread
|
|
virtual ColumnFamilyData* current() override { return current_; }
|
|
|
|
private:
|
|
ColumnFamilySet* column_family_set_;
|
|
ColumnFamilyData* current_;
|
|
ColumnFamilyHandleInternal handle_;
|
|
};
|
|
|
|
extern uint32_t GetColumnFamilyID(ColumnFamilyHandle* column_family);
|
|
|
|
extern const Comparator* GetColumnFamilyUserComparator(
|
|
ColumnFamilyHandle* column_family);
|
|
|
|
} // namespace ROCKSDB_NAMESPACE
|