db_impl.h 36.9 KB
Newer Older
1
//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
2 3 4 5
//  This source code is licensed under the BSD-style license found in the
//  LICENSE file in the root directory of this source tree. An additional grant
//  of patent rights can be found in the PATENTS file in the same directory.
//
J
jorlow@chromium.org 已提交
6 7 8
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
9
#pragma once
K
Kai Liu 已提交
10

H
Haobo Xu 已提交
11
#include <atomic>
12
#include <deque>
13
#include <limits>
I
Igor Canadi 已提交
14
#include <list>
15
#include <set>
I
Igor Canadi 已提交
16
#include <string>
17 18
#include <utility>
#include <vector>
K
kailiu 已提交
19

20
#include "db/column_family.h"
21
#include "db/compaction_job.h"
22
#include "db/dbformat.h"
23
#include "db/flush_job.h"
24 25
#include "db/flush_scheduler.h"
#include "db/internal_stats.h"
A
agiardullo 已提交
26 27
#include "db/log_writer.h"
#include "db/snapshot_impl.h"
28
#include "db/version_edit.h"
I
Igor Canadi 已提交
29
#include "db/wal_manager.h"
30 31
#include "db/write_controller.h"
#include "db/write_thread.h"
32
#include "db/writebuffer.h"
K
Kai Liu 已提交
33 34
#include "memtable_list.h"
#include "port/port.h"
35 36 37 38
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/memtablerep.h"
#include "rocksdb/transaction_log.h"
S
sdong 已提交
39
#include "table/scoped_arena_iterator.h"
40
#include "util/autovector.h"
I
Igor Canadi 已提交
41 42
#include "util/event_logger.h"
#include "util/hash.h"
43
#include "util/instrumented_mutex.h"
44 45
#include "util/stop_watch.h"
#include "util/thread_local.h"
46

47
namespace rocksdb {
J
jorlow@chromium.org 已提交
48 49 50 51 52 53

class MemTable;
class TableCache;
class Version;
class VersionEdit;
class VersionSet;
54
class Arena;
A
agiardullo 已提交
55
class WriteCallback;
I
Igor Canadi 已提交
56
struct JobContext;
57
struct ExternalSstFileInfo;
J
jorlow@chromium.org 已提交
58 59 60

class DBImpl : public DB {
 public:
I
Igor Canadi 已提交
61
  DBImpl(const DBOptions& options, const std::string& dbname);
J
jorlow@chromium.org 已提交
62 63 64
  virtual ~DBImpl();

