rocksdb/utilities/blob_db/blob_db_impl.h
Hui Xiao 06e593376c Group SST write in flush, compaction and db open with new stats (#11910)
Summary:
## Context/Summary
Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity.

For that, this PR does the following:
- Tag different write IOs by passing down and converting WriteOptions to IOOptions
- Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS

Some related code refactory to make implementation cleaner:
- Blob stats
   - Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS  include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info.
   - Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write.
- Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority
- Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification
- Build table
   - TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables
   - Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder.
This parameter is used for dynamically changing file io priority for flush, see  https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more
   - Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority

## Test
### db bench

Flush
```
./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100

rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
```

compaction, db oopen
```
Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench

Run:./db_bench --statistics=1 --benchmarks=compact  --db=../db_bench --use_existing_db=1

rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279
rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213
rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66
```

blob stats - just to make sure they aren't broken by this PR
```
Integrated Blob DB

Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench

Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact  --db=../db_bench --use_existing_db=1

pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600
rocksdb.blobdb.blob.file.synced COUNT : 1
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842

post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164

rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same)
```

```
Stacked Blob DB

Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench

pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876
rocksdb.blobdb.blob.file.synced COUNT : 8
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445

post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164

rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same)
```

###  Rehearsal CI stress test
Trigger 3 full runs of all our CI stress tests

###  Performance

Flush
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark; enable_statistics = true

Pre-pr: avg 507515519.3 ns
497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908,

Post-pr: avg 511971266.5 ns, regressed 0.88%
502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408,
```

Compaction
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1  --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark

Pre-pr: avg 495346098.30 ns
492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846

Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97%
502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007
```

Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats)
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1  --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark

Pre-pr: avg 3848.10 ns
3814,3838,3839,3848,3854,3854,3854,3860,3860,3860

