column_family.h 32 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. //
  6. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
  7. // Use of this source code is governed by a BSD-style license that can be
  8. // found in the LICENSE file. See the AUTHORS file for names of contributors.
  9. #pragma once
  10. #include <unordered_map>
  11. #include <string>
  12. #include <vector>
  13. #include <atomic>
  14. #include "db/memtable_list.h"
  15. #include "db/table_cache.h"
  16. #include "db/table_properties_collector.h"
  17. #include "db/write_batch_internal.h"
  18. #include "db/write_controller.h"
  19. #include "options/cf_options.h"
  20. #include "rocksdb/compaction_job_stats.h"
  21. #include "rocksdb/db.h"
  22. #include "rocksdb/env.h"
  23. #include "rocksdb/options.h"
  24. #include "trace_replay/block_cache_tracer.h"
  25. #include "util/thread_local.h"
  26. namespace ROCKSDB_NAMESPACE {
  27. class Version;
  28. class VersionSet;
  29. class VersionStorageInfo;
  30. class MemTable;
  31. class MemTableListVersion;
  32. class CompactionPicker;
  33. class Compaction;
  34. class InternalKey;
  35. class InternalStats;
  36. class ColumnFamilyData;
  37. class DBImpl;
  38. class LogBuffer;
  39. class InstrumentedMutex;
  40. class InstrumentedMutexLock;
  41. struct SuperVersionContext;
  42. extern const double kIncSlowdownRatio;
  43. // This file contains a list of data structures for managing column family
  44. // level metadata.
  45. //
  46. // The basic relationships among classes declared here are illustrated as
  47. // following:
  48. //
  49. // +----------------------+ +----------------------+ +--------+
  50. // +---+ ColumnFamilyHandle 1 | +--+ ColumnFamilyHandle 2 | | DBImpl |
  51. // | +----------------------+ | +----------------------+ +----+---+
  52. // | +--------------------------+ |
  53. // | | +-----------------------------+
  54. // | | |
  55. // | | +-----------------------------v-------------------------------+
  56. // | | | |
  57. // | | | ColumnFamilySet |
  58. // | | | |
  59. // | | +-------------+--------------------------+----------------+---+
  60. // | | | | |
  61. // | +-------------------------------------+ | |
  62. // | | | | v
  63. // | +-------------v-------------+ +-----v----v---------+
  64. // | | | | |
  65. // | | ColumnFamilyData 1 | | ColumnFamilyData 2 | ......
  66. // | | | | |
  67. // +---> | | |
  68. // | +---------+ | |
  69. // | | MemTable| | |
  70. // | | List | | |
  71. // +--------+---+--+-+----+----+ +--------------------++
  72. // | | | |
  73. // | | | |
  74. // | | | +-----------------------+
  75. // | | +-----------+ |
  76. // v +--------+ | |
  77. // +--------+--------+ | | |
  78. // | | | | +----------v----------+
  79. // +---> |SuperVersion 1.a +-----------------> |
  80. // | +------+ | | MemTableListVersion |
  81. // +---+-------------+ | | | | |
  82. // | | | | +----+------------+---+
  83. // | current | | | | |
  84. // | +-------------+ | |mem | |
  85. // | | | | | |
  86. // +-v---v-------+ +---v--v---+ +-----v----+ +----v-----+
  87. // | | | | | | | |
  88. // | Version 1.a | | memtable | | memtable | | memtable |
  89. // | | | 1.a | | 1.b | | 1.c |
  90. // +-------------+ | | | | | |
  91. // +----------+ +----------+ +----------+
  92. //
  93. // DBImpl keeps a ColumnFamilySet, which references to all column families by
  94. // pointing to respective ColumnFamilyData object of each column family.
  95. // This is how DBImpl can list and operate on all the column families.
  96. // ColumnFamilyHandle also points to ColumnFamilyData directly, so that
  97. // when a user executes a query, it can directly find memtables and Version
  98. // as well as SuperVersion to the column family, without going through
  99. // ColumnFamilySet.
  100. //
  101. // ColumnFamilySet points to the latest view of the LSM-tree (list of memtables
  102. // and SST files) indirectly, while ongoing operations may hold references
  103. // to a current or an out-of-date SuperVersion, which in turn points to a
  104. // point-in-time view of the LSM-tree. This guarantees the memtables and SST
  105. // files being operated on will not go away, until the SuperVersion is
  106. // unreferenced to 0 and destoryed.
  107. //
  108. // The following graph illustrates a possible referencing relationships:
  109. //
  110. // Column +--------------+ current +-----------+
  111. // Family +---->+ +------------------->+ |
  112. // Data | SuperVersion +----------+ | Version A |
  113. // | 3 | imm | | |
  114. // Iter2 +----->+ | +-------v------+ +-----------+
  115. // +-----+--------+ | MemtableList +----------------> Empty
  116. // | | Version r | +-----------+
  117. // | +--------------+ | |
  118. // +------------------+ current| Version B |
  119. // +--------------+ | +----->+ |
  120. // | | | | +-----+-----+
  121. // Compaction +>+ SuperVersion +-------------+ ^
  122. // Job | 2 +------+ | |current
  123. // | +----+ | | mem | +------------+
  124. // +--------------+ | | +---------------------> |
  125. // | +------------------------> MemTable a |
  126. // | mem | | |
  127. // +--------------+ | | +------------+
  128. // | +--------------------------+
  129. // Iter1 +-----> SuperVersion | | +------------+
  130. // | 1 +------------------------------>+ |
  131. // | +-+ | mem | MemTable b |
  132. // +--------------+ | | | |
  133. // | | +--------------+ +-----^------+
  134. // | |imm | MemtableList | |
  135. // | +--->+ Version s +------------+
  136. // | +--------------+
  137. // | +--------------+
  138. // | | MemtableList |
  139. // +------>+ Version t +--------> Empty
  140. // imm +--------------+
  141. //
  142. // In this example, even if the current LSM-tree consists of Version A and
  143. // memtable a, which is also referenced by SuperVersion, two older SuperVersion
  144. // SuperVersion2 and Superversion1 still exist, and are referenced by a
  145. // compaction job and an old iterator Iter1, respectively. SuperVersion2
  146. // contains Version B, memtable a and memtable b; SuperVersion1 contains
  147. // Version B and memtable b (mutable). As a result, Version B and memtable b
  148. // are prevented from being destroyed or deleted.
  149. // ColumnFamilyHandleImpl is the class that clients use to access different
  150. // column families. It has non-trivial destructor, which gets called when client
  151. // is done using the column family
  152. class ColumnFamilyHandleImpl : public ColumnFamilyHandle {
  153. public:
  154. // create while holding the mutex
  155. ColumnFamilyHandleImpl(
  156. ColumnFamilyData* cfd, DBImpl* db, InstrumentedMutex* mutex);
  157. // destroy without mutex
  158. virtual ~ColumnFamilyHandleImpl();
  159. virtual ColumnFamilyData* cfd() const { return cfd_; }
  160. virtual uint32_t GetID() const override;
  161. virtual const std::string& GetName() const override;
  162. virtual Status GetDescriptor(ColumnFamilyDescriptor* desc) override;
  163. virtual const Comparator* GetComparator() const override;
  164. private:
  165. ColumnFamilyData* cfd_;
  166. DBImpl* db_;
  167. InstrumentedMutex* mutex_;
  168. };
  169. // Does not ref-count ColumnFamilyData
  170. // We use this dummy ColumnFamilyHandleImpl because sometimes MemTableInserter
  171. // calls DBImpl methods. When this happens, MemTableInserter need access to
  172. // ColumnFamilyHandle (same as the client would need). In that case, we feed
  173. // MemTableInserter dummy ColumnFamilyHandle and enable it to call DBImpl
  174. // methods
  175. class ColumnFamilyHandleInternal : public ColumnFamilyHandleImpl {
  176. public:
  177. ColumnFamilyHandleInternal()
  178. : ColumnFamilyHandleImpl(nullptr, nullptr, nullptr), internal_cfd_(nullptr) {}
  179. void SetCFD(ColumnFamilyData* _cfd) { internal_cfd_ = _cfd; }
  180. virtual ColumnFamilyData* cfd() const override { return internal_cfd_; }
  181. private:
  182. ColumnFamilyData* internal_cfd_;
  183. };
  184. // holds references to memtable, all immutable memtables and version
  185. struct SuperVersion {
  186. // Accessing members of this class is not thread-safe and requires external
  187. // synchronization (ie db mutex held or on write thread).
  188. ColumnFamilyData* cfd;
  189. MemTable* mem;
  190. MemTableListVersion* imm;
  191. Version* current;
  192. MutableCFOptions mutable_cf_options;
  193. // Version number of the current SuperVersion
  194. uint64_t version_number;
  195. WriteStallCondition write_stall_condition;
  196. InstrumentedMutex* db_mutex;
  197. // should be called outside the mutex
  198. SuperVersion() = default;
  199. ~SuperVersion();
  200. SuperVersion* Ref();
  201. // If Unref() returns true, Cleanup() should be called with mutex held
  202. // before deleting this SuperVersion.
  203. bool Unref();
  204. // call these two methods with db mutex held
  205. // Cleanup unrefs mem, imm and current. Also, it stores all memtables
  206. // that needs to be deleted in to_delete vector. Unrefing those
  207. // objects needs to be done in the mutex
  208. void Cleanup();
  209. void Init(ColumnFamilyData* new_cfd, MemTable* new_mem,
  210. MemTableListVersion* new_imm, Version* new_current);
  211. // The value of dummy is not actually used. kSVInUse takes its address as a
  212. // mark in the thread local storage to indicate the SuperVersion is in use
  213. // by thread. This way, the value of kSVInUse is guaranteed to have no
  214. // conflict with SuperVersion object address and portable on different
  215. // platform.
  216. static int dummy;
  217. static void* const kSVInUse;
  218. static void* const kSVObsolete;
  219. private:
  220. std::atomic<uint32_t> refs;
  221. // We need to_delete because during Cleanup(), imm->Unref() returns
  222. // all memtables that we need to free through this vector. We then
  223. // delete all those memtables outside of mutex, during destruction
  224. autovector<MemTable*> to_delete;
  225. };
  226. extern Status CheckCompressionSupported(const ColumnFamilyOptions& cf_options);
  227. extern Status CheckConcurrentWritesSupported(
  228. const ColumnFamilyOptions& cf_options);
  229. extern Status CheckCFPathsSupported(const DBOptions& db_options,
  230. const ColumnFamilyOptions& cf_options);
  231. extern ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
  232. const ColumnFamilyOptions& src);
  233. // Wrap user defined table proproties collector factories `from cf_options`
  234. // into internal ones in int_tbl_prop_collector_factories. Add a system internal
  235. // one too.
  236. extern void GetIntTblPropCollectorFactory(
  237. const ImmutableCFOptions& ioptions,
  238. std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
  239. int_tbl_prop_collector_factories);
  240. class ColumnFamilySet;
  241. // This class keeps all the data that a column family needs.
  242. // Most methods require DB mutex held, unless otherwise noted
  243. class ColumnFamilyData {
  244. public:
  245. ~ColumnFamilyData();
  246. // thread-safe
  247. uint32_t GetID() const { return id_; }
  248. // thread-safe
  249. const std::string& GetName() const { return name_; }
  250. // Ref() can only be called from a context where the caller can guarantee
  251. // that ColumnFamilyData is alive (while holding a non-zero ref already,
  252. // holding a DB mutex, or as the leader in a write batch group).
  253. void Ref() { refs_.fetch_add(1); }
  254. // Unref decreases the reference count, but does not handle deletion
  255. // when the count goes to 0. If this method returns true then the
  256. // caller should delete the instance immediately, or later, by calling
  257. // FreeDeadColumnFamilies(). Unref() can only be called while holding
  258. // a DB mutex, or during single-threaded recovery.
  259. bool Unref() {
  260. int old_refs = refs_.fetch_sub(1);
  261. assert(old_refs > 0);
  262. return old_refs == 1;
  263. }
  264. // UnrefAndTryDelete() decreases the reference count and do free if needed,
  265. // return true if this is freed else false, UnrefAndTryDelete() can only
  266. // be called while holding a DB mutex, or during single-threaded recovery.
  267. bool UnrefAndTryDelete();
  268. // SetDropped() can only be called under following conditions:
  269. // 1) Holding a DB mutex,
  270. // 2) from single-threaded write thread, AND
  271. // 3) from single-threaded VersionSet::LogAndApply()
  272. // After dropping column family no other operation on that column family
  273. // will be executed. All the files and memory will be, however, kept around
  274. // until client drops the column family handle. That way, client can still
  275. // access data from dropped column family.
  276. // Column family can be dropped and still alive. In that state:
  277. // *) Compaction and flush is not executed on the dropped column family.
  278. // *) Client can continue reading from column family. Writes will fail unless
  279. // WriteOptions::ignore_missing_column_families is true
  280. // When the dropped column family is unreferenced, then we:
  281. // *) Remove column family from the linked list maintained by ColumnFamilySet
  282. // *) delete all memory associated with that column family
  283. // *) delete all the files associated with that column family
  284. void SetDropped();
  285. bool IsDropped() const { return dropped_.load(std::memory_order_relaxed); }
  286. // thread-safe
  287. int NumberLevels() const { return ioptions_.num_levels; }
  288. void SetLogNumber(uint64_t log_number) { log_number_ = log_number; }
  289. uint64_t GetLogNumber() const { return log_number_; }
  290. void SetFlushReason(FlushReason flush_reason) {
  291. flush_reason_ = flush_reason;
  292. }
  293. FlushReason GetFlushReason() const { return flush_reason_; }
  294. // thread-safe
  295. const FileOptions* soptions() const;
  296. const ImmutableCFOptions* ioptions() const { return &ioptions_; }
  297. // REQUIRES: DB mutex held
  298. // This returns the MutableCFOptions used by current SuperVersion
  299. // You should use this API to reference MutableCFOptions most of the time.
  300. const MutableCFOptions* GetCurrentMutableCFOptions() const {
  301. return &(super_version_->mutable_cf_options);
  302. }
  303. // REQUIRES: DB mutex held
  304. // This returns the latest MutableCFOptions, which may be not in effect yet.
  305. const MutableCFOptions* GetLatestMutableCFOptions() const {
  306. return &mutable_cf_options_;
  307. }
  308. // REQUIRES: DB mutex held
  309. // Build ColumnFamiliesOptions with immutable options and latest mutable
  310. // options.
  311. ColumnFamilyOptions GetLatestCFOptions() const;
  312. bool is_delete_range_supported() { return is_delete_range_supported_; }
  313. // Validate CF options against DB options
  314. static Status ValidateOptions(const DBOptions& db_options,
  315. const ColumnFamilyOptions& cf_options);
  316. #ifndef ROCKSDB_LITE
  317. // REQUIRES: DB mutex held
  318. Status SetOptions(
  319. const DBOptions& db_options,
  320. const std::unordered_map<std::string, std::string>& options_map);
  321. #endif // ROCKSDB_LITE
  322. InternalStats* internal_stats() { return internal_stats_.get(); }
  323. MemTableList* imm() { return &imm_; }
  324. MemTable* mem() { return mem_; }
  325. Version* current() { return current_; }
  326. Version* dummy_versions() { return dummy_versions_; }
  327. void SetCurrent(Version* _current);
  328. uint64_t GetNumLiveVersions() const; // REQUIRE: DB mutex held
  329. uint64_t GetTotalSstFilesSize() const; // REQUIRE: DB mutex held
  330. uint64_t GetLiveSstFilesSize() const; // REQUIRE: DB mutex held
  331. void SetMemtable(MemTable* new_mem) {
  332. uint64_t memtable_id = last_memtable_id_.fetch_add(1) + 1;
  333. new_mem->SetID(memtable_id);
  334. mem_ = new_mem;
  335. }
  336. // calculate the oldest log needed for the durability of this column family
  337. uint64_t OldestLogToKeep();
  338. // See Memtable constructor for explanation of earliest_seq param.
  339. MemTable* ConstructNewMemtable(const MutableCFOptions& mutable_cf_options,
  340. SequenceNumber earliest_seq);
  341. void CreateNewMemtable(const MutableCFOptions& mutable_cf_options,
  342. SequenceNumber earliest_seq);
  343. TableCache* table_cache() const { return table_cache_.get(); }
  344. // See documentation in compaction_picker.h
  345. // REQUIRES: DB mutex held
  346. bool NeedsCompaction() const;
  347. // REQUIRES: DB mutex held
  348. Compaction* PickCompaction(const MutableCFOptions& mutable_options,
  349. LogBuffer* log_buffer);
  350. // Check if the passed range overlap with any running compactions.
  351. // REQUIRES: DB mutex held
  352. bool RangeOverlapWithCompaction(const Slice& smallest_user_key,
  353. const Slice& largest_user_key,
  354. int level) const;
  355. // Check if the passed ranges overlap with any unflushed memtables
  356. // (immutable or mutable).
  357. //
  358. // @param super_version A referenced SuperVersion that will be held for the
  359. // duration of this function.
  360. //
  361. // Thread-safe
  362. Status RangesOverlapWithMemtables(const autovector<Range>& ranges,
  363. SuperVersion* super_version, bool* overlap);
  364. // A flag to tell a manual compaction is to compact all levels together
  365. // instead of a specific level.
  366. static const int kCompactAllLevels;
  367. // A flag to tell a manual compaction's output is base level.
  368. static const int kCompactToBaseLevel;
  369. // REQUIRES: DB mutex held
  370. Compaction* CompactRange(const MutableCFOptions& mutable_cf_options,
  371. int input_level, int output_level,
  372. const CompactRangeOptions& compact_range_options,
  373. const InternalKey* begin, const InternalKey* end,
  374. InternalKey** compaction_end, bool* manual_conflict,
  375. uint64_t max_file_num_to_ignore);
  376. CompactionPicker* compaction_picker() { return compaction_picker_.get(); }
  377. // thread-safe
  378. const Comparator* user_comparator() const {
  379. return internal_comparator_.user_comparator();
  380. }
  381. // thread-safe
  382. const InternalKeyComparator& internal_comparator() const {
  383. return internal_comparator_;
  384. }
  385. const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
  386. int_tbl_prop_collector_factories() const {
  387. return &int_tbl_prop_collector_factories_;
  388. }
  389. SuperVersion* GetSuperVersion() { return super_version_; }
  390. // thread-safe
  391. // Return a already referenced SuperVersion to be used safely.
  392. SuperVersion* GetReferencedSuperVersion(DBImpl* db);
  393. // thread-safe
  394. // Get SuperVersion stored in thread local storage. If it does not exist,
  395. // get a reference from a current SuperVersion.
  396. SuperVersion* GetThreadLocalSuperVersion(DBImpl* db);
  397. // Try to return SuperVersion back to thread local storage. Retrun true on
  398. // success and false on failure. It fails when the thread local storage
  399. // contains anything other than SuperVersion::kSVInUse flag.
  400. bool ReturnThreadLocalSuperVersion(SuperVersion* sv);
  401. // thread-safe
  402. uint64_t GetSuperVersionNumber() const {
  403. return super_version_number_.load();
  404. }
  405. // will return a pointer to SuperVersion* if previous SuperVersion
  406. // if its reference count is zero and needs deletion or nullptr if not
  407. // As argument takes a pointer to allocated SuperVersion to enable
  408. // the clients to allocate SuperVersion outside of mutex.
  409. // IMPORTANT: Only call this from DBImpl::InstallSuperVersion()
  410. void InstallSuperVersion(SuperVersionContext* sv_context,
  411. InstrumentedMutex* db_mutex,
  412. const MutableCFOptions& mutable_cf_options);
  413. void InstallSuperVersion(SuperVersionContext* sv_context,
  414. InstrumentedMutex* db_mutex);
  415. void ResetThreadLocalSuperVersions();
  416. // Protected by DB mutex
  417. void set_queued_for_flush(bool value) { queued_for_flush_ = value; }
  418. void set_queued_for_compaction(bool value) { queued_for_compaction_ = value; }
  419. bool queued_for_flush() { return queued_for_flush_; }
  420. bool queued_for_compaction() { return queued_for_compaction_; }
  421. enum class WriteStallCause {
  422. kNone,
  423. kMemtableLimit,
  424. kL0FileCountLimit,
  425. kPendingCompactionBytes,
  426. };
  427. static std::pair<WriteStallCondition, WriteStallCause>
  428. GetWriteStallConditionAndCause(int num_unflushed_memtables, int num_l0_files,
  429. uint64_t num_compaction_needed_bytes,
  430. const MutableCFOptions& mutable_cf_options);
  431. // Recalculate some small conditions, which are changed only during
  432. // compaction, adding new memtable and/or
  433. // recalculation of compaction score. These values are used in
  434. // DBImpl::MakeRoomForWrite function to decide, if it need to make
  435. // a write stall
  436. WriteStallCondition RecalculateWriteStallConditions(
  437. const MutableCFOptions& mutable_cf_options);
  438. void set_initialized() { initialized_.store(true); }
  439. bool initialized() const { return initialized_.load(); }
  440. const ColumnFamilyOptions& initial_cf_options() {
  441. return initial_cf_options_;
  442. }
  443. Env::WriteLifeTimeHint CalculateSSTWriteHint(int level);
  444. // created_dirs remembers directory created, so that we don't need to call
  445. // the same data creation operation again.
  446. Status AddDirectories(
  447. std::map<std::string, std::shared_ptr<Directory>>* created_dirs);
  448. Directory* GetDataDir(size_t path_id) const;
  449. ThreadLocalPtr* TEST_GetLocalSV() { return local_sv_.get(); }
  450. private:
  451. friend class ColumnFamilySet;
  452. ColumnFamilyData(uint32_t id, const std::string& name,
  453. Version* dummy_versions, Cache* table_cache,
  454. WriteBufferManager* write_buffer_manager,
  455. const ColumnFamilyOptions& options,
  456. const ImmutableDBOptions& db_options,
  457. const FileOptions& file_options,
  458. ColumnFamilySet* column_family_set,
  459. BlockCacheTracer* const block_cache_tracer);
  460. uint32_t id_;
  461. const std::string name_;
  462. Version* dummy_versions_; // Head of circular doubly-linked list of versions.
  463. Version* current_; // == dummy_versions->prev_
  464. std::atomic<int> refs_; // outstanding references to ColumnFamilyData
  465. std::atomic<bool> initialized_;
  466. std::atomic<bool> dropped_; // true if client dropped it
  467. const InternalKeyComparator internal_comparator_;
  468. std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
  469. int_tbl_prop_collector_factories_;
  470. const ColumnFamilyOptions initial_cf_options_;
  471. const ImmutableCFOptions ioptions_;
  472. MutableCFOptions mutable_cf_options_;
  473. const bool is_delete_range_supported_;
  474. std::unique_ptr<TableCache> table_cache_;
  475. std::unique_ptr<InternalStats> internal_stats_;
  476. WriteBufferManager* write_buffer_manager_;
  477. MemTable* mem_;
  478. MemTableList imm_;
  479. SuperVersion* super_version_;
  480. // An ordinal representing the current SuperVersion. Updated by
  481. // InstallSuperVersion(), i.e. incremented every time super_version_
  482. // changes.
  483. std::atomic<uint64_t> super_version_number_;
  484. // Thread's local copy of SuperVersion pointer
  485. // This needs to be destructed before mutex_
  486. std::unique_ptr<ThreadLocalPtr> local_sv_;
  487. // pointers for a circular linked list. we use it to support iterations over
  488. // all column families that are alive (note: dropped column families can also
  489. // be alive as long as client holds a reference)
  490. ColumnFamilyData* next_;
  491. ColumnFamilyData* prev_;
  492. // This is the earliest log file number that contains data from this
  493. // Column Family. All earlier log files must be ignored and not
  494. // recovered from
  495. uint64_t log_number_;
  496. std::atomic<FlushReason> flush_reason_;
  497. // An object that keeps all the compaction stats
  498. // and picks the next compaction
  499. std::unique_ptr<CompactionPicker> compaction_picker_;
  500. ColumnFamilySet* column_family_set_;
  501. std::unique_ptr<WriteControllerToken> write_controller_token_;
  502. // If true --> this ColumnFamily is currently present in DBImpl::flush_queue_
  503. bool queued_for_flush_;
  504. // If true --> this ColumnFamily is currently present in
  505. // DBImpl::compaction_queue_
  506. bool queued_for_compaction_;
  507. uint64_t prev_compaction_needed_bytes_;
  508. // if the database was opened with 2pc enabled
  509. bool allow_2pc_;
  510. // Memtable id to track flush.
  511. std::atomic<uint64_t> last_memtable_id_;
  512. // Directories corresponding to cf_paths.
  513. std::vector<std::shared_ptr<Directory>> data_dirs_;
  514. };
  515. // ColumnFamilySet has interesting thread-safety requirements
  516. // * CreateColumnFamily() or RemoveColumnFamily() -- need to be protected by DB
  517. // mutex AND executed in the write thread.
  518. // CreateColumnFamily() should ONLY be called from VersionSet::LogAndApply() AND
  519. // single-threaded write thread. It is also called during Recovery and in
  520. // DumpManifest().
  521. // RemoveColumnFamily() is only called from SetDropped(). DB mutex needs to be
  522. // held and it needs to be executed from the write thread. SetDropped() also
  523. // guarantees that it will be called only from single-threaded LogAndApply(),
  524. // but this condition is not that important.
  525. // * Iteration -- hold DB mutex, but you can release it in the body of
  526. // iteration. If you release DB mutex in body, reference the column
  527. // family before the mutex and unreference after you unlock, since the column
  528. // family might get dropped when the DB mutex is released
  529. // * GetDefault() -- thread safe
  530. // * GetColumnFamily() -- either inside of DB mutex or from a write thread
  531. // * GetNextColumnFamilyID(), GetMaxColumnFamily(), UpdateMaxColumnFamily(),
  532. // NumberOfColumnFamilies -- inside of DB mutex
  533. class ColumnFamilySet {
  534. public:
  535. // ColumnFamilySet supports iteration
  536. class iterator {
  537. public:
  538. explicit iterator(ColumnFamilyData* cfd)
  539. : current_(cfd) {}
  540. iterator& operator++() {
  541. // dropped column families might still be included in this iteration
  542. // (we're only removing them when client drops the last reference to the
  543. // column family).
  544. // dummy is never dead, so this will never be infinite
  545. do {
  546. current_ = current_->next_;
  547. } while (current_->refs_.load(std::memory_order_relaxed) == 0);
  548. return *this;
  549. }
  550. bool operator!=(const iterator& other) {
  551. return this->current_ != other.current_;
  552. }
  553. ColumnFamilyData* operator*() { return current_; }
  554. private:
  555. ColumnFamilyData* current_;
  556. };
  557. ColumnFamilySet(const std::string& dbname,
  558. const ImmutableDBOptions* db_options,
  559. const FileOptions& file_options, Cache* table_cache,
  560. WriteBufferManager* write_buffer_manager,
  561. WriteController* write_controller,
  562. BlockCacheTracer* const block_cache_tracer);
  563. ~ColumnFamilySet();
  564. ColumnFamilyData* GetDefault() const;
  565. // GetColumnFamily() calls return nullptr if column family is not found
  566. ColumnFamilyData* GetColumnFamily(uint32_t id) const;
  567. ColumnFamilyData* GetColumnFamily(const std::string& name) const;
  568. // this call will return the next available column family ID. it guarantees
  569. // that there is no column family with id greater than or equal to the
  570. // returned value in the current running instance or anytime in RocksDB
  571. // instance history.
  572. uint32_t GetNextColumnFamilyID();
  573. uint32_t GetMaxColumnFamily();
  574. void UpdateMaxColumnFamily(uint32_t new_max_column_family);
  575. size_t NumberOfColumnFamilies() const;
  576. ColumnFamilyData* CreateColumnFamily(const std::string& name, uint32_t id,
  577. Version* dummy_version,
  578. const ColumnFamilyOptions& options);
  579. iterator begin() { return iterator(dummy_cfd_->next_); }
  580. iterator end() { return iterator(dummy_cfd_); }
  581. // REQUIRES: DB mutex held
  582. // Don't call while iterating over ColumnFamilySet
  583. void FreeDeadColumnFamilies();
  584. Cache* get_table_cache() { return table_cache_; }
  585. private:
  586. friend class ColumnFamilyData;
  587. // helper function that gets called from cfd destructor
  588. // REQUIRES: DB mutex held
  589. void RemoveColumnFamily(ColumnFamilyData* cfd);
  590. // column_families_ and column_family_data_ need to be protected:
  591. // * when mutating both conditions have to be satisfied:
  592. // 1. DB mutex locked
  593. // 2. thread currently in single-threaded write thread
  594. // * when reading, at least one condition needs to be satisfied:
  595. // 1. DB mutex locked
  596. // 2. accessed from a single-threaded write thread
  597. std::unordered_map<std::string, uint32_t> column_families_;
  598. std::unordered_map<uint32_t, ColumnFamilyData*> column_family_data_;
  599. uint32_t max_column_family_;
  600. ColumnFamilyData* dummy_cfd_;
  601. // We don't hold the refcount here, since default column family always exists
  602. // We are also not responsible for cleaning up default_cfd_cache_. This is
  603. // just a cache that makes common case (accessing default column family)
  604. // faster
  605. ColumnFamilyData* default_cfd_cache_;
  606. const std::string db_name_;
  607. const ImmutableDBOptions* const db_options_;
  608. const FileOptions file_options_;
  609. Cache* table_cache_;
  610. WriteBufferManager* write_buffer_manager_;
  611. WriteController* write_controller_;
  612. BlockCacheTracer* const block_cache_tracer_;
  613. };
  614. // We use ColumnFamilyMemTablesImpl to provide WriteBatch a way to access
  615. // memtables of different column families (specified by ID in the write batch)
  616. class ColumnFamilyMemTablesImpl : public ColumnFamilyMemTables {
  617. public:
  618. explicit ColumnFamilyMemTablesImpl(ColumnFamilySet* column_family_set)
  619. : column_family_set_(column_family_set), current_(nullptr) {}
  620. // Constructs a ColumnFamilyMemTablesImpl equivalent to one constructed
  621. // with the arguments used to construct *orig.
  622. explicit ColumnFamilyMemTablesImpl(ColumnFamilyMemTablesImpl* orig)
  623. : column_family_set_(orig->column_family_set_), current_(nullptr) {}
  624. // sets current_ to ColumnFamilyData with column_family_id
  625. // returns false if column family doesn't exist
  626. // REQUIRES: use this function of DBImpl::column_family_memtables_ should be
  627. // under a DB mutex OR from a write thread
  628. bool Seek(uint32_t column_family_id) override;
  629. // Returns log number of the selected column family
  630. // REQUIRES: under a DB mutex OR from a write thread
  631. uint64_t GetLogNumber() const override;
  632. // REQUIRES: Seek() called first
  633. // REQUIRES: use this function of DBImpl::column_family_memtables_ should be
  634. // under a DB mutex OR from a write thread
  635. virtual MemTable* GetMemTable() const override;
  636. // Returns column family handle for the selected column family
  637. // REQUIRES: use this function of DBImpl::column_family_memtables_ should be
  638. // under a DB mutex OR from a write thread
  639. virtual ColumnFamilyHandle* GetColumnFamilyHandle() override;
  640. // Cannot be called while another thread is calling Seek().
  641. // REQUIRES: use this function of DBImpl::column_family_memtables_ should be
  642. // under a DB mutex OR from a write thread
  643. virtual ColumnFamilyData* current() override { return current_; }
  644. private:
  645. ColumnFamilySet* column_family_set_;
  646. ColumnFamilyData* current_;
  647. ColumnFamilyHandleInternal handle_;
  648. };
  649. extern uint32_t GetColumnFamilyID(ColumnFamilyHandle* column_family);
  650. extern const Comparator* GetColumnFamilyUserComparator(
  651. ColumnFamilyHandle* column_family);
  652. } // namespace ROCKSDB_NAMESPACE