  // Implementations of the DB interface
65 66
  using DB::Put;
  virtual Status Put(const WriteOptions& options,
67
                     ColumnFamilyHandle* column_family, const Slice& key,
I
Igor Sugak 已提交
68
                     const Slice& value) override;
69 70
  using DB::Merge;
  virtual Status Merge(const WriteOptions& options,
71
                       ColumnFamilyHandle* column_family, const Slice& key,
I
Igor Sugak 已提交
72
                       const Slice& value) override;
73 74
  using DB::Delete;
  virtual Status Delete(const WriteOptions& options,
I
Igor Sugak 已提交
75 76
                        ColumnFamilyHandle* column_family,
                        const Slice& key) override;
A
Andres Noetzli 已提交
77 78 79 80
  using DB::SingleDelete;
  virtual Status SingleDelete(const WriteOptions& options,
                              ColumnFamilyHandle* column_family,
                              const Slice& key) override;
81
  using DB::Write;
I
Igor Sugak 已提交
82 83
  virtual Status Write(const WriteOptions& options,
                       WriteBatch* updates) override;
A
agiardullo 已提交
84

85
  using DB::Get;
J
jorlow@chromium.org 已提交
86
  virtual Status Get(const ReadOptions& options,
87
                     ColumnFamilyHandle* column_family, const Slice& key,
I
Igor Sugak 已提交
88
                     std::string* value) override;
89 90 91
  using DB::MultiGet;
  virtual std::vector<Status> MultiGet(
      const ReadOptions& options,
92
      const std::vector<ColumnFamilyHandle*>& column_family,
I
Igor Sugak 已提交
93 94
      const std::vector<Slice>& keys,
      std::vector<std::string>* values) override;
95

96
  virtual Status CreateColumnFamily(const ColumnFamilyOptions& options,
97
                                    const std::string& column_family,
I
Igor Sugak 已提交
98 99
                                    ColumnFamilyHandle** handle) override;
  virtual Status DropColumnFamily(ColumnFamilyHandle* column_family) override;
100

101 102 103 104
  // Returns false if key doesn't exist in the database and true if it may.
  // If value_found is not passed in as null, then return the value if found in
  // memory. On return, if value was found, then value_found will be set to true
  // , otherwise false.
105
  using DB::KeyMayExist;
106
  virtual bool KeyMayExist(const ReadOptions& options,
107
                           ColumnFamilyHandle* column_family, const Slice& key,
I
Igor Sugak 已提交
108 109
                           std::string* value,
                           bool* value_found = nullptr) override;
110 111
  using DB::NewIterator;
  virtual Iterator* NewIterator(const ReadOptions& options,
I
Igor Sugak 已提交
112
                                ColumnFamilyHandle* column_family) override;
113 114
  virtual Status NewIterators(
      const ReadOptions& options,
I
Igor Canadi 已提交
115
      const std::vector<ColumnFamilyHandle*>& column_families,
I
Igor Sugak 已提交
116 117 118
      std::vector<Iterator*>* iterators) override;
  virtual const Snapshot* GetSnapshot() override;
  virtual void ReleaseSnapshot(const Snapshot* snapshot) override;
119
  using DB::GetProperty;
120
  virtual bool GetProperty(ColumnFamilyHandle* column_family,
I
Igor Sugak 已提交
121
                           const Slice& property, std::string* value) override;
122 123 124
  using DB::GetIntProperty;
  virtual bool GetIntProperty(ColumnFamilyHandle* column_family,
                              const Slice& property, uint64_t* value) override;
125 126 127
  using DB::GetAggregatedIntProperty;
  virtual bool GetAggregatedIntProperty(const Slice& property,
                                        uint64_t* aggregated_value) override;
128
  using DB::GetApproximateSizes;
129
  virtual void GetApproximateSizes(ColumnFamilyHandle* column_family,
130 131
                                   const Range* range, int n, uint64_t* sizes,
                                   bool include_memtable = false) override;
132
  using DB::CompactRange;
133 134 135
  virtual Status CompactRange(const CompactRangeOptions& options,
                              ColumnFamilyHandle* column_family,
                              const Slice* begin, const Slice* end) override;
136

137
  using DB::CompactFiles;
I
Igor Sugak 已提交
138 139 140 141 142
  virtual Status CompactFiles(const CompactionOptions& compact_options,
                              ColumnFamilyHandle* column_family,
                              const std::vector<std::string>& input_file_names,
                              const int output_level,
                              const int output_path_id = -1) override;
143

144 145 146
  virtual Status PauseBackgroundWork() override;
  virtual Status ContinueBackgroundWork() override;

147 148 149
  virtual Status EnableAutoCompaction(
      const std::vector<ColumnFamilyHandle*>& column_family_handles) override;

150
  using DB::SetOptions;
I
Igor Sugak 已提交
151 152 153
  Status SetOptions(
      ColumnFamilyHandle* column_family,
      const std::unordered_map<std::string, std::string>& options_map) override;
154

155
  using DB::NumberLevels;
I
Igor Sugak 已提交
156
  virtual int NumberLevels(ColumnFamilyHandle* column_family) override;
157
  using DB::MaxMemCompactionLevel;
I
Igor Sugak 已提交
158
  virtual int MaxMemCompactionLevel(ColumnFamilyHandle* column_family) override;
159
  using DB::Level0StopWriteTrigger;
I
Igor Sugak 已提交
160 161 162 163
  virtual int Level0StopWriteTrigger(
      ColumnFamilyHandle* column_family) override;
  virtual const std::string& GetName() const override;
  virtual Env* GetEnv() const override;
164
  using DB::GetOptions;
I
Igor Sugak 已提交
165 166
  virtual const Options& GetOptions(
      ColumnFamilyHandle* column_family) const override;
167 168
  using DB::GetDBOptions;
  virtual const DBOptions& GetDBOptions() const override;
169 170
  using DB::Flush;
  virtual Status Flush(const FlushOptions& options,
I
Igor Sugak 已提交
171
                       ColumnFamilyHandle* column_family) override;
172
  virtual Status SyncWAL() override;
I
Igor Canadi 已提交
173

I
Igor Sugak 已提交
174
  virtual SequenceNumber GetLatestSequenceNumber() const override;
I
Igor Canadi 已提交
175 176

#ifndef ROCKSDB_LITE
I
Igor Sugak 已提交
177 178
  virtual Status DisableFileDeletions() override;
  virtual Status EnableFileDeletions(bool force) override;
179
  virtual int IsFileDeletionsEnabled() const;
I
Igor Canadi 已提交
180
  // All the returned filenames start with "/"
181
  virtual Status GetLiveFiles(std::vector<std::string>&,
182
                              uint64_t* manifest_file_size,
I
Igor Sugak 已提交
183 184
                              bool flush_memtable = true) override;
  virtual Status GetSortedWalFiles(VectorLogPtr& files) override;
I
Igor Canadi 已提交
185

186 187 188
  virtual Status GetUpdatesSince(
      SequenceNumber seq_number, unique_ptr<TransactionLogIterator>* iter,
      const TransactionLogIterator::ReadOptions&
I
Igor Sugak 已提交
189 190
          read_options = TransactionLogIterator::ReadOptions()) override;
  virtual Status DeleteFile(std::string name) override;
191 192
  Status DeleteFilesInRange(ColumnFamilyHandle* column_family,
                            const Slice* begin, const Slice* end);
193

I
Igor Sugak 已提交
194 195
  virtual void GetLiveFilesMetaData(
      std::vector<LiveFileMetaData>* metadata) override;
196 197 198 199 200 201 202 203 204

  // Obtains the meta data of the specified column family of the DB.
  // Status::NotFound() will be returned if the current DB does not have
  // any column family match the specified name.
  // TODO(yhchiang): output parameter is placed in the end in this codebase.
  virtual void GetColumnFamilyMetaData(
      ColumnFamilyHandle* column_family,
      ColumnFamilyMetaData* metadata) override;

205 206 207 208
  // experimental API
  Status SuggestCompactRange(ColumnFamilyHandle* column_family,
                             const Slice* begin, const Slice* end);

209 210
  Status PromoteL0(ColumnFamilyHandle* column_family, int target_level);

A
agiardullo 已提交
211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230
  // Similar to Write() but will call the callback once on the single write
  // thread to determine whether it is safe to perform the write.
  virtual Status WriteWithCallback(const WriteOptions& write_options,
                                   WriteBatch* my_batch,
                                   WriteCallback* callback);

  // Returns the sequence number that is guaranteed to be smaller than or equal
  // to the sequence number of any key that could be inserted into the current
  // memtables. It can then be assumed that any write with a larger(or equal)
  // sequence number will be present in this memtable or a later memtable.
  //
  // If the earliest sequence number could not be determined,
  // kMaxSequenceNumber will be returned.
  //
  // If include_history=true, will also search Memtables in MemTableList
  // History.
  SequenceNumber GetEarliestMemTableSequenceNumber(SuperVersion* sv,
                                                   bool include_history);

