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