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 
11 #include <atomic>
12 #include <deque>
13 #include <functional>
14 #include <limits>
15 #include <list>
16 #include <map>
17 #include <set>
18 #include <string>
19 #include <utility>
20 #include <vector>
21 
22 #include "db/column_family.h"
23 #include "db/compaction/compaction_job.h"
24 #include "db/dbformat.h"
25 #include "db/error_handler.h"
26 #include "db/event_helpers.h"
27 #include "db/external_sst_file_ingestion_job.h"
28 #include "db/flush_job.h"
29 #include "db/flush_scheduler.h"
30 #include "db/import_column_family_job.h"
31 #include "db/internal_stats.h"
32 #include "db/log_writer.h"
33 #include "db/logs_with_prep_tracker.h"
34 #include "db/memtable_list.h"
35 #include "db/pre_release_callback.h"
36 #include "db/range_del_aggregator.h"
37 #include "db/read_callback.h"
38 #include "db/snapshot_checker.h"
39 #include "db/snapshot_impl.h"
40 #include "db/trim_history_scheduler.h"
41 #include "db/version_edit.h"
42 #include "db/wal_manager.h"
43 #include "db/write_controller.h"
44 #include "db/write_thread.h"
45 #include "logging/event_logger.h"
46 #include "monitoring/instrumented_mutex.h"
47 #include "options/db_options.h"
48 #include "port/port.h"
49 #include "rocksdb/db.h"
50 #include "rocksdb/env.h"
51 #include "rocksdb/memtablerep.h"
52 #include "rocksdb/status.h"
53 #include "rocksdb/trace_reader_writer.h"
54 #include "rocksdb/transaction_log.h"
55 #include "rocksdb/write_buffer_manager.h"
56 #include "table/scoped_arena_iterator.h"
57 #include "trace_replay/block_cache_tracer.h"
58 #include "trace_replay/trace_replay.h"
59 #include "util/autovector.h"
60 #include "util/hash.h"
61 #include "util/repeatable_thread.h"
62 #include "util/stop_watch.h"
63 #include "util/thread_local.h"
64 
65 namespace ROCKSDB_NAMESPACE {
66 
67 class Arena;
68 class ArenaWrappedDBIter;
69 class InMemoryStatsHistoryIterator;
70 class MemTable;
71 class PersistentStatsHistoryIterator;
72 class TableCache;
73 class TaskLimiterToken;
74 class Version;
75 class VersionEdit;
76 class VersionSet;
77 class WriteCallback;
78 struct JobContext;
79 struct ExternalSstFileInfo;
80 struct MemTableInfo;
81 
82 // Class to maintain directories for all database paths other than main one.
83 class Directories {
84  public:
85   IOStatus SetDirectories(FileSystem* fs, const std::string& dbname,
86                           const std::string& wal_dir,
87                           const std::vector<DbPath>& data_paths);
88 
GetDataDir(size_t path_id)89   FSDirectory* GetDataDir(size_t path_id) const {
90     assert(path_id < data_dirs_.size());
91     FSDirectory* ret_dir = data_dirs_[path_id].get();
92     if (ret_dir == nullptr) {
93       // Should use db_dir_
94       return db_dir_.get();
95     }
96     return ret_dir;
97   }
98 
GetWalDir()99   FSDirectory* GetWalDir() {
100     if (wal_dir_) {
101       return wal_dir_.get();
102     }
103     return db_dir_.get();
104   }
105 
GetDbDir()106   FSDirectory* GetDbDir() { return db_dir_.get(); }
107 
108  private:
109   std::unique_ptr<FSDirectory> db_dir_;
110   std::vector<std::unique_ptr<FSDirectory>> data_dirs_;
111   std::unique_ptr<FSDirectory> wal_dir_;
112 };
113 
114 // While DB is the public interface of RocksDB, and DBImpl is the actual
115 // class implementing it. It's the entrance of the core RocksdB engine.
116 // All other DB implementations, e.g. TransactionDB, BlobDB, etc, wrap a
117 // DBImpl internally.
118 // Other than functions implementing the DB interface, some public
119 // functions are there for other internal components to call. For
120 // example, TransactionDB directly calls DBImpl::WriteImpl() and
121 // BlobDB directly calls DBImpl::GetImpl(). Some other functions
122 // are for sub-components to call. For example, ColumnFamilyHandleImpl
123 // calls DBImpl::FindObsoleteFiles().
124 //
125 // Since it's a very large class, the definition of the functions is
126 // divided in several db_impl_*.cc files, besides db_impl.cc.
127 class DBImpl : public DB {
128  public:
129   DBImpl(const DBOptions& options, const std::string& dbname,
130          const bool seq_per_batch = false, const bool batch_per_txn = true);
131   // No copying allowed
132   DBImpl(const DBImpl&) = delete;
133   void operator=(const DBImpl&) = delete;
134 
135   virtual ~DBImpl();
136 
137   // ---- Implementations of the DB interface ----
138 
139   using DB::Resume;
140   virtual Status Resume() override;
141 
142   using DB::Put;
143   virtual Status Put(const WriteOptions& options,
144                      ColumnFamilyHandle* column_family, const Slice& key,
145                      const Slice& value) override;
146   using DB::Merge;
147   virtual Status Merge(const WriteOptions& options,
148                        ColumnFamilyHandle* column_family, const Slice& key,
149                        const Slice& value) override;
150   using DB::Delete;
151   virtual Status Delete(const WriteOptions& options,
152                         ColumnFamilyHandle* column_family,
153                         const Slice& key) override;
154   using DB::SingleDelete;
155   virtual Status SingleDelete(const WriteOptions& options,
156                               ColumnFamilyHandle* column_family,
157                               const Slice& key) override;
158   using DB::Write;
159   virtual Status Write(const WriteOptions& options,
160                        WriteBatch* updates) override;
161 
162   using DB::Get;
163   virtual Status Get(const ReadOptions& options,
164                      ColumnFamilyHandle* column_family, const Slice& key,
165                      PinnableSlice* value) override;
166   virtual Status Get(const ReadOptions& options,
167                      ColumnFamilyHandle* column_family, const Slice& key,
168                      PinnableSlice* value, std::string* timestamp) override;
169 
170   using DB::GetMergeOperands;
GetMergeOperands(const ReadOptions & options,ColumnFamilyHandle * column_family,const Slice & key,PinnableSlice * merge_operands,GetMergeOperandsOptions * get_merge_operands_options,int * number_of_operands)171   Status GetMergeOperands(const ReadOptions& options,
172                           ColumnFamilyHandle* column_family, const Slice& key,
173                           PinnableSlice* merge_operands,
174                           GetMergeOperandsOptions* get_merge_operands_options,
175                           int* number_of_operands) override {
176     GetImplOptions get_impl_options;
177     get_impl_options.column_family = column_family;
178     get_impl_options.merge_operands = merge_operands;
179     get_impl_options.get_merge_operands_options = get_merge_operands_options;
180     get_impl_options.number_of_operands = number_of_operands;
181     get_impl_options.get_value = false;
182     return GetImpl(options, key, get_impl_options);
183   }
184 
185   using DB::MultiGet;
186   virtual std::vector<Status> MultiGet(
187       const ReadOptions& options,
188       const std::vector<ColumnFamilyHandle*>& column_family,
189       const std::vector<Slice>& keys,
190       std::vector<std::string>* values) override;
191   virtual std::vector<Status> MultiGet(
192       const ReadOptions& options,
193       const std::vector<ColumnFamilyHandle*>& column_family,
194       const std::vector<Slice>& keys, std::vector<std::string>* values,
195       std::vector<std::string>* timestamps) override;
196 
197   // This MultiGet is a batched version, which may be faster than calling Get
198   // multiple times, especially if the keys have some spatial locality that
199   // enables them to be queried in the same SST files/set of files. The larger
200   // the batch size, the more scope for batching and performance improvement
201   // The values and statuses parameters are arrays with number of elements
202   // equal to keys.size(). This allows the storage for those to be alloacted
203   // by the caller on the stack for small batches
204   virtual void MultiGet(const ReadOptions& options,
205                         ColumnFamilyHandle* column_family,
206                         const size_t num_keys, const Slice* keys,
207                         PinnableSlice* values, Status* statuses,
208                         const bool sorted_input = false) override;
209   virtual void MultiGet(const ReadOptions& options,
210                         ColumnFamilyHandle* column_family,
211                         const size_t num_keys, const Slice* keys,
212                         PinnableSlice* values, std::string* timestamps,
213                         Status* statuses,
214                         const bool sorted_input = false) override;
215 
216   virtual void MultiGet(const ReadOptions& options, const size_t num_keys,
217                         ColumnFamilyHandle** column_families, const Slice* keys,
218                         PinnableSlice* values, Status* statuses,
219                         const bool sorted_input = false) override;
220   virtual void MultiGet(const ReadOptions& options, const size_t num_keys,
221                         ColumnFamilyHandle** column_families, const Slice* keys,
222                         PinnableSlice* values, std::string* timestamps,
223                         Status* statuses,
224                         const bool sorted_input = false) override;
225 
226   virtual void MultiGetWithCallback(
227       const ReadOptions& options, ColumnFamilyHandle* column_family,
228       ReadCallback* callback,
229       autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE>* sorted_keys);
230 
231   virtual Status CreateColumnFamily(const ColumnFamilyOptions& cf_options,
232                                     const std::string& column_family,
233                                     ColumnFamilyHandle** handle) override;
234   virtual Status CreateColumnFamilies(
235       const ColumnFamilyOptions& cf_options,
236       const std::vector<std::string>& column_family_names,
237       std::vector<ColumnFamilyHandle*>* handles) override;
238   virtual Status CreateColumnFamilies(
239       const std::vector<ColumnFamilyDescriptor>& column_families,
240       std::vector<ColumnFamilyHandle*>* handles) override;
241   virtual Status DropColumnFamily(ColumnFamilyHandle* column_family) override;
242   virtual Status DropColumnFamilies(
243       const std::vector<ColumnFamilyHandle*>& column_families) override;
244 
245   // Returns false if key doesn't exist in the database and true if it may.
246   // If value_found is not passed in as null, then return the value if found in
247   // memory. On return, if value was found, then value_found will be set to true
248   // , otherwise false.
249   using DB::KeyMayExist;
250   virtual bool KeyMayExist(const ReadOptions& options,
251                            ColumnFamilyHandle* column_family, const Slice& key,
252                            std::string* value, std::string* timestamp,
253                            bool* value_found = nullptr) override;
254 
255   using DB::NewIterator;
256   virtual Iterator* NewIterator(const ReadOptions& options,
257                                 ColumnFamilyHandle* column_family) override;
258   virtual Status NewIterators(
259       const ReadOptions& options,
260       const std::vector<ColumnFamilyHandle*>& column_families,
261       std::vector<Iterator*>* iterators) override;
262 
263   virtual const Snapshot* GetSnapshot() override;
264   virtual void ReleaseSnapshot(const Snapshot* snapshot) override;
265   using DB::GetProperty;
266   virtual bool GetProperty(ColumnFamilyHandle* column_family,
267                            const Slice& property, std::string* value) override;
268   using DB::GetMapProperty;
269   virtual bool GetMapProperty(
270       ColumnFamilyHandle* column_family, const Slice& property,
271       std::map<std::string, std::string>* value) override;
272   using DB::GetIntProperty;
273   virtual bool GetIntProperty(ColumnFamilyHandle* column_family,
274                               const Slice& property, uint64_t* value) override;
275   using DB::GetAggregatedIntProperty;
276   virtual bool GetAggregatedIntProperty(const Slice& property,
277                                         uint64_t* aggregated_value) override;
278   using DB::GetApproximateSizes;
279   virtual Status GetApproximateSizes(const SizeApproximationOptions& options,
280                                      ColumnFamilyHandle* column_family,
281                                      const Range* range, int n,
282                                      uint64_t* sizes) override;
283   using DB::GetApproximateMemTableStats;
284   virtual void GetApproximateMemTableStats(ColumnFamilyHandle* column_family,
285                                            const Range& range,
286                                            uint64_t* const count,
287                                            uint64_t* const size) override;
288   using DB::CompactRange;
289   virtual Status CompactRange(const CompactRangeOptions& options,
290                               ColumnFamilyHandle* column_family,
291                               const Slice* begin, const Slice* end) override;
292 
293   using DB::CompactFiles;
294   virtual Status CompactFiles(
295       const CompactionOptions& compact_options,
296       ColumnFamilyHandle* column_family,
297       const std::vector<std::string>& input_file_names, const int output_level,
298       const int output_path_id = -1,
299       std::vector<std::string>* const output_file_names = nullptr,
300       CompactionJobInfo* compaction_job_info = nullptr) override;
301 
302   virtual Status PauseBackgroundWork() override;
303   virtual Status ContinueBackgroundWork() override;
304 
305   virtual Status EnableAutoCompaction(
306       const std::vector<ColumnFamilyHandle*>& column_family_handles) override;
307 
308   virtual void EnableManualCompaction() override;
309   virtual void DisableManualCompaction() override;
310 
311   using DB::SetOptions;
312   Status SetOptions(
313       ColumnFamilyHandle* column_family,
314       const std::unordered_map<std::string, std::string>& options_map) override;
315 
316   virtual Status SetDBOptions(
317       const std::unordered_map<std::string, std::string>& options_map) override;
318 
319   using DB::NumberLevels;
320   virtual int NumberLevels(ColumnFamilyHandle* column_family) override;
321   using DB::MaxMemCompactionLevel;
322   virtual int MaxMemCompactionLevel(ColumnFamilyHandle* column_family) override;
323   using DB::Level0StopWriteTrigger;
324   virtual int Level0StopWriteTrigger(
325       ColumnFamilyHandle* column_family) override;
326   virtual const std::string& GetName() const override;
327   virtual Env* GetEnv() const override;
328   virtual FileSystem* GetFileSystem() const override;
329   using DB::GetOptions;
330   virtual Options GetOptions(ColumnFamilyHandle* column_family) const override;
331   using DB::GetDBOptions;
332   virtual DBOptions GetDBOptions() const override;
333   using DB::Flush;
334   virtual Status Flush(const FlushOptions& options,
335                        ColumnFamilyHandle* column_family) override;
336   virtual Status Flush(
337       const FlushOptions& options,
338       const std::vector<ColumnFamilyHandle*>& column_families) override;
339   virtual Status FlushWAL(bool sync) override;
340   bool TEST_WALBufferIsEmpty(bool lock = true);
341   virtual Status SyncWAL() override;
342   virtual Status LockWAL() override;
343   virtual Status UnlockWAL() override;
344 
345   virtual SequenceNumber GetLatestSequenceNumber() const override;
346 
347   virtual bool SetPreserveDeletesSequenceNumber(SequenceNumber seqnum) override;
348 
349   virtual Status GetDbIdentity(std::string& identity) const override;
350 
351   virtual Status GetDbIdentityFromIdentityFile(std::string* identity) const;
352 
353   ColumnFamilyHandle* DefaultColumnFamily() const override;
354 
355   ColumnFamilyHandle* PersistentStatsColumnFamily() const;
356 
357   virtual Status Close() override;
358 
359   Status GetStatsHistory(
360       uint64_t start_time, uint64_t end_time,
361       std::unique_ptr<StatsHistoryIterator>* stats_iterator) override;
362 
363 #ifndef ROCKSDB_LITE
364   using DB::ResetStats;
365   virtual Status ResetStats() override;
366   virtual Status DisableFileDeletions() override;
367   virtual Status EnableFileDeletions(bool force) override;
368   virtual int IsFileDeletionsEnabled() const;
369   // All the returned filenames start with "/"
370   virtual Status GetLiveFiles(std::vector<std::string>&,
371                               uint64_t* manifest_file_size,
372                               bool flush_memtable = true) override;
373   virtual Status GetSortedWalFiles(VectorLogPtr& files) override;
374   virtual Status GetCurrentWalFile(
375       std::unique_ptr<LogFile>* current_log_file) override;
376   virtual Status GetCreationTimeOfOldestFile(
377       uint64_t* creation_time) override;
378 
379   virtual Status GetUpdatesSince(
380       SequenceNumber seq_number, std::unique_ptr<TransactionLogIterator>* iter,
381       const TransactionLogIterator::ReadOptions& read_options =
382           TransactionLogIterator::ReadOptions()) override;
383   virtual Status DeleteFile(std::string name) override;
384   Status DeleteFilesInRanges(ColumnFamilyHandle* column_family,
385                              const RangePtr* ranges, size_t n,
386                              bool include_end = true);
387 
388   virtual void GetLiveFilesMetaData(
389       std::vector<LiveFileMetaData>* metadata) override;
390 
391   // Obtains the meta data of the specified column family of the DB.
392   // Status::NotFound() will be returned if the current DB does not have
393   // any column family match the specified name.
394   // TODO(yhchiang): output parameter is placed in the end in this codebase.
395   virtual void GetColumnFamilyMetaData(ColumnFamilyHandle* column_family,
396                                        ColumnFamilyMetaData* metadata) override;
397 
398   Status SuggestCompactRange(ColumnFamilyHandle* column_family,
399                              const Slice* begin, const Slice* end) override;
400 
401   Status PromoteL0(ColumnFamilyHandle* column_family,
402                    int target_level) override;
403 
404   using DB::IngestExternalFile;
405   virtual Status IngestExternalFile(
406       ColumnFamilyHandle* column_family,
407       const std::vector<std::string>& external_files,
408       const IngestExternalFileOptions& ingestion_options) override;
409 
410   using DB::IngestExternalFiles;
411   virtual Status IngestExternalFiles(
412       const std::vector<IngestExternalFileArg>& args) override;
413 
414   using DB::CreateColumnFamilyWithImport;
415   virtual Status CreateColumnFamilyWithImport(
416       const ColumnFamilyOptions& options, const std::string& column_family_name,
417       const ImportColumnFamilyOptions& import_options,
418       const ExportImportFilesMetaData& metadata,
419       ColumnFamilyHandle** handle) override;
420 
421   using DB::VerifyChecksum;
422   virtual Status VerifyChecksum(const ReadOptions& /*read_options*/) override;
423 
424   using DB::StartTrace;
425   virtual Status StartTrace(
426       const TraceOptions& options,
427       std::unique_ptr<TraceWriter>&& trace_writer) override;
428 
429   using DB::EndTrace;
430   virtual Status EndTrace() override;
431 
432   using DB::StartBlockCacheTrace;
433   Status StartBlockCacheTrace(
434       const TraceOptions& options,
435       std::unique_ptr<TraceWriter>&& trace_writer) override;
436 
437   using DB::EndBlockCacheTrace;
438   Status EndBlockCacheTrace() override;
439 
440   using DB::GetPropertiesOfAllTables;
441   virtual Status GetPropertiesOfAllTables(
442       ColumnFamilyHandle* column_family,
443       TablePropertiesCollection* props) override;
444   virtual Status GetPropertiesOfTablesInRange(
445       ColumnFamilyHandle* column_family, const Range* range, std::size_t n,
446       TablePropertiesCollection* props) override;
447 
448 #endif  // ROCKSDB_LITE
449 
450   // ---- End of implementations of the DB interface ----
451 
452   struct GetImplOptions {
453     ColumnFamilyHandle* column_family = nullptr;
454     PinnableSlice* value = nullptr;
455     std::string* timestamp = nullptr;
456     bool* value_found = nullptr;
457     ReadCallback* callback = nullptr;
458     bool* is_blob_index = nullptr;
459     // If true return value associated with key via value pointer else return
460     // all merge operands for key via merge_operands pointer
461     bool get_value = true;
462     // Pointer to an array of size
463     // get_merge_operands_options.expected_max_number_of_operands allocated by
464     // user
465     PinnableSlice* merge_operands = nullptr;
466     GetMergeOperandsOptions* get_merge_operands_options = nullptr;
467     int* number_of_operands = nullptr;
468   };
469 
470   // Function that Get and KeyMayExist call with no_io true or false
471   // Note: 'value_found' from KeyMayExist propagates here
472   // This function is also called by GetMergeOperands
473   // If get_impl_options.get_value = true get value associated with
474   // get_impl_options.key via get_impl_options.value
475   // If get_impl_options.get_value = false get merge operands associated with
476   // get_impl_options.key via get_impl_options.merge_operands
477   Status GetImpl(const ReadOptions& options, const Slice& key,
478                  GetImplOptions& get_impl_options);
479 
480   ArenaWrappedDBIter* NewIteratorImpl(const ReadOptions& options,
481                                       ColumnFamilyData* cfd,
482                                       SequenceNumber snapshot,
483                                       ReadCallback* read_callback,
484                                       bool allow_blob = false,
485                                       bool allow_refresh = true);
486 
GetLastPublishedSequence()487   virtual SequenceNumber GetLastPublishedSequence() const {
488     if (last_seq_same_as_publish_seq_) {
489       return versions_->LastSequence();
490     } else {
491       return versions_->LastPublishedSequence();
492     }
493   }
494 
495   // REQUIRES: joined the main write queue if two_write_queues is disabled, and
496   // the second write queue otherwise.
497   virtual void SetLastPublishedSequence(SequenceNumber seq);
498   // Returns LastSequence in last_seq_same_as_publish_seq_
499   // mode and LastAllocatedSequence otherwise. This is useful when visiblility
500   // depends also on data written to the WAL but not to the memtable.
501   SequenceNumber TEST_GetLastVisibleSequence() const;
502 
503 #ifndef ROCKSDB_LITE
504   // Similar to Write() but will call the callback once on the single write
505   // thread to determine whether it is safe to perform the write.
506   virtual Status WriteWithCallback(const WriteOptions& write_options,
507                                    WriteBatch* my_batch,
508                                    WriteCallback* callback);
509 
510   // Returns the sequence number that is guaranteed to be smaller than or equal
511   // to the sequence number of any key that could be inserted into the current
512   // memtables. It can then be assumed that any write with a larger(or equal)
513   // sequence number will be present in this memtable or a later memtable.
514   //
515   // If the earliest sequence number could not be determined,
516   // kMaxSequenceNumber will be returned.
517   //
518   // If include_history=true, will also search Memtables in MemTableList
519   // History.
520   SequenceNumber GetEarliestMemTableSequenceNumber(SuperVersion* sv,
521                                                    bool include_history);
522 
523   // For a given key, check to see if there are any records for this key
524   // in the memtables, including memtable history.  If cache_only is false,
525   // SST files will also be checked.
526   //
527   // If a key is found, *found_record_for_key will be set to true and
528   // *seq will be set to the stored sequence number for the latest
529   // operation on this key or kMaxSequenceNumber if unknown.
530   // If no key is found, *found_record_for_key will be set to false.
531   //
532   // Note: If cache_only=false, it is possible for *seq to be set to 0 if
533   // the sequence number has been cleared from the record.  If the caller is
534   // holding an active db snapshot, we know the missing sequence must be less
535   // than the snapshot's sequence number (sequence numbers are only cleared
536   // when there are no earlier active snapshots).
537   //
538   // If NotFound is returned and found_record_for_key is set to false, then no
539   // record for this key was found.  If the caller is holding an active db
540   // snapshot, we know that no key could have existing after this snapshot
541   // (since we do not compact keys that have an earlier snapshot).
542   //
543   // Only records newer than or at `lower_bound_seq` are guaranteed to be
544   // returned. Memtables and files may not be checked if it only contains data
545   // older than `lower_bound_seq`.
546   //
547   // Returns OK or NotFound on success,
548   // other status on unexpected error.
549   // TODO(andrewkr): this API need to be aware of range deletion operations
550   Status GetLatestSequenceForKey(SuperVersion* sv, const Slice& key,
551                                  bool cache_only,
552                                  SequenceNumber lower_bound_seq,
553                                  SequenceNumber* seq,
554                                  bool* found_record_for_key,
555                                  bool* is_blob_index = nullptr);
556 
557   Status TraceIteratorSeek(const uint32_t& cf_id, const Slice& key);
558   Status TraceIteratorSeekForPrev(const uint32_t& cf_id, const Slice& key);
559 #endif  // ROCKSDB_LITE
560 
561   // Similar to GetSnapshot(), but also lets the db know that this snapshot
562   // will be used for transaction write-conflict checking.  The DB can then
563   // make sure not to compact any keys that would prevent a write-conflict from
564   // being detected.
565   const Snapshot* GetSnapshotForWriteConflictBoundary();
566 
567   // checks if all live files exist on file system and that their file sizes
568   // match to our in-memory records
569   virtual Status CheckConsistency();
570 
571   // max_file_num_to_ignore allows bottom level compaction to filter out newly
572   // compacted SST files. Setting max_file_num_to_ignore to kMaxUint64 will
573   // disable the filtering
574   Status RunManualCompaction(ColumnFamilyData* cfd, int input_level,
575                              int output_level,
576                              const CompactRangeOptions& compact_range_options,
577                              const Slice* begin, const Slice* end,
578                              bool exclusive, bool disallow_trivial_move,
579                              uint64_t max_file_num_to_ignore);
580 
581   // Return an internal iterator over the current state of the database.
582   // The keys of this iterator are internal keys (see format.h).
583   // The returned iterator should be deleted when no longer needed.
584   InternalIterator* NewInternalIterator(
585       Arena* arena, RangeDelAggregator* range_del_agg, SequenceNumber sequence,
586       ColumnFamilyHandle* column_family = nullptr);
587 
logs_with_prep_tracker()588   LogsWithPrepTracker* logs_with_prep_tracker() {
589     return &logs_with_prep_tracker_;
590   }
591 
592   struct BGJobLimits {
593     int max_flushes;
594     int max_compactions;
595   };
596   // Returns maximum background flushes and compactions allowed to be scheduled
597   BGJobLimits GetBGJobLimits() const;
598   // Need a static version that can be called during SanitizeOptions().
599   static BGJobLimits GetBGJobLimits(int max_background_flushes,
600                                     int max_background_compactions,
601                                     int max_background_jobs,
602                                     bool parallelize_compactions);
603 
604   // move logs pending closing from job_context to the DB queue and
605   // schedule a purge
606   void ScheduleBgLogWriterClose(JobContext* job_context);
607 
608   uint64_t MinLogNumberToKeep();
609 
610   // Returns the lower bound file number for SSTs that won't be deleted, even if
611   // they're obsolete. This lower bound is used internally to prevent newly
612   // created flush/compaction output files from being deleted before they're
613   // installed. This technique avoids the need for tracking the exact numbers of
614   // files pending creation, although it prevents more files than necessary from
615   // being deleted.
616   uint64_t MinObsoleteSstNumberToKeep();
617 
618   // Returns the list of live files in 'live' and the list
619   // of all files in the filesystem in 'candidate_files'.
620   // If force == false and the last call was less than
621   // db_options_.delete_obsolete_files_period_micros microseconds ago,
622   // it will not fill up the job_context
623   void FindObsoleteFiles(JobContext* job_context, bool force,
624                          bool no_full_scan = false);
625 
626   // Diffs the files listed in filenames and those that do not
627   // belong to live files are possibly removed. Also, removes all the
628   // files in sst_delete_files and log_delete_files.
629   // It is not necessary to hold the mutex when invoking this method.
630   // If FindObsoleteFiles() was run, we need to also run
631   // PurgeObsoleteFiles(), even if disable_delete_obsolete_files_ is true
632   void PurgeObsoleteFiles(JobContext& background_contet,
633                           bool schedule_only = false);
634 
635   // Schedule a background job to actually delete obsolete files.
636   void SchedulePurge();
637 
snapshots()638   const SnapshotList& snapshots() const { return snapshots_; }
639 
640   // load list of snapshots to `snap_vector` that is no newer than `max_seq`
641   // in ascending order.
642   // `oldest_write_conflict_snapshot` is filled with the oldest snapshot
643   // which satisfies SnapshotImpl.is_write_conflict_boundary_ = true.
LoadSnapshots(std::vector<SequenceNumber> * snap_vector,SequenceNumber * oldest_write_conflict_snapshot,const SequenceNumber & max_seq)644   void LoadSnapshots(std::vector<SequenceNumber>* snap_vector,
645                      SequenceNumber* oldest_write_conflict_snapshot,
646                      const SequenceNumber& max_seq) const {
647     InstrumentedMutexLock l(mutex());
648     snapshots().GetAll(snap_vector, oldest_write_conflict_snapshot, max_seq);
649   }
650 
immutable_db_options()651   const ImmutableDBOptions& immutable_db_options() const {
652     return immutable_db_options_;
653   }
654 
655   // Cancel all background jobs, including flush, compaction, background
656   // purging, stats dumping threads, etc. If `wait` = true, wait for the
657   // running jobs to abort or finish before returning. Otherwise, only
658   // sends the signals.
659   void CancelAllBackgroundWork(bool wait);
660 
661   // Find Super version and reference it. Based on options, it might return
662   // the thread local cached one.
663   // Call ReturnAndCleanupSuperVersion() when it is no longer needed.
664   SuperVersion* GetAndRefSuperVersion(ColumnFamilyData* cfd);
665 
666   // Similar to the previous function but looks up based on a column family id.
667   // nullptr will be returned if this column family no longer exists.
668   // REQUIRED: this function should only be called on the write thread or if the
669   // mutex is held.
670   SuperVersion* GetAndRefSuperVersion(uint32_t column_family_id);
671 
672   // Un-reference the super version and clean it up if it is the last reference.
673   void CleanupSuperVersion(SuperVersion* sv);
674 
675   // Un-reference the super version and return it to thread local cache if
676   // needed. If it is the last reference of the super version. Clean it up
677   // after un-referencing it.
678   void ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd, SuperVersion* sv);
679 
680   // Similar to the previous function but looks up based on a column family id.
681   // nullptr will be returned if this column family no longer exists.
682   // REQUIRED: this function should only be called on the write thread.
683   void ReturnAndCleanupSuperVersion(uint32_t colun_family_id, SuperVersion* sv);
684 
685   // REQUIRED: this function should only be called on the write thread or if the
686   // mutex is held.  Return value only valid until next call to this function or
687   // mutex is released.
688   ColumnFamilyHandle* GetColumnFamilyHandle(uint32_t column_family_id);
689 
690   // Same as above, should called without mutex held and not on write thread.
691   std::unique_ptr<ColumnFamilyHandle> GetColumnFamilyHandleUnlocked(
692       uint32_t column_family_id);
693 
694   // Returns the number of currently running flushes.
695   // REQUIREMENT: mutex_ must be held when calling this function.
num_running_flushes()696   int num_running_flushes() {
697     mutex_.AssertHeld();
698     return num_running_flushes_;
699   }
700 
701   // Returns the number of currently running compactions.
702   // REQUIREMENT: mutex_ must be held when calling this function.
num_running_compactions()703   int num_running_compactions() {
704     mutex_.AssertHeld();
705     return num_running_compactions_;
706   }
707 
write_controller()708   const WriteController& write_controller() { return write_controller_; }
709 
710   InternalIterator* NewInternalIterator(
711       const ReadOptions&, ColumnFamilyData* cfd, SuperVersion* super_version,
712       Arena* arena, RangeDelAggregator* range_del_agg, SequenceNumber sequence);
713 
714   // hollow transactions shell used for recovery.
715   // these will then be passed to TransactionDB so that
716   // locks can be reacquired before writing can resume.
717   struct RecoveredTransaction {
718     std::string name_;
719     bool unprepared_;
720 
721     struct BatchInfo {
722       uint64_t log_number_;
723       // TODO(lth): For unprepared, the memory usage here can be big for
724       // unprepared transactions. This is only useful for rollbacks, and we
725       // can in theory just keep keyset for that.
726       WriteBatch* batch_;
727       // Number of sub-batches. A new sub-batch is created if txn attempts to
728       // insert a duplicate key,seq to memtable. This is currently used in
729       // WritePreparedTxn/WriteUnpreparedTxn.
730       size_t batch_cnt_;
731     };
732 
733     // This maps the seq of the first key in the batch to BatchInfo, which
734     // contains WriteBatch and other information relevant to the batch.
735     //
736     // For WriteUnprepared, batches_ can have size greater than 1, but for
737     // other write policies, it must be of size 1.
738     std::map<SequenceNumber, BatchInfo> batches_;
739 
RecoveredTransactionRecoveredTransaction740     explicit RecoveredTransaction(const uint64_t log, const std::string& name,
741                                   WriteBatch* batch, SequenceNumber seq,
742                                   size_t batch_cnt, bool unprepared)
743         : name_(name), unprepared_(unprepared) {
744       batches_[seq] = {log, batch, batch_cnt};
745     }
746 
~RecoveredTransactionRecoveredTransaction747     ~RecoveredTransaction() {
748       for (auto& it : batches_) {
749         delete it.second.batch_;
750       }
751     }
752 
AddBatchRecoveredTransaction753     void AddBatch(SequenceNumber seq, uint64_t log_number, WriteBatch* batch,
754                   size_t batch_cnt, bool unprepared) {
755       assert(batches_.count(seq) == 0);
756       batches_[seq] = {log_number, batch, batch_cnt};
757       // Prior state must be unprepared, since the prepare batch must be the
758       // last batch.
759       assert(unprepared_);
760       unprepared_ = unprepared;
761     }
762   };
763 
allow_2pc()764   bool allow_2pc() const { return immutable_db_options_.allow_2pc; }
765 
766   std::unordered_map<std::string, RecoveredTransaction*>
recovered_transactions()767   recovered_transactions() {
768     return recovered_transactions_;
769   }
770 
GetRecoveredTransaction(const std::string & name)771   RecoveredTransaction* GetRecoveredTransaction(const std::string& name) {
772     auto it = recovered_transactions_.find(name);
773     if (it == recovered_transactions_.end()) {
774       return nullptr;
775     } else {
776       return it->second;
777     }
778   }
779 
InsertRecoveredTransaction(const uint64_t log,const std::string & name,WriteBatch * batch,SequenceNumber seq,size_t batch_cnt,bool unprepared_batch)780   void InsertRecoveredTransaction(const uint64_t log, const std::string& name,
781                                   WriteBatch* batch, SequenceNumber seq,
782                                   size_t batch_cnt, bool unprepared_batch) {
783     // For WriteUnpreparedTxn, InsertRecoveredTransaction is called multiple
784     // times for every unprepared batch encountered during recovery.
785     //
786     // If the transaction is prepared, then the last call to
787     // InsertRecoveredTransaction will have unprepared_batch = false.
788     auto rtxn = recovered_transactions_.find(name);
789     if (rtxn == recovered_transactions_.end()) {
790       recovered_transactions_[name] = new RecoveredTransaction(
791           log, name, batch, seq, batch_cnt, unprepared_batch);
792     } else {
793       rtxn->second->AddBatch(seq, log, batch, batch_cnt, unprepared_batch);
794     }
795     logs_with_prep_tracker_.MarkLogAsContainingPrepSection(log);
796   }
797 
DeleteRecoveredTransaction(const std::string & name)798   void DeleteRecoveredTransaction(const std::string& name) {
799     auto it = recovered_transactions_.find(name);
800     assert(it != recovered_transactions_.end());
801     auto* trx = it->second;
802     recovered_transactions_.erase(it);
803     for (const auto& info : trx->batches_) {
804       logs_with_prep_tracker_.MarkLogAsHavingPrepSectionFlushed(
805           info.second.log_number_);
806     }
807     delete trx;
808   }
809 
DeleteAllRecoveredTransactions()810   void DeleteAllRecoveredTransactions() {
811     for (auto it = recovered_transactions_.begin();
812          it != recovered_transactions_.end(); ++it) {
813       delete it->second;
814     }
815     recovered_transactions_.clear();
816   }
817 
AddToLogsToFreeQueue(log::Writer * log_writer)818   void AddToLogsToFreeQueue(log::Writer* log_writer) {
819     logs_to_free_queue_.push_back(log_writer);
820   }
821 
AddSuperVersionsToFreeQueue(SuperVersion * sv)822   void AddSuperVersionsToFreeQueue(SuperVersion* sv) {
823     superversions_to_free_queue_.push_back(sv);
824   }
825 
826   void SetSnapshotChecker(SnapshotChecker* snapshot_checker);
827 
828   // Fill JobContext with snapshot information needed by flush and compaction.
829   void GetSnapshotContext(JobContext* job_context,
830                           std::vector<SequenceNumber>* snapshot_seqs,
831                           SequenceNumber* earliest_write_conflict_snapshot,
832                           SnapshotChecker** snapshot_checker);
833 
834   // Not thread-safe.
835   void SetRecoverableStatePreReleaseCallback(PreReleaseCallback* callback);
836 
mutex()837   InstrumentedMutex* mutex() const { return &mutex_; }
838 
839   // Initialize a brand new DB. The DB directory is expected to be empty before
840   // calling it.
841   Status NewDB();
842 
843   // This is to be used only by internal rocksdb classes.
844   static Status Open(const DBOptions& db_options, const std::string& name,
845                      const std::vector<ColumnFamilyDescriptor>& column_families,
846                      std::vector<ColumnFamilyHandle*>* handles, DB** dbptr,
847                      const bool seq_per_batch, const bool batch_per_txn);
848 
849   static IOStatus CreateAndNewDirectory(
850       FileSystem* fs, const std::string& dirname,
851       std::unique_ptr<FSDirectory>* directory);
852 
853   // find stats map from stats_history_ with smallest timestamp in
854   // the range of [start_time, end_time)
855   bool FindStatsByTime(uint64_t start_time, uint64_t end_time,
856                        uint64_t* new_time,
857                        std::map<std::string, uint64_t>* stats_map);
858 
859   // Print information of all tombstones of all iterators to the std::string
860   // This is only used by ldb. The output might be capped. Tombstones
861   // printed out are not guaranteed to be in any order.
862   Status TablesRangeTombstoneSummary(ColumnFamilyHandle* column_family,
863                                      int max_entries_to_print,
864                                      std::string* out_str);
865 
866 #ifndef NDEBUG
867   // Compact any files in the named level that overlap [*begin, *end]
868   Status TEST_CompactRange(int level, const Slice* begin, const Slice* end,
869                            ColumnFamilyHandle* column_family = nullptr,
870                            bool disallow_trivial_move = false);
871 
872   void TEST_SwitchWAL();
873 
TEST_UnableToReleaseOldestLog()874   bool TEST_UnableToReleaseOldestLog() { return unable_to_release_oldest_log_; }
875 
TEST_IsLogGettingFlushed()876   bool TEST_IsLogGettingFlushed() {
877     return alive_log_files_.begin()->getting_flushed;
878   }
879 
880   Status TEST_SwitchMemtable(ColumnFamilyData* cfd = nullptr);
881 
882   // Force current memtable contents to be flushed.
883   Status TEST_FlushMemTable(bool wait = true, bool allow_write_stall = false,
884                             ColumnFamilyHandle* cfh = nullptr);
885 
886   Status TEST_FlushMemTable(ColumnFamilyData* cfd,
887                             const FlushOptions& flush_opts);
888 
889   // Flush (multiple) ColumnFamilyData without using ColumnFamilyHandle. This
890   // is because in certain cases, we can flush column families, wait for the
891   // flush to complete, but delete the column family handle before the wait
892   // finishes. For example in CompactRange.
893   Status TEST_AtomicFlushMemTables(const autovector<ColumnFamilyData*>& cfds,
894                                    const FlushOptions& flush_opts);
895 
896   // Wait for memtable compaction
897   Status TEST_WaitForFlushMemTable(ColumnFamilyHandle* column_family = nullptr);
898 
899   // Wait for any compaction
900   // We add a bool parameter to wait for unscheduledCompactions_ == 0, but this
901   // is only for the special test of CancelledCompactions
902   Status TEST_WaitForCompact(bool waitUnscheduled = false);
903 
904   // Return the maximum overlapping data (in bytes) at next level for any
905   // file at a level >= 1.
906   int64_t TEST_MaxNextLevelOverlappingBytes(
907       ColumnFamilyHandle* column_family = nullptr);
908 
909   // Return the current manifest file no.
910   uint64_t TEST_Current_Manifest_FileNo();
911 
912   // Returns the number that'll be assigned to the next file that's created.
913   uint64_t TEST_Current_Next_FileNo();
914 
915   // get total level0 file size. Only for testing.
916   uint64_t TEST_GetLevel0TotalSize();
917 
918   void TEST_GetFilesMetaData(ColumnFamilyHandle* column_family,
919                              std::vector<std::vector<FileMetaData>>* metadata);
920 
921   void TEST_LockMutex();
922 
923   void TEST_UnlockMutex();
924 
925   // REQUIRES: mutex locked
926   void* TEST_BeginWrite();
927 
928   // REQUIRES: mutex locked
929   // pass the pointer that you got from TEST_BeginWrite()
930   void TEST_EndWrite(void* w);
931 
TEST_MaxTotalInMemoryState()932   uint64_t TEST_MaxTotalInMemoryState() const {
933     return max_total_in_memory_state_;
934   }
935 
936   size_t TEST_LogsToFreeSize();
937 
938   uint64_t TEST_LogfileNumber();
939 
TEST_total_log_size()940   uint64_t TEST_total_log_size() const { return total_log_size_; }
941 
942   // Returns column family name to ImmutableCFOptions map.
943   Status TEST_GetAllImmutableCFOptions(
944       std::unordered_map<std::string, const ImmutableCFOptions*>* iopts_map);
945 
946   // Return the lastest MutableCFOptions of a column family
947   Status TEST_GetLatestMutableCFOptions(ColumnFamilyHandle* column_family,
948                                         MutableCFOptions* mutable_cf_options);
949 
TEST_table_cache()950   Cache* TEST_table_cache() { return table_cache_.get(); }
951 
TEST_write_controler()952   WriteController& TEST_write_controler() { return write_controller_; }
953 
954   uint64_t TEST_FindMinLogContainingOutstandingPrep();
955   uint64_t TEST_FindMinPrepLogReferencedByMemTable();
956   size_t TEST_PreparedSectionCompletedSize();
957   size_t TEST_LogsWithPrepSize();
958 
959   int TEST_BGCompactionsAllowed() const;
960   int TEST_BGFlushesAllowed() const;
961   size_t TEST_GetWalPreallocateBlockSize(uint64_t write_buffer_size) const;
962   void TEST_WaitForDumpStatsRun(std::function<void()> callback) const;
963   void TEST_WaitForPersistStatsRun(std::function<void()> callback) const;
964   bool TEST_IsPersistentStatsEnabled() const;
965   size_t TEST_EstimateInMemoryStatsHistorySize() const;
966 #endif  // NDEBUG
967 
968  protected:
969   const std::string dbname_;
970   std::string db_id_;
971   std::unique_ptr<VersionSet> versions_;
972   // Flag to check whether we allocated and own the info log file
973   bool own_info_log_;
974   const DBOptions initial_db_options_;
975   Env* const env_;
976   std::shared_ptr<FileSystem> fs_;
977   const ImmutableDBOptions immutable_db_options_;
978   MutableDBOptions mutable_db_options_;
979   Statistics* stats_;
980   std::unordered_map<std::string, RecoveredTransaction*>
981       recovered_transactions_;
982   std::unique_ptr<Tracer> tracer_;
983   InstrumentedMutex trace_mutex_;
984   BlockCacheTracer block_cache_tracer_;
985 
986   // State below is protected by mutex_
987   // With two_write_queues enabled, some of the variables that accessed during
988   // WriteToWAL need different synchronization: log_empty_, alive_log_files_,
989   // logs_, logfile_number_. Refer to the definition of each variable below for
990   // more description.
991   mutable InstrumentedMutex mutex_;
992 
993   ColumnFamilyHandleImpl* default_cf_handle_;
994   InternalStats* default_cf_internal_stats_;
995 
996   // only used for dynamically adjusting max_total_wal_size. it is a sum of
997   // [write_buffer_size * max_write_buffer_number] over all column families
998   uint64_t max_total_in_memory_state_;
999   // If true, we have only one (default) column family. We use this to optimize
1000   // some code-paths
1001   bool single_column_family_mode_;
1002 
1003   // The options to access storage files
1004   const FileOptions file_options_;
1005 
1006   // Additonal options for compaction and flush
1007   FileOptions file_options_for_compaction_;
1008 
1009   std::unique_ptr<ColumnFamilyMemTablesImpl> column_family_memtables_;
1010 
1011   // Increase the sequence number after writing each batch, whether memtable is
1012   // disabled for that or not. Otherwise the sequence number is increased after
1013   // writing each key into memtable. This implies that when disable_memtable is
1014   // set, the seq is not increased at all.
1015   //
1016   // Default: false
1017   const bool seq_per_batch_;
1018   // This determines during recovery whether we expect one writebatch per
1019   // recovered transaction, or potentially multiple writebatches per
1020   // transaction. For WriteUnprepared, this is set to false, since multiple
1021   // batches can exist per transaction.
1022   //
1023   // Default: true
1024   const bool batch_per_txn_;
1025 
1026   // Except in DB::Open(), WriteOptionsFile can only be called when:
1027   // Persist options to options file.
1028   // If need_mutex_lock = false, the method will lock DB mutex.
1029   // If need_enter_write_thread = false, the method will enter write thread.
1030   Status WriteOptionsFile(bool need_mutex_lock, bool need_enter_write_thread);
1031 
1032   // The following two functions can only be called when:
1033   // 1. WriteThread::Writer::EnterUnbatched() is used.
1034   // 2. db_mutex is NOT held
1035   Status RenameTempFileToOptionsFile(const std::string& file_name);
1036   Status DeleteObsoleteOptionsFiles();
1037 
1038   void NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta,
1039                           const MutableCFOptions& mutable_cf_options,
1040                           int job_id);
1041 
1042   void NotifyOnFlushCompleted(
1043       ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
1044       std::list<std::unique_ptr<FlushJobInfo>>* flush_jobs_info);
1045 
1046   void NotifyOnCompactionBegin(ColumnFamilyData* cfd, Compaction* c,
1047                                const Status& st,
1048                                const CompactionJobStats& job_stats, int job_id);
1049 
1050   void NotifyOnCompactionCompleted(ColumnFamilyData* cfd, Compaction* c,
1051                                    const Status& st,
1052                                    const CompactionJobStats& job_stats,
1053                                    int job_id);
1054   void NotifyOnMemTableSealed(ColumnFamilyData* cfd,
1055                               const MemTableInfo& mem_table_info);
1056 
1057 #ifndef ROCKSDB_LITE
1058   void NotifyOnExternalFileIngested(
1059       ColumnFamilyData* cfd, const ExternalSstFileIngestionJob& ingestion_job);
1060 #endif  // !ROCKSDB_LITE
1061 
1062   void NewThreadStatusCfInfo(ColumnFamilyData* cfd) const;
1063 
1064   void EraseThreadStatusCfInfo(ColumnFamilyData* cfd) const;
1065 
1066   void EraseThreadStatusDbInfo() const;
1067 
1068   // If disable_memtable is set the application logic must guarantee that the
1069   // batch will still be skipped from memtable during the recovery. An excption
1070   // to this is seq_per_batch_ mode, in which since each batch already takes one
1071   // seq, it is ok for the batch to write to memtable during recovery as long as
1072   // it only takes one sequence number: i.e., no duplicate keys.
1073   // In WriteCommitted it is guarnateed since disable_memtable is used for
1074   // prepare batch which will be written to memtable later during the commit,
1075   // and in WritePrepared it is guaranteed since it will be used only for WAL
1076   // markers which will never be written to memtable. If the commit marker is
1077   // accompanied with CommitTimeWriteBatch that is not written to memtable as
1078   // long as it has no duplicate keys, it does not violate the one-seq-per-batch
1079   // policy.
1080   // batch_cnt is expected to be non-zero in seq_per_batch mode and
1081   // indicates the number of sub-patches. A sub-patch is a subset of the write
1082   // batch that does not have duplicate keys.
1083   Status WriteImpl(const WriteOptions& options, WriteBatch* updates,
1084                    WriteCallback* callback = nullptr,
1085                    uint64_t* log_used = nullptr, uint64_t log_ref = 0,
1086                    bool disable_memtable = false, uint64_t* seq_used = nullptr,
1087                    size_t batch_cnt = 0,
1088                    PreReleaseCallback* pre_release_callback = nullptr);
1089 
1090   Status PipelinedWriteImpl(const WriteOptions& options, WriteBatch* updates,
1091                             WriteCallback* callback = nullptr,
1092                             uint64_t* log_used = nullptr, uint64_t log_ref = 0,
1093                             bool disable_memtable = false,
1094                             uint64_t* seq_used = nullptr);
1095 
1096   // Write only to memtables without joining any write queue
1097   Status UnorderedWriteMemtable(const WriteOptions& write_options,
1098                                 WriteBatch* my_batch, WriteCallback* callback,
1099                                 uint64_t log_ref, SequenceNumber seq,
1100                                 const size_t sub_batch_cnt);
1101 
1102   // Whether the batch requires to be assigned with an order
1103   enum AssignOrder : bool { kDontAssignOrder, kDoAssignOrder };
1104   // Whether it requires publishing last sequence or not
1105   enum PublishLastSeq : bool { kDontPublishLastSeq, kDoPublishLastSeq };
1106 
1107   // Join the write_thread to write the batch only to the WAL. It is the
1108   // responsibility of the caller to also write the write batch to the memtable
1109   // if it required.
1110   //
1111   // sub_batch_cnt is expected to be non-zero when assign_order = kDoAssignOrder
1112   // indicating the number of sub-batches in my_batch. A sub-patch is a subset
1113   // of the write batch that does not have duplicate keys. When seq_per_batch is
1114   // not set, each key is a separate sub_batch. Otherwise each duplicate key
1115   // marks start of a new sub-batch.
1116   Status WriteImplWALOnly(
1117       WriteThread* write_thread, const WriteOptions& options,
1118       WriteBatch* updates, WriteCallback* callback, uint64_t* log_used,
1119       const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt,
1120       PreReleaseCallback* pre_release_callback, const AssignOrder assign_order,
1121       const PublishLastSeq publish_last_seq, const bool disable_memtable);
1122 
1123   // write cached_recoverable_state_ to memtable if it is not empty
1124   // The writer must be the leader in write_thread_ and holding mutex_
1125   Status WriteRecoverableState();
1126 
1127   // Actual implementation of Close()
1128   Status CloseImpl();
1129 
1130   // Recover the descriptor from persistent storage.  May do a significant
1131   // amount of work to recover recently logged updates.  Any changes to
1132   // be made to the descriptor are added to *edit.
1133   // recovered_seq is set to less than kMaxSequenceNumber if the log's tail is
1134   // skipped.
1135   virtual Status Recover(
1136       const std::vector<ColumnFamilyDescriptor>& column_families,
1137       bool read_only = false, bool error_if_log_file_exist = false,
1138       bool error_if_data_exists_in_logs = false,
1139       uint64_t* recovered_seq = nullptr);
1140 
OwnTablesAndLogs()1141   virtual bool OwnTablesAndLogs() const { return true; }
1142 
1143   // REQUIRES: db mutex held when calling this function, but the db mutex can
1144   // be released and re-acquired. Db mutex will be held when the function
1145   // returns.
1146   // Currently, this function should be called only in best-efforts recovery
1147   // mode.
1148   // After best-efforts recovery, there may be SST files in db/cf paths that are
1149   // not referenced in the MANIFEST. We delete these SST files. In the
1150   // meantime, we find out the largest file number present in the paths, and
1151   // bump up the version set's next_file_number_ to be 1 + largest_file_number.
1152   Status CleanupFilesAfterRecovery();
1153 
1154  private:
1155   friend class DB;
1156   friend class ErrorHandler;
1157   friend class InternalStats;
1158   friend class PessimisticTransaction;
1159   friend class TransactionBaseImpl;
1160   friend class WriteCommittedTxn;
1161   friend class WritePreparedTxn;
1162   friend class WritePreparedTxnDB;
1163   friend class WriteBatchWithIndex;
1164   friend class WriteUnpreparedTxnDB;
1165   friend class WriteUnpreparedTxn;
1166 
1167 #ifndef ROCKSDB_LITE
1168   friend class ForwardIterator;
1169 #endif
1170   friend struct SuperVersion;
1171   friend class CompactedDBImpl;
1172   friend class DBTest_ConcurrentFlushWAL_Test;
1173   friend class DBTest_MixedSlowdownOptionsStop_Test;
1174   friend class DBCompactionTest_CompactBottomLevelFilesWithDeletions_Test;
1175   friend class DBCompactionTest_CompactionDuringShutdown_Test;
1176   friend class StatsHistoryTest_PersistentStatsCreateColumnFamilies_Test;
1177 #ifndef NDEBUG
1178   friend class DBTest2_ReadCallbackTest_Test;
1179   friend class WriteCallbackTest_WriteWithCallbackTest_Test;
1180   friend class XFTransactionWriteHandler;
1181   friend class DBBlobIndexTest;
1182   friend class WriteUnpreparedTransactionTest_RecoveryTest_Test;
1183 #endif
1184 
1185   struct CompactionState;
1186   struct PrepickedCompaction;
1187   struct PurgeFileInfo;
1188 
1189   struct WriteContext {
1190     SuperVersionContext superversion_context;
1191     autovector<MemTable*> memtables_to_free_;
1192 
1193     explicit WriteContext(bool create_superversion = false)
superversion_contextWriteContext1194         : superversion_context(create_superversion) {}
1195 
~WriteContextWriteContext1196     ~WriteContext() {
1197       superversion_context.Clean();
1198       for (auto& m : memtables_to_free_) {
1199         delete m;
1200       }
1201     }
1202   };
1203 
1204   struct LogFileNumberSize {
LogFileNumberSizeLogFileNumberSize1205     explicit LogFileNumberSize(uint64_t _number) : number(_number) {}
AddSizeLogFileNumberSize1206     void AddSize(uint64_t new_size) { size += new_size; }
1207     uint64_t number;
1208     uint64_t size = 0;
1209     bool getting_flushed = false;
1210   };
1211 
1212   struct LogWriterNumber {
1213     // pass ownership of _writer
LogWriterNumberLogWriterNumber1214     LogWriterNumber(uint64_t _number, log::Writer* _writer)
1215         : number(_number), writer(_writer) {}
1216 
ReleaseWriterLogWriterNumber1217     log::Writer* ReleaseWriter() {
1218       auto* w = writer;
1219       writer = nullptr;
1220       return w;
1221     }
ClearWriterLogWriterNumber1222     Status ClearWriter() {
1223       Status s = writer->WriteBuffer();
1224       delete writer;
1225       writer = nullptr;
1226       return s;
1227     }
1228 
1229     uint64_t number;
1230     // Visual Studio doesn't support deque's member to be noncopyable because
1231     // of a std::unique_ptr as a member.
1232     log::Writer* writer;  // own
1233     // true for some prefix of logs_
1234     bool getting_synced = false;
1235   };
1236 
1237   // PurgeFileInfo is a structure to hold information of files to be deleted in
1238   // purge_files_
1239   struct PurgeFileInfo {
1240     std::string fname;
1241     std::string dir_to_sync;
1242     FileType type;
1243     uint64_t number;
1244     int job_id;
PurgeFileInfoPurgeFileInfo1245     PurgeFileInfo(std::string fn, std::string d, FileType t, uint64_t num,
1246                   int jid)
1247         : fname(fn), dir_to_sync(d), type(t), number(num), job_id(jid) {}
1248   };
1249 
1250   // Argument required by background flush thread.
1251   struct BGFlushArg {
BGFlushArgBGFlushArg1252     BGFlushArg()
1253         : cfd_(nullptr), max_memtable_id_(0), superversion_context_(nullptr) {}
BGFlushArgBGFlushArg1254     BGFlushArg(ColumnFamilyData* cfd, uint64_t max_memtable_id,
1255                SuperVersionContext* superversion_context)
1256         : cfd_(cfd),
1257           max_memtable_id_(max_memtable_id),
1258           superversion_context_(superversion_context) {}
1259 
1260     // Column family to flush.
1261     ColumnFamilyData* cfd_;
1262     // Maximum ID of memtable to flush. In this column family, memtables with
1263     // IDs smaller than this value must be flushed before this flush completes.
1264     uint64_t max_memtable_id_;
1265     // Pointer to a SuperVersionContext object. After flush completes, RocksDB
1266     // installs a new superversion for the column family. This operation
1267     // requires a SuperVersionContext object (currently embedded in JobContext).
1268     SuperVersionContext* superversion_context_;
1269   };
1270 
1271   // Argument passed to flush thread.
1272   struct FlushThreadArg {
1273     DBImpl* db_;
1274 
1275     Env::Priority thread_pri_;
1276   };
1277 
1278   // Information for a manual compaction
1279   struct ManualCompactionState {
1280     ColumnFamilyData* cfd;
1281     int input_level;
1282     int output_level;
1283     uint32_t output_path_id;
1284     Status status;
1285     bool done;
1286     bool in_progress;            // compaction request being processed?
1287     bool incomplete;             // only part of requested range compacted
1288     bool exclusive;              // current behavior of only one manual
1289     bool disallow_trivial_move;  // Force actual compaction to run
1290     const InternalKey* begin;    // nullptr means beginning of key range
1291     const InternalKey* end;      // nullptr means end of key range
1292     InternalKey* manual_end;     // how far we are compacting
1293     InternalKey tmp_storage;     // Used to keep track of compaction progress
1294     InternalKey tmp_storage1;    // Used to keep track of compaction progress
1295   };
1296   struct PrepickedCompaction {
1297     // background compaction takes ownership of `compaction`.
1298     Compaction* compaction;
1299     // caller retains ownership of `manual_compaction_state` as it is reused
1300     // across background compactions.
1301     ManualCompactionState* manual_compaction_state;  // nullptr if non-manual
1302     // task limiter token is requested during compaction picking.
1303     std::unique_ptr<TaskLimiterToken> task_token;
1304   };
1305 
1306   struct CompactionArg {
1307     // caller retains ownership of `db`.
1308     DBImpl* db;
1309     // background compaction takes ownership of `prepicked_compaction`.
1310     PrepickedCompaction* prepicked_compaction;
1311   };
1312 
1313   // Initialize the built-in column family for persistent stats. Depending on
1314   // whether on-disk persistent stats have been enabled before, it may either
1315   // create a new column family and column family handle or just a column family
1316   // handle.
1317   // Required: DB mutex held
1318   Status InitPersistStatsColumnFamily();
1319 
1320   // Persistent Stats column family has two format version key which are used
1321   // for compatibility check. Write format version if it's created for the
1322   // first time, read format version and check compatibility if recovering
1323   // from disk. This function requires DB mutex held at entrance but may
1324   // release and re-acquire DB mutex in the process.
1325   // Required: DB mutex held
1326   Status PersistentStatsProcessFormatVersion();
1327 
1328   Status ResumeImpl();
1329 
1330   void MaybeIgnoreError(Status* s) const;
1331 
1332   const Status CreateArchivalDirectory();
1333 
1334   Status CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options,
1335                                 const std::string& cf_name,
1336                                 ColumnFamilyHandle** handle);
1337 
1338   Status DropColumnFamilyImpl(ColumnFamilyHandle* column_family);
1339 
1340   // Delete any unneeded files and stale in-memory entries.
1341   void DeleteObsoleteFiles();
1342   // Delete obsolete files and log status and information of file deletion
1343   void DeleteObsoleteFileImpl(int job_id, const std::string& fname,
1344                               const std::string& path_to_sync, FileType type,
1345                               uint64_t number);
1346 
1347   // Background process needs to call
1348   //     auto x = CaptureCurrentFileNumberInPendingOutputs()
1349   //     auto file_num = versions_->NewFileNumber();
1350   //     <do something>
1351   //     ReleaseFileNumberFromPendingOutputs(x)
1352   // This will protect any file with number `file_num` or greater from being
1353   // deleted while <do something> is running.
1354   // -----------
1355   // This function will capture current file number and append it to
1356   // pending_outputs_. This will prevent any background process to delete any
1357   // file created after this point.
1358   std::list<uint64_t>::iterator CaptureCurrentFileNumberInPendingOutputs();
1359   // This function should be called with the result of
1360   // CaptureCurrentFileNumberInPendingOutputs(). It then marks that any file
1361   // created between the calls CaptureCurrentFileNumberInPendingOutputs() and
1362   // ReleaseFileNumberFromPendingOutputs() can now be deleted (if it's not live
1363   // and blocked by any other pending_outputs_ calls)
1364   void ReleaseFileNumberFromPendingOutputs(
1365       std::unique_ptr<std::list<uint64_t>::iterator>& v);
1366 
1367   IOStatus SyncClosedLogs(JobContext* job_context);
1368 
1369   // Flush the in-memory write buffer to storage.  Switches to a new
1370   // log-file/memtable and writes a new descriptor iff successful. Then
1371   // installs a new super version for the column family.
1372   Status FlushMemTableToOutputFile(
1373       ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
1374       bool* madeProgress, JobContext* job_context,
1375       SuperVersionContext* superversion_context,
1376       std::vector<SequenceNumber>& snapshot_seqs,
1377       SequenceNumber earliest_write_conflict_snapshot,
1378       SnapshotChecker* snapshot_checker, LogBuffer* log_buffer,
1379       Env::Priority thread_pri);
1380 
1381   // Flush the memtables of (multiple) column families to multiple files on
1382   // persistent storage.
1383   Status FlushMemTablesToOutputFiles(
1384       const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
1385       JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri);
1386 
1387   Status AtomicFlushMemTablesToOutputFiles(
1388       const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
1389       JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri);
1390 
1391   // REQUIRES: log_numbers are sorted in ascending order
1392   // corrupted_log_found is set to true if we recover from a corrupted log file.
1393   Status RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
1394                          SequenceNumber* next_sequence, bool read_only,
1395                          bool* corrupted_log_found);
1396 
1397   // The following two methods are used to flush a memtable to
1398   // storage. The first one is used at database RecoveryTime (when the
1399   // database is opened) and is heavyweight because it holds the mutex
1400   // for the entire period. The second method WriteLevel0Table supports
1401   // concurrent flush memtables to storage.
1402   Status WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
1403                                      MemTable* mem, VersionEdit* edit);
1404 
1405   // Restore alive_log_files_ and total_log_size_ after recovery.
1406   // It needs to run only when there's no flush during recovery
1407   // (e.g. avoid_flush_during_recovery=true). May also trigger flush
1408   // in case total_log_size > max_total_wal_size.
1409   Status RestoreAliveLogFiles(const std::vector<uint64_t>& log_numbers);
1410 
1411   // num_bytes: for slowdown case, delay time is calculated based on
1412   //            `num_bytes` going through.
1413   Status DelayWrite(uint64_t num_bytes, const WriteOptions& write_options);
1414 
1415   Status ThrottleLowPriWritesIfNeeded(const WriteOptions& write_options,
1416                                       WriteBatch* my_batch);
1417 
1418   // REQUIRES: mutex locked and in write thread.
1419   Status ScheduleFlushes(WriteContext* context);
1420 
1421   void MaybeFlushStatsCF(autovector<ColumnFamilyData*>* cfds);
1422 
1423   Status TrimMemtableHistory(WriteContext* context);
1424 
1425   Status SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context);
1426 
1427   void SelectColumnFamiliesForAtomicFlush(autovector<ColumnFamilyData*>* cfds);
1428 
1429   // Force current memtable contents to be flushed.
1430   Status FlushMemTable(ColumnFamilyData* cfd, const FlushOptions& options,
1431                        FlushReason flush_reason, bool writes_stopped = false);
1432 
1433   Status AtomicFlushMemTables(
1434       const autovector<ColumnFamilyData*>& column_family_datas,
1435       const FlushOptions& options, FlushReason flush_reason,
1436       bool writes_stopped = false);
1437 
1438   // Wait until flushing this column family won't stall writes
1439   Status WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd,
1440                                            bool* flush_needed);
1441 
1442   // Wait for memtable flushed.
1443   // If flush_memtable_id is non-null, wait until the memtable with the ID
1444   // gets flush. Otherwise, wait until the column family don't have any
1445   // memtable pending flush.
1446   // resuming_from_bg_err indicates whether the caller is attempting to resume
1447   // from background error.
1448   Status WaitForFlushMemTable(ColumnFamilyData* cfd,
1449                               const uint64_t* flush_memtable_id = nullptr,
1450                               bool resuming_from_bg_err = false) {
1451     return WaitForFlushMemTables({cfd}, {flush_memtable_id},
1452                                  resuming_from_bg_err);
1453   }
1454   // Wait for memtables to be flushed for multiple column families.
1455   Status WaitForFlushMemTables(
1456       const autovector<ColumnFamilyData*>& cfds,
1457       const autovector<const uint64_t*>& flush_memtable_ids,
1458       bool resuming_from_bg_err);
1459 
WaitForPendingWrites()1460   inline void WaitForPendingWrites() {
1461     mutex_.AssertHeld();
1462     TEST_SYNC_POINT("DBImpl::WaitForPendingWrites:BeforeBlock");
1463     // In case of pipelined write is enabled, wait for all pending memtable
1464     // writers.
1465     if (immutable_db_options_.enable_pipelined_write) {
1466       // Memtable writers may call DB::Get in case max_successive_merges > 0,
1467       // which may lock mutex. Unlocking mutex here to avoid deadlock.
1468       mutex_.Unlock();
1469       write_thread_.WaitForMemTableWriters();
1470       mutex_.Lock();
1471     }
1472 
1473     if (!immutable_db_options_.unordered_write) {
1474       // Then the writes are finished before the next write group starts
1475       return;
1476     }
1477 
1478     // Wait for the ones who already wrote to the WAL to finish their
1479     // memtable write.
1480     if (pending_memtable_writes_.load() != 0) {
1481       std::unique_lock<std::mutex> guard(switch_mutex_);
1482       switch_cv_.wait(guard,
1483                       [&] { return pending_memtable_writes_.load() == 0; });
1484     }
1485   }
1486 
1487   // REQUIRES: mutex locked and in write thread.
1488   void AssignAtomicFlushSeq(const autovector<ColumnFamilyData*>& cfds);
1489 
1490   // REQUIRES: mutex locked and in write thread.
1491   Status SwitchWAL(WriteContext* write_context);
1492 
1493   // REQUIRES: mutex locked and in write thread.
1494   Status HandleWriteBufferFull(WriteContext* write_context);
1495 
1496   // REQUIRES: mutex locked
1497   Status PreprocessWrite(const WriteOptions& write_options, bool* need_log_sync,
1498                          WriteContext* write_context);
1499 
1500   WriteBatch* MergeBatch(const WriteThread::WriteGroup& write_group,
1501                          WriteBatch* tmp_batch, size_t* write_with_wal,
1502                          WriteBatch** to_be_cached_state);
1503 
1504   IOStatus WriteToWAL(const WriteBatch& merged_batch, log::Writer* log_writer,
1505                       uint64_t* log_used, uint64_t* log_size);
1506 
1507   IOStatus WriteToWAL(const WriteThread::WriteGroup& write_group,
1508                       log::Writer* log_writer, uint64_t* log_used,
1509                       bool need_log_sync, bool need_log_dir_sync,
1510                       SequenceNumber sequence);
1511 
1512   IOStatus ConcurrentWriteToWAL(const WriteThread::WriteGroup& write_group,
1513                                 uint64_t* log_used,
1514                                 SequenceNumber* last_sequence, size_t seq_inc);
1515 
1516   // Used by WriteImpl to update bg_error_ if paranoid check is enabled.
1517   void WriteStatusCheck(const Status& status);
1518 
1519   // Used by WriteImpl to update bg_error_ when IO error happens, e.g., write
1520   // WAL, sync WAL fails, if paranoid check is enabled.
1521   void IOStatusCheck(const IOStatus& status);
1522 
1523   // Used by WriteImpl to update bg_error_ in case of memtable insert error.
1524   void MemTableInsertStatusCheck(const Status& memtable_insert_status);
1525 
1526 #ifndef ROCKSDB_LITE
1527 
1528   Status CompactFilesImpl(const CompactionOptions& compact_options,
1529                           ColumnFamilyData* cfd, Version* version,
1530                           const std::vector<std::string>& input_file_names,
1531                           std::vector<std::string>* const output_file_names,
1532                           const int output_level, int output_path_id,
1533                           JobContext* job_context, LogBuffer* log_buffer,
1534                           CompactionJobInfo* compaction_job_info);
1535 
1536   // Wait for current IngestExternalFile() calls to finish.
1537   // REQUIRES: mutex_ held
1538   void WaitForIngestFile();
1539 
1540 #else
1541   // IngestExternalFile is not supported in ROCKSDB_LITE so this function
1542   // will be no-op
WaitForIngestFile()1543   void WaitForIngestFile() {}
1544 #endif  // ROCKSDB_LITE
1545 
1546   ColumnFamilyData* GetColumnFamilyDataByName(const std::string& cf_name);
1547 
1548   void MaybeScheduleFlushOrCompaction();
1549 
1550   // A flush request specifies the column families to flush as well as the
1551   // largest memtable id to persist for each column family. Once all the
1552   // memtables whose IDs are smaller than or equal to this per-column-family
1553   // specified value, this flush request is considered to have completed its
1554   // work of flushing this column family. After completing the work for all
1555   // column families in this request, this flush is considered complete.
1556   typedef std::vector<std::pair<ColumnFamilyData*, uint64_t>> FlushRequest;
1557 
1558   void GenerateFlushRequest(const autovector<ColumnFamilyData*>& cfds,
1559                             FlushRequest* req);
1560 
1561   void SchedulePendingFlush(const FlushRequest& req, FlushReason flush_reason);
1562 
1563   void SchedulePendingCompaction(ColumnFamilyData* cfd);
1564   void SchedulePendingPurge(std::string fname, std::string dir_to_sync,
1565                             FileType type, uint64_t number, int job_id);
1566   static void BGWorkCompaction(void* arg);
1567   // Runs a pre-chosen universal compaction involving bottom level in a
1568   // separate, bottom-pri thread pool.
1569   static void BGWorkBottomCompaction(void* arg);
1570   static void BGWorkFlush(void* arg);
1571   static void BGWorkPurge(void* arg);
1572   static void UnscheduleCompactionCallback(void* arg);
1573   static void UnscheduleFlushCallback(void* arg);
1574   void BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction,
1575                                 Env::Priority thread_pri);
1576   void BackgroundCallFlush(Env::Priority thread_pri);
1577   void BackgroundCallPurge();
1578   Status BackgroundCompaction(bool* madeProgress, JobContext* job_context,
1579                               LogBuffer* log_buffer,
1580                               PrepickedCompaction* prepicked_compaction,
1581                               Env::Priority thread_pri);
1582   Status BackgroundFlush(bool* madeProgress, JobContext* job_context,
1583                          LogBuffer* log_buffer, FlushReason* reason,
1584                          Env::Priority thread_pri);
1585 
1586   bool EnoughRoomForCompaction(ColumnFamilyData* cfd,
1587                                const std::vector<CompactionInputFiles>& inputs,
1588                                bool* sfm_bookkeeping, LogBuffer* log_buffer);
1589 
1590   // Request compaction tasks token from compaction thread limiter.
1591   // It always succeeds if force = true or limiter is disable.
1592   bool RequestCompactionToken(ColumnFamilyData* cfd, bool force,
1593                               std::unique_ptr<TaskLimiterToken>* token,
1594                               LogBuffer* log_buffer);
1595 
1596   // Schedule background tasks
1597   void StartTimedTasks();
1598 
1599   void PrintStatistics();
1600 
1601   size_t EstimateInMemoryStatsHistorySize() const;
1602 
1603   // persist stats to column family "_persistent_stats"
1604   void PersistStats();
1605 
1606   // dump rocksdb.stats to LOG
1607   void DumpStats();
1608 
1609   // Return the minimum empty level that could hold the total data in the
1610   // input level. Return the input level, if such level could not be found.
1611   int FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
1612                                    const MutableCFOptions& mutable_cf_options,
1613                                    int level);
1614 
1615   // Move the files in the input level to the target level.
1616   // If target_level < 0, automatically calculate the minimum level that could
1617   // hold the data set.
1618   Status ReFitLevel(ColumnFamilyData* cfd, int level, int target_level = -1);
1619 
1620   // helper functions for adding and removing from flush & compaction queues
1621   void AddToCompactionQueue(ColumnFamilyData* cfd);
1622   ColumnFamilyData* PopFirstFromCompactionQueue();
1623   FlushRequest PopFirstFromFlushQueue();
1624 
1625   // Pick the first unthrottled compaction with task token from queue.
1626   ColumnFamilyData* PickCompactionFromQueue(
1627       std::unique_ptr<TaskLimiterToken>* token, LogBuffer* log_buffer);
1628 
1629   // helper function to call after some of the logs_ were synced
1630   void MarkLogsSynced(uint64_t up_to, bool synced_dir, const Status& status);
1631 
1632   SnapshotImpl* GetSnapshotImpl(bool is_write_conflict_boundary,
1633                                 bool lock = true);
1634 
1635   uint64_t GetMaxTotalWalSize() const;
1636 
1637   FSDirectory* GetDataDir(ColumnFamilyData* cfd, size_t path_id) const;
1638 
1639   Status CloseHelper();
1640 
1641   void WaitForBackgroundWork();
1642 
1643   // Background threads call this function, which is just a wrapper around
1644   // the InstallSuperVersion() function. Background threads carry
1645   // sv_context which can have new_superversion already
1646   // allocated.
1647   // All ColumnFamily state changes go through this function. Here we analyze
1648   // the new state and we schedule background work if we detect that the new
1649   // state needs flush or compaction.
1650   void InstallSuperVersionAndScheduleWork(
1651       ColumnFamilyData* cfd, SuperVersionContext* sv_context,
1652       const MutableCFOptions& mutable_cf_options);
1653 
1654   bool GetIntPropertyInternal(ColumnFamilyData* cfd,
1655                               const DBPropertyInfo& property_info,
1656                               bool is_locked, uint64_t* value);
1657   bool GetPropertyHandleOptionsStatistics(std::string* value);
1658 
1659   bool HasPendingManualCompaction();
1660   bool HasExclusiveManualCompaction();
1661   void AddManualCompaction(ManualCompactionState* m);
1662   void RemoveManualCompaction(ManualCompactionState* m);
1663   bool ShouldntRunManualCompaction(ManualCompactionState* m);
1664   bool HaveManualCompaction(ColumnFamilyData* cfd);
1665   bool MCOverlap(ManualCompactionState* m, ManualCompactionState* m1);
1666 #ifndef ROCKSDB_LITE
1667   void BuildCompactionJobInfo(const ColumnFamilyData* cfd, Compaction* c,
1668                               const Status& st,
1669                               const CompactionJobStats& compaction_job_stats,
1670                               const int job_id, const Version* current,
1671                               CompactionJobInfo* compaction_job_info) const;
1672   // Reserve the next 'num' file numbers for to-be-ingested external SST files,
1673   // and return the current file_number in 'next_file_number'.
1674   // Write a version edit to the MANIFEST.
1675   Status ReserveFileNumbersBeforeIngestion(
1676       ColumnFamilyData* cfd, uint64_t num,
1677       std::unique_ptr<std::list<uint64_t>::iterator>& pending_output_elem,
1678       uint64_t* next_file_number);
1679 #endif  //! ROCKSDB_LITE
1680 
1681   bool ShouldPurge(uint64_t file_number) const;
1682   void MarkAsGrabbedForPurge(uint64_t file_number);
1683 
1684   size_t GetWalPreallocateBlockSize(uint64_t write_buffer_size) const;
CalculateWALWriteHint()1685   Env::WriteLifeTimeHint CalculateWALWriteHint() { return Env::WLTH_SHORT; }
1686 
1687   Status CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number,
1688                    size_t preallocate_block_size, log::Writer** new_log);
1689 
1690   // Validate self-consistency of DB options
1691   static Status ValidateOptions(const DBOptions& db_options);
1692   // Validate self-consistency of DB options and its consistency with cf options
1693   static Status ValidateOptions(
1694       const DBOptions& db_options,
1695       const std::vector<ColumnFamilyDescriptor>& column_families);
1696 
1697   // Utility function to do some debug validation and sort the given vector
1698   // of MultiGet keys
1699   void PrepareMultiGetKeys(
1700       const size_t num_keys, bool sorted,
1701       autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE>* key_ptrs);
1702 
1703   // A structure to hold the information required to process MultiGet of keys
1704   // belonging to one column family. For a multi column family MultiGet, there
1705   // will be a container of these objects.
1706   struct MultiGetColumnFamilyData {
1707     ColumnFamilyHandle* cf;
1708     ColumnFamilyData* cfd;
1709 
1710     // For the batched MultiGet which relies on sorted keys, start specifies
1711     // the index of first key belonging to this column family in the sorted
1712     // list.
1713     size_t start;
1714 
1715     // For the batched MultiGet case, num_keys specifies the number of keys
1716     // belonging to this column family in the sorted list
1717     size_t num_keys;
1718 
1719     // SuperVersion for the column family obtained in a manner that ensures a
1720     // consistent view across all column families in the DB
1721     SuperVersion* super_version;
MultiGetColumnFamilyDataMultiGetColumnFamilyData1722     MultiGetColumnFamilyData(ColumnFamilyHandle* column_family,
1723                              SuperVersion* sv)
1724         : cf(column_family),
1725           cfd(static_cast<ColumnFamilyHandleImpl*>(cf)->cfd()),
1726           start(0),
1727           num_keys(0),
1728           super_version(sv) {}
1729 
MultiGetColumnFamilyDataMultiGetColumnFamilyData1730     MultiGetColumnFamilyData(ColumnFamilyHandle* column_family, size_t first,
1731                              size_t count, SuperVersion* sv)
1732         : cf(column_family),
1733           cfd(static_cast<ColumnFamilyHandleImpl*>(cf)->cfd()),
1734           start(first),
1735           num_keys(count),
1736           super_version(sv) {}
1737 
1738     MultiGetColumnFamilyData() = default;
1739   };
1740 
1741   // A common function to obtain a consistent snapshot, which can be implicit
1742   // if the user doesn't specify a snapshot in read_options, across
1743   // multiple column families for MultiGet. It will attempt to get an implicit
1744   // snapshot without acquiring the db_mutes, but will give up after a few
1745   // tries and acquire the mutex if a memtable flush happens. The template
1746   // allows both the batched and non-batched MultiGet to call this with
1747   // either an std::unordered_map or autovector of column families.
1748   //
1749   // If callback is non-null, the callback is refreshed with the snapshot
1750   // sequence number
1751   //
1752   // A return value of true indicates that the SuperVersions were obtained
1753   // from the ColumnFamilyData, whereas false indicates they are thread
1754   // local
1755   template <class T>
1756   bool MultiCFSnapshot(
1757       const ReadOptions& read_options, ReadCallback* callback,
1758       std::function<MultiGetColumnFamilyData*(typename T::iterator&)>&
1759           iter_deref_func,
1760       T* cf_list, SequenceNumber* snapshot);
1761 
1762   // The actual implementation of the batching MultiGet. The caller is expected
1763   // to have acquired the SuperVersion and pass in a snapshot sequence number
1764   // in order to construct the LookupKeys. The start_key and num_keys specify
1765   // the range of keys in the sorted_keys vector for a single column family.
1766   void MultiGetImpl(
1767       const ReadOptions& read_options, size_t start_key, size_t num_keys,
1768       autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE>* sorted_keys,
1769       SuperVersion* sv, SequenceNumber snap_seqnum, ReadCallback* callback,
1770       bool* is_blob_index);
1771 
1772   // table_cache_ provides its own synchronization
1773   std::shared_ptr<Cache> table_cache_;
1774 
1775   // Lock over the persistent DB state.  Non-nullptr iff successfully acquired.
1776   FileLock* db_lock_;
1777 
1778   // In addition to mutex_, log_write_mutex_ protected writes to stats_history_
1779   InstrumentedMutex stats_history_mutex_;
1780   // In addition to mutex_, log_write_mutex_ protected writes to logs_ and
1781   // logfile_number_. With two_write_queues it also protects alive_log_files_,
1782   // and log_empty_. Refer to the definition of each variable below for more
1783   // details.
1784   // Note: to avoid dealock, if needed to acquire both log_write_mutex_ and
1785   // mutex_, the order should be first mutex_ and then log_write_mutex_.
1786   InstrumentedMutex log_write_mutex_;
1787 
1788   std::atomic<bool> shutting_down_;
1789   std::atomic<bool> manual_compaction_paused_;
1790   // This condition variable is signaled on these conditions:
1791   // * whenever bg_compaction_scheduled_ goes down to 0
1792   // * if AnyManualCompaction, whenever a compaction finishes, even if it hasn't
1793   // made any progress
1794   // * whenever a compaction made any progress
1795   // * whenever bg_flush_scheduled_ or bg_purge_scheduled_ value decreases
1796   // (i.e. whenever a flush is done, even if it didn't make any progress)
1797   // * whenever there is an error in background purge, flush or compaction
1798   // * whenever num_running_ingest_file_ goes to 0.
1799   // * whenever pending_purge_obsolete_files_ goes to 0.
1800   // * whenever disable_delete_obsolete_files_ goes to 0.
1801   // * whenever SetOptions successfully updates options.
1802   // * whenever a column family is dropped.
1803   InstrumentedCondVar bg_cv_;
1804   // Writes are protected by locking both mutex_ and log_write_mutex_, and reads
1805   // must be under either mutex_ or log_write_mutex_. Since after ::Open,
1806   // logfile_number_ is currently updated only in write_thread_, it can be read
1807   // from the same write_thread_ without any locks.
1808   uint64_t logfile_number_;
1809   std::deque<uint64_t>
1810       log_recycle_files_;  // a list of log files that we can recycle
1811   bool log_dir_synced_;
1812   // Without two_write_queues, read and writes to log_empty_ are protected by
1813   // mutex_. Since it is currently updated/read only in write_thread_, it can be
1814   // accessed from the same write_thread_ without any locks. With
1815   // two_write_queues writes, where it can be updated in different threads,
1816   // read and writes are protected by log_write_mutex_ instead. This is to avoid
1817   // expesnive mutex_ lock during WAL write, which update log_empty_.
1818   bool log_empty_;
1819 
1820   ColumnFamilyHandleImpl* persist_stats_cf_handle_;
1821 
1822   bool persistent_stats_cfd_exists_ = true;
1823 
1824   // Without two_write_queues, read and writes to alive_log_files_ are
1825   // protected by mutex_. However since back() is never popped, and push_back()
1826   // is done only from write_thread_, the same thread can access the item
1827   // reffered by back() without mutex_. With two_write_queues_, writes
1828   // are protected by locking both mutex_ and log_write_mutex_, and reads must
1829   // be under either mutex_ or log_write_mutex_.
1830   std::deque<LogFileNumberSize> alive_log_files_;
1831   // Log files that aren't fully synced, and the current log file.
1832   // Synchronization:
1833   //  - push_back() is done from write_thread_ with locked mutex_ and
1834   //  log_write_mutex_
1835   //  - pop_front() is done from any thread with locked mutex_ and
1836   //  log_write_mutex_
1837   //  - reads are done with either locked mutex_ or log_write_mutex_
1838   //  - back() and items with getting_synced=true are not popped,
1839   //  - The same thread that sets getting_synced=true will reset it.
1840   //  - it follows that the object referred by back() can be safely read from
1841   //  the write_thread_ without using mutex
1842   //  - it follows that the items with getting_synced=true can be safely read
1843   //  from the same thread that has set getting_synced=true
1844   std::deque<LogWriterNumber> logs_;
1845   // Signaled when getting_synced becomes false for some of the logs_.
1846   InstrumentedCondVar log_sync_cv_;
1847   // This is the app-level state that is written to the WAL but will be used
1848   // only during recovery. Using this feature enables not writing the state to
1849   // memtable on normal writes and hence improving the throughput. Each new
1850   // write of the state will replace the previous state entirely even if the
1851   // keys in the two consecuitive states do not overlap.
1852   // It is protected by log_write_mutex_ when two_write_queues_ is enabled.
1853   // Otherwise only the heaad of write_thread_ can access it.
1854   WriteBatch cached_recoverable_state_;
1855   std::atomic<bool> cached_recoverable_state_empty_ = {true};
1856   std::atomic<uint64_t> total_log_size_;
1857 
1858   // If this is non-empty, we need to delete these log files in background
1859   // threads. Protected by db mutex.
1860   autovector<log::Writer*> logs_to_free_;
1861 
1862   bool is_snapshot_supported_;
1863 
1864   std::map<uint64_t, std::map<std::string, uint64_t>> stats_history_;
1865 
1866   std::map<std::string, uint64_t> stats_slice_;
1867 
1868   bool stats_slice_initialized_ = false;
1869 
1870   Directories directories_;
1871 
1872   WriteBufferManager* write_buffer_manager_;
1873 
1874   WriteThread write_thread_;
1875   WriteBatch tmp_batch_;
1876   // The write thread when the writers have no memtable write. This will be used
1877   // in 2PC to batch the prepares separately from the serial commit.
1878   WriteThread nonmem_write_thread_;
1879 
1880   WriteController write_controller_;
1881 
1882   // Size of the last batch group. In slowdown mode, next write needs to
1883   // sleep if it uses up the quota.
1884   // Note: This is to protect memtable and compaction. If the batch only writes
1885   // to the WAL its size need not to be included in this.
1886   uint64_t last_batch_group_size_;
1887 
1888   FlushScheduler flush_scheduler_;
1889 
1890   TrimHistoryScheduler trim_history_scheduler_;
1891 
1892   SnapshotList snapshots_;
1893 
1894   // For each background job, pending_outputs_ keeps the current file number at
1895   // the time that background job started.
1896   // FindObsoleteFiles()/PurgeObsoleteFiles() never deletes any file that has
1897   // number bigger than any of the file number in pending_outputs_. Since file
1898   // numbers grow monotonically, this also means that pending_outputs_ is always
1899   // sorted. After a background job is done executing, its file number is
1900   // deleted from pending_outputs_, which allows PurgeObsoleteFiles() to clean
1901   // it up.
1902   // State is protected with db mutex.
1903   std::list<uint64_t> pending_outputs_;
1904 
1905   // flush_queue_ and compaction_queue_ hold column families that we need to
1906   // flush and compact, respectively.
1907   // A column family is inserted into flush_queue_ when it satisfies condition
1908   // cfd->imm()->IsFlushPending()
1909   // A column family is inserted into compaction_queue_ when it satisfied
1910   // condition cfd->NeedsCompaction()
1911   // Column families in this list are all Ref()-erenced
1912   // TODO(icanadi) Provide some kind of ReferencedColumnFamily class that will
1913   // do RAII on ColumnFamilyData
1914   // Column families are in this queue when they need to be flushed or
1915   // compacted. Consumers of these queues are flush and compaction threads. When
1916   // column family is put on this queue, we increase unscheduled_flushes_ and
1917   // unscheduled_compactions_. When these variables are bigger than zero, that
1918   // means we need to schedule background threads for flush and compaction.
1919   // Once the background threads are scheduled, we decrease unscheduled_flushes_
1920   // and unscheduled_compactions_. That way we keep track of number of
1921   // compaction and flush threads we need to schedule. This scheduling is done
1922   // in MaybeScheduleFlushOrCompaction()
1923   // invariant(column family present in flush_queue_ <==>
1924   // ColumnFamilyData::pending_flush_ == true)
1925   std::deque<FlushRequest> flush_queue_;
1926   // invariant(column family present in compaction_queue_ <==>
1927   // ColumnFamilyData::pending_compaction_ == true)
1928   std::deque<ColumnFamilyData*> compaction_queue_;
1929 
1930   // A map to store file numbers and filenames of the files to be purged
1931   std::unordered_map<uint64_t, PurgeFileInfo> purge_files_;
1932 
1933   // A vector to store the file numbers that have been assigned to certain
1934   // JobContext. Current implementation tracks ssts only.
1935   std::unordered_set<uint64_t> files_grabbed_for_purge_;
1936 
1937   // A queue to store log writers to close
1938   std::deque<log::Writer*> logs_to_free_queue_;
1939   std::deque<SuperVersion*> superversions_to_free_queue_;
1940   int unscheduled_flushes_;
1941   int unscheduled_compactions_;
1942 
1943   // count how many background compactions are running or have been scheduled in
1944   // the BOTTOM pool
1945   int bg_bottom_compaction_scheduled_;
1946 
1947   // count how many background compactions are running or have been scheduled
1948   int bg_compaction_scheduled_;
1949 
1950   // stores the number of compactions are currently running
1951   int num_running_compactions_;
1952 
1953   // number of background memtable flush jobs, submitted to the HIGH pool
1954   int bg_flush_scheduled_;
1955 
1956   // stores the number of flushes are currently running
1957   int num_running_flushes_;
1958 
1959   // number of background obsolete file purge jobs, submitted to the HIGH pool
1960   int bg_purge_scheduled_;
1961 
1962   std::deque<ManualCompactionState*> manual_compaction_dequeue_;
1963 
1964   // shall we disable deletion of obsolete files
1965   // if 0 the deletion is enabled.
1966   // if non-zero, files will not be getting deleted
1967   // This enables two different threads to call
1968   // EnableFileDeletions() and DisableFileDeletions()
1969   // without any synchronization
1970   int disable_delete_obsolete_files_;
1971 
1972   // Number of times FindObsoleteFiles has found deletable files and the
1973   // corresponding call to PurgeObsoleteFiles has not yet finished.
1974   int pending_purge_obsolete_files_;
1975 
1976   // last time when DeleteObsoleteFiles with full scan was executed. Originally
1977   // initialized with startup time.
1978   uint64_t delete_obsolete_files_last_run_;
1979 
1980   // last time stats were dumped to LOG
1981   std::atomic<uint64_t> last_stats_dump_time_microsec_;
1982 
1983   // The thread that wants to switch memtable, can wait on this cv until the
1984   // pending writes to memtable finishes.
1985   std::condition_variable switch_cv_;
1986   // The mutex used by switch_cv_. mutex_ should be acquired beforehand.
1987   std::mutex switch_mutex_;
1988   // Number of threads intending to write to memtable
1989   std::atomic<size_t> pending_memtable_writes_ = {};
1990 
1991   // Each flush or compaction gets its own job id. this counter makes sure
1992   // they're unique
1993   std::atomic<int> next_job_id_;
1994 
1995   // A flag indicating whether the current rocksdb database has any
1996   // data that is not yet persisted into either WAL or SST file.
1997   // Used when disableWAL is true.
1998   std::atomic<bool> has_unpersisted_data_;
1999 
2000   // if an attempt was made to flush all column families that
2001   // the oldest log depends on but uncommitted data in the oldest
2002   // log prevents the log from being released.
2003   // We must attempt to free the dependent memtables again
2004   // at a later time after the transaction in the oldest
2005   // log is fully commited.
2006   bool unable_to_release_oldest_log_;
2007 
2008   static const int KEEP_LOG_FILE_NUM = 1000;
2009   // MSVC version 1800 still does not have constexpr for ::max()
2010   static const uint64_t kNoTimeOut = port::kMaxUint64;
2011 
2012   std::string db_absolute_path_;
2013 
2014   // Number of running IngestExternalFile() or CreateColumnFamilyWithImport()
2015   // calls.
2016   // REQUIRES: mutex held
2017   int num_running_ingest_file_;
2018 
2019 #ifndef ROCKSDB_LITE
2020   WalManager wal_manager_;
2021 #endif  // ROCKSDB_LITE
2022 
2023   // Unified interface for logging events
2024   EventLogger event_logger_;
2025 
2026   // A value of > 0 temporarily disables scheduling of background work
2027   int bg_work_paused_;
2028 
2029   // A value of > 0 temporarily disables scheduling of background compaction
2030   int bg_compaction_paused_;
2031 
2032   // Guard against multiple concurrent refitting
2033   bool refitting_level_;
2034 
2035   // Indicate DB was opened successfully
2036   bool opened_successfully_;
2037 
2038   // The min threshold to triggere bottommost compaction for removing
2039   // garbages, among all column families.
2040   SequenceNumber bottommost_files_mark_threshold_ = kMaxSequenceNumber;
2041 
2042   LogsWithPrepTracker logs_with_prep_tracker_;
2043 
2044   // Callback for compaction to check if a key is visible to a snapshot.
2045   // REQUIRES: mutex held
2046   std::unique_ptr<SnapshotChecker> snapshot_checker_;
2047 
2048   // Callback for when the cached_recoverable_state_ is written to memtable
2049   // Only to be set during initialization
2050   std::unique_ptr<PreReleaseCallback> recoverable_state_pre_release_callback_;
2051 
2052   // handle for scheduling stats dumping at fixed intervals
2053   // REQUIRES: mutex locked
2054   std::unique_ptr<ROCKSDB_NAMESPACE::RepeatableThread> thread_dump_stats_;
2055 
2056   // handle for scheduling stats snapshoting at fixed intervals
2057   // REQUIRES: mutex locked
2058   std::unique_ptr<ROCKSDB_NAMESPACE::RepeatableThread> thread_persist_stats_;
2059 
2060   // When set, we use a separate queue for writes that don't write to memtable.
2061   // In 2PC these are the writes at Prepare phase.
2062   const bool two_write_queues_;
2063   const bool manual_wal_flush_;
2064 
2065   // LastSequence also indicates last published sequence visibile to the
2066   // readers. Otherwise LastPublishedSequence should be used.
2067   const bool last_seq_same_as_publish_seq_;
2068   // It indicates that a customized gc algorithm must be used for
2069   // flush/compaction and if it is not provided vis SnapshotChecker, we should
2070   // disable gc to be safe.
2071   const bool use_custom_gc_;
2072   // Flag to indicate that the DB instance shutdown has been initiated. This
2073   // different from shutting_down_ atomic in that it is set at the beginning
2074   // of shutdown sequence, specifically in order to prevent any background
2075   // error recovery from going on in parallel. The latter, shutting_down_,
2076   // is set a little later during the shutdown after scheduling memtable
2077   // flushes
2078   std::atomic<bool> shutdown_initiated_;
2079   // Flag to indicate whether sst_file_manager object was allocated in
2080   // DB::Open() or passed to us
2081   bool own_sfm_;
2082 
2083   // Clients must periodically call SetPreserveDeletesSequenceNumber()
2084   // to advance this seqnum. Default value is 0 which means ALL deletes are
2085   // preserved. Note that this has no effect if DBOptions.preserve_deletes
2086   // is set to false.
2087   std::atomic<SequenceNumber> preserve_deletes_seqnum_;
2088   const bool preserve_deletes_;
2089 
2090   // Flag to check whether Close() has been called on this DB
2091   bool closed_;
2092 
2093   ErrorHandler error_handler_;
2094 
2095   // Conditional variable to coordinate installation of atomic flush results.
2096   // With atomic flush, each bg thread installs the result of flushing multiple
2097   // column families, and different threads can flush different column
2098   // families. It's difficult to rely on one thread to perform batch
2099   // installation for all threads. This is different from the non-atomic flush
2100   // case.
2101   // atomic_flush_install_cv_ makes sure that threads install atomic flush
2102   // results sequentially. Flush results of memtables with lower IDs get
2103   // installed to MANIFEST first.
2104   InstrumentedCondVar atomic_flush_install_cv_;
2105 
2106   bool wal_in_db_path_;
2107 };
2108 
2109 extern Options SanitizeOptions(const std::string& db, const Options& src);
2110 
2111 extern DBOptions SanitizeOptions(const std::string& db, const DBOptions& src);
2112 
2113 extern CompressionType GetCompressionFlush(
2114     const ImmutableCFOptions& ioptions,
2115     const MutableCFOptions& mutable_cf_options);
2116 
2117 // Return the earliest log file to keep after the memtable flush is
2118 // finalized.
2119 // `cfd_to_flush` is the column family whose memtable (specified in
2120 // `memtables_to_flush`) will be flushed and thus will not depend on any WAL
2121 // file.
2122 // The function is only applicable to 2pc mode.
2123 extern uint64_t PrecomputeMinLogNumberToKeep(
2124     VersionSet* vset, const ColumnFamilyData& cfd_to_flush,
2125     autovector<VersionEdit*> edit_list,
2126     const autovector<MemTable*>& memtables_to_flush,
2127     LogsWithPrepTracker* prep_tracker);
2128 
2129 // `cfd_to_flush` is the column family whose memtable will be flushed and thus
2130 // will not depend on any WAL file. nullptr means no memtable is being flushed.
2131 // The function is only applicable to 2pc mode.
2132 extern uint64_t FindMinPrepLogReferencedByMemTable(
2133     VersionSet* vset, const ColumnFamilyData* cfd_to_flush,
2134     const autovector<MemTable*>& memtables_to_flush);
2135 
2136 // Fix user-supplied options to be reasonable
2137 template <class T, class V>
ClipToRange(T * ptr,V minvalue,V maxvalue)2138 static void ClipToRange(T* ptr, V minvalue, V maxvalue) {
2139   if (static_cast<V>(*ptr) > maxvalue) *ptr = maxvalue;
2140   if (static_cast<V>(*ptr) < minvalue) *ptr = minvalue;
2141 }
2142 
2143 }  // namespace ROCKSDB_NAMESPACE
2144