  // For a given key, check to see if there are any records for this key
231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254
  // in the memtables, including memtable history.  If cache_only is false,
  // SST files will also be checked.
  //
  // If a key is found, *found_record_for_key will be set to true and
  // *seq will will be set to the stored sequence number for the latest
  // operation on this key or kMaxSequenceNumber if unknown.
  // If no key is found, *found_record_for_key will be set to false.
  //
  // Note: If cache_only=false, it is possible for *seq to be set to 0 if
  // the sequence number has been cleared from the record.  If the caller is
  // holding an active db snapshot, we know the missing sequence must be less
  // than the snapshot's sequence number (sequence numbers are only cleared
  // when there are no earlier active snapshots).
  //
  // If NotFound is returned and found_record_for_key is set to false, then no
  // record for this key was found.  If the caller is holding an active db
  // snapshot, we know that no key could have existing after this snapshot
  // (since we do not compact keys that have an earlier snapshot).
  //
  // Returns OK or NotFound on success,
  // other status on unexpected error.
  Status GetLatestSequenceForKey(SuperVersion* sv, const Slice& key,
                                 bool cache_only, SequenceNumber* seq,
                                 bool* found_record_for_key);
A
agiardullo 已提交
255

256 257 258 259 260 261 262
  using DB::AddFile;
  virtual Status AddFile(ColumnFamilyHandle* column_family,
                         const ExternalSstFileInfo* file_info,
                         bool move_file) override;
  virtual Status AddFile(ColumnFamilyHandle* column_family,
                         const std::string& file_path, bool move_file) override;

I
Igor Canadi 已提交
263
#endif  // ROCKSDB_LITE
264

265 266 267 268 269 270
  // Similar to GetSnapshot(), but also lets the db know that this snapshot
  // will be used for transaction write-conflict checking.  The DB can then
  // make sure not to compact any keys that would prevent a write-conflict from
  // being detected.
  const Snapshot* GetSnapshotForWriteConflictBoundary();

I
Igor Canadi 已提交
271 272 273 274
  // checks if all live files exist on file system and that their file sizes
  // match to our in-memory records
  virtual Status CheckConsistency();

275
  virtual Status GetDbIdentity(std::string& identity) const override;
276

I
Igor Canadi 已提交
277
  Status RunManualCompaction(ColumnFamilyData* cfd, int input_level,
278
                             int output_level, uint32_t output_path_id,
279
                             const Slice* begin, const Slice* end,
280
                             bool exclusive,
281
                             bool disallow_trivial_move = false);
282

283 284 285
  // Return an internal iterator over the current state of the database.
  // The keys of this iterator are internal keys (see format.h).
  // The returned iterator should be deleted when no longer needed.
S
sdong 已提交
286 287
  InternalIterator* NewInternalIterator(
      Arena* arena, ColumnFamilyHandle* column_family = nullptr);
288

289
#ifndef NDEBUG
J
jorlow@chromium.org 已提交
290
  // Extra methods (for testing) that are not in the public DB interface
I
Igor Canadi 已提交
291
  // Implemented in db_impl_debug.cc
J
jorlow@chromium.org 已提交
292

293
  // Compact any files in the named level that overlap [*begin, *end]
294
  Status TEST_CompactRange(int level, const Slice* begin, const Slice* end,
295 296
                           ColumnFamilyHandle* column_family = nullptr,
                           bool disallow_trivial_move = false);
J
jorlow@chromium.org 已提交
297

298
  // Force current memtable contents to be flushed.
299
  Status TEST_FlushMemTable(bool wait = true);
J
jorlow@chromium.org 已提交
300

301
  // Wait for memtable compaction
302
  Status TEST_WaitForFlushMemTable(ColumnFamilyHandle* column_family = nullptr);
303 304 305 306

  // Wait for any compaction
  Status TEST_WaitForCompact();

307 308
  // Return the maximum overlapping data (in bytes) at next level for any
  // file at a level >= 1.
309 310
  int64_t TEST_MaxNextLevelOverlappingBytes(ColumnFamilyHandle* column_family =
                                                nullptr);
311

A
Abhishek Kona 已提交
312 313
  // Return the current manifest file no.
  uint64_t TEST_Current_Manifest_FileNo();
314

315
  // get total level0 file size. Only for testing.
316
  uint64_t TEST_GetLevel0TotalSize();
317

318 319
  void TEST_GetFilesMetaData(ColumnFamilyHandle* column_family,
                             std::vector<std::vector<FileMetaData>>* metadata);
320

321 322 323 324 325 326 327 328 329 330
  void TEST_LockMutex();

  void TEST_UnlockMutex();

  // REQUIRES: mutex locked
  void* TEST_BeginWrite();

  // REQUIRES: mutex locked
  // pass the pointer that you got from TEST_BeginWrite()
  void TEST_EndWrite(void* w);
331

332
  uint64_t TEST_MaxTotalInMemoryState() const {
333 334
    return max_total_in_memory_state_;
  }
335

336 337
  size_t TEST_LogsToFreeSize();

338 339
  uint64_t TEST_LogfileNumber();

340 341 342 343 344 345
  // Returns column family name to ImmutableCFOptions map.
  Status TEST_GetAllImmutableCFOptions(
      std::unordered_map<std::string, const ImmutableCFOptions*>* iopts_map);