Post-pr: avg 3874.20 ns, regressed 0.68%
3863,3867,3871,3874,3875,3877,3877,3877,3880,3881
```

Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910

Reviewed By: ajkr

Differential Revision: D49788060

Pulled By: hx235

fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 15:29:23 -08:00

512 lines
18 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).
#pragma once
#include <atomic>
#include <condition_variable>
#include <limits>
#include <list>
#include <memory>
#include <set>
#include <string>
#include <thread>
#include <unordered_map>
#include <utility>
#include <vector>
#include "db/blob/blob_log_format.h"
#include "db/blob/blob_log_writer.h"
#include "db/db_iter.h"
#include "rocksdb/compaction_filter.h"
#include "rocksdb/db.h"
#include "rocksdb/file_system.h"
#include "rocksdb/listener.h"
#include "rocksdb/options.h"
#include "rocksdb/statistics.h"
#include "rocksdb/wal_filter.h"
#include "util/mutexlock.h"
#include "util/timer_queue.h"
#include "utilities/blob_db/blob_db.h"
#include "utilities/blob_db/blob_file.h"
namespace ROCKSDB_NAMESPACE {
class DBImpl;
class ColumnFamilyHandle;
class ColumnFamilyData;
class SystemClock;
struct FlushJobInfo;
namespace blob_db {
struct BlobCompactionContext;
struct BlobCompactionContextGC;
class BlobDBImpl;
class BlobFile;
// Comparator to sort "TTL" aware Blob files based on the lower value of
// TTL range.
struct BlobFileComparatorTTL {
bool operator()(const std::shared_ptr<BlobFile>& lhs,
const std::shared_ptr<BlobFile>& rhs) const;
};
struct BlobFileComparator {
bool operator()(const std::shared_ptr<BlobFile>& lhs,
const std::shared_ptr<BlobFile>& rhs) const;
};
/**
* The implementation class for BlobDB. It manages the blob logs, which
* are sequentially written files. Blob logs can be of the TTL or non-TTL
* varieties; the former are cleaned up when they expire, while the latter
* are (optionally) garbage collected.
*/
class BlobDBImpl : public BlobDB {
friend class BlobFile;
friend class BlobDBIterator;
friend class BlobDBListener;
friend class BlobDBListenerGC;
friend class BlobIndexCompactionFilterBase;
friend class BlobIndexCompactionFilterGC;
public:
// deletions check period
static constexpr uint32_t kDeleteCheckPeriodMillisecs = 2 * 1000;
// sanity check task
static constexpr uint32_t kSanityCheckPeriodMillisecs = 20 * 60 * 1000;
// how many random access open files can we tolerate
static constexpr uint32_t kOpenFilesTrigger = 100;
// how often to schedule reclaim open files.
static constexpr uint32_t kReclaimOpenFilesPeriodMillisecs = 1 * 1000;
// how often to schedule delete obs files periods
static constexpr uint32_t kDeleteObsoleteFilesPeriodMillisecs = 10 * 1000;
// how often to schedule expired files eviction.
static constexpr uint32_t kEvictExpiredFilesPeriodMillisecs = 10 * 1000;
// when should oldest file be evicted:
// on reaching 90% of blob_dir_size
static constexpr double kEvictOldestFileAtSize = 0.9;
using BlobDB::Put;
Status Put(const WriteOptions& options, const Slice& key,
const Slice& value) override;
using BlobDB::Get;
Status Get(const ReadOptions& _read_options,
ColumnFamilyHandle* column_family, const Slice& key,
PinnableSlice* value) override;
Status Get(const ReadOptions& _read_options,
ColumnFamilyHandle* column_family, const Slice& key,
PinnableSlice* value, uint64_t* expiration) override;
using BlobDB::NewIterator;
virtual Iterator* NewIterator(const ReadOptions& read_options) override;
using BlobDB::NewIterators;
virtual Status NewIterators(
const ReadOptions& /*read_options*/,
const std::vector<ColumnFamilyHandle*>& /*column_families*/,
std::vector<Iterator*>* /*iterators*/) override {
return Status::NotSupported("Not implemented");
}
using BlobDB::MultiGet;
virtual std::vector<Status> MultiGet(
const ReadOptions& _read_options, const std::vector<Slice>& keys,
std::vector<std::string>* values) override;
using BlobDB::Write;
virtual Status Write(const WriteOptions& opts, WriteBatch* updates) override;
virtual Status Close() override;
using BlobDB::PutWithTTL;
Status PutWithTTL(const WriteOptions& options, const Slice& key,
const Slice& value, uint64_t ttl) override;
using BlobDB::PutUntil;
Status PutUntil(const WriteOptions& options, const Slice& key,
const Slice& value, uint64_t expiration) override;
using BlobDB::CompactFiles;
Status CompactFiles(
const CompactionOptions& compact_options,
const std::vector<std::string>& input_file_names, const int output_level,
const int output_path_id = -1,
std::vector<std::string>* const output_file_names = nullptr,
CompactionJobInfo* compaction_job_info = nullptr) override;
BlobDBOptions GetBlobDBOptions() const override;
BlobDBImpl(const std::string& dbname, const BlobDBOptions& bdb_options,
const DBOptions& db_options,
const ColumnFamilyOptions& cf_options);
virtual Status DisableFileDeletions() override;
virtual Status EnableFileDeletions(bool force) override;
virtual Status GetLiveFiles(std::vector<std::string>&,
uint64_t* manifest_file_size,
bool flush_memtable = true) override;
virtual void GetLiveFilesMetaData(std::vector<LiveFileMetaData>*) override;
~BlobDBImpl();
Status Open(std::vector<ColumnFamilyHandle*>* handles);
Status SyncBlobFiles(const WriteOptions& write_options) override;
// Common part of the two GetCompactionContext methods below.
// REQUIRES: read lock on mutex_
void GetCompactionContextCommon(BlobCompactionContext* context);
void GetCompactionContext(BlobCompactionContext* context);
void GetCompactionContext(BlobCompactionContext* context,
BlobCompactionContextGC* context_gc);
#ifndef NDEBUG
Status TEST_GetBlobValue(const Slice& key, const Slice& index_entry,
PinnableSlice* value);
void TEST_AddDummyBlobFile(uint64_t blob_file_number,
SequenceNumber immutable_sequence);
std::vector<std::shared_ptr<BlobFile>> TEST_GetBlobFiles() const;
std::vector<std::shared_ptr<BlobFile>> TEST_GetLiveImmNonTTLFiles() const;
std::vector<std::shared_ptr<BlobFile>> TEST_GetObsoleteFiles() const;
Status TEST_CloseBlobFile(std::shared_ptr<BlobFile>& bfile);
void TEST_ObsoleteBlobFile(std::shared_ptr<BlobFile>& blob_file,
SequenceNumber obsolete_seq = 0,
bool update_size = true);
void TEST_EvictExpiredFiles();
void TEST_DeleteObsoleteFiles();
uint64_t TEST_live_sst_size();
const std::string& TEST_blob_dir() const { return blob_dir_; }
void TEST_InitializeBlobFileToSstMapping(
const std::vector<LiveFileMetaData>& live_files);
void TEST_ProcessFlushJobInfo(const FlushJobInfo& info);
void TEST_ProcessCompactionJobInfo(const CompactionJobInfo& info);
#endif // !NDEBUG
private:
class BlobInserter;
// Create a snapshot if there isn't one in read options.
// Return true if a snapshot is created.
bool SetSnapshotIfNeeded(ReadOptions* read_options);
Status GetImpl(const ReadOptions& read_options,
ColumnFamilyHandle* column_family, const Slice& key,
PinnableSlice* value, uint64_t* expiration = nullptr);
Status GetBlobValue(const Slice& key, const Slice& index_entry,
PinnableSlice* value, uint64_t* expiration = nullptr);
Status GetRawBlobFromFile(const Slice& key, uint64_t file_number,
uint64_t offset, uint64_t size,
PinnableSlice* value,
CompressionType* compression_type);
Slice GetCompressedSlice(const Slice& raw,
std::string* compression_output) const;
Status DecompressSlice(const Slice& compressed_value,
CompressionType compression_type,
PinnableSlice* value_output) const;
// Close a file by appending a footer, and removes file from open files list.
// REQUIRES: lock held on write_mutex_, write lock held on both the db mutex_
// and the blob file's mutex_. If called on a blob file which is visible only
// to a single thread (like in the case of new files written during
// compaction/GC), the locks on write_mutex_ and the blob file's mutex_ can be
// avoided.
Status CloseBlobFile(const WriteOptions& write_options,
std::shared_ptr<BlobFile> bfile);
// Close a file if its size exceeds blob_file_size
// REQUIRES: lock held on write_mutex_.
Status CloseBlobFileIfNeeded(const WriteOptions& write_options,
std::shared_ptr<BlobFile>& bfile);
// Mark file as obsolete and move the file to obsolete file list.
//
// REQUIRED: hold write lock of mutex_ or during DB open.
void ObsoleteBlobFile(std::shared_ptr<BlobFile> blob_file,
SequenceNumber obsolete_seq, bool update_size);
Status PutBlobValue(const WriteOptions& options, const Slice& key,
const Slice& value, uint64_t expiration,
WriteBatch* batch);
Status AppendBlob(const WriteOptions& write_options,
const std::shared_ptr<BlobFile>& bfile,
const std::string& headerbuf, const Slice& key,
const Slice& value, uint64_t expiration,
std::string* index_entry);
// Create a new blob file and associated writer.
Status CreateBlobFileAndWriter(const WriteOptions& write_options,
bool has_ttl,
const ExpirationRange& expiration_range,
const std::string& reason,
std::shared_ptr<BlobFile>* blob_file,
std::shared_ptr<BlobLogWriter>* writer);
// Get the open non-TTL blob log file, or create a new one if no such file
// exists.
Status SelectBlobFile(const WriteOptions& write_options,
std::shared_ptr<BlobFile>* blob_file);
// Get the open TTL blob log file for a certain expiration, or create a new
// one if no such file exists.
Status SelectBlobFileTTL(const WriteOptions& write_options,
uint64_t expiration,
std::shared_ptr<BlobFile>* blob_file);
std::shared_ptr<BlobFile> FindBlobFileLocked(uint64_t expiration) const;
// periodic sanity check. Bunch of checks
std::pair<bool, int64_t> SanityCheck(bool aborted);
// Delete files that have been marked obsolete (either because of TTL
// or GC). Check whether any snapshots exist which refer to the same.
std::pair<bool, int64_t> DeleteObsoleteFiles(bool aborted);
// periodically check if open blob files and their TTL's has expired
// if expired, close the sequential writer and make the file immutable
std::pair<bool, int64_t> EvictExpiredFiles(bool aborted);
// if the number of open files, approaches ULIMIT's this
// task will close random readers, which are kept around for
// efficiency
std::pair<bool, int64_t> ReclaimOpenFiles(bool aborted);
std::pair<bool, int64_t> RemoveTimerQ(TimerQueue* tq, bool aborted);
// Adds the background tasks to the timer queue
void StartBackgroundTasks();
// add a new Blob File
std::shared_ptr<BlobFile> NewBlobFile(bool has_ttl,
const ExpirationRange& expiration_range,
const std::string& reason);
// Register a new blob file.
// REQUIRES: write lock on mutex_.
void RegisterBlobFile(std::shared_ptr<BlobFile> blob_file);
// collect all the blob log files from the blob directory
Status GetAllBlobFiles(std::set<uint64_t>* file_numbers);
// Open all blob files found in blob_dir.
Status OpenAllBlobFiles();
// Link an SST to a blob file. Comes in locking and non-locking varieties
// (the latter is used during Open).
template <typename Linker>
void LinkSstToBlobFileImpl(uint64_t sst_file_number,
uint64_t blob_file_number, Linker linker);
void LinkSstToBlobFile(uint64_t sst_file_number, uint64_t blob_file_number);
void LinkSstToBlobFileNoLock(uint64_t sst_file_number,
uint64_t blob_file_number);
// Unlink an SST from a blob file.
void UnlinkSstFromBlobFile(uint64_t sst_file_number,
uint64_t blob_file_number);
// Initialize the mapping between blob files and SSTs during Open.
void InitializeBlobFileToSstMapping(
const std::vector<LiveFileMetaData>& live_files);
// Update the mapping between blob files and SSTs after a flush and mark
// any unneeded blob files obsolete.
void ProcessFlushJobInfo(const FlushJobInfo& info);
// Update the mapping between blob files and SSTs after a compaction and
// mark any unneeded blob files obsolete.
void ProcessCompactionJobInfo(const CompactionJobInfo& info);
// Mark an immutable non-TTL blob file obsolete assuming it has no more SSTs
// linked to it, and all memtables from before the blob file became immutable
// have been flushed. Note: should only be called if the condition holds for
// all lower-numbered non-TTL blob files as well.
bool MarkBlobFileObsoleteIfNeeded(const std::shared_ptr<BlobFile>& blob_file,
SequenceNumber obsolete_seq);
// Mark all immutable non-TTL blob files that aren't needed by any SSTs as
// obsolete. Comes in two varieties; the version used during Open need not
// worry about locking or snapshots.
template <class Functor>
void MarkUnreferencedBlobFilesObsoleteImpl(Functor mark_if_needed);
void MarkUnreferencedBlobFilesObsolete();
void MarkUnreferencedBlobFilesObsoleteDuringOpen();
void UpdateLiveSSTSize(const WriteOptions& write_options);
Status GetBlobFileReader(const std::shared_ptr<BlobFile>& blob_file,
std::shared_ptr<RandomAccessFileReader>* reader);
// hold write mutex on file and call.
// Close the above Random Access reader
void CloseRandomAccessLocked(const std::shared_ptr<BlobFile>& bfile);
// hold write mutex on file and call
// creates a sequential (append) writer for this blobfile
Status CreateWriterLocked(const std::shared_ptr<BlobFile>& bfile);
// returns a BlobLogWriter object for the file. If writer is not
// already present, creates one. Needs Write Mutex to be held
Status CheckOrCreateWriterLocked(const std::shared_ptr<BlobFile>& blob_file,
std::shared_ptr<BlobLogWriter>* writer);
// checks if there is no snapshot which is referencing the
// blobs
bool VisibleToActiveSnapshot(const std::shared_ptr<BlobFile>& file);
bool FileDeleteOk_SnapshotCheckLocked(const std::shared_ptr<BlobFile>& bfile);
void CopyBlobFiles(std::vector<std::shared_ptr<BlobFile>>* bfiles_copy);
uint64_t EpochNow() { return clock_->NowMicros() / 1000000; }
// Check if inserting a new blob will make DB grow out of space.
// If is_fifo = true, FIFO eviction will be triggered to make room for the
// new blob. If force_evict = true, FIFO eviction will evict blob files
// even eviction will not make enough room for the new blob.
Status CheckSizeAndEvictBlobFiles(const WriteOptions& write_options,
uint64_t blob_size,
bool force_evict = false);
Status CloseImpl();
// name of the database directory
std::string dbname_;
// the base DB
DBImpl* db_impl_;
Env* env_;
SystemClock* clock_;
// the options that govern the behavior of Blob Storage
BlobDBOptions bdb_options_;
DBOptions db_options_;
ColumnFamilyOptions cf_options_;
FileOptions file_options_;
// Raw pointer of statistic. db_options_ has a std::shared_ptr to hold
// ownership.
Statistics* statistics_;
// by default this is "blob_dir" under dbname_
// but can be configured
std::string blob_dir_;
// pointer to directory
std::unique_ptr<FSDirectory> dir_ent_;
// Read Write Mutex, which protects all the data structures
// HEAVILY TRAFFICKED
mutable port::RWMutex mutex_;
// Writers has to hold write_mutex_ before writing.
mutable port::Mutex write_mutex_;
// counter for blob file number
std::atomic<uint64_t> next_file_number_;
// entire metadata of all the BLOB files memory
std::map<uint64_t, std::shared_ptr<BlobFile>> blob_files_;
// All live immutable non-TTL blob files.
std::map<uint64_t, std::shared_ptr<BlobFile>> live_imm_non_ttl_blob_files_;
// The largest sequence number that has been flushed.
SequenceNumber flush_sequence_;
// opened non-TTL blob file.
std::shared_ptr<BlobFile> open_non_ttl_file_;
// all the blob files which are currently being appended to based
// on variety of incoming TTL's
std::set<std::shared_ptr<BlobFile>, BlobFileComparatorTTL> open_ttl_files_;
// Flag to check whether Close() has been called on this DB
bool closed_;
// timer based queue to execute tasks
TimerQueue tqueue_;
// number of files opened for random access/GET
// counter is used to monitor and close excess RA files.
std::atomic<uint32_t> open_file_count_;
// Total size of all live blob files (i.e. exclude obsolete files).
std::atomic<uint64_t> total_blob_size_;
// total size of SST files.
std::atomic<uint64_t> live_sst_size_;
// Latest FIFO eviction timestamp
//
// REQUIRES: access with metex_ lock held.
uint64_t fifo_eviction_seq_;
// The expiration up to which latest FIFO eviction evicts.
//
// REQUIRES: access with metex_ lock held.
uint64_t evict_expiration_up_to_;
std::list<std::shared_ptr<BlobFile>> obsolete_files_;
// DeleteObsoleteFiles, DiableFileDeletions and EnableFileDeletions block
// on the mutex to avoid contention.
//
// While DeleteObsoleteFiles hold both mutex_ and delete_file_mutex_, note
// the difference. mutex_ only needs to be held when access the
// data-structure, and delete_file_mutex_ needs to be held the whole time
// during DeleteObsoleteFiles to avoid being run simultaneously with
// DisableFileDeletions.
//
// If both of mutex_ and delete_file_mutex_ needs to be held, it is adviced
// to hold delete_file_mutex_ first to avoid deadlock.
mutable port::Mutex delete_file_mutex_;
// Each call of DisableFileDeletions will increase disable_file_deletion_
// by 1. EnableFileDeletions will either decrease the count by 1 or reset
// it to zero, depending on the force flag.
//
// REQUIRES: access with delete_file_mutex_ held.
int disable_file_deletions_ = 0;
uint32_t debug_level_;
};
} // namespace blob_db
} // namespace ROCKSDB_NAMESPACE