  Cache* TEST_table_cache() { return table_cache_.get(); }

346
  WriteController& TEST_write_controler() { return write_controller_; }
347

348
#endif  // NDEBUG
I
Igor Canadi 已提交
349

350 351 352 353
  // Return maximum background compaction alowed to be scheduled based on
  // compaction status.
  int BGCompactionsAllowed() const;

I
Igor Canadi 已提交
354
  // Returns the list of live files in 'live' and the list
K
kailiu 已提交
355
  // of all files in the filesystem in 'candidate_files'.
I
Igor Canadi 已提交
356
  // If force == false and the last call was less than
357
  // db_options_.delete_obsolete_files_period_micros microseconds ago,
I
Igor Canadi 已提交
358 359
  // it will not fill up the job_context
  void FindObsoleteFiles(JobContext* job_context, bool force,
I
Igor Canadi 已提交
360 361 362 363 364 365
                         bool no_full_scan = false);

  // Diffs the files listed in filenames and those that do not
  // belong to live files are posibly removed. Also, removes all the
  // files in sst_delete_files and log_delete_files.
  // It is not necessary to hold the mutex when invoking this method.
I
Igor Canadi 已提交
366
  void PurgeObsoleteFiles(const JobContext& background_contet);
I
Igor Canadi 已提交
367

I
Igor Sugak 已提交
368
  ColumnFamilyHandle* DefaultColumnFamily() const override;
369

370 371
  const SnapshotList& snapshots() const { return snapshots_; }

372
  void CancelAllBackgroundWork(bool wait);
373

A
agiardullo 已提交
374 375 376 377 378 379 380 381 382 383 384
  // Find Super version and reference it. Based on options, it might return
  // the thread local cached one.
  // Call ReturnAndCleanupSuperVersion() when it is no longer needed.
  SuperVersion* GetAndRefSuperVersion(ColumnFamilyData* cfd);

  // Similar to the previous function but looks up based on a column family id.
  // nullptr will be returned if this column family no longer exists.
  // REQUIRED: this function should only be called on the write thread or if the
  // mutex is held.
  SuperVersion* GetAndRefSuperVersion(uint32_t column_family_id);

A
agiardullo 已提交
385 386 387
  // Same as above, should called without mutex held and not on write thread.
  SuperVersion* GetAndRefSuperVersionUnlocked(uint32_t column_family_id);

A
agiardullo 已提交
388 389 390 391 392 393 394 395 396 397
  // Un-reference the super version and return it to thread local cache if
  // needed. If it is the last reference of the super version. Clean it up
  // after un-referencing it.
  void ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd, SuperVersion* sv);

  // Similar to the previous function but looks up based on a column family id.
  // nullptr will be returned if this column family no longer exists.
  // REQUIRED: this function should only be called on the write thread.
  void ReturnAndCleanupSuperVersion(uint32_t colun_family_id, SuperVersion* sv);

A
agiardullo 已提交
398 399 400 401
  // Same as above, should called without mutex held and not on write thread.
  void ReturnAndCleanupSuperVersionUnlocked(uint32_t colun_family_id,
                                            SuperVersion* sv);

A
agiardullo 已提交
402 403 404 405 406
  // REQUIRED: this function should only be called on the write thread or if the
  // mutex is held.  Return value only valid until next call to this function or
  // mutex is released.
  ColumnFamilyHandle* GetColumnFamilyHandle(uint32_t column_family_id);

A
agiardullo 已提交
407 408 409
  // Same as above, should called without mutex held and not on write thread.
  ColumnFamilyHandle* GetColumnFamilyHandleUnlocked(uint32_t column_family_id);

410 411 412 413 414 415 416 417 418 419 420 421 422 423
  // Returns the number of currently running flushes.
  // REQUIREMENT: mutex_ must be held when calling this function.
  int num_running_flushes() {
    mutex_.AssertHeld();
    return num_running_flushes_;
  }

  // Returns the number of currently running compactions.
  // REQUIREMENT: mutex_ must be held when calling this function.
  int num_running_compactions() {
    mutex_.AssertHeld();
    return num_running_compactions_;
  }

424
 protected:
H
heyongqiang 已提交
425 426
  Env* const env_;
  const std::string dbname_;
427
  unique_ptr<VersionSet> versions_;
428
  const DBOptions db_options_;
L
Lei Jin 已提交
429
  Statistics* stats_;
H
heyongqiang 已提交
430

S
sdong 已提交
431 432 433 434
  InternalIterator* NewInternalIterator(const ReadOptions&,
                                        ColumnFamilyData* cfd,
                                        SuperVersion* super_version,
                                        Arena* arena);
435

436 437 438
  // Except in DB::Open(), WriteOptionsFile can only be called when:
  // 1. WriteThread::Writer::EnterUnbatched() is used.
  // 2. db_mutex is held
439
  Status WriteOptionsFile();
440 441 442 443

  // The following two functions can only be called when:
  // 1. WriteThread::Writer::EnterUnbatched() is used.
  // 2. db_mutex is NOT held
444 445 446
  Status RenameTempFileToOptionsFile(const std::string& file_name);
  Status DeleteObsoleteOptionsFiles();

447
  void NotifyOnFlushCompleted(ColumnFamilyData* cfd, FileMetaData* file_meta,
448
                              const MutableCFOptions& mutable_cf_options,
449
                              int job_id, TableProperties prop);
450

O
Ori Bernstein 已提交
451
  void NotifyOnCompactionCompleted(ColumnFamilyData* cfd,
452 453
                                   Compaction *c, const Status &st,
                                   const CompactionJobStats& job_stats,
454
                                   int job_id);
O
Ori Bernstein 已提交
455

Y
Yueh-Hsuan Chiang 已提交
456 457 458 459 460 461
  void NewThreadStatusCfInfo(ColumnFamilyData* cfd) const;

  void EraseThreadStatusCfInfo(ColumnFamilyData* cfd) const;

  void EraseThreadStatusDbInfo() const;

A
agiardullo 已提交
462 463 464
  Status WriteImpl(const WriteOptions& options, WriteBatch* updates,
                   WriteCallback* callback);

J
jorlow@chromium.org 已提交
465 466
 private:
  friend class DB;
467
  friend class InternalStats;
I
Igor Canadi 已提交
468
#ifndef ROCKSDB_LITE
L
Lei Jin 已提交
469
  friend class ForwardIterator;
I
Igor Canadi 已提交
470
#endif
471
  friend struct SuperVersion;
L
Lei Jin 已提交
472
  friend class CompactedDBImpl;
A
agiardullo 已提交
473 474 475
#ifndef NDEBUG
  friend class XFTransactionWriteHandler;
#endif
476
  struct CompactionState;
477

S
Stanislau Hlebik 已提交
478
  struct WriteContext;
J
jorlow@chromium.org 已提交
479 480 481 482 483 484

  Status NewDB();

  // Recover the descriptor from persistent storage.  May do a significant
  // amount of work to recover recently logged updates.  Any changes to
  // be made to the descriptor are added to *edit.
485 486
  Status Recover(const std::vector<ColumnFamilyDescriptor>& column_families,
                 bool read_only = false, bool error_if_log_file_exist = false);
J
jorlow@chromium.org 已提交
487 488 489

  void MaybeIgnoreError(Status* s) const;

490 491
  const Status CreateArchivalDirectory();

J
jorlow@chromium.org 已提交
492 493 494
  // Delete any unneeded files and stale in-memory entries.
  void DeleteObsoleteFiles();

I
Igor Canadi 已提交
495 496
  // Background process needs to call
  //     auto x = CaptureCurrentFileNumberInPendingOutputs()
497
  //     auto file_num = versions_->NewFileNumber();
I
Igor Canadi 已提交
498 499
  //     <do something>
  //     ReleaseFileNumberFromPendingOutputs(x)
500 501
  // This will protect any file with number `file_num` or greater from being
  // deleted while <do something> is running.
I
Igor Canadi 已提交
502 503 504 505 506 507 508 509 510 511 512 513
  // -----------
  // This function will capture current file number and append it to
  // pending_outputs_. This will prevent any background process to delete any
  // file created after this point.
  std::list<uint64_t>::iterator CaptureCurrentFileNumberInPendingOutputs();
  // This function should be called with the result of
  // CaptureCurrentFileNumberInPendingOutputs(). It then marks that any file
  // created between the calls CaptureCurrentFileNumberInPendingOutputs() and
  // ReleaseFileNumberFromPendingOutputs() can now be deleted (if it's not live
  // and blocked by any other pending_outputs_ calls)
  void ReleaseFileNumberFromPendingOutputs(std::list<uint64_t>::iterator v);

514
  // Flush the in-memory write buffer to storage.  Switches to a new
J
jorlow@chromium.org 已提交
515
  // log-file/memtable and writes a new descriptor iff successful.
I
Igor Canadi 已提交
516 517 518 519
  Status FlushMemTableToOutputFile(ColumnFamilyData* cfd,
                                   const MutableCFOptions& mutable_cf_options,
                                   bool* madeProgress, JobContext* job_context,
                                   LogBuffer* log_buffer);
J
jorlow@chromium.org 已提交
520

S
Stanislau Hlebik 已提交
521 522 523
  // REQUIRES: log_numbers are sorted in ascending order
  Status RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
                         SequenceNumber* max_sequence, bool read_only);
J
jorlow@chromium.org 已提交
524

525
  // The following two methods are used to flush a memtable to
Y
Yueh-Hsuan Chiang 已提交
526
  // storage. The first one is used at database RecoveryTime (when the
527 528 529
  // database is opened) and is heavyweight because it holds the mutex
  // for the entire period. The second method WriteLevel0Table supports
  // concurrent flush memtables to storage.
530 531
  Status WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
                                     MemTable* mem, VersionEdit* edit);
S
sdong 已提交
532 533 534

  // num_bytes: for slowdown case, delay time is calculated based on
  //            `num_bytes` going through.
535
  Status DelayWrite(uint64_t num_bytes);
536

I
Igor Canadi 已提交
537
  Status ScheduleFlushes(WriteContext* context);
538

I
Igor Canadi 已提交
539
  Status SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context);
S
Stanislau Hlebik 已提交
540

H
heyongqiang 已提交
541
  // Force current memtable contents to be flushed.
542
  Status FlushMemTable(ColumnFamilyData* cfd, const FlushOptions& options);
H
heyongqiang 已提交
543

544
  // Wait for memtable flushed
545
  Status WaitForFlushMemTable(ColumnFamilyData* cfd);
H
heyongqiang 已提交
546

I
Igor Canadi 已提交
547
#ifndef ROCKSDB_LITE
548 549 550
  Status CompactFilesImpl(
      const CompactionOptions& compact_options, ColumnFamilyData* cfd,
      Version* version, const std::vector<std::string>& input_file_names,
551 552
      const int output_level, int output_path_id, JobContext* job_context,
      LogBuffer* log_buffer);
I
Igor Canadi 已提交
553
#endif  // ROCKSDB_LITE
554 555 556

  ColumnFamilyData* GetColumnFamilyDataByName(const std::string& cf_name);

557
  void MaybeScheduleFlushOrCompaction();
558 559
  void SchedulePendingFlush(ColumnFamilyData* cfd);
  void SchedulePendingCompaction(ColumnFamilyData* cfd);
560
  static void BGWorkCompaction(void* arg);
561
  static void BGWorkFlush(void* db);
562 563
  static void UnscheduleCallback(void* arg);
  void BackgroundCallCompaction(void* arg);
564
  void BackgroundCallFlush();
I
Igor Canadi 已提交
565
  Status BackgroundCompaction(bool* madeProgress, JobContext* job_context,
566
                              LogBuffer* log_buffer, void* m = 0);
I
Igor Canadi 已提交
567
  Status BackgroundFlush(bool* madeProgress, JobContext* job_context,
H
Haobo Xu 已提交
568
                         LogBuffer* log_buffer);
J
jorlow@chromium.org 已提交
569

570 571
  void PrintStatistics();

572
  // dump rocksdb.stats to LOG
573 574
  void MaybeDumpStats();

575 576
  // Return the minimum empty level that could hold the total data in the
  // input level. Return the input level, if such level could not be found.
577 578
  int FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
      const MutableCFOptions& mutable_cf_options, int level);
579

580 581 582
  // Move the files in the input level to the target level.
  // If target_level < 0, automatically calculate the minimum level that could
  // hold the data set.
I
Igor Canadi 已提交
583
  Status ReFitLevel(ColumnFamilyData* cfd, int level, int target_level = -1);
584

585 586 587 588 589 590
  // helper functions for adding and removing from flush & compaction queues
  void AddToCompactionQueue(ColumnFamilyData* cfd);
  ColumnFamilyData* PopFirstFromCompactionQueue();
  void AddToFlushQueue(ColumnFamilyData* cfd);
  ColumnFamilyData* PopFirstFromFlushQueue();

591 592 593
  // helper function to call after some of the logs_ were synced
  void MarkLogsSynced(uint64_t up_to, bool synced_dir, const Status& status);

594 595
  const Snapshot* GetSnapshotImpl(bool is_write_conflict_boundary);

J
jorlow@chromium.org 已提交
596
  // table_cache_ provides its own synchronization
I
Igor Canadi 已提交
597
  std::shared_ptr<Cache> table_cache_;
J
jorlow@chromium.org 已提交
598

599
  // Lock over the persistent DB state.  Non-nullptr iff successfully acquired.
J
jorlow@chromium.org 已提交
600 601
  FileLock* db_lock_;

602 603 604 605
  // The mutex for options file related operations.
  // NOTE: should never acquire options_file_mutex_ and mutex_ at the
  //       same time.
  InstrumentedMutex options_files_mutex_;
J
jorlow@chromium.org 已提交
606
  // State below is protected by mutex_
607
  InstrumentedMutex mutex_;
608

I
Igor Canadi 已提交
609
  std::atomic<bool> shutting_down_;
610 611
  // This condition variable is signaled on these conditions:
  // * whenever bg_compaction_scheduled_ goes down to 0
612
  // * if AnyManualCompaction, whenever a compaction finishes, even if it hasn't
613 614 615 616 617
  // made any progress
  // * whenever a compaction made any progress
  // * whenever bg_flush_scheduled_ value decreases (i.e. whenever a flush is
  // done, even if it didn't make any progress)
  // * whenever there is an error in background flush or compaction
618
  InstrumentedCondVar bg_cv_;
619
  uint64_t logfile_number_;
S
Sage Weil 已提交
620 621
  std::deque<uint64_t>
      log_recycle_files;  // a list of log files that we can recycle
622
  bool log_dir_synced_;
I
Igor Canadi 已提交
623
  bool log_empty_;
624
  ColumnFamilyHandleImpl* default_cf_handle_;
625
  InternalStats* default_cf_internal_stats_;
626
  unique_ptr<ColumnFamilyMemTablesImpl> column_family_memtables_;
I
Igor Canadi 已提交
627 628
  struct LogFileNumberSize {
    explicit LogFileNumberSize(uint64_t _number)
629
        : number(_number) {}
I
Igor Canadi 已提交
630 631
    void AddSize(uint64_t new_size) { size += new_size; }
    uint64_t number;
632 633 634 635
    uint64_t size = 0;
    bool getting_flushed = false;
  };
  struct LogWriterNumber {
636 637 638 639 640 641 642 643 644 645 646 647 648 649
    // pass ownership of _writer
    LogWriterNumber(uint64_t _number, log::Writer* _writer)
        : number(_number), writer(_writer) {}

    log::Writer* ReleaseWriter() {
      auto* w = writer;
      writer = nullptr;
      return w;
    }
    void ClearWriter() {
      delete writer;
      writer = nullptr;
    }

650
    uint64_t number;
651 652 653
    // Visual Studio doesn't support deque's member to be noncopyable because
    // of a unique_ptr as a member.
    log::Writer* writer;  // own
654 655
    // true for some prefix of logs_
    bool getting_synced = false;
I
Igor Canadi 已提交
656 657
  };
  std::deque<LogFileNumberSize> alive_log_files_;
658 659 660 661 662 663 664 665 666 667
  // Log files that aren't fully synced, and the current log file.
  // Synchronization:
  //  - push_back() is done from write thread with locked mutex_,
  //  - pop_front() is done from any thread with locked mutex_,
  //  - back() and items with getting_synced=true are not popped,
  //  - it follows that write thread with unlocked mutex_ can safely access
  //    back() and items with getting_synced=true.
  std::deque<LogWriterNumber> logs_;
  // Signaled when getting_synced becomes false for some of the logs_.
  InstrumentedCondVar log_sync_cv_;
I
Igor Canadi 已提交
668 669 670 671
  uint64_t total_log_size_;
  // only used for dynamically adjusting max_total_wal_size. it is a sum of
  // [write_buffer_size * max_write_buffer_number] over all column families
  uint64_t max_total_in_memory_state_;
672 673 674
  // If true, we have only one (default) column family. We use this to optimize
  // some code-paths
  bool single_column_family_mode_;
675 676 677
  // If this is non-empty, we need to delete these log files in background
  // threads. Protected by db mutex.
  autovector<log::Writer*> logs_to_free_;
I
Igor Canadi 已提交
678

S
sdong 已提交
679 680
  bool is_snapshot_supported_;

681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708
  // Class to maintain directories for all database paths other than main one.
  class Directories {
   public:
    Status SetDirectories(Env* env, const std::string& dbname,
                          const std::string& wal_dir,
                          const std::vector<DbPath>& data_paths);

    Directory* GetDataDir(size_t path_id);

    Directory* GetWalDir() {
      if (wal_dir_) {
        return wal_dir_.get();
      }
      return db_dir_.get();
    }

    Directory* GetDbDir() { return db_dir_.get(); }

   private:
    std::unique_ptr<Directory> db_dir_;
    std::vector<std::unique_ptr<Directory>> data_dirs_;
    std::unique_ptr<Directory> wal_dir_;

    Status CreateAndNewDirectory(Env* env, const std::string& dirname,
                                 std::unique_ptr<Directory>* directory) const;
  };

  Directories directories_;
709

710 711
  WriteBuffer write_buffer_;

I
Igor Canadi 已提交
712 713
  WriteThread write_thread_;

714
  WriteBatch tmp_batch_;
715

716
  WriteController write_controller_;
S
sdong 已提交
717 718 719 720 721

  // Size of the last batch group. In slowdown mode, next write needs to
  // sleep if it uses up the quota.
  uint64_t last_batch_group_size_;

I
Igor Canadi 已提交
722
  FlushScheduler flush_scheduler_;
723

J
jorlow@chromium.org 已提交
724 725
  SnapshotList snapshots_;

I
Igor Canadi 已提交
726 727 728 729 730 731 732 733 734 735
  // For each background job, pending_outputs_ keeps the current file number at
  // the time that background job started.
  // FindObsoleteFiles()/PurgeObsoleteFiles() never deletes any file that has
  // number bigger than any of the file number in pending_outputs_. Since file
  // numbers grow monotonically, this also means that pending_outputs_ is always
  // sorted. After a background job is done executing, its file number is
  // deleted from pending_outputs_, which allows PurgeObsoleteFiles() to clean
  // it up.
  // State is protected with db mutex.
  std::list<uint64_t> pending_outputs_;
J
jorlow@chromium.org 已提交
736

737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762
  // flush_queue_ and compaction_queue_ hold column families that we need to
  // flush and compact, respectively.
  // A column family is inserted into flush_queue_ when it satisfies condition
  // cfd->imm()->IsFlushPending()
  // A column family is inserted into compaction_queue_ when it satisfied
  // condition cfd->NeedsCompaction()
  // Column families in this list are all Ref()-erenced
  // TODO(icanadi) Provide some kind of ReferencedColumnFamily class that will
  // do RAII on ColumnFamilyData
  // Column families are in this queue when they need to be flushed or
  // compacted. Consumers of these queues are flush and compaction threads. When
  // column family is put on this queue, we increase unscheduled_flushes_ and
  // unscheduled_compactions_. When these variables are bigger than zero, that
  // means we need to schedule background threads for compaction and thread.
  // Once the background threads are scheduled, we decrease unscheduled_flushes_
  // and unscheduled_compactions_. That way we keep track of number of
  // compaction and flush threads we need to schedule. This scheduling is done
  // in MaybeScheduleFlushOrCompaction()
  // invariant(column family present in flush_queue_ <==>
  // ColumnFamilyData::pending_flush_ == true)
  std::deque<ColumnFamilyData*> flush_queue_;
  // invariant(column family present in compaction_queue_ <==>
  // ColumnFamilyData::pending_compaction_ == true)
  std::deque<ColumnFamilyData*> compaction_queue_;
  int unscheduled_flushes_;
  int unscheduled_compactions_;
763

764
  // count how many background compactions are running or have been scheduled
765
  int bg_compaction_scheduled_;
J
jorlow@chromium.org 已提交
766

767 768 769
  // stores the number of compactions are currently running
  int num_running_compactions_;

770 771 772
  // number of background memtable flush jobs, submitted to the HIGH pool
  int bg_flush_scheduled_;

773 774 775
  // stores the number of flushes are currently running
  int num_running_flushes_;

H
hans@chromium.org 已提交
776 777
  // Information for a manual compaction
  struct ManualCompaction {
I
Igor Canadi 已提交
778
    ColumnFamilyData* cfd;
779 780
    int input_level;
    int output_level;
781
    uint32_t output_path_id;
L
Lei Jin 已提交
782
    Status status;
783
    bool done;
784
    bool in_progress;             // compaction request being processed?
785 786 787
    bool incomplete;              // only part of requested range compacted
    bool exclusive;               // current behavior of only one manual
    bool disallow_trivial_move;   // Force actual compaction to run
788 789
    const InternalKey* begin;     // nullptr means beginning of key range
    const InternalKey* end;       // nullptr means end of key range
790
    InternalKey* manual_end;      // how far we are compacting
791
    InternalKey tmp_storage;      // Used to keep track of compaction progress
792 793 794 795 796 797 798 799
    InternalKey tmp_storage1;     // Used to keep track of compaction progress
    Compaction* compaction;
  };
  std::deque<ManualCompaction*> manual_compaction_dequeue_;

  struct CompactionArg {
    DBImpl* db;
    ManualCompaction* m;
H
hans@chromium.org 已提交
800
  };
J
jorlow@chromium.org 已提交
801 802 803 804

  // Have we encountered a background error in paranoid mode?
  Status bg_error_;

805
  // shall we disable deletion of obsolete files
806 807 808 809 810 811
  // if 0 the deletion is enabled.
  // if non-zero, files will not be getting deleted
  // This enables two different threads to call
  // EnableFileDeletions() and DisableFileDeletions()
  // without any synchronization
  int disable_delete_obsolete_files_;
812

I
Igor Canadi 已提交
813 814
  // next time when we should run DeleteObsoleteFiles with full scan
  uint64_t delete_obsolete_files_next_run_;
815

816
  // last time stats were dumped to LOG
H
Haobo Xu 已提交
817
  std::atomic<uint64_t> last_stats_dump_time_microsec_;
818

819 820 821 822
  // Each flush or compaction gets its own job id. this counter makes sure
  // they're unique
  std::atomic<int> next_job_id_;

823 824 825 826
  // A flag indicating whether the current rocksdb database has any
  // data that is not yet persisted into either WAL or SST file.
  // Used when disableWAL is true.
  bool has_unpersisted_data_;
827

H
heyongqiang 已提交
828
  static const int KEEP_LOG_FILE_NUM = 1000;
D
Dmitri Smirnov 已提交
829
  // MSVC version 1800 still does not have constexpr for ::max()
830
  static const uint64_t kNoTimeOut = port::kMaxUint64;
D
Dmitri Smirnov 已提交
831

H
heyongqiang 已提交
832
  std::string db_absolute_path_;
H
heyongqiang 已提交
833

834
  // The options to access storage files
L
Lei Jin 已提交
835
  const EnvOptions env_options_;
836

I
Igor Canadi 已提交
837 838 839 840
#ifndef ROCKSDB_LITE
  WalManager wal_manager_;
#endif  // ROCKSDB_LITE

I
Igor Canadi 已提交
841 842 843
  // Unified interface for logging events
  EventLogger event_logger_;

844
  // A value of > 0 temporarily disables scheduling of background work
845
  int bg_work_paused_;
846

847 848 849
  // A value of > 0 temporarily disables scheduling of background compaction
  int bg_compaction_paused_;

850 851 852
  // Guard against multiple concurrent refitting
  bool refitting_level_;

853 854 855
  // Indicate DB was opened successfully
  bool opened_successfully_;

J
jorlow@chromium.org 已提交
856 857 858 859
  // No copying allowed
  DBImpl(const DBImpl&);
  void operator=(const DBImpl&);

860 861 862 863 864 865
  // Return the earliest snapshot where seqno is visible.
  // Store the snapshot right before that, if any, in prev_snapshot
  inline SequenceNumber findEarliestVisibleSnapshot(
    SequenceNumber in,
    std::vector<SequenceNumber>& snapshots,
    SequenceNumber* prev_snapshot);
866

I
Igor Canadi 已提交
867
  // Background threads call this function, which is just a wrapper around
I
Igor Canadi 已提交
868 869 870
  // the InstallSuperVersion() function. Background threads carry
  // job_context which can have new_superversion already
  // allocated.
I
Igor Canadi 已提交
871
  void InstallSuperVersionAndScheduleWorkWrapper(
I
Igor Canadi 已提交
872 873
      ColumnFamilyData* cfd, JobContext* job_context,
      const MutableCFOptions& mutable_cf_options);
L
Lei Jin 已提交
874

875 876 877
  // All ColumnFamily state changes go through this function. Here we analyze
  // the new state and we schedule background work if we detect that the new
  // state needs flush or compaction.
I
Igor Canadi 已提交
878 879 880
  SuperVersion* InstallSuperVersionAndScheduleWork(
      ColumnFamilyData* cfd, SuperVersion* new_sv,
      const MutableCFOptions& mutable_cf_options);
I
Igor Canadi 已提交
881

I
Igor Canadi 已提交
882
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
883 884 885
  using DB::GetPropertiesOfAllTables;
  virtual Status GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                          TablePropertiesCollection* props)
886
      override;
887
  virtual Status GetPropertiesOfTablesInRange(
888
      ColumnFamilyHandle* column_family, const Range* range, std::size_t n,
889 890
      TablePropertiesCollection* props) override;

I
Igor Canadi 已提交
891
#endif  // ROCKSDB_LITE
892

893 894
  // Function that Get and KeyMayExist call with no_io true or false
  // Note: 'value_found' from KeyMayExist propagates here
895 896 897
  Status GetImpl(const ReadOptions& options, ColumnFamilyHandle* column_family,
                 const Slice& key, std::string* value,
                 bool* value_found = nullptr);
898

899
  bool GetIntPropertyInternal(ColumnFamilyData* cfd,
900 901
                              const DBPropertyInfo& property_info,
                              bool is_locked, uint64_t* value);
902 903 904 905 906

  bool HasPendingManualCompaction();
  bool HasExclusiveManualCompaction();
  void AddManualCompaction(ManualCompaction* m);
  void RemoveManualCompaction(ManualCompaction* m);
907
  bool ShouldntRunManualCompaction(ManualCompaction* m);
908 909
  bool HaveManualCompaction(ColumnFamilyData* cfd);
  bool MCOverlap(ManualCompaction* m, ManualCompaction* m1);
J
jorlow@chromium.org 已提交
910 911 912 913 914 915 916
};

// Sanitize db options.  The caller should delete result.info_log if
// it is not equal to src.info_log.
extern Options SanitizeOptions(const std::string& db,
                               const InternalKeyComparator* icmp,
                               const Options& src);
917
extern DBOptions SanitizeOptions(const std::string& db, const DBOptions& src);
S
Siying Dong 已提交
918

M
miguelportilla 已提交
919 920 921 922 923 924 925
// Fix user-supplied options to be reasonable
template <class T, class V>
static void ClipToRange(T* ptr, V minvalue, V maxvalue) {
  if (static_cast<V>(*ptr) > maxvalue) *ptr = maxvalue;
  if (static_cast<V>(*ptr) < minvalue) *ptr = minvalue;
}

926
}  // namespace rocksdb