db_impl.cc 170.7 KB
Newer Older
1 2 3 4 5
//  Copyright (c) 2013, Facebook, Inc.  All rights reserved.
//  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 9 10 11
// 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.

#include "db/db_impl.h"

I
Igor Canadi 已提交
12 13
#define __STDC_FORMAT_MACROS
#include <inttypes.h>
J
jorlow@chromium.org 已提交
14
#include <algorithm>
15 16
#include <climits>
#include <cstdio>
J
jorlow@chromium.org 已提交
17
#include <set>
18
#include <stdexcept>
19 20
#include <stdint.h>
#include <string>
21
#include <unordered_set>
22
#include <unordered_map>
T
Tomislav Novak 已提交
23
#include <utility>
24
#include <vector>
25

J
jorlow@chromium.org 已提交
26
#include "db/builder.h"
27
#include "db/db_iter.h"
K
kailiu 已提交
28
#include "db/dbformat.h"
J
jorlow@chromium.org 已提交
29 30 31 32
#include "db/filename.h"
#include "db/log_reader.h"
#include "db/log_writer.h"
#include "db/memtable.h"
K
kailiu 已提交
33
#include "db/memtable_list.h"
34
#include "db/merge_context.h"
35
#include "db/merge_helper.h"
J
jorlow@chromium.org 已提交
36
#include "db/table_cache.h"
K
kailiu 已提交
37
#include "db/table_properties_collector.h"
T
Tomislav Novak 已提交
38
#include "db/tailing_iter.h"
L
Lei Jin 已提交
39
#include "db/forward_iterator.h"
40
#include "db/transaction_log_impl.h"
J
jorlow@chromium.org 已提交
41 42
#include "db/version_set.h"
#include "db/write_batch_internal.h"
43
#include "port/port.h"
I
Igor Canadi 已提交
44
#include "rocksdb/cache.h"
45
#include "port/likely.h"
46 47 48 49 50 51
#include "rocksdb/compaction_filter.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
S
Siying Dong 已提交
52
#include "rocksdb/table.h"
J
jorlow@chromium.org 已提交
53
#include "table/block.h"
54
#include "table/block_based_table_factory.h"
J
jorlow@chromium.org 已提交
55
#include "table/merger.h"
K
kailiu 已提交
56
#include "table/table_builder.h"
J
jorlow@chromium.org 已提交
57
#include "table/two_level_iterator.h"
58
#include "util/auto_roll_logger.h"
K
kailiu 已提交
59
#include "util/autovector.h"
60
#include "util/build_version.h"
J
jorlow@chromium.org 已提交
61
#include "util/coding.h"
I
Igor Canadi 已提交
62
#include "util/hash_skiplist_rep.h"
63
#include "util/hash_linklist_rep.h"
J
jorlow@chromium.org 已提交
64
#include "util/logging.h"
H
Haobo Xu 已提交
65
#include "util/log_buffer.h"
J
jorlow@chromium.org 已提交
66
#include "util/mutexlock.h"
67
#include "util/perf_context_imp.h"
68
#include "util/iostats_context_imp.h"
69
#include "util/stop_watch.h"
70
#include "util/sync_point.h"
J
jorlow@chromium.org 已提交
71

72
namespace rocksdb {
J
jorlow@chromium.org 已提交
73

74
const std::string kDefaultColumnFamilyName("default");
75

K
kailiu 已提交
76
void DumpLeveldbBuildVersion(Logger * log);
77

78 79 80 81 82
// Information kept for every waiting writer
struct DBImpl::Writer {
  Status status;
  WriteBatch* batch;
  bool sync;
H
heyongqiang 已提交
83
  bool disableWAL;
84
  bool in_batch_group;
85
  bool done;
86
  uint64_t timeout_hint_us;
87 88 89 90 91
  port::CondVar cv;

  explicit Writer(port::Mutex* mu) : cv(mu) { }
};

J
jorlow@chromium.org 已提交
92 93 94
struct DBImpl::CompactionState {
  Compaction* const compaction;

95 96 97 98 99
  // If there were two snapshots with seq numbers s1 and
  // s2 and s1 < s2, and if we find two instances of a key k1 then lies
  // entirely within s1 and s2, then the earlier version of k1 can be safely
  // deleted because that version is not visible in any snapshot.
  std::vector<SequenceNumber> existing_snapshots;
J
jorlow@chromium.org 已提交
100 101 102 103

  // Files produced by compaction
  struct Output {
    uint64_t number;
104
    uint32_t path_id;
J
jorlow@chromium.org 已提交
105 106
    uint64_t file_size;
    InternalKey smallest, largest;
107
    SequenceNumber smallest_seqno, largest_seqno;
J
jorlow@chromium.org 已提交
108 109
  };
  std::vector<Output> outputs;
110
  std::list<uint64_t> allocated_file_numbers;
J
jorlow@chromium.org 已提交
111 112

  // State kept for output being generated
113 114
  unique_ptr<WritableFile> outfile;
  unique_ptr<TableBuilder> builder;
J
jorlow@chromium.org 已提交
115 116 117 118 119 120 121 122 123

  uint64_t total_bytes;

  Output* current_output() { return &outputs[outputs.size()-1]; }

  explicit CompactionState(Compaction* c)
      : compaction(c),
        total_bytes(0) {
  }
124

125 126 127 128 129 130 131 132
  // Create a client visible context of this compaction
  CompactionFilter::Context GetFilterContextV1() {
    CompactionFilter::Context context;
    context.is_full_compaction = compaction->IsFullCompaction();
    context.is_manual_compaction = compaction->IsManualCompaction();
    return context;
  }

133
  // Create a client visible context of this compaction
D
Danny Guo 已提交
134 135
  CompactionFilterContext GetFilterContext() {
    CompactionFilterContext context;
136
    context.is_full_compaction = compaction->IsFullCompaction();
137
    context.is_manual_compaction = compaction->IsManualCompaction();
138 139
    return context;
  }
D
Danny Guo 已提交
140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256

  std::vector<Slice> key_buf_;
  std::vector<Slice> existing_value_buf_;
  std::vector<std::string> key_str_buf_;
  std::vector<std::string> existing_value_str_buf_;
  // new_value_buf_ will only be appended if a value changes
  std::vector<std::string> new_value_buf_;
  // if values_changed_buf_[i] is true
  // new_value_buf_ will add a new entry with the changed value
  std::vector<bool> value_changed_buf_;
  // to_delete_buf_[i] is true iff key_buf_[i] is deleted
  std::vector<bool> to_delete_buf_;
  // buffer for the parsed internal keys, the string buffer is backed
  // by key_str_buf_
  std::vector<ParsedInternalKey> ikey_buf_;

  std::vector<Slice> other_key_buf_;
  std::vector<Slice> other_value_buf_;
  std::vector<std::string> other_key_str_buf_;
  std::vector<std::string> other_value_str_buf_;

  std::vector<Slice> combined_key_buf_;
  std::vector<Slice> combined_value_buf_;

  std::string cur_prefix_;

  // Buffers the kv-pair that will be run through compaction filter V2
  // in the future.
  void BufferKeyValueSlices(const Slice& key, const Slice& value) {
    key_str_buf_.emplace_back(key.ToString());
    existing_value_str_buf_.emplace_back(value.ToString());
    key_buf_.emplace_back(Slice(key_str_buf_.back()));
    existing_value_buf_.emplace_back(Slice(existing_value_str_buf_.back()));

    ParsedInternalKey ikey;
    ParseInternalKey(key_buf_.back(), &ikey);
    ikey_buf_.emplace_back(ikey);
  }

  // Buffers the kv-pair that will not be run through compaction filter V2
  // in the future.
  void BufferOtherKeyValueSlices(const Slice& key, const Slice& value) {
    other_key_str_buf_.emplace_back(key.ToString());
    other_value_str_buf_.emplace_back(value.ToString());
    other_key_buf_.emplace_back(Slice(other_key_str_buf_.back()));
    other_value_buf_.emplace_back(Slice(other_value_str_buf_.back()));
  }

  // Add a kv-pair to the combined buffer
  void AddToCombinedKeyValueSlices(const Slice& key, const Slice& value) {
    // The real strings are stored in the batch buffers
    combined_key_buf_.emplace_back(key);
    combined_value_buf_.emplace_back(value);
  }

  // Merging the two buffers
  void MergeKeyValueSliceBuffer(const InternalKeyComparator* comparator) {
    size_t i = 0;
    size_t j = 0;
    size_t total_size = key_buf_.size() + other_key_buf_.size();
    combined_key_buf_.reserve(total_size);
    combined_value_buf_.reserve(total_size);

    while (i + j < total_size) {
      int comp_res = 0;
      if (i < key_buf_.size() && j < other_key_buf_.size()) {
        comp_res = comparator->Compare(key_buf_[i], other_key_buf_[j]);
      } else if (i >= key_buf_.size() && j < other_key_buf_.size()) {
        comp_res = 1;
      } else if (j >= other_key_buf_.size() && i < key_buf_.size()) {
        comp_res = -1;
      }
      if (comp_res > 0) {
        AddToCombinedKeyValueSlices(other_key_buf_[j], other_value_buf_[j]);
        j++;
      } else if (comp_res < 0) {
        AddToCombinedKeyValueSlices(key_buf_[i], existing_value_buf_[i]);
        i++;
      }
    }
  }

  void CleanupBatchBuffer() {
    to_delete_buf_.clear();
    key_buf_.clear();
    existing_value_buf_.clear();
    key_str_buf_.clear();
    existing_value_str_buf_.clear();
    new_value_buf_.clear();
    value_changed_buf_.clear();
    ikey_buf_.clear();

    to_delete_buf_.shrink_to_fit();
    key_buf_.shrink_to_fit();
    existing_value_buf_.shrink_to_fit();
    key_str_buf_.shrink_to_fit();
    existing_value_str_buf_.shrink_to_fit();
    new_value_buf_.shrink_to_fit();
    value_changed_buf_.shrink_to_fit();
    ikey_buf_.shrink_to_fit();

    other_key_buf_.clear();
    other_value_buf_.clear();
    other_key_str_buf_.clear();
    other_value_str_buf_.clear();
    other_key_buf_.shrink_to_fit();
    other_value_buf_.shrink_to_fit();
    other_key_str_buf_.shrink_to_fit();
    other_value_str_buf_.shrink_to_fit();
  }

  void CleanupMergedBuffer() {
    combined_key_buf_.clear();
    combined_value_buf_.clear();
    combined_key_buf_.shrink_to_fit();
    combined_value_buf_.shrink_to_fit();
  }
J
jorlow@chromium.org 已提交
257 258
};

259
namespace {
J
jorlow@chromium.org 已提交
260
// Fix user-supplied options to be reasonable
261
template <class T, class V>
J
jorlow@chromium.org 已提交
262
static void ClipToRange(T* ptr, V minvalue, V maxvalue) {
D
dgrogan@chromium.org 已提交
263 264
  if (static_cast<V>(*ptr) > maxvalue) *ptr = maxvalue;
  if (static_cast<V>(*ptr) < minvalue) *ptr = minvalue;
J
jorlow@chromium.org 已提交
265
}
266 267
}  // anonymous namespace

J
jorlow@chromium.org 已提交
268 269
Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
S
Sanjay Ghemawat 已提交
270
                        const InternalFilterPolicy* ipolicy,
J
jorlow@chromium.org 已提交
271
                        const Options& src) {
272 273 274 275 276 277 278
  auto db_options = SanitizeOptions(dbname, DBOptions(src));
  auto cf_options = SanitizeOptions(icmp, ipolicy, ColumnFamilyOptions(src));
  return Options(db_options, cf_options);
}

DBOptions SanitizeOptions(const std::string& dbname, const DBOptions& src) {
  DBOptions result = src;
L
Lei Jin 已提交
279

280 281
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
282
    ClipToRange(&result.max_open_files, 20, 1000000);
283
  }
284

285
  if (result.info_log == nullptr) {
K
Kai Liu 已提交
286 287
    Status s = CreateLoggerFromOptions(dbname, result.db_log_dir, src.env,
                                       result, &result.info_log);
J
jorlow@chromium.org 已提交
288 289
    if (!s.ok()) {
      // No place suitable for logging
290
      result.info_log = nullptr;
J
jorlow@chromium.org 已提交
291 292
    }
  }
293 294 295 296 297

  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
298
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
299
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
300
  }
301

302
  if (result.db_paths.size() == 0) {
303
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
304 305
  }

J
jorlow@chromium.org 已提交
306 307 308
  return result;
}

309
namespace {
310 311 312 313 314 315 316
CompressionType GetCompressionFlush(const Options& options) {
  // Compressing memtable flushes might not help unless the sequential load
  // optimization is used for leveled compaction. Otherwise the CPU and
  // latency overhead is not offset by saving much space.

  bool can_compress;

317
  if (options.compaction_style == kCompactionStyleUniversal) {
318 319 320 321
    can_compress =
        (options.compaction_options_universal.compression_size_percent < 0);
  } else {
    // For leveled compress when min_level_to_compress == 0.
322 323
    can_compress = options.compression_per_level.empty() ||
                   options.compression_per_level[0] != kNoCompression;
324 325 326 327 328 329 330 331
  }

  if (can_compress) {
    return options.compression;
  } else {
    return kNoCompression;
  }
}
332
}  // namespace
333

I
Igor Canadi 已提交
334
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
J
jorlow@chromium.org 已提交
335
    : env_(options.env),
H
heyongqiang 已提交
336
      dbname_(dbname),
I
Igor Canadi 已提交
337
      options_(SanitizeOptions(dbname, options)),
L
Lei Jin 已提交
338
      stats_(options_.statistics.get()),
339
      db_lock_(nullptr),
H
Haobo Xu 已提交
340
      mutex_(options.use_adaptive_mutex),
341
      shutting_down_(nullptr),
J
jorlow@chromium.org 已提交
342
      bg_cv_(&mutex_),
343
      logfile_number_(0),
I
Igor Canadi 已提交
344
      log_empty_(true),
345
      default_cf_handle_(nullptr),
I
Igor Canadi 已提交
346 347
      total_log_size_(0),
      max_total_in_memory_state_(0),
348
      tmp_batch_(),
349
      bg_schedule_needed_(false),
350
      bg_compaction_scheduled_(0),
351
      bg_manual_only_(0),
352
      bg_flush_scheduled_(0),
353
      manual_compaction_(nullptr),
354
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
355
      delete_obsolete_files_last_run_(options.env->NowMicros()),
356
      purge_wal_files_last_run_(0),
357
      last_stats_dump_time_microsec_(0),
358
      default_interval_to_delete_obsolete_WAL_(600),
359
      flush_on_destroy_(false),
360
      delayed_writes_(0),
361 362
      storage_options_(options),
      bg_work_gate_closed_(false),
363 364
      refitting_level_(false),
      opened_successfully_(false) {
H
heyongqiang 已提交
365
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
366

J
jorlow@chromium.org 已提交
367
  // Reserve ten files or so for other uses and give the rest to TableCache.
368 369
  // Give a large number for setting of "infinite" open files.
  const int table_cache_size =
370 371 372 373 374 375
      (options_.max_open_files == -1) ? 4194304 : options_.max_open_files - 10;
  // Reserve ten files or so for other uses and give the rest to TableCache.
  table_cache_ =
      NewLRUCache(table_cache_size, options_.table_cache_numshardbits,
                  options_.table_cache_remove_scan_count_limit);

I
Igor Canadi 已提交
376 377
  versions_.reset(
      new VersionSet(dbname_, &options_, storage_options_, table_cache_.get()));
378 379
  column_family_memtables_.reset(
      new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet()));
380

K
kailiu 已提交
381
  DumpLeveldbBuildVersion(options_.info_log.get());
382
  options_.Dump(options_.info_log.get());
383

I
Igor Canadi 已提交
384
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
385 386 387
}

DBImpl::~DBImpl() {
388 389 390 391 392 393 394 395
  mutex_.Lock();
  if (flush_on_destroy_) {
    for (auto cfd : *versions_->GetColumnFamilySet()) {
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
396
        cfd->Unref();
397 398
      }
    }
399
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
400
  }
I
Igor Canadi 已提交
401 402

  // Wait for background work to finish
403
  shutting_down_.Release_Store(this);  // Any non-nullptr value is ok
I
Igor Canadi 已提交
404
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
H
hans@chromium.org 已提交
405
    bg_cv_.Wait();
J
jorlow@chromium.org 已提交
406
  }
407

I
Igor Canadi 已提交
408 409 410 411 412
  if (default_cf_handle_ != nullptr) {
    // we need to delete handle outside of lock because it does its own locking
    mutex_.Unlock();
    delete default_cf_handle_;
    mutex_.Lock();
413 414
  }

415 416 417 418 419 420 421 422 423 424 425 426 427 428 429
  if (options_.allow_thread_local) {
    // Clean up obsolete files due to SuperVersion release.
    // (1) Need to delete to obsolete files before closing because RepairDB()
    // scans all existing files in the file system and builds manifest file.
    // Keeping obsolete files confuses the repair process.
    // (2) Need to check if we Open()/Recover() the DB successfully before
    // deleting because if VersionSet recover fails (may be due to corrupted
    // manifest file), it is not able to identify live files correctly. As a
    // result, all "live" files can get deleted by accident. However, corrupted
    // manifest is recoverable by RepairDB().
    if (opened_successfully_) {
      DeletionState deletion_state;
      FindObsoleteFiles(deletion_state, true);
      // manifest number starting from 2
      deletion_state.manifest_file_number = 1;
430 431 432
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
433 434 435
    }
  }

436
  // versions need to be destroyed before table_cache since it can hold
437 438
  // references to table_cache.
  versions_.reset();
439
  mutex_.Unlock();
I
Igor Canadi 已提交
440 441 442
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
443

I
Igor Canadi 已提交
444
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
445 446 447
}

Status DBImpl::NewDB() {
448
  VersionEdit new_db;
449
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
450 451 452 453
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

  const std::string manifest = DescriptorFileName(dbname_, 1);
454
  unique_ptr<WritableFile> file;
I
Igor Canadi 已提交
455 456
  Status s = env_->NewWritableFile(
      manifest, &file, env_->OptimizeForManifestWrite(storage_options_));
J
jorlow@chromium.org 已提交
457 458 459
  if (!s.ok()) {
    return s;
  }
460
  file->SetPreallocationBlockSize(options_.manifest_preallocation_size);
J
jorlow@chromium.org 已提交
461
  {
462
    log::Writer log(std::move(file));
J
jorlow@chromium.org 已提交
463 464 465 466 467 468
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
469
    s = SetCurrentFile(env_, dbname_, 1, db_directory_.get());
J
jorlow@chromium.org 已提交
470 471 472 473 474 475 476 477 478 479
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
  if (s->ok() || options_.paranoid_checks) {
    // No change needed
  } else {
480
    Log(options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
481 482 483 484
    *s = Status::OK();
  }
}

485
const Status DBImpl::CreateArchivalDirectory() {
486
  if (options_.WAL_ttl_seconds > 0 || options_.WAL_size_limit_MB > 0) {
487
    std::string archivalPath = ArchivalDirectory(options_.wal_dir);
488 489 490 491 492
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

493
void DBImpl::PrintStatistics() {
494
  auto dbstats = options_.statistics.get();
495 496
  if (dbstats) {
    Log(options_.info_log,
497 498
        "STATISTCS:\n %s",
        dbstats->ToString().c_str());
499 500 501
  }
}

502
void DBImpl::MaybeDumpStats() {
H
Haobo Xu 已提交
503 504 505 506 507 508 509 510 511 512 513 514
  if (options_.stats_dump_period_sec == 0) return;

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
      options_.stats_dump_period_sec * 1000000
      <= now_micros) {
    // Multiple threads could race in here simultaneously.
    // However, the last one will update last_stats_dump_time_microsec_
    // atomically. We could see more than one dump during one dump
    // period in rare cases.
    last_stats_dump_time_microsec_ = now_micros;
515 516

    DBPropertyType cf_property_type = GetPropertyType("rocksdb.cfstats");
517
    DBPropertyType db_property_type = GetPropertyType("rocksdb.dbstats");
H
Haobo Xu 已提交
518
    std::string stats;
519 520 521
    {
      MutexLock l(&mutex_);
      for (auto cfd : *versions_->GetColumnFamilySet()) {
522 523
        cfd->internal_stats()->GetProperty(
            cf_property_type, "rocksdb.cfstats", &stats);
524 525
      }
      default_cf_internal_stats_->GetProperty(
526
          db_property_type, "rocksdb.dbstats", &stats);
527 528
    }
    Log(options_.info_log, "------- DUMPING STATS -------");
H
Haobo Xu 已提交
529
    Log(options_.info_log, "%s", stats.c_str());
530

531
    PrintStatistics();
532 533 534
  }
}

535
// Returns the list of live files in 'sst_live' and the list
K
kailiu 已提交
536
// of all files in the filesystem in 'candidate_files'.
I
Igor Canadi 已提交
537 538 539 540 541 542 543
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
//  options_.delete_obsolete_files_period_micros
// force = true -- force the full scan
void DBImpl::FindObsoleteFiles(DeletionState& deletion_state,
                               bool force,
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
544 545
  mutex_.AssertHeld();

546
  // if deletion is disabled, do nothing
547
  if (disable_delete_obsolete_files_ > 0) {
548 549 550
    return;
  }

551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566
  bool doing_the_full_scan = false;

  // logic for figurint out if we're doing the full scan
  if (no_full_scan) {
    doing_the_full_scan = false;
  } else if (force || options_.delete_obsolete_files_period_micros == 0) {
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
    if (delete_obsolete_files_last_run_ +
        options_.delete_obsolete_files_period_micros < now_micros) {
      doing_the_full_scan = true;
      delete_obsolete_files_last_run_ = now_micros;
    }
  }

I
Igor Canadi 已提交
567 568 569
  // get obsolete files
  versions_->GetObsoleteFiles(&deletion_state.sst_delete_files);

I
Igor Canadi 已提交
570 571
  // store the current filenum, lognum, etc
  deletion_state.manifest_file_number = versions_->ManifestFileNumber();
572 573
  deletion_state.pending_manifest_file_number =
      versions_->PendingManifestFileNumber();
574
  deletion_state.log_number = versions_->MinLogNumber();
I
Igor Canadi 已提交
575 576
  deletion_state.prev_log_number = versions_->PrevLogNumber();

577 578 579 580 581 582 583 584
  if (!doing_the_full_scan && !deletion_state.HaveSomethingToDelete()) {
    // avoid filling up sst_live if we're sure that we
    // are not going to do the full scan and that we don't have
    // anything to delete at the moment
    return;
  }

  // don't delete live files
585 586 587 588 589
  for (auto pair : pending_outputs_) {
    deletion_state.sst_live.emplace_back(pair.first, pair.second, 0);
  }
  /*  deletion_state.sst_live.insert(pending_outputs_.begin(),
                                   pending_outputs_.end());*/
I
Igor Canadi 已提交
590 591
  versions_->AddLiveFiles(&deletion_state.sst_live);

592
  if (doing_the_full_scan) {
593 594 595 596 597 598 599 600 601
    for (uint32_t path_id = 0; path_id < options_.db_paths.size(); path_id++) {
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
      env_->GetChildren(dbname_, &files);  // Ignore errors
      for (std::string file : files) {
        deletion_state.candidate_files.emplace_back(file, path_id);
      }
    }
602 603 604 605 606

    //Add log files in wal_dir
    if (options_.wal_dir != dbname_) {
      std::vector<std::string> log_files;
      env_->GetChildren(options_.wal_dir, &log_files); // Ignore errors
607 608 609
      for (std::string log_file : log_files) {
        deletion_state.candidate_files.emplace_back(log_file, 0);
      }
610
    }
611
  }
612 613
}

614 615 616 617 618 619 620 621 622 623 624 625 626
namespace {
bool CompareCandidateFile(const rocksdb::DBImpl::CandidateFileInfo& first,
                          const rocksdb::DBImpl::CandidateFileInfo& second) {
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
    return (first.path_id > first.path_id);
  }
}
};  // namespace

D
Dhruba Borthakur 已提交
627
// Diffs the files listed in filenames and those that do not
I
Igor Canadi 已提交
628
// belong to live files are posibly removed. Also, removes all the
629
// files in sst_delete_files and log_delete_files.
630
// It is not necessary to hold the mutex when invoking this method.
D
Dhruba Borthakur 已提交
631
void DBImpl::PurgeObsoleteFiles(DeletionState& state) {
632 633
  // we'd better have sth to delete
  assert(state.HaveSomethingToDelete());
634

I
Igor Canadi 已提交
635 636 637 638
  // this checks if FindObsoleteFiles() was run before. If not, don't do
  // PurgeObsoleteFiles(). If FindObsoleteFiles() was run, we need to also
  // run PurgeObsoleteFiles(), even if disable_delete_obsolete_files_ is true
  if (state.manifest_file_number == 0) {
I
Igor Canadi 已提交
639 640
    return;
  }
641

642
  // Now, convert live list to an unordered map, WITHOUT mutex held;
643
  // set is slow.
644 645 646 647
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
  for (FileDescriptor& fd : state.sst_live) {
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
648

K
kailiu 已提交
649 650 651 652 653 654 655
  auto& candidate_files = state.candidate_files;
  candidate_files.reserve(
      candidate_files.size() +
      state.sst_delete_files.size() +
      state.log_delete_files.size());
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
656
  for (auto file : state.sst_delete_files) {
657 658 659
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
660
    delete file;
I
Igor Canadi 已提交
661 662
  }

K
kailiu 已提交
663 664
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
665 666
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
667 668
    }
  }
669

K
kailiu 已提交
670
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
671
  // file twice
672
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
673 674
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
675

676 677
  std::vector<std::string> old_info_log_files;

678 679 680
  for (const auto& candidate_file : candidate_files) {
    std::string to_delete = candidate_file.file_name;
    uint32_t path_id = candidate_file.path_id;
K
kailiu 已提交
681 682 683 684 685 686
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
    if (!ParseFileName(to_delete, &number, &type)) {
      continue;
    }
J
jorlow@chromium.org 已提交
687

K
kailiu 已提交
688 689 690 691 692 693 694 695
    bool keep = true;
    switch (type) {
      case kLogFile:
        keep = ((number >= state.log_number) ||
                (number == state.prev_log_number));
        break;
      case kDescriptorFile:
        // Keep my manifest file, and any newer incarnations'
696
        // (can happen during manifest roll)
K
kailiu 已提交
697 698 699
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
700
        keep = (sst_live_map.find(number) != sst_live_map.end());
K
kailiu 已提交
701 702 703
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
704 705 706 707
        // be recorded in pending_outputs_, which is inserted into "live".
        // Also, SetCurrentFile creates a temp file when writing out new
        // manifest, which is equal to state.pending_manifest_file_number. We
        // should not delete that file
708
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
709
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
710 711 712 713
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
714
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
715
        }
K
kailiu 已提交
716 717 718 719 720 721 722 723 724 725 726 727 728
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

729
    std::string fname;
K
kailiu 已提交
730 731
    if (type == kTableFile) {
      // evict from cache
732
      TableCache::Evict(table_cache_.get(), number);
733 734 735 736
      fname = TableFileName(options_.db_paths, number, path_id);
    } else {
      fname =
          ((type == kLogFile) ? options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
737
    }
738

K
kailiu 已提交
739 740
    if (type == kLogFile &&
        (options_.WAL_ttl_seconds > 0 || options_.WAL_size_limit_MB > 0)) {
741
      auto archived_log_name = ArchivedLogFileName(options_.wal_dir, number);
742 743
      // The sync point below is used in (DBTest,TransactionLogIteratorRace)
      TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:1");
744
      Status s = env_->RenameFile(fname, archived_log_name);
745 746
      // The sync point below is used in (DBTest,TransactionLogIteratorRace)
      TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:2");
747 748 749
      Log(options_.info_log,
          "Move log file %s to %s -- %s\n",
          fname.c_str(), archived_log_name.c_str(), s.ToString().c_str());
K
kailiu 已提交
750 751
    } else {
      Status s = env_->DeleteFile(fname);
752 753
      Log(options_.info_log, "Delete %s type=%d #%" PRIu64 " -- %s\n",
          fname.c_str(), type, number, s.ToString().c_str());
J
jorlow@chromium.org 已提交
754 755
    }
  }
H
heyongqiang 已提交
756

757
  // Delete old info log files.
758
  size_t old_info_log_file_count = old_info_log_files.size();
K
Kai Liu 已提交
759 760
  // NOTE: Currently we only support log purge when options_.db_log_dir is
  // located in `dbname` directory.
761
  if (old_info_log_file_count >= options_.keep_log_file_num &&
K
Kai Liu 已提交
762
      options_.db_log_dir.empty()) {
763 764
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
    size_t end = old_info_log_file_count - options_.keep_log_file_num;
765
    for (unsigned int i = 0; i <= end; i++) {
766 767 768 769 770 771 772
      std::string& to_delete = old_info_log_files.at(i);
      Log(options_.info_log, "Delete info log file %s\n", to_delete.c_str());
      Status s = env_->DeleteFile(dbname_ + "/" + to_delete);
      if (!s.ok()) {
        Log(options_.info_log, "Delete info log file %s FAILED -- %s\n",
            to_delete.c_str(), s.ToString().c_str());
      }
H
heyongqiang 已提交
773 774
    }
  }
775
  PurgeObsoleteWALFiles();
I
Igor Canadi 已提交
776
  LogFlush(options_.info_log);
D
Dhruba Borthakur 已提交
777 778 779 780 781
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
  DeletionState deletion_state;
I
Igor Canadi 已提交
782
  FindObsoleteFiles(deletion_state, true);
783 784 785
  if (deletion_state.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(deletion_state);
  }
786 787
}

I
Igor Canadi 已提交
788
#ifndef ROCKSDB_LITE
789 790 791 792 793 794 795 796
// 1. Go through all archived files and
//    a. if ttl is enabled, delete outdated files
//    b. if archive size limit is enabled, delete empty files,
//        compute file number and size.
// 2. If size limit is enabled:
//    a. compute how many files should be deleted
//    b. get sorted non-empty archived logs
//    c. delete what should be deleted
797
void DBImpl::PurgeObsoleteWALFiles() {
798 799 800 801 802 803
  bool const ttl_enabled = options_.WAL_ttl_seconds > 0;
  bool const size_limit_enabled =  options_.WAL_size_limit_MB > 0;
  if (!ttl_enabled && !size_limit_enabled) {
    return;
  }

804 805
  int64_t current_time;
  Status s = env_->GetCurrentTime(&current_time);
806 807 808 809 810 811 812 813
  if (!s.ok()) {
    Log(options_.info_log, "Can't get current time: %s", s.ToString().c_str());
    assert(false);
    return;
  }
  uint64_t const now_seconds = static_cast<uint64_t>(current_time);
  uint64_t const time_to_check = (ttl_enabled && !size_limit_enabled) ?
    options_.WAL_ttl_seconds / 2 : default_interval_to_delete_obsolete_WAL_;
814

815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852
  if (purge_wal_files_last_run_ + time_to_check > now_seconds) {
    return;
  }

  purge_wal_files_last_run_ = now_seconds;

  std::string archival_dir = ArchivalDirectory(options_.wal_dir);
  std::vector<std::string> files;
  s = env_->GetChildren(archival_dir, &files);
  if (!s.ok()) {
    Log(options_.info_log, "Can't get archive files: %s", s.ToString().c_str());
    assert(false);
    return;
  }

  size_t log_files_num = 0;
  uint64_t log_file_size = 0;

  for (auto& f : files) {
    uint64_t number;
    FileType type;
    if (ParseFileName(f, &number, &type) && type == kLogFile) {
      std::string const file_path = archival_dir + "/" + f;
      if (ttl_enabled) {
        uint64_t file_m_time;
        Status const s = env_->GetFileModificationTime(file_path,
          &file_m_time);
        if (!s.ok()) {
          Log(options_.info_log, "Can't get file mod time: %s: %s",
              file_path.c_str(), s.ToString().c_str());
          continue;
        }
        if (now_seconds - file_m_time > options_.WAL_ttl_seconds) {
          Status const s = env_->DeleteFile(file_path);
          if (!s.ok()) {
            Log(options_.info_log, "Can't delete file: %s: %s",
                file_path.c_str(), s.ToString().c_str());
            continue;
I
Igor Canadi 已提交
853 854 855
          } else {
            MutexLock l(&read_first_record_cache_mutex_);
            read_first_record_cache_.erase(number);
856 857
          }
          continue;
858
        }
859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877
      }

      if (size_limit_enabled) {
        uint64_t file_size;
        Status const s = env_->GetFileSize(file_path, &file_size);
        if (!s.ok()) {
          Log(options_.info_log, "Can't get file size: %s: %s",
              file_path.c_str(), s.ToString().c_str());
          return;
        } else {
          if (file_size > 0) {
            log_file_size = std::max(log_file_size, file_size);
            ++log_files_num;
          } else {
            Status s = env_->DeleteFile(file_path);
            if (!s.ok()) {
              Log(options_.info_log, "Can't delete file: %s: %s",
                  file_path.c_str(), s.ToString().c_str());
              continue;
I
Igor Canadi 已提交
878 879 880
            } else {
              MutexLock l(&read_first_record_cache_mutex_);
              read_first_record_cache_.erase(number);
881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899
            }
          }
        }
      }
    }
  }

  if (0 == log_files_num || !size_limit_enabled) {
    return;
  }

  size_t const files_keep_num = options_.WAL_size_limit_MB *
    1024 * 1024 / log_file_size;
  if (log_files_num <= files_keep_num) {
    return;
  }

  size_t files_del_num = log_files_num - files_keep_num;
  VectorLogPtr archived_logs;
900
  GetSortedWalsOfType(archival_dir, archived_logs, kArchivedLogFile);
901 902 903 904 905 906 907 908 909 910 911 912 913 914

  if (files_del_num > archived_logs.size()) {
    Log(options_.info_log, "Trying to delete more archived log files than "
        "exist. Deleting all");
    files_del_num = archived_logs.size();
  }

  for (size_t i = 0; i < files_del_num; ++i) {
    std::string const file_path = archived_logs[i]->PathName();
    Status const s = DeleteFile(file_path);
    if (!s.ok()) {
      Log(options_.info_log, "Can't delete file: %s: %s",
          file_path.c_str(), s.ToString().c_str());
      continue;
I
Igor Canadi 已提交
915 916 917
    } else {
      MutexLock l(&read_first_record_cache_mutex_);
      read_first_record_cache_.erase(archived_logs[i]->LogNumber());
918 919
    }
  }
D
Dhruba Borthakur 已提交
920 921
}

922 923 924 925 926 927 928 929 930 931
namespace {
struct CompareLogByPointer {
  bool operator()(const unique_ptr<LogFile>& a, const unique_ptr<LogFile>& b) {
    LogFileImpl* a_impl = dynamic_cast<LogFileImpl*>(a.get());
    LogFileImpl* b_impl = dynamic_cast<LogFileImpl*>(b.get());
    return *a_impl < *b_impl;
  }
};
}

I
Igor Canadi 已提交
932 933 934 935 936 937 938 939 940 941 942 943 944
Status DBImpl::GetSortedWalsOfType(const std::string& path,
                                   VectorLogPtr& log_files,
                                   WalFileType log_type) {
  std::vector<std::string> all_files;
  const Status status = env_->GetChildren(path, &all_files);
  if (!status.ok()) {
    return status;
  }
  log_files.reserve(all_files.size());
  for (const auto& f : all_files) {
    uint64_t number;
    FileType type;
    if (ParseFileName(f, &number, &type) && type == kLogFile) {
I
Igor Canadi 已提交
945 946
      SequenceNumber sequence;
      Status s = ReadFirstRecord(log_type, number, &sequence);
I
Igor Canadi 已提交
947 948 949
      if (!s.ok()) {
        return s;
      }
I
Igor Canadi 已提交
950 951 952 953
      if (sequence == 0) {
        // empty file
        continue;
      }
I
Igor Canadi 已提交
954

955 956 957 958 959 960
      // Reproduce the race condition where a log file is moved
      // to archived dir, between these two sync points, used in
      // (DBTest,TransactionLogIteratorRace)
      TEST_SYNC_POINT("DBImpl::GetSortedWalsOfType:1");
      TEST_SYNC_POINT("DBImpl::GetSortedWalsOfType:2");

I
Igor Canadi 已提交
961 962
      uint64_t size_bytes;
      s = env_->GetFileSize(LogFileName(path, number), &size_bytes);
963 964 965 966 967
      // re-try in case the alive log file has been moved to archive.
      if (!s.ok() && log_type == kAliveLogFile &&
          env_->FileExists(ArchivedLogFileName(path, number))) {
        s = env_->GetFileSize(ArchivedLogFileName(path, number), &size_bytes);
      }
I
Igor Canadi 已提交
968 969 970 971 972
      if (!s.ok()) {
        return s;
      }

      log_files.push_back(std::move(unique_ptr<LogFile>(
I
Igor Canadi 已提交
973
          new LogFileImpl(number, log_type, sequence, size_bytes))));
I
Igor Canadi 已提交
974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997
    }
  }
  CompareLogByPointer compare_log_files;
  std::sort(log_files.begin(), log_files.end(), compare_log_files);
  return status;
}

Status DBImpl::RetainProbableWalFiles(VectorLogPtr& all_logs,
                                      const SequenceNumber target) {
  int64_t start = 0;  // signed to avoid overflow when target is < first file.
  int64_t end = static_cast<int64_t>(all_logs.size()) - 1;
  // Binary Search. avoid opening all files.
  while (end >= start) {
    int64_t mid = start + (end - start) / 2;  // Avoid overflow.
    SequenceNumber current_seq_num = all_logs.at(mid)->StartSequence();
    if (current_seq_num == target) {
      end = mid;
      break;
    } else if (current_seq_num < target) {
      start = mid + 1;
    } else {
      end = mid - 1;
    }
  }
I
Igor Canadi 已提交
998 999
  // end could be -ve.
  size_t start_index = std::max(static_cast<int64_t>(0), end);
I
Igor Canadi 已提交
1000 1001 1002 1003 1004 1005
  // The last wal file is always included
  all_logs.erase(all_logs.begin(), all_logs.begin() + start_index);
  return Status::OK();
}

Status DBImpl::ReadFirstRecord(const WalFileType type, const uint64_t number,
I
Igor Canadi 已提交
1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018
                               SequenceNumber* sequence) {
  if (type != kAliveLogFile && type != kArchivedLogFile) {
    return Status::NotSupported("File Type Not Known " + std::to_string(type));
  }
  {
    MutexLock l(&read_first_record_cache_mutex_);
    auto itr = read_first_record_cache_.find(number);
    if (itr != read_first_record_cache_.end()) {
      *sequence = itr->second;
      return Status::OK();
    }
  }
  Status s;
I
Igor Canadi 已提交
1019 1020
  if (type == kAliveLogFile) {
    std::string fname = LogFileName(options_.wal_dir, number);
I
Igor Canadi 已提交
1021 1022 1023 1024
    s = ReadFirstLine(fname, sequence);
    if (env_->FileExists(fname) && !s.ok()) {
      // return any error that is not caused by non-existing file
      return s;
I
Igor Canadi 已提交
1025
    }
I
Igor Canadi 已提交
1026
  }
I
Igor Canadi 已提交
1027

I
Igor Canadi 已提交
1028
  if (type == kArchivedLogFile || !s.ok()) {
I
Igor Canadi 已提交
1029 1030
    //  check if the file got moved to archive.
    std::string archived_file = ArchivedLogFileName(options_.wal_dir, number);
I
Igor Canadi 已提交
1031
    s = ReadFirstLine(archived_file, sequence);
I
Igor Canadi 已提交
1032
  }
I
Igor Canadi 已提交
1033 1034 1035 1036 1037 1038

  if (s.ok() && *sequence != 0) {
    MutexLock l(&read_first_record_cache_mutex_);
    read_first_record_cache_.insert({number, *sequence});
  }
  return s;
I
Igor Canadi 已提交
1039 1040
}

I
Igor Canadi 已提交
1041 1042
// the function returns status.ok() and sequence == 0 if the file exists, but is
// empty
I
Igor Canadi 已提交
1043
Status DBImpl::ReadFirstLine(const std::string& fname,
I
Igor Canadi 已提交
1044
                             SequenceNumber* sequence) {
I
Igor Canadi 已提交
1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
    Logger* info_log;
    const char* fname;

    Status* status;
    bool ignore_error;  // true if options_.paranoid_checks==false
    virtual void Corruption(size_t bytes, const Status& s) {
      Log(info_log, "%s%s: dropping %d bytes; %s",
          (this->ignore_error ? "(ignoring error) " : ""), fname,
          static_cast<int>(bytes), s.ToString().c_str());
      if (this->status->ok()) {
        // only keep the first error
        *this->status = s;
      }
    }
  };

  unique_ptr<SequentialFile> file;
  Status status = env_->NewSequentialFile(fname, &file, storage_options_);

  if (!status.ok()) {
    return status;
  }

  LogReporter reporter;
  reporter.env = env_;
  reporter.info_log = options_.info_log.get();
  reporter.fname = fname.c_str();
  reporter.status = &status;
  reporter.ignore_error = !options_.paranoid_checks;
  log::Reader reader(std::move(file), &reporter, true /*checksum*/,
                     0 /*initial_offset*/);
  std::string scratch;
  Slice record;

  if (reader.ReadRecord(&record, &scratch) &&
      (status.ok() || !options_.paranoid_checks)) {
    if (record.size() < 12) {
      reporter.Corruption(record.size(),
                          Status::Corruption("log record too small"));
      // TODO read record's till the first no corrupt entry?
    } else {
I
Igor Canadi 已提交
1088 1089 1090
      WriteBatch batch;
      WriteBatchInternal::SetContents(&batch, record);
      *sequence = WriteBatchInternal::Sequence(&batch);
I
Igor Canadi 已提交
1091 1092 1093 1094
      return Status::OK();
    }
  }

I
Igor Canadi 已提交
1095 1096 1097
  // ReadRecord returns false on EOF, which means that the log file is empty. we
  // return status.ok() in that case and set sequence number to 0
  *sequence = 0;
I
Igor Canadi 已提交
1098 1099 1100 1101 1102
  return status;
}

#endif  // ROCKSDB_LITE

1103
Status DBImpl::Recover(
1104 1105
    const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
    bool error_if_log_file_exist) {
J
jorlow@chromium.org 已提交
1106 1107
  mutex_.AssertHeld();

1108
  bool is_new_db = false;
1109
  assert(db_lock_ == nullptr);
I
Igor Canadi 已提交
1110
  if (!read_only) {
1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122
    // We call CreateDirIfMissing() as the directory may already exist (if we
    // are reopening a DB), when this happens we don't want creating the
    // directory to cause an error. However, we need to check if creating the
    // directory fails or else we may get an obscure message about the lock
    // file not existing. One real-world example of this occurring is if
    // env->CreateDirIfMissing() doesn't create intermediate directories, e.g.
    // when dbname_ is "dir/db" but when "dir" doesn't exist.
    Status s = env_->CreateDirIfMissing(dbname_);
    if (!s.ok()) {
      return s;
    }

1123 1124
    for (auto& db_path : options_.db_paths) {
      s = env_->CreateDirIfMissing(db_path.path);
1125 1126 1127 1128 1129
      if (!s.ok()) {
        return s;
      }
    }

1130 1131 1132 1133 1134
    s = env_->NewDirectory(dbname_, &db_directory_);
    if (!s.ok()) {
      return s;
    }

1135
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
1136 1137 1138
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
1139

1140 1141
    if (!env_->FileExists(CurrentFileName(dbname_))) {
      if (options_.create_if_missing) {
1142
        // TODO: add merge_operator name check
1143
        s = NewDB();
1144
        is_new_db = true;
1145 1146 1147 1148 1149 1150
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
1151 1152
      }
    } else {
1153 1154 1155 1156
      if (options_.error_if_exists) {
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
1157
    }
M
Mayank Agarwal 已提交
1158 1159 1160 1161 1162 1163 1164
    // Check for the IDENTITY file and create it if not there
    if (!env_->FileExists(IdentityFileName(dbname_))) {
      s = SetIdentityFile(env_, dbname_);
      if (!s.ok()) {
        return s;
      }
    }
J
jorlow@chromium.org 已提交
1165 1166
  }

1167
  Status s = versions_->Recover(column_families, read_only);
I
Igor Canadi 已提交
1168 1169 1170
  if (options_.paranoid_checks && s.ok()) {
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
1171 1172
  if (s.ok()) {
    SequenceNumber max_sequence(0);
1173 1174
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
1175
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
1176 1177
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
1178 1179 1180 1181 1182 1183 1184

    // Recover from all newer log files than the ones named in the
    // descriptor (new log files may have been added by the previous
    // incarnation without registering them in the descriptor).
    //
    // Note that PrevLogNumber() is no longer used, but we pay
    // attention to it in case we are recovering a database
1185
    // produced by an older version of rocksdb.
1186
    const uint64_t min_log = versions_->MinLogNumber();
1187 1188
    const uint64_t prev_log = versions_->PrevLogNumber();
    std::vector<std::string> filenames;
1189
    s = env_->GetChildren(options_.wal_dir, &filenames);
1190 1191
    if (!s.ok()) {
      return s;
1192
    }
K
kailiu 已提交
1193

1194 1195
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
1196 1197
      uint64_t number;
      FileType type;
1198 1199 1200 1201 1202 1203 1204 1205 1206
      if (ParseFileName(filenames[i], &number, &type) && type == kLogFile) {
        if (is_new_db) {
          return Status::Corruption(
              "While creating a new Db, wal_dir contains "
              "existing log file: ",
              filenames[i]);
        } else if ((number >= min_log) || (number == prev_log)) {
          logs.push_back(number);
        }
1207
      }
J
jorlow@chromium.org 已提交
1208
    }
1209

H
heyongqiang 已提交
1210 1211 1212 1213 1214 1215
    if (logs.size() > 0 && error_if_log_file_exist) {
      return Status::Corruption(""
          "The db was opened in readonly mode with error_if_log_file_exist"
          "flag but a log file already exists");
    }

1216 1217
    // Recover in the order in which the logs were generated
    std::sort(logs.begin(), logs.end());
K
kailiu 已提交
1218
    for (const auto& log : logs) {
1219 1220 1221
      // The previous incarnation may not have written any MANIFEST
      // records after allocating this log number.  So we manually
      // update the file number allocation counter in VersionSet.
K
kailiu 已提交
1222
      versions_->MarkFileNumberUsed(log);
K
Kai Liu 已提交
1223
      s = RecoverLogFile(log, &max_sequence, read_only);
1224
    }
L
Lei Jin 已提交
1225
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
1226 1227
  }

I
Igor Canadi 已提交
1228 1229 1230 1231 1232
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    max_total_in_memory_state_ += cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
  }

J
jorlow@chromium.org 已提交
1233 1234 1235
  return s;
}

I
Igor Canadi 已提交
1236 1237
Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
                              bool read_only) {
J
jorlow@chromium.org 已提交
1238 1239
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
1240
    Logger* info_log;
J
jorlow@chromium.org 已提交
1241
    const char* fname;
1242 1243
    Status* status;  // nullptr if options_.paranoid_checks==false or
                     //            options_.skip_log_error_on_recovery==true
J
jorlow@chromium.org 已提交
1244
    virtual void Corruption(size_t bytes, const Status& s) {
1245
      Log(info_log, "%s%s: dropping %d bytes; %s",
1246
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
1247
          fname, static_cast<int>(bytes), s.ToString().c_str());
1248
      if (this->status != nullptr && this->status->ok()) *this->status = s;
J
jorlow@chromium.org 已提交
1249 1250 1251 1252 1253
    }
  };

  mutex_.AssertHeld();

1254
  std::unordered_map<int, VersionEdit> version_edits;
1255
  // no need to refcount because iteration is under mutex
1256 1257
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1258 1259
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1260
  }
I
Igor Canadi 已提交
1261

J
jorlow@chromium.org 已提交
1262
  // Open the log file
1263
  std::string fname = LogFileName(options_.wal_dir, log_number);
1264
  unique_ptr<SequentialFile> file;
1265
  Status status = env_->NewSequentialFile(fname, &file, storage_options_);
J
jorlow@chromium.org 已提交
1266 1267 1268 1269 1270 1271 1272 1273
  if (!status.ok()) {
    MaybeIgnoreError(&status);
    return status;
  }

  // Create the log reader.
  LogReporter reporter;
  reporter.env = env_;
1274
  reporter.info_log = options_.info_log.get();
J
jorlow@chromium.org 已提交
1275
  reporter.fname = fname.c_str();
1276 1277
  reporter.status = (options_.paranoid_checks &&
                     !options_.skip_log_error_on_recovery ? &status : nullptr);
J
jorlow@chromium.org 已提交
1278 1279 1280 1281
  // We intentially make log::Reader do checksumming even if
  // paranoid_checks==false so that corruptions cause entire commits
  // to be skipped instead of propagating bad information (like overly
  // large sequence numbers).
1282
  log::Reader reader(std::move(file), &reporter, true/*checksum*/,
1283
                     0/*initial_offset*/);
1284
  Log(options_.info_log, "Recovering log #%" PRIu64 "", log_number);
J
jorlow@chromium.org 已提交
1285 1286 1287 1288 1289

  // Read all the records and add to a memtable
  std::string scratch;
  Slice record;
  WriteBatch batch;
I
Igor Canadi 已提交
1290
  while (reader.ReadRecord(&record, &scratch)) {
J
jorlow@chromium.org 已提交
1291 1292 1293 1294 1295 1296 1297
    if (record.size() < 12) {
      reporter.Corruption(
          record.size(), Status::Corruption("log record too small"));
      continue;
    }
    WriteBatchInternal::SetContents(&batch, record);

1298
    status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
1299
        &batch, column_family_memtables_.get(), true, log_number);
1300

J
jorlow@chromium.org 已提交
1301 1302
    MaybeIgnoreError(&status);
    if (!status.ok()) {
I
Igor Canadi 已提交
1303
      return status;
J
jorlow@chromium.org 已提交
1304 1305 1306 1307 1308 1309 1310 1311
    }
    const SequenceNumber last_seq =
        WriteBatchInternal::Sequence(&batch) +
        WriteBatchInternal::Count(&batch) - 1;
    if (last_seq > *max_sequence) {
      *max_sequence = last_seq;
    }

1312
    if (!read_only) {
1313 1314
      // no need to refcount since client still doesn't have access
      // to the DB and can not drop column families while we iterate
1315
      for (auto cfd : *versions_->GetColumnFamilySet()) {
1316
        if (cfd->mem()->ShouldFlush()) {
1317
          // If this asserts, it means that InsertInto failed in
1318 1319
          // filtering updates to already-flushed column families
          assert(cfd->GetLogNumber() <= log_number);
1320
          auto iter = version_edits.find(cfd->GetID());
1321 1322
          assert(iter != version_edits.end());
          VersionEdit* edit = &iter->second;
1323
          status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
1324 1325 1326 1327 1328 1329 1330 1331
          // we still want to clear the memtable, even if the recovery failed
          cfd->CreateNewMemtable();
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
          }
        }
J
jorlow@chromium.org 已提交
1332 1333 1334 1335
      }
    }
  }

1336 1337 1338 1339
  if (versions_->LastSequence() < *max_sequence) {
    versions_->SetLastSequence(*max_sequence);
  }

1340
  if (!read_only) {
1341 1342
    // no need to refcount since client still doesn't have access
    // to the DB and can not drop column families while we iterate
1343
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1344
      auto iter = version_edits.find(cfd->GetID());
1345 1346 1347
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

1348 1349 1350 1351
      if (cfd->GetLogNumber() > log_number) {
        // Column family cfd has already flushed the data
        // from log_number. Memtable has to be empty because
        // we filter the updates based on log_number
1352
        // (in WriteBatch::InsertInto)
1353 1354 1355 1356 1357 1358 1359
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1360
        status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
1361
      }
1362 1363 1364 1365 1366
      // we still want to clear the memtable, even if the recovery failed
      cfd->CreateNewMemtable();
      if (!status.ok()) {
        return status;
      }
J
jorlow@chromium.org 已提交
1367

1368 1369 1370 1371 1372 1373 1374
      // write MANIFEST with update
      // writing log number in the manifest means that any log file
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
      // Since we already recovered log_number, we want all logs
      // with numbers `<= log_number` (includes this one) to be ignored
      edit->SetLogNumber(log_number + 1);
1375 1376 1377
      // we must mark the next log number as used, even though it's
      // not actually used. that is because VersionSet assumes
      // VersionSet::next_file_number_ always to be strictly greater than any
I
Igor Canadi 已提交
1378
      // log number
1379
      versions_->MarkFileNumberUsed(log_number + 1);
1380 1381 1382 1383 1384
      status = versions_->LogAndApply(cfd, edit, &mutex_);
      if (!status.ok()) {
        return status;
      }
    }
1385
  }
I
Igor Canadi 已提交
1386

J
jorlow@chromium.org 已提交
1387 1388 1389
  return status;
}

1390 1391
Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
                                           VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1392
  mutex_.AssertHeld();
1393
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1394
  FileMetaData meta;
1395 1396
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
  pending_outputs_[meta.fd.GetNumber()] = 0;  // path 0 for level 0 file.
1397
  Iterator* iter = mem->NewIterator(ReadOptions(), true);
1398 1399 1400
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
    mem->GetFirstSequenceNumber();
1401 1402
  Log(options_.info_log, "[%s] Level-0 table #%" PRIu64 ": started",
      cfd->GetName().c_str(), meta.fd.GetNumber());
1403 1404 1405 1406

  Status s;
  {
    mutex_.Unlock();
I
Igor Canadi 已提交
1407
    s = BuildTable(dbname_, env_, *cfd->options(), storage_options_,
1408
                   cfd->table_cache(), iter, &meta, cfd->internal_comparator(),
1409
                   newest_snapshot, earliest_seqno_in_memtable,
L
Lei Jin 已提交
1410
                   GetCompressionFlush(*cfd->options()), Env::IO_HIGH);
I
Igor Canadi 已提交
1411
    LogFlush(options_.info_log);
1412 1413 1414
    mutex_.Lock();
  }

1415 1416 1417 1418
  Log(options_.info_log,
      "[%s] Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
      cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
      s.ToString().c_str());
J
jorlow@chromium.org 已提交
1419
  delete iter;
1420

1421
  pending_outputs_.erase(meta.fd.GetNumber());
1422 1423 1424 1425

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1426
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1427 1428 1429
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1430 1431
  }

L
Lei Jin 已提交
1432
  InternalStats::CompactionStats stats(1);
1433
  stats.micros = env_->NowMicros() - start_micros;
1434
  stats.bytes_written = meta.fd.GetFileSize();
M
Mark Callaghan 已提交
1435
  stats.files_out_levelnp1 = 1;
1436
  cfd->internal_stats()->AddCompactionStats(level, stats);
1437 1438
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1439
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1440 1441 1442
  return s;
}

1443
Status DBImpl::WriteLevel0Table(ColumnFamilyData* cfd,
1444
                                autovector<MemTable*>& mems, VersionEdit* edit,
1445
                                uint64_t* filenumber, LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
1446
  mutex_.AssertHeld();
1447 1448
  const uint64_t start_micros = env_->NowMicros();
  FileMetaData meta;
1449 1450

  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
1451
  *filenumber = meta.fd.GetNumber();
1452
  pending_outputs_[meta.fd.GetNumber()] = 0;  // path 0 for level 0 file.
1453

1454 1455
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
1456
    mems[0]->GetFirstSequenceNumber();
1457
  Version* base = cfd->current();
1458
  base->Ref();          // it is likely that we do not need this reference
1459 1460 1461
  Status s;
  {
    mutex_.Unlock();
H
Haobo Xu 已提交
1462
    log_buffer->FlushBufferToLog();
K
Kai Liu 已提交
1463
    std::vector<Iterator*> memtables;
1464
    for (MemTable* m : mems) {
1465 1466 1467
      Log(options_.info_log,
          "[%s] Flushing memtable with next log file: %" PRIu64 "\n",
          cfd->GetName().c_str(), m->GetNextLogNumber());
1468
      memtables.push_back(m->NewIterator(ReadOptions(), true));
1469
    }
I
Igor Canadi 已提交
1470
    Iterator* iter = NewMergingIterator(&cfd->internal_comparator(),
1471
                                        &memtables[0], memtables.size());
1472 1473
    Log(options_.info_log, "[%s] Level-0 flush table #%" PRIu64 ": started",
        cfd->GetName().c_str(), meta.fd.GetNumber());
1474

I
Igor Canadi 已提交
1475
    s = BuildTable(dbname_, env_, *cfd->options(), storage_options_,
1476
                   cfd->table_cache(), iter, &meta, cfd->internal_comparator(),
1477
                   newest_snapshot, earliest_seqno_in_memtable,
L
Lei Jin 已提交
1478
                   GetCompressionFlush(*cfd->options()), Env::IO_HIGH);
I
Igor Canadi 已提交
1479
    LogFlush(options_.info_log);
1480
    delete iter;
1481 1482
    Log(options_.info_log,
        "[%s] Level-0 flush table #%" PRIu64 ": %" PRIu64 " bytes %s",
F
Feng Zhu 已提交
1483
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
1484
        s.ToString().c_str());
I
Igor Canadi 已提交
1485

1486 1487 1488
    if (!options_.disableDataSync) {
      db_directory_->Fsync();
    }
1489 1490
    mutex_.Lock();
  }
1491 1492
  base->Unref();

1493
  // re-acquire the most current version
1494
  base = cfd->current();
1495 1496 1497 1498 1499 1500

  // There could be multiple threads writing to its own level-0 file.
  // The pending_outputs cannot be cleared here, otherwise this newly
  // created file might not be considered as a live-file by another
  // compaction thread that is concurrently deleting obselete files.
  // The pending_outputs can be cleared only after the new version is
A
Abhishek Kona 已提交
1501
  // committed so that other threads can recognize this file as a
1502 1503 1504 1505 1506 1507
  // valid one.
  // pending_outputs_.erase(meta.number);

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1508
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1509 1510 1511 1512 1513 1514
    const Slice min_user_key = meta.smallest.user_key();
    const Slice max_user_key = meta.largest.user_key();
    // if we have more than 1 background thread, then we cannot
    // insert files directly into higher levels because some other
    // threads could be concurrently producing compacted files for
    // that key range.
1515
    if (base != nullptr && options_.max_background_compactions <= 1 &&
1516
        cfd->options()->compaction_style == kCompactionStyleLevel) {
1517 1518
      level = base->PickLevelForMemTableOutput(min_user_key, max_user_key);
    }
1519 1520 1521
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1522 1523
  }

L
Lei Jin 已提交
1524
  InternalStats::CompactionStats stats(1);
1525
  stats.micros = env_->NowMicros() - start_micros;
1526
  stats.bytes_written = meta.fd.GetFileSize();
1527
  cfd->internal_stats()->AddCompactionStats(level, stats);
1528 1529
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1530
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
1531 1532 1533
  return s;
}

1534 1535
Status DBImpl::FlushMemTableToOutputFile(ColumnFamilyData* cfd,
                                         bool* madeProgress,
H
Haobo Xu 已提交
1536 1537
                                         DeletionState& deletion_state,
                                         LogBuffer* log_buffer) {
1538
  mutex_.AssertHeld();
1539
  assert(cfd->imm()->size() != 0);
1540
  assert(cfd->imm()->IsFlushPending());
1541 1542 1543

  // Save the contents of the earliest memtable as a new Table
  uint64_t file_number;
K
Kai Liu 已提交
1544
  autovector<MemTable*> mems;
1545
  cfd->imm()->PickMemtablesToFlush(&mems);
1546
  if (mems.empty()) {
I
Igor Canadi 已提交
1547 1548
    LogToBuffer(log_buffer, "[%s] Nothing in memtable to flush",
                cfd->GetName().c_str());
L
Lei Jin 已提交
1549
    return Status::OK();
1550 1551 1552
  }

  // record the logfile_number_ before we release the mutex
1553 1554 1555
  // entries mems are (implicitly) sorted in ascending order by their created
  // time. We will use the first memtable's `edit` to keep the meta info for
  // this flush.
1556
  MemTable* m = mems[0];
1557 1558
  VersionEdit* edit = m->GetEdits();
  edit->SetPrevLogNumber(0);
1559 1560
  // SetLogNumber(log_num) indicates logs with number smaller than log_num
  // will no longer be picked up for recovery.
1561 1562
  edit->SetLogNumber(mems.back()->GetNextLogNumber());
  edit->SetColumnFamily(cfd->GetID());
1563

1564
  // This will release and re-acquire the mutex.
1565
  Status s = WriteLevel0Table(cfd, mems, edit, &file_number, log_buffer);
1566

1567
  if (s.ok() && shutting_down_.Acquire_Load() && cfd->IsDropped()) {
L
Lei Jin 已提交
1568
    s = Status::ShutdownInProgress(
I
Igor Canadi 已提交
1569
        "Database shutdown or Column family drop during flush");
1570
  }
J
jorlow@chromium.org 已提交
1571

L
Lei Jin 已提交
1572
  if (!s.ok()) {
1573
    cfd->imm()->RollbackMemtableFlush(mems, file_number, &pending_outputs_);
1574 1575
  } else {
    // Replace immutable memtable with the generated Table
1576 1577
    s = cfd->imm()->InstallMemtableFlushResults(
        cfd, mems, versions_.get(), &mutex_, options_.info_log.get(),
1578
        file_number, &pending_outputs_, &deletion_state.memtables_to_free,
1579
        db_directory_.get(), log_buffer);
1580
  }
J
jorlow@chromium.org 已提交
1581 1582

  if (s.ok()) {
1583
    InstallSuperVersion(cfd, deletion_state);
1584 1585 1586
    if (madeProgress) {
      *madeProgress = 1;
    }
I
Igor Canadi 已提交
1587 1588 1589
    Version::LevelSummaryStorage tmp;
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
                cfd->current()->LevelSummary(&tmp));
1590

1591
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1592
      // add to deletion state
1593
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1594 1595 1596 1597
             alive_log_files_.begin()->number < versions_->MinLogNumber()) {
        const auto& earliest = *alive_log_files_.begin();
        deletion_state.log_delete_files.push_back(earliest.number);
        total_log_size_ -= earliest.size;
1598 1599
        alive_log_files_.pop_front();
      }
1600
    }
J
jorlow@chromium.org 已提交
1601
  }
1602 1603 1604 1605 1606 1607 1608

  if (!s.ok() && !s.IsShutdownInProgress() && options_.paranoid_checks &&
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1609
  RecordFlushIOStats();
J
jorlow@chromium.org 已提交
1610 1611 1612
  return s;
}

1613
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1614
                            const Slice* begin, const Slice* end,
1615 1616 1617 1618 1619 1620
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
  if (target_path_id >= options_.db_paths.size()) {
    return Status::InvalidArgument("Invalid target path ID");
  }

1621 1622
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1623 1624

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1625 1626 1627 1628 1629
  if (!s.ok()) {
    LogFlush(options_.info_log);
    return s;
  }

I
Igor Canadi 已提交
1630
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1631 1632
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1633 1634
    Version* base = cfd->current();
    for (int level = 1; level < cfd->NumberLevels(); level++) {
G
Gabor Cselle 已提交
1635 1636 1637 1638 1639
      if (base->OverlapInLevel(level, begin, end)) {
        max_level_with_files = level;
      }
    }
  }
1640 1641 1642
  for (int level = 0; level <= max_level_with_files; level++) {
    // in case the compaction is unversal or if we're compacting the
    // bottom-most level, the output level will be the same as input one
I
Igor Canadi 已提交
1643
    if (cfd->options()->compaction_style == kCompactionStyleUniversal ||
I
Igor Canadi 已提交
1644
        cfd->options()->compaction_style == kCompactionStyleFIFO ||
1645
        level == max_level_with_files) {
1646
      s = RunManualCompaction(cfd, level, level, target_path_id, begin, end);
1647
    } else {
1648 1649
      s = RunManualCompaction(cfd, level, level + 1, target_path_id, begin,
                              end);
L
Lei Jin 已提交
1650 1651 1652 1653
    }
    if (!s.ok()) {
      LogFlush(options_.info_log);
      return s;
1654
    }
G
Gabor Cselle 已提交
1655
  }
1656 1657

  if (reduce_level) {
I
Igor Canadi 已提交
1658
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1659
  }
I
Igor Canadi 已提交
1660
  LogFlush(options_.info_log);
L
Lei Jin 已提交
1661 1662

  return s;
1663 1664 1665
}

// return the same level if it cannot be moved
I
Igor Canadi 已提交
1666
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd, int level) {
1667
  mutex_.AssertHeld();
I
Igor Canadi 已提交
1668
  Version* current = cfd->current();
1669
  int minimum_level = level;
1670
  for (int i = level - 1; i > 0; --i) {
1671
    // stop if level i is not empty
1672
    if (current->NumLevelFiles(i) > 0) break;
1673
    // stop if level i is too small (cannot fit the level files)
I
Igor Canadi 已提交
1674
    if (cfd->compaction_picker()->MaxBytesForLevel(i) <
1675 1676 1677
        current->NumLevelBytes(level)) {
      break;
    }
1678 1679 1680 1681 1682 1683

    minimum_level = i;
  }
  return minimum_level;
}

I
Igor Canadi 已提交
1684 1685
Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
  assert(level < cfd->NumberLevels());
1686

I
Igor Canadi 已提交
1687
  SuperVersion* superversion_to_free = nullptr;
1688
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1689 1690

  mutex_.Lock();
1691 1692 1693

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1694
    mutex_.Unlock();
1695
    Log(options_.info_log, "ReFitLevel: another thread is refitting");
I
Igor Canadi 已提交
1696
    delete new_superversion;
L
Lei Jin 已提交
1697
    return Status::NotSupported("another thread is refitting");
1698 1699 1700 1701 1702
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1703
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1704
    Log(options_.info_log,
1705 1706
        "RefitLevel: waiting for background threads to stop: %d %d",
        bg_compaction_scheduled_, bg_flush_scheduled_);
1707 1708 1709 1710
    bg_cv_.Wait();
  }

  // move to a smaller level
1711 1712
  int to_level = target_level;
  if (target_level < 0) {
I
Igor Canadi 已提交
1713
    to_level = FindMinimumEmptyLevelFitting(cfd, level);
1714
  }
1715 1716 1717

  assert(to_level <= level);

L
Lei Jin 已提交
1718
  Status status;
1719
  if (to_level < level) {
I
Igor Canadi 已提交
1720
    Log(options_.info_log, "[%s] Before refitting:\n%s", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1721
        cfd->current()->DebugString().data());
1722

1723
    VersionEdit edit;
I
Igor Canadi 已提交
1724 1725
    edit.SetColumnFamily(cfd->GetID());
    for (const auto& f : cfd->current()->files_[level]) {
1726
      edit.DeleteFile(level, f->fd.GetNumber());
1727 1728 1729
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1730
    }
I
Igor Canadi 已提交
1731 1732
    Log(options_.info_log, "[%s] Apply version edit:\n%s",
        cfd->GetName().c_str(), edit.DebugString().data());
1733

I
Igor Canadi 已提交
1734
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
1735
    superversion_to_free = cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
1736
    new_superversion = nullptr;
1737

I
Igor Canadi 已提交
1738 1739
    Log(options_.info_log, "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
        status.ToString().data());
1740 1741

    if (status.ok()) {
I
Igor Canadi 已提交
1742 1743
      Log(options_.info_log, "[%s] After refitting:\n%s",
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1744 1745 1746 1747 1748
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1749 1750 1751 1752

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1753
  return status;
G
Gabor Cselle 已提交
1754 1755
}

1756 1757 1758
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1759 1760
}

1761 1762 1763
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->max_mem_compaction_level;
1764 1765
}

1766 1767 1768
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->level0_stop_writes_trigger;
1769 1770
}

1771
Status DBImpl::Flush(const FlushOptions& options,
1772 1773 1774
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return FlushMemTable(cfh->cfd(), options);
H
heyongqiang 已提交
1775 1776
}

1777
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1778 1779 1780
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1781
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1782 1783
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1784
  assert(input_level >= 0);
1785

G
Gabor Cselle 已提交
1786 1787
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1788
  ManualCompaction manual;
I
Igor Canadi 已提交
1789
  manual.cfd = cfd;
1790 1791
  manual.input_level = input_level;
  manual.output_level = output_level;
1792
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
1793
  manual.done = false;
1794
  manual.in_progress = false;
1795 1796 1797
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
I
Igor Canadi 已提交
1798 1799
      cfd->options()->compaction_style == kCompactionStyleUniversal ||
      cfd->options()->compaction_style == kCompactionStyleFIFO) {
1800
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1801 1802 1803 1804
  } else {
    begin_storage = InternalKey(*begin, kMaxSequenceNumber, kValueTypeForSeek);
    manual.begin = &begin_storage;
  }
1805
  if (end == nullptr ||
I
Igor Canadi 已提交
1806 1807
      cfd->options()->compaction_style == kCompactionStyleUniversal ||
      cfd->options()->compaction_style == kCompactionStyleFIFO) {
1808
    manual.end = nullptr;
G
Gabor Cselle 已提交
1809 1810 1811 1812 1813 1814
  } else {
    end_storage = InternalKey(*end, 0, static_cast<ValueType>(0));
    manual.end = &end_storage;
  }

  MutexLock l(&mutex_);
1815

1816 1817 1818 1819 1820 1821
  // When a manual compaction arrives, temporarily disable scheduling of
  // non-manual compactions and wait until the number of scheduled compaction
  // jobs drops to zero. This is needed to ensure that this manual compaction
  // can compact any range of keys/files.
  //
  // bg_manual_only_ is non-zero when at least one thread is inside
1822
  // RunManualCompaction(), i.e. during that time no other compaction will
1823 1824 1825
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1826
  // RunManualCompaction() from getting to the second while loop below.
1827 1828 1829 1830 1831 1832
  // However, only one of them will actually schedule compaction, while
  // others will wait on a condition variable until it completes.

  ++bg_manual_only_;
  while (bg_compaction_scheduled_ > 0) {
    Log(options_.info_log,
I
Igor Canadi 已提交
1833 1834 1835
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1836 1837
    bg_cv_.Wait();
  }
1838

I
Igor Canadi 已提交
1839 1840
  Log(options_.info_log, "[%s] Manual compaction starting",
      cfd->GetName().c_str());
1841

1842 1843 1844 1845
  while (!manual.done && !shutting_down_.Acquire_Load() && bg_error_.ok()) {
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
1846
      bg_cv_.Wait();
1847 1848 1849
    } else {
      manual_compaction_ = &manual;
      MaybeScheduleFlushOrCompaction();
G
Gabor Cselle 已提交
1850
    }
H
hans@chromium.org 已提交
1851
  }
1852

1853 1854 1855
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1856
  return manual.status;
J
jorlow@chromium.org 已提交
1857 1858
}

1859 1860
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
                             const FlushOptions& options) {
1861 1862
  // nullptr batch means just wait for earlier writes to be done
  Status s = Write(WriteOptions(), nullptr);
H
heyongqiang 已提交
1863
  if (s.ok() && options.wait) {
1864
    // Wait until the compaction completes
1865
    s = WaitForFlushMemTable(cfd);
1866 1867
  }
  return s;
J
jorlow@chromium.org 已提交
1868 1869
}

1870
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1871 1872 1873
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1874
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1875 1876
    bg_cv_.Wait();
  }
1877
  if (!bg_error_.ok()) {
1878 1879 1880
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1881 1882
}

1883
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1884
  mutex_.AssertHeld();
1885
  bg_schedule_needed_ = false;
1886 1887
  if (bg_work_gate_closed_) {
    // gate closed for backgrond work
J
jorlow@chromium.org 已提交
1888 1889 1890
  } else if (shutting_down_.Acquire_Load()) {
    // DB is being deleted; no more background compactions
  } else {
1891
    bool is_flush_pending = false;
1892
    // no need to refcount since we're under a mutex
1893
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1894
      if (cfd->imm()->IsFlushPending()) {
1895 1896 1897
        is_flush_pending = true;
      }
    }
1898
    if (is_flush_pending) {
1899
      // memtable flush needed
1900 1901 1902
      if (bg_flush_scheduled_ < options_.max_background_flushes) {
        bg_flush_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH);
1903
      } else if (options_.max_background_flushes > 0) {
1904
        bg_schedule_needed_ = true;
1905
      }
1906
    }
1907
    bool is_compaction_needed = false;
1908
    // no need to refcount since we're under a mutex
1909
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1910
      if (cfd->current()->NeedsCompaction()) {
1911 1912 1913 1914
        is_compaction_needed = true;
        break;
      }
    }
1915

1916 1917
    // Schedule BGWorkCompaction if there's a compaction pending (or a memtable
    // flush, but the HIGH pool is not enabled)
1918
    // Do it only if max_background_compactions hasn't been reached and, in case
1919
    // bg_manual_only_ > 0, if it's a manual compaction.
1920 1921
    if ((manual_compaction_ || is_compaction_needed ||
         (is_flush_pending && options_.max_background_flushes == 0)) &&
1922
        (!bg_manual_only_ || manual_compaction_)) {
1923 1924 1925 1926 1927 1928
      if (bg_compaction_scheduled_ < options_.max_background_compactions) {
        bg_compaction_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
      } else {
        bg_schedule_needed_ = true;
      }
1929 1930 1931 1932
    }
  }
}

1933
void DBImpl::RecordFlushIOStats() {
L
Lei Jin 已提交
1934
  RecordTick(stats_, FLUSH_WRITE_BYTES, iostats_context.bytes_written);
1935 1936 1937 1938
  IOSTATS_RESET(bytes_written);
}

void DBImpl::RecordCompactionIOStats() {
L
Lei Jin 已提交
1939
  RecordTick(stats_, COMPACT_READ_BYTES, IOSTATS(bytes_read));
1940
  IOSTATS_RESET(bytes_read);
L
Lei Jin 已提交
1941
  RecordTick(stats_, COMPACT_WRITE_BYTES, IOSTATS(bytes_written));
1942 1943 1944
  IOSTATS_RESET(bytes_written);
}

1945
void DBImpl::BGWorkFlush(void* db) {
1946
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
1947 1948 1949 1950
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
1951
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
1952 1953 1954
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
1955
Status DBImpl::BackgroundFlush(bool* madeProgress,
H
Haobo Xu 已提交
1956 1957
                               DeletionState& deletion_state,
                               LogBuffer* log_buffer) {
1958 1959 1960 1961 1962 1963
  mutex_.AssertHeld();
  // call_status is failure if at least one flush was a failure. even if
  // flushing one column family reports a failure, we will continue flushing
  // other column families. however, call_status will be a failure in that case.
  Status call_status;
  // refcounting in iteration
1964
  for (auto cfd : *versions_->GetColumnFamilySet()) {
1965 1966 1967
    cfd->Ref();
    Status flush_status;
    while (flush_status.ok() && cfd->imm()->IsFlushPending()) {
1968 1969
      LogToBuffer(
          log_buffer,
1970
          "BackgroundCallFlush doing FlushMemTableToOutputFile with column "
I
Igor Canadi 已提交
1971 1972 1973
          "family [%s], flush slots available %d",
          cfd->GetName().c_str(),
          options_.max_background_flushes - bg_flush_scheduled_);
1974 1975
      flush_status = FlushMemTableToOutputFile(cfd, madeProgress,
                                               deletion_state, log_buffer);
1976 1977 1978
    }
    if (call_status.ok() && !flush_status.ok()) {
      call_status = flush_status;
1979
    }
1980
    cfd->Unref();
J
jorlow@chromium.org 已提交
1981
  }
1982
  versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
1983
  return call_status;
J
jorlow@chromium.org 已提交
1984 1985
}

1986
void DBImpl::BackgroundCallFlush() {
1987
  bool madeProgress = false;
K
Kai Liu 已提交
1988
  DeletionState deletion_state(true);
1989 1990
  assert(bg_flush_scheduled_);

1991
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, options_.info_log.get());
H
Haobo Xu 已提交
1992 1993 1994 1995 1996 1997 1998 1999 2000 2001 2002
  {
    MutexLock l(&mutex_);

    Status s;
    if (!shutting_down_.Acquire_Load()) {
      s = BackgroundFlush(&madeProgress, deletion_state, &log_buffer);
      if (!s.ok()) {
        // Wait a little bit before retrying background compaction in
        // case this is an environmental problem and we do not want to
        // chew up resources for failed compactions for the duration of
        // the problem.
2003 2004
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
H
Haobo Xu 已提交
2005 2006
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
2007 2008 2009 2010
        Log(options_.info_log,
            "Waiting after background flush error: %s"
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
H
Haobo Xu 已提交
2011 2012 2013 2014 2015 2016 2017 2018 2019 2020 2021
        log_buffer.FlushBufferToLog();
        LogFlush(options_.info_log);
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }

    // If !s.ok(), this means that Flush failed. In that case, we want
    // to delete all obsolete files and we force FindObsoleteFiles()
    FindObsoleteFiles(deletion_state, !s.ok());
    // delete unnecessary files if any, this is done outside the mutex
2022
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2023
      mutex_.Unlock();
2024 2025 2026 2027 2028
      // Have to flush the info logs before bg_flush_scheduled_--
      // because if bg_flush_scheduled_ becomes 0 and the lock is
      // released, the deconstructor of DB can kick in and destroy all the
      // states of DB so info_log might not be available after that point.
      // It also applies to access other states that DB owns.
H
Haobo Xu 已提交
2029
      log_buffer.FlushBufferToLog();
2030 2031 2032
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
2033 2034
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2035

H
Haobo Xu 已提交
2036
    bg_flush_scheduled_--;
2037 2038 2039 2040 2041
    // Any time the mutex is released After finding the work to do, another
    // thread might execute MaybeScheduleFlushOrCompaction(). It is possible
    // that there is a pending job but it is not scheduled because of the
    // max thread limit.
    if (madeProgress || bg_schedule_needed_) {
H
Haobo Xu 已提交
2042 2043 2044
      MaybeScheduleFlushOrCompaction();
    }
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2045 2046 2047 2048
    // IMPORTANT: there should be no code after calling SignalAll. This call may
    // signal the DB destructor that it's OK to proceed with destruction. In
    // that case, all DB variables will be dealloacated and referencing them
    // will cause trouble.
2049
  }
2050
  RecordFlushIOStats();
J
jorlow@chromium.org 已提交
2051 2052
}

2053
void DBImpl::BackgroundCallCompaction() {
2054
  bool madeProgress = false;
K
Kai Liu 已提交
2055
  DeletionState deletion_state(true);
H
Haobo Xu 已提交
2056 2057

  MaybeDumpStats();
2058
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, options_.info_log.get());
2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069
  {
    MutexLock l(&mutex_);
    assert(bg_compaction_scheduled_);
    Status s;
    if (!shutting_down_.Acquire_Load()) {
      s = BackgroundCompaction(&madeProgress, deletion_state, &log_buffer);
      if (!s.ok()) {
        // Wait a little bit before retrying background compaction in
        // case this is an environmental problem and we do not want to
        // chew up resources for failed compactions for the duration of
        // the problem.
2070 2071
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2072 2073
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
H
Haobo Xu 已提交
2074
        log_buffer.FlushBufferToLog();
2075 2076 2077 2078
        Log(options_.info_log,
            "Waiting after background compaction error: %s, "
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
2079 2080 2081 2082 2083
        LogFlush(options_.info_log);
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }
H
Haobo Xu 已提交
2084

2085 2086 2087 2088 2089 2090 2091
    // If !s.ok(), this means that Compaction failed. In that case, we want
    // to delete all obsolete files we might have created and we force
    // FindObsoleteFiles(). This is because deletion_state does not catch
    // all created files if compaction failed.
    FindObsoleteFiles(deletion_state, !s.ok());

    // delete unnecessary files if any, this is done outside the mutex
2092
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2093
      mutex_.Unlock();
2094 2095 2096 2097 2098
      // Have to flush the info logs before bg_compaction_scheduled_--
      // because if bg_flush_scheduled_ becomes 0 and the lock is
      // released, the deconstructor of DB can kick in and destroy all the
      // states of DB so info_log might not be available after that point.
      // It also applies to access other states that DB owns.
H
Haobo Xu 已提交
2099
      log_buffer.FlushBufferToLog();
2100 2101 2102
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
2103 2104
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2105

2106
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2107

2108 2109
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2110 2111 2112
    // Previous compaction may have produced too many files in a level,
    // So reschedule another compaction if we made progress in the
    // last compaction.
2113 2114 2115 2116 2117 2118
    //
    // Also, any time the mutex is released After finding the work to do,
    // another thread might execute MaybeScheduleFlushOrCompaction(). It is
    // possible  that there is a pending job but it is not scheduled because of
    // the max thread limit.
    if (madeProgress || bg_schedule_needed_) {
2119 2120
      MaybeScheduleFlushOrCompaction();
    }
2121 2122 2123 2124 2125 2126 2127 2128 2129
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
      // * madeProgress -- need to wakeup MakeRoomForWrite
      // * bg_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl
      // * bg_manual_only_ > 0 -- need to wakeup RunManualCompaction
      // If none of this is true, there is no need to signal since nobody is
      // waiting for it
      bg_cv_.SignalAll();
    }
I
Igor Canadi 已提交
2130 2131 2132 2133
    // IMPORTANT: there should be no code after calling SignalAll. This call may
    // signal the DB destructor that it's OK to proceed with destruction. In
    // that case, all DB variables will be dealloacated and referencing them
    // will cause trouble.
2134
  }
J
jorlow@chromium.org 已提交
2135 2136
}

A
Abhishek Kona 已提交
2137
Status DBImpl::BackgroundCompaction(bool* madeProgress,
2138 2139
                                    DeletionState& deletion_state,
                                    LogBuffer* log_buffer) {
2140
  *madeProgress = false;
J
jorlow@chromium.org 已提交
2141
  mutex_.AssertHeld();
2142

2143 2144
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2145 2146 2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158 2159 2160 2161 2162

  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
  }

  // FLUSH preempts compaction
  Status flush_stat;
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    while (cfd->imm()->IsFlushPending()) {
      LogToBuffer(
          log_buffer,
          "BackgroundCompaction doing FlushMemTableToOutputFile, "
          "compaction slots available %d",
          options_.max_background_compactions - bg_compaction_scheduled_);
      cfd->Ref();
      flush_stat = FlushMemTableToOutputFile(cfd, madeProgress, deletion_state,
                                             log_buffer);
2163
      cfd->Unref();
2164 2165 2166 2167 2168 2169 2170
      if (!flush_stat.ok()) {
        if (is_manual) {
          manual_compaction_->status = flush_stat;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
2171
        return flush_stat;
2172 2173 2174 2175 2176
      }
    }
  }

  unique_ptr<Compaction> c;
2177 2178
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2179
  if (is_manual) {
G
Gabor Cselle 已提交
2180
    ManualCompaction* m = manual_compaction_;
2181
    assert(m->in_progress);
2182 2183 2184
    c.reset(m->cfd->CompactRange(m->input_level, m->output_level,
                                 m->output_path_id, m->begin, m->end,
                                 &manual_end));
2185
    if (!c) {
2186
      m->done = true;
G
Gabor Cselle 已提交
2187
    }
I
Igor Canadi 已提交
2188 2189 2190 2191 2192 2193 2194 2195 2196
    LogToBuffer(log_buffer,
                "[%s] Manual compaction from level-%d to level-%d from %s .. "
                "%s; will stop at %s\n",
                m->cfd->GetName().c_str(), m->input_level, m->output_level,
                (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                (m->end ? m->end->DebugString().c_str() : "(end)"),
                ((m->done || manual_end == nullptr)
                     ? "(end)"
                     : manual_end->DebugString().c_str()));
I
Igor Canadi 已提交
2197
  } else {
2198
    // no need to refcount in iteration since it's always under a mutex
I
Igor Canadi 已提交
2199
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
2200
      if (!cfd->options()->disable_auto_compactions) {
2201
        c.reset(cfd->PickCompaction(log_buffer));
I
Igor Canadi 已提交
2202 2203
        if (c != nullptr) {
          // update statistics
L
Lei Jin 已提交
2204
          MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
I
Igor Canadi 已提交
2205 2206 2207
                      c->inputs(0)->size());
          break;
        }
I
Igor Canadi 已提交
2208 2209
      }
    }
J
jorlow@chromium.org 已提交
2210 2211 2212
  }

  Status status;
2213
  if (!c) {
H
hans@chromium.org 已提交
2214
    // Nothing to do
2215
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2216 2217 2218 2219 2220 2221 2222 2223
  } else if (c->IsDeletionCompaction()) {
    // TODO(icanadi) Do we want to honor snapshots here? i.e. not delete old
    // file if there is alive snapshot pointing to it
    assert(c->num_input_files(1) == 0);
    assert(c->level() == 0);
    assert(c->column_family_data()->options()->compaction_style ==
           kCompactionStyleFIFO);
    for (const auto& f : *c->inputs(0)) {
2224
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2225 2226 2227 2228 2229 2230 2231 2232 2233
    }
    status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_,
                                    db_directory_.get());
    InstallSuperVersion(c->column_family_data(), deletion_state);
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
    c->ReleaseCompactionFiles(status);
    *madeProgress = true;
H
hans@chromium.org 已提交
2234
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
2235
    // Move file to next level
2236
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2237
    FileMetaData* f = c->input(0, 0);
2238
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
2239 2240 2241
    c->edit()->AddFile(c->level() + 1, f->fd.GetNumber(), f->fd.GetPathId(),
                       f->fd.GetFileSize(), f->smallest, f->largest,
                       f->smallest_seqno, f->largest_seqno);
I
Igor Canadi 已提交
2242
    status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_,
2243
                                    db_directory_.get());
I
Igor Canadi 已提交
2244
    InstallSuperVersion(c->column_family_data(), deletion_state);
2245

2246
    Version::LevelSummaryStorage tmp;
2247 2248 2249 2250 2251 2252
    LogToBuffer(
        log_buffer, "[%s] Moved #%lld to level-%d %lld bytes %s: %s\n",
        c->column_family_data()->GetName().c_str(),
        static_cast<unsigned long long>(f->fd.GetNumber()), c->level() + 1,
        static_cast<unsigned long long>(f->fd.GetFileSize()),
        status.ToString().c_str(), c->input_version()->LevelSummary(&tmp));
I
Igor Canadi 已提交
2253
    c->ReleaseCompactionFiles(status);
2254
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2255
  } else {
2256
    MaybeScheduleFlushOrCompaction(); // do more compaction work in parallel.
2257
    CompactionState* compact = new CompactionState(c.get());
H
Haobo Xu 已提交
2258
    status = DoCompactionWork(compact, deletion_state, log_buffer);
2259
    CleanupCompaction(compact, status);
I
Igor Canadi 已提交
2260
    c->ReleaseCompactionFiles(status);
2261
    c->ReleaseInputs();
2262
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2263
  }
2264
  c.reset();
J
jorlow@chromium.org 已提交
2265 2266 2267 2268 2269 2270

  if (status.ok()) {
    // Done
  } else if (shutting_down_.Acquire_Load()) {
    // Ignore compaction errors found during shutting down
  } else {
2271
    Log(InfoLogLevel::WARN_LEVEL, options_.info_log, "Compaction error: %s",
2272
        status.ToString().c_str());
J
jorlow@chromium.org 已提交
2273 2274 2275 2276
    if (options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2277 2278

  if (is_manual) {
G
Gabor Cselle 已提交
2279
    ManualCompaction* m = manual_compaction_;
2280
    if (!status.ok()) {
L
Lei Jin 已提交
2281
      m->status = status;
2282 2283
      m->done = true;
    }
2284 2285 2286 2287 2288 2289 2290 2291 2292
    // For universal compaction:
    //   Because universal compaction always happens at level 0, so one
    //   compaction will pick up all overlapped files. No files will be
    //   filtered out due to size limit and left for a successive compaction.
    //   So we can safely conclude the current compaction.
    //
    //   Also note that, if we don't stop here, then the current compaction
    //   writes a new file back to level 0, which will be used in successive
    //   compaction. Hence the manual compaction will never finish.
2293 2294 2295 2296 2297
    //
    // Stop the compaction if manual_end points to nullptr -- this means
    // that we compacted the whole range. manual_end should always point
    // to nullptr in case of universal compaction
    if (manual_end == nullptr) {
2298 2299
      m->done = true;
    }
G
Gabor Cselle 已提交
2300 2301 2302
    if (!m->done) {
      // We only compacted part of the requested range.  Update *m
      // to the range that is left to be compacted.
I
Igor Canadi 已提交
2303
      // Universal and FIFO compactions should always compact the whole range
I
Igor Canadi 已提交
2304
      assert(m->cfd->options()->compaction_style != kCompactionStyleUniversal);
I
Igor Canadi 已提交
2305
      assert(m->cfd->options()->compaction_style != kCompactionStyleFIFO);
2306
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2307 2308
      m->begin = &m->tmp_storage;
    }
2309
    m->in_progress = false; // not being processed anymore
2310
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2311
  }
2312
  return status;
J
jorlow@chromium.org 已提交
2313 2314
}

2315
void DBImpl::CleanupCompaction(CompactionState* compact, Status status) {
J
jorlow@chromium.org 已提交
2316
  mutex_.AssertHeld();
2317
  if (compact->builder != nullptr) {
J
jorlow@chromium.org 已提交
2318 2319
    // May happen if we get a shutdown call in the middle of compaction
    compact->builder->Abandon();
2320
    compact->builder.reset();
J
jorlow@chromium.org 已提交
2321
  } else {
2322
    assert(compact->outfile == nullptr);
J
jorlow@chromium.org 已提交
2323
  }
D
dgrogan@chromium.org 已提交
2324
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2325 2326
    const CompactionState::Output& out = compact->outputs[i];
    pending_outputs_.erase(out.number);
2327 2328 2329 2330

    // If this file was inserted into the table cache then remove
    // them here because this compaction was not committed.
    if (!status.ok()) {
I
Igor Canadi 已提交
2331
      TableCache::Evict(table_cache_.get(), out.number);
2332
    }
J
jorlow@chromium.org 已提交
2333 2334 2335 2336
  }
  delete compact;
}

2337
// Allocate the file numbers for the output file. We allocate as
2338
// many output file numbers as there are files in level+1 (at least one)
2339 2340 2341
// Insert them into pending_outputs so that they do not get deleted.
void DBImpl::AllocateCompactionOutputFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2342 2343
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
2344
  int filesNeeded = compact->compaction->num_input_files(1);
2345
  for (int i = 0; i < std::max(filesNeeded, 1); i++) {
2346
    uint64_t file_number = versions_->NewFileNumber();
2347
    pending_outputs_[file_number] = compact->compaction->GetOutputPathId();
2348 2349 2350 2351 2352 2353 2354
    compact->allocated_file_numbers.push_back(file_number);
  }
}

// Frees up unused file number.
void DBImpl::ReleaseCompactionUnusedFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2355
  for (const auto file_number : compact->allocated_file_numbers) {
2356 2357 2358 2359
    pending_outputs_.erase(file_number);
  }
}

J
jorlow@chromium.org 已提交
2360
Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
2361 2362
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
J
jorlow@chromium.org 已提交
2363
  uint64_t file_number;
2364 2365 2366 2367 2368 2369 2370
  // If we have not yet exhausted the pre-allocated file numbers,
  // then use the one from the front. Otherwise, we have to acquire
  // the heavyweight lock and allocate a new file number.
  if (!compact->allocated_file_numbers.empty()) {
    file_number = compact->allocated_file_numbers.front();
    compact->allocated_file_numbers.pop_front();
  } else {
J
jorlow@chromium.org 已提交
2371 2372
    mutex_.Lock();
    file_number = versions_->NewFileNumber();
2373
    pending_outputs_[file_number] = compact->compaction->GetOutputPathId();
J
jorlow@chromium.org 已提交
2374 2375
    mutex_.Unlock();
  }
2376 2377
  CompactionState::Output out;
  out.number = file_number;
2378
  out.path_id = compact->compaction->GetOutputPathId();
2379 2380
  out.smallest.Clear();
  out.largest.Clear();
2381
  out.smallest_seqno = out.largest_seqno = 0;
2382
  compact->outputs.push_back(out);
J
jorlow@chromium.org 已提交
2383 2384

  // Make the output file
2385 2386
  std::string fname = TableFileName(options_.db_paths, file_number,
                                    compact->compaction->GetOutputPathId());
2387
  Status s = env_->NewWritableFile(fname, &compact->outfile, storage_options_);
2388

J
jorlow@chromium.org 已提交
2389
  if (s.ok()) {
L
Lei Jin 已提交
2390
    compact->outfile->SetIOPriority(Env::IO_LOW);
2391
    compact->outfile->SetPreallocationBlockSize(
2392
        compact->compaction->OutputFilePreallocationSize());
2393

2394
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
2395 2396 2397
    compact->builder.reset(NewTableBuilder(
        *cfd->options(), cfd->internal_comparator(), compact->outfile.get(),
        compact->compaction->OutputCompressionType()));
J
jorlow@chromium.org 已提交
2398
  }
I
Igor Canadi 已提交
2399
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
2400 2401 2402 2403 2404
  return s;
}

Status DBImpl::FinishCompactionOutputFile(CompactionState* compact,
                                          Iterator* input) {
2405
  assert(compact != nullptr);
2406
  assert(compact->outfile);
2407
  assert(compact->builder != nullptr);
J
jorlow@chromium.org 已提交
2408 2409

  const uint64_t output_number = compact->current_output()->number;
2410
  const uint32_t output_path_id = compact->current_output()->path_id;
J
jorlow@chromium.org 已提交
2411 2412 2413 2414 2415 2416 2417 2418 2419 2420 2421 2422 2423
  assert(output_number != 0);

  // Check for iterator errors
  Status s = input->status();
  const uint64_t current_entries = compact->builder->NumEntries();
  if (s.ok()) {
    s = compact->builder->Finish();
  } else {
    compact->builder->Abandon();
  }
  const uint64_t current_bytes = compact->builder->FileSize();
  compact->current_output()->file_size = current_bytes;
  compact->total_bytes += current_bytes;
2424
  compact->builder.reset();
J
jorlow@chromium.org 已提交
2425 2426

  // Finish and check for file errors
2427
  if (s.ok() && !options_.disableDataSync) {
2428
    if (options_.use_fsync) {
L
Lei Jin 已提交
2429
      StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
2430 2431
      s = compact->outfile->Fsync();
    } else {
L
Lei Jin 已提交
2432
      StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
2433 2434
      s = compact->outfile->Sync();
    }
J
jorlow@chromium.org 已提交
2435 2436 2437 2438
  }
  if (s.ok()) {
    s = compact->outfile->Close();
  }
2439
  compact->outfile.reset();
J
jorlow@chromium.org 已提交
2440 2441 2442

  if (s.ok() && current_entries > 0) {
    // Verify that the table is usable
I
Igor Canadi 已提交
2443
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
2444
    FileDescriptor fd(output_number, output_path_id, current_bytes);
I
Igor Canadi 已提交
2445
    Iterator* iter = cfd->table_cache()->NewIterator(
2446
        ReadOptions(), storage_options_, cfd->internal_comparator(), fd);
J
jorlow@chromium.org 已提交
2447 2448 2449
    s = iter->status();
    delete iter;
    if (s.ok()) {
2450 2451 2452 2453
      Log(options_.info_log, "[%s] Generated table #%" PRIu64 ": %" PRIu64
                             " keys, %" PRIu64 " bytes",
          cfd->GetName().c_str(), output_number, current_entries,
          current_bytes);
J
jorlow@chromium.org 已提交
2454 2455 2456 2457 2458 2459
    }
  }
  return s;
}


2460 2461
Status DBImpl::InstallCompactionResults(CompactionState* compact,
                                        LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
2462
  mutex_.AssertHeld();
2463 2464 2465 2466 2467

  // paranoia: verify that the files that we started with
  // still exist in the current version and in the same original level.
  // This ensures that a concurrent compaction did not erroneously
  // pick the same files to compact.
2468
  if (!versions_->VerifyCompactionFileConsistency(compact->compaction)) {
I
Igor Canadi 已提交
2469 2470 2471 2472 2473
    Log(options_.info_log, "[%s] Compaction %d@%d + %d@%d files aborted",
        compact->compaction->column_family_data()->GetName().c_str(),
        compact->compaction->num_input_files(0), compact->compaction->level(),
        compact->compaction->num_input_files(1),
        compact->compaction->output_level());
L
Lei Jin 已提交
2474
    return Status::Corruption("Compaction input files inconsistent");
2475 2476
  }

I
Igor Canadi 已提交
2477 2478
  LogToBuffer(log_buffer, "[%s] Compacted %d@%d + %d@%d files => %lld bytes",
              compact->compaction->column_family_data()->GetName().c_str(),
2479 2480 2481 2482 2483
              compact->compaction->num_input_files(0),
              compact->compaction->level(),
              compact->compaction->num_input_files(1),
              compact->compaction->output_level(),
              static_cast<long long>(compact->total_bytes));
J
jorlow@chromium.org 已提交
2484 2485 2486

  // Add compaction outputs
  compact->compaction->AddInputDeletions(compact->compaction->edit());
D
dgrogan@chromium.org 已提交
2487
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2488
    const CompactionState::Output& out = compact->outputs[i];
2489 2490 2491 2492
    compact->compaction->edit()->AddFile(compact->compaction->output_level(),
                                         out.number, out.path_id, out.file_size,
                                         out.smallest, out.largest,
                                         out.smallest_seqno, out.largest_seqno);
J
jorlow@chromium.org 已提交
2493
  }
I
Igor Canadi 已提交
2494 2495 2496
  return versions_->LogAndApply(compact->compaction->column_family_data(),
                                compact->compaction->edit(), &mutex_,
                                db_directory_.get());
J
jorlow@chromium.org 已提交
2497 2498
}

2499 2500 2501 2502 2503 2504 2505
// Given a sequence number, return the sequence number of the
// earliest snapshot that this sequence number is visible in.
// The snapshots themselves are arranged in ascending order of
// sequence numbers.
// Employ a sequential search because the total number of
// snapshots are typically small.
inline SequenceNumber DBImpl::findEarliestVisibleSnapshot(
2506 2507
  SequenceNumber in, std::vector<SequenceNumber>& snapshots,
  SequenceNumber* prev_snapshot) {
2508
  SequenceNumber prev __attribute__((unused)) = 0;
2509 2510 2511
  for (const auto cur : snapshots) {
    assert(prev <= cur);
    if (cur >= in) {
2512
      *prev_snapshot = prev;
2513
      return cur;
2514
    }
2515 2516
    prev = cur; // assignment
    assert(prev);
2517 2518
  }
  Log(options_.info_log,
2519 2520
      "Looking for seqid %" PRIu64 " but maxseqid is %" PRIu64 "", in,
      snapshots[snapshots.size() - 1]);
2521 2522 2523 2524
  assert(0);
  return 0;
}

I
Igor Canadi 已提交
2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538 2539 2540 2541 2542 2543
uint64_t DBImpl::CallFlushDuringCompaction(ColumnFamilyData* cfd,
                                           DeletionState& deletion_state,
                                           LogBuffer* log_buffer) {
  if (cfd->imm()->imm_flush_needed.NoBarrier_Load() != nullptr) {
    const uint64_t imm_start = env_->NowMicros();
    mutex_.Lock();
    if (cfd->imm()->IsFlushPending()) {
      cfd->Ref();
      FlushMemTableToOutputFile(cfd, nullptr, deletion_state, log_buffer);
      cfd->Unref();
      bg_cv_.SignalAll();  // Wakeup MakeRoomForWrite() if necessary
    }
    mutex_.Unlock();
    log_buffer->FlushBufferToLog();
    return env_->NowMicros() - imm_start;
  }
  return 0;
}

D
Danny Guo 已提交
2544
Status DBImpl::ProcessKeyValueCompaction(
2545
    bool is_snapshot_supported,
D
Danny Guo 已提交
2546 2547 2548 2549 2550 2551 2552 2553 2554 2555 2556
    SequenceNumber visible_at_tip,
    SequenceNumber earliest_snapshot,
    SequenceNumber latest_snapshot,
    DeletionState& deletion_state,
    bool bottommost_level,
    int64_t& imm_micros,
    Iterator* input,
    CompactionState* compact,
    bool is_compaction_v2,
    LogBuffer* log_buffer) {
  size_t combined_idx = 0;
J
jorlow@chromium.org 已提交
2557
  Status status;
D
Danny Guo 已提交
2558
  std::string compaction_filter_value;
J
jorlow@chromium.org 已提交
2559
  ParsedInternalKey ikey;
2560
  IterKey current_user_key;
J
jorlow@chromium.org 已提交
2561
  bool has_current_user_key = false;
2562
  IterKey delete_key;
2563 2564
  SequenceNumber last_sequence_for_key __attribute__((unused)) =
    kMaxSequenceNumber;
2565
  SequenceNumber visible_in_snapshot = kMaxSequenceNumber;
2566
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
I
Igor Canadi 已提交
2567 2568
  MergeHelper merge(
      cfd->user_comparator(), cfd->options()->merge_operator.get(),
2569 2570
      options_.info_log.get(), cfd->options()->min_partial_merge_operands,
      false /* internal key corruption is expected */);
I
Igor Canadi 已提交
2571
  auto compaction_filter = cfd->options()->compaction_filter;
2572 2573
  std::unique_ptr<CompactionFilter> compaction_filter_from_factory = nullptr;
  if (!compaction_filter) {
2574
    auto context = compact->GetFilterContextV1();
2575
    compaction_filter_from_factory =
I
Igor Canadi 已提交
2576 2577
        cfd->options()->compaction_filter_factory->CreateCompactionFilter(
            context);
2578 2579
    compaction_filter = compaction_filter_from_factory.get();
  }
2580

2581 2582
  while (input->Valid() && !shutting_down_.Acquire_Load() &&
         !cfd->IsDropped()) {
2583
    RecordCompactionIOStats();
2584 2585 2586 2587
    // FLUSH preempts compaction
    // TODO(icanadi) this currently only checks if flush is necessary on
    // compacting column family. we should also check if flush is necessary on
    // other column families, too
I
Igor Canadi 已提交
2588
    imm_micros += CallFlushDuringCompaction(cfd, deletion_state, log_buffer);
2589

D
Danny Guo 已提交
2590 2591 2592 2593 2594 2595 2596 2597 2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610
    Slice key;
    Slice value;
    // If is_compaction_v2 is on, kv-pairs are reset to the prefix batch.
    // This prefix batch should contain results after calling
    // compaction_filter_v2.
    //
    // If is_compaction_v2 is off, this function will go through all the
    // kv-pairs in input.
    if (!is_compaction_v2) {
      key = input->key();
      value = input->value();
    } else {
      if (combined_idx >= compact->combined_key_buf_.size()) {
        break;
      }
      assert(combined_idx < compact->combined_key_buf_.size());
      key = compact->combined_key_buf_[combined_idx];
      value = compact->combined_value_buf_[combined_idx];

      ++combined_idx;
    }
H
Haobo Xu 已提交
2611

2612
    if (compact->compaction->ShouldStopBefore(key) &&
2613
        compact->builder != nullptr) {
D
Danny Guo 已提交
2614
      status = FinishCompactionOutputFile(compact, input);
2615 2616 2617 2618 2619 2620
      if (!status.ok()) {
        break;
      }
    }

    // Handle key/value, add to state, etc.
J
jorlow@chromium.org 已提交
2621
    bool drop = false;
2622
    bool current_entry_is_merging = false;
J
jorlow@chromium.org 已提交
2623 2624
    if (!ParseInternalKey(key, &ikey)) {
      // Do not hide error keys
2625 2626
      // TODO: error key stays in db forever? Figure out the intention/rationale
      // v10 error v8 : we cannot hide v8 even though it's pretty obvious.
2627
      current_user_key.Clear();
J
jorlow@chromium.org 已提交
2628 2629
      has_current_user_key = false;
      last_sequence_for_key = kMaxSequenceNumber;
2630
      visible_in_snapshot = kMaxSequenceNumber;
J
jorlow@chromium.org 已提交
2631 2632
    } else {
      if (!has_current_user_key ||
2633
          cfd->user_comparator()->Compare(ikey.user_key,
2634
                                          current_user_key.GetKey()) != 0) {
J
jorlow@chromium.org 已提交
2635
        // First occurrence of this user key
L
Lei Jin 已提交
2636
        current_user_key.SetKey(ikey.user_key);
J
jorlow@chromium.org 已提交
2637 2638
        has_current_user_key = true;
        last_sequence_for_key = kMaxSequenceNumber;
2639
        visible_in_snapshot = kMaxSequenceNumber;
H
Haobo Xu 已提交
2640
        // apply the compaction filter to the first occurrence of the user key
D
Danny Guo 已提交
2641
        if (compaction_filter && !is_compaction_v2 &&
H
Haobo Xu 已提交
2642 2643 2644 2645 2646 2647 2648 2649 2650
            ikey.type == kTypeValue &&
            (visible_at_tip || ikey.sequence > latest_snapshot)) {
          // If the user has specified a compaction filter and the sequence
          // number is greater than any external snapshot, then invoke the
          // filter.
          // If the return value of the compaction filter is true, replace
          // the entry with a delete marker.
          bool value_changed = false;
          compaction_filter_value.clear();
I
Igor Canadi 已提交
2651 2652 2653
          bool to_delete = compaction_filter->Filter(
              compact->compaction->level(), ikey.user_key, value,
              &compaction_filter_value, &value_changed);
H
Haobo Xu 已提交
2654
          if (to_delete) {
2655 2656 2657
            // make a copy of the original key and convert it to a delete
            delete_key.SetInternalKey(ExtractUserKey(key), ikey.sequence,
                                      kTypeDeletion);
H
Haobo Xu 已提交
2658
            // anchor the key again
2659
            key = delete_key.GetKey();
H
Haobo Xu 已提交
2660 2661 2662 2663
            // needed because ikey is backed by key
            ParseInternalKey(key, &ikey);
            // no value associated with delete
            value.clear();
L
Lei Jin 已提交
2664
            RecordTick(stats_, COMPACTION_KEY_DROP_USER);
H
Haobo Xu 已提交
2665 2666 2667 2668
          } else if (value_changed) {
            value = compaction_filter_value;
          }
        }
J
jorlow@chromium.org 已提交
2669 2670
      }

2671 2672 2673
      // If there are no snapshots, then this kv affect visibility at tip.
      // Otherwise, search though all existing snapshots to find
      // the earlist snapshot that is affected by this kv.
2674
      SequenceNumber prev_snapshot = 0; // 0 means no previous snapshot
2675 2676 2677 2678
      SequenceNumber visible = visible_at_tip ? visible_at_tip :
        is_snapshot_supported ?  findEarliestVisibleSnapshot(ikey.sequence,
                                  compact->existing_snapshots, &prev_snapshot)
                              : 0;
2679 2680 2681 2682 2683

      if (visible_in_snapshot == visible) {
        // If the earliest snapshot is which this key is visible in
        // is the same as the visibily of a previous instance of the
        // same key, then this kv is not visible in any snapshot.
J
jorlow@chromium.org 已提交
2684
        // Hidden by an newer entry for same user key
2685
        // TODO: why not > ?
2686
        assert(last_sequence_for_key >= ikey.sequence);
J
jorlow@chromium.org 已提交
2687
        drop = true;    // (A)
L
Lei Jin 已提交
2688
        RecordTick(stats_, COMPACTION_KEY_DROP_NEWER_ENTRY);
J
jorlow@chromium.org 已提交
2689
      } else if (ikey.type == kTypeDeletion &&
D
Danny Guo 已提交
2690
          ikey.sequence <= earliest_snapshot &&
2691
          compact->compaction->KeyNotExistsBeyondOutputLevel(ikey.user_key)) {
J
jorlow@chromium.org 已提交
2692 2693 2694 2695 2696 2697 2698 2699
        // For this user key:
        // (1) there is no data in higher levels
        // (2) data in lower levels will have larger sequence numbers
        // (3) data in layers that are being compacted here and have
        //     smaller sequence numbers will be dropped in the next
        //     few iterations of this loop (by rule (A) above).
        // Therefore this deletion marker is obsolete and can be dropped.
        drop = true;
L
Lei Jin 已提交
2700
        RecordTick(stats_, COMPACTION_KEY_DROP_OBSOLETE);
2701 2702 2703 2704 2705 2706 2707
      } else if (ikey.type == kTypeMerge) {
        // We know the merge type entry is not hidden, otherwise we would
        // have hit (A)
        // We encapsulate the merge related state machine in a different
        // object to minimize change to the existing flow. Turn out this
        // logic could also be nicely re-used for memtable flush purge
        // optimization in BuildTable.
D
Danny Guo 已提交
2708 2709 2710 2711 2712 2713
        int steps = 0;
        merge.MergeUntil(input, prev_snapshot, bottommost_level,
            options_.statistics.get(), &steps);
        // Skip the Merge ops
        combined_idx = combined_idx - 1 + steps;

2714 2715 2716 2717 2718 2719 2720 2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731
        current_entry_is_merging = true;
        if (merge.IsSuccess()) {
          // Successfully found Put/Delete/(end-of-key-range) while merging
          // Get the merge result
          key = merge.key();
          ParseInternalKey(key, &ikey);
          value = merge.value();
        } else {
          // Did not find a Put/Delete/(end-of-key-range) while merging
          // We now have some stack of merge operands to write out.
          // NOTE: key,value, and ikey are now referring to old entries.
          //       These will be correctly set below.
          assert(!merge.keys().empty());
          assert(merge.keys().size() == merge.values().size());

          // Hack to make sure last_sequence_for_key is correct
          ParseInternalKey(merge.keys().front(), &ikey);
        }
J
jorlow@chromium.org 已提交
2732 2733 2734
      }

      last_sequence_for_key = ikey.sequence;
2735
      visible_in_snapshot = visible;
J
jorlow@chromium.org 已提交
2736 2737 2738
    }

    if (!drop) {
2739 2740 2741 2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752 2753 2754
      // We may write a single key (e.g.: for Put/Delete or successful merge).
      // Or we may instead have to write a sequence/list of keys.
      // We have to write a sequence iff we have an unsuccessful merge
      bool has_merge_list = current_entry_is_merging && !merge.IsSuccess();
      const std::deque<std::string>* keys = nullptr;
      const std::deque<std::string>* values = nullptr;
      std::deque<std::string>::const_reverse_iterator key_iter;
      std::deque<std::string>::const_reverse_iterator value_iter;
      if (has_merge_list) {
        keys = &merge.keys();
        values = &merge.values();
        key_iter = keys->rbegin();    // The back (*rbegin()) is the first key
        value_iter = values->rbegin();

        key = Slice(*key_iter);
        value = Slice(*value_iter);
2755
      }
2756

2757 2758 2759 2760 2761 2762 2763 2764 2765 2766 2767
      // If we have a list of keys to write, traverse the list.
      // If we have a single key to write, simply write that key.
      while (true) {
        // Invariant: key,value,ikey will always be the next entry to write
        char* kptr = (char*)key.data();
        std::string kstr;

        // Zeroing out the sequence number leads to better compression.
        // If this is the bottommost level (no files in lower levels)
        // and the earliest snapshot is larger than this seqno
        // then we can squash the seqno to zero.
2768
        if (bottommost_level && ikey.sequence < earliest_snapshot &&
2769 2770 2771 2772 2773 2774 2775 2776 2777 2778 2779
            ikey.type != kTypeMerge) {
          assert(ikey.type != kTypeDeletion);
          // make a copy because updating in place would cause problems
          // with the priority queue that is managing the input key iterator
          kstr.assign(key.data(), key.size());
          kptr = (char *)kstr.c_str();
          UpdateInternalKey(kptr, key.size(), (uint64_t)0, ikey.type);
        }

        Slice newkey(kptr, key.size());
        assert((key.clear(), 1)); // we do not need 'key' anymore
2780

2781 2782 2783 2784 2785 2786 2787
        // Open output file if necessary
        if (compact->builder == nullptr) {
          status = OpenCompactionOutputFile(compact);
          if (!status.ok()) {
            break;
          }
        }
2788 2789

        SequenceNumber seqno = GetInternalKeySeqno(newkey);
2790 2791
        if (compact->builder->NumEntries() == 0) {
          compact->current_output()->smallest.DecodeFrom(newkey);
2792 2793 2794 2795
          compact->current_output()->smallest_seqno = seqno;
        } else {
          compact->current_output()->smallest_seqno =
            std::min(compact->current_output()->smallest_seqno, seqno);
2796 2797 2798
        }
        compact->current_output()->largest.DecodeFrom(newkey);
        compact->builder->Add(newkey, value);
2799 2800
        compact->current_output()->largest_seqno =
          std::max(compact->current_output()->largest_seqno, seqno);
2801 2802 2803 2804

        // Close output file if it is big enough
        if (compact->builder->FileSize() >=
            compact->compaction->MaxOutputFileSize()) {
D
Danny Guo 已提交
2805
          status = FinishCompactionOutputFile(compact, input);
2806 2807 2808
          if (!status.ok()) {
            break;
          }
J
jorlow@chromium.org 已提交
2809 2810
        }

2811 2812 2813 2814 2815 2816 2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828 2829 2830
        // If we have a list of entries, move to next element
        // If we only had one entry, then break the loop.
        if (has_merge_list) {
          ++key_iter;
          ++value_iter;

          // If at end of list
          if (key_iter == keys->rend() || value_iter == values->rend()) {
            // Sanity Check: if one ends, then both end
            assert(key_iter == keys->rend() && value_iter == values->rend());
            break;
          }

          // Otherwise not at end of list. Update key, value, and ikey.
          key = Slice(*key_iter);
          value = Slice(*value_iter);
          ParseInternalKey(key, &ikey);

        } else{
          // Only had one item to begin with (Put/Delete)
J
jorlow@chromium.org 已提交
2831 2832 2833 2834 2835
          break;
        }
      }
    }

2836
    // MergeUntil has moved input to the next entry
2837
    if (!current_entry_is_merging) {
2838 2839
      input->Next();
    }
J
jorlow@chromium.org 已提交
2840 2841
  }

2842 2843
  RecordCompactionIOStats();

D
Danny Guo 已提交
2844 2845 2846 2847 2848 2849 2850 2851 2852 2853 2854 2855 2856 2857 2858 2859 2860 2861 2862 2863 2864 2865 2866 2867 2868 2869 2870 2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886 2887 2888 2889
  return status;
}

void DBImpl::CallCompactionFilterV2(CompactionState* compact,
  CompactionFilterV2* compaction_filter_v2) {
  if (compact == nullptr || compaction_filter_v2 == nullptr) {
    return;
  }

  std::vector<Slice> user_key_buf;
  for (const auto& key : compact->ikey_buf_) {
    user_key_buf.emplace_back(key.user_key);
  }

  // If the user has specified a compaction filter and the sequence
  // number is greater than any external snapshot, then invoke the
  // filter.
  // If the return value of the compaction filter is true, replace
  // the entry with a delete marker.
  compact->to_delete_buf_ = compaction_filter_v2->Filter(
      compact->compaction->level(),
      user_key_buf, compact->existing_value_buf_,
      &compact->new_value_buf_,
      &compact->value_changed_buf_);

  // new_value_buf_.size() <= to_delete__buf_.size(). "=" iff all
  // kv-pairs in this compaction run needs to be deleted.
  assert(compact->to_delete_buf_.size() ==
      compact->key_buf_.size());
  assert(compact->to_delete_buf_.size() ==
      compact->existing_value_buf_.size());
  assert(compact->to_delete_buf_.size() ==
      compact->value_changed_buf_.size());

  int new_value_idx = 0;
  for (unsigned int i = 0; i < compact->to_delete_buf_.size(); ++i) {
    if (compact->to_delete_buf_[i]) {
      // update the string buffer directly
      // the Slice buffer points to the updated buffer
      UpdateInternalKey(&compact->key_str_buf_[i][0],
          compact->key_str_buf_[i].size(),
          compact->ikey_buf_[i].sequence,
          kTypeDeletion);

      // no value associated with delete
      compact->existing_value_buf_[i].clear();
L
Lei Jin 已提交
2890
      RecordTick(stats_, COMPACTION_KEY_DROP_USER);
D
Danny Guo 已提交
2891 2892 2893 2894 2895 2896 2897 2898 2899 2900 2901 2902 2903
    } else if (compact->value_changed_buf_[i]) {
      compact->existing_value_buf_[i] =
        Slice(compact->new_value_buf_[new_value_idx++]);
    }
  }  // for
}

Status DBImpl::DoCompactionWork(CompactionState* compact,
                                DeletionState& deletion_state,
                                LogBuffer* log_buffer) {
  assert(compact);
  compact->CleanupBatchBuffer();
  compact->CleanupMergedBuffer();
2904
  bool prefix_initialized = false;
D
Danny Guo 已提交
2905

F
Feng Zhu 已提交
2906 2907
  // Generate file_levels_ for compaction berfore making Iterator
  compact->compaction->GenerateFileLevels();
D
Danny Guo 已提交
2908
  int64_t imm_micros = 0;  // Micros spent doing imm_ compactions
2909
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
2910 2911
  LogToBuffer(
      log_buffer,
I
Igor Canadi 已提交
2912 2913
      "[%s] Compacting %d@%d + %d@%d files, score %.2f slots available %d",
      cfd->GetName().c_str(), compact->compaction->num_input_files(0),
2914 2915
      compact->compaction->level(), compact->compaction->num_input_files(1),
      compact->compaction->output_level(), compact->compaction->score(),
D
Danny Guo 已提交
2916 2917 2918
      options_.max_background_compactions - bg_compaction_scheduled_);
  char scratch[2345];
  compact->compaction->Summary(scratch, sizeof(scratch));
I
Igor Canadi 已提交
2919 2920
  LogToBuffer(log_buffer, "[%s] Compaction start summary: %s\n",
              cfd->GetName().c_str(), scratch);
D
Danny Guo 已提交
2921

2922
  assert(cfd->current()->NumLevelFiles(compact->compaction->level()) > 0);
D
Danny Guo 已提交
2923 2924 2925 2926 2927 2928 2929 2930 2931 2932 2933 2934 2935 2936 2937 2938 2939 2940 2941 2942 2943 2944 2945 2946 2947
  assert(compact->builder == nullptr);
  assert(!compact->outfile);

  SequenceNumber visible_at_tip = 0;
  SequenceNumber earliest_snapshot;
  SequenceNumber latest_snapshot = 0;
  snapshots_.getAll(compact->existing_snapshots);
  if (compact->existing_snapshots.size() == 0) {
    // optimize for fast path if there are no snapshots
    visible_at_tip = versions_->LastSequence();
    earliest_snapshot = visible_at_tip;
  } else {
    latest_snapshot = compact->existing_snapshots.back();
    // Add the current seqno as the 'latest' virtual
    // snapshot to the end of this list.
    compact->existing_snapshots.push_back(versions_->LastSequence());
    earliest_snapshot = compact->existing_snapshots[0];
  }

  // Is this compaction producing files at the bottommost level?
  bool bottommost_level = compact->compaction->BottomMostLevel();

  // Allocate the output file numbers before we release the lock
  AllocateCompactionOutputFileNumbers(compact);

2948
  bool is_snapshot_supported = IsSnapshotSupported();
D
Danny Guo 已提交
2949 2950
  // Release mutex while we're actually doing the compaction work
  mutex_.Unlock();
2951
  log_buffer->FlushBufferToLog();
D
Danny Guo 已提交
2952 2953 2954 2955 2956 2957 2958 2959 2960 2961 2962 2963 2964 2965

  const uint64_t start_micros = env_->NowMicros();
  unique_ptr<Iterator> input(versions_->MakeInputIterator(compact->compaction));
  input->SeekToFirst();
  shared_ptr<Iterator> backup_input(
      versions_->MakeInputIterator(compact->compaction));
  backup_input->SeekToFirst();

  Status status;
  ParsedInternalKey ikey;
  std::unique_ptr<CompactionFilterV2> compaction_filter_from_factory_v2
    = nullptr;
  auto context = compact->GetFilterContext();
  compaction_filter_from_factory_v2 =
2966 2967
      cfd->options()->compaction_filter_factory_v2->CreateCompactionFilterV2(
          context);
D
Danny Guo 已提交
2968 2969 2970 2971 2972 2973 2974 2975 2976 2977 2978
  auto compaction_filter_v2 =
    compaction_filter_from_factory_v2.get();

  // temp_backup_input always point to the start of the current buffer
  // temp_backup_input = backup_input;
  // iterate through input,
  // 1) buffer ineligible keys and value keys into 2 separate buffers;
  // 2) send value_buffer to compaction filter and alternate the values;
  // 3) merge value_buffer with ineligible_value_buffer;
  // 4) run the modified "compaction" using the old for loop.
  if (compaction_filter_v2) {
2979 2980
    while (backup_input->Valid() && !shutting_down_.Acquire_Load() &&
           !cfd->IsDropped()) {
I
Igor Canadi 已提交
2981 2982 2983 2984 2985 2986
      // FLUSH preempts compaction
      // TODO(icanadi) this currently only checks if flush is necessary on
      // compacting column family. we should also check if flush is necessary on
      // other column families, too
      imm_micros += CallFlushDuringCompaction(cfd, deletion_state, log_buffer);

D
Danny Guo 已提交
2987 2988 2989 2990
      Slice key = backup_input->key();
      Slice value = backup_input->value();

      const SliceTransform* transformer =
2991
          cfd->options()->compaction_filter_factory_v2->GetPrefixExtractor();
2992
      const auto key_prefix = transformer->Transform(key);
2993
      if (!prefix_initialized) {
2994
        compact->cur_prefix_ = key_prefix.ToString();
2995
        prefix_initialized = true;
D
Danny Guo 已提交
2996 2997 2998
      }
      if (!ParseInternalKey(key, &ikey)) {
        // log error
I
Igor Canadi 已提交
2999 3000
        Log(options_.info_log, "[%s] Failed to parse key: %s",
            cfd->GetName().c_str(), key.ToString().c_str());
D
Danny Guo 已提交
3001 3002 3003
        continue;
      } else {
        // If the prefix remains the same, keep buffering
3004
        if (key_prefix.compare(Slice(compact->cur_prefix_)) == 0) {
D
Danny Guo 已提交
3005 3006 3007 3008 3009 3010 3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022 3023 3024
          // Apply the compaction filter V2 to all the kv pairs sharing
          // the same prefix
          if (ikey.type == kTypeValue &&
              (visible_at_tip || ikey.sequence > latest_snapshot)) {
            // Buffer all keys sharing the same prefix for CompactionFilterV2
            // Iterate through keys to check prefix
            compact->BufferKeyValueSlices(key, value);
          } else {
            // buffer ineligible keys
            compact->BufferOtherKeyValueSlices(key, value);
          }
          backup_input->Next();
          continue;
          // finish changing values for eligible keys
        } else {
          // Now prefix changes, this batch is done.
          // Call compaction filter on the buffered values to change the value
          if (compact->key_buf_.size() > 0) {
            CallCompactionFilterV2(compact, compaction_filter_v2);
          }
3025
          compact->cur_prefix_ = key_prefix.ToString();
D
Danny Guo 已提交
3026 3027 3028 3029
        }
      }

      // Merge this batch of data (values + ineligible keys)
3030
      compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3031 3032 3033 3034

      // Done buffering for the current prefix. Spit it out to disk
      // Now just iterate through all the kv-pairs
      status = ProcessKeyValueCompaction(
3035
          is_snapshot_supported,
D
Danny Guo 已提交
3036 3037 3038 3039 3040 3041 3042 3043 3044 3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062 3063 3064 3065 3066 3067
          visible_at_tip,
          earliest_snapshot,
          latest_snapshot,
          deletion_state,
          bottommost_level,
          imm_micros,
          input.get(),
          compact,
          true,
          log_buffer);

      if (!status.ok()) {
        break;
      }

      // After writing the kv-pairs, we can safely remove the reference
      // to the string buffer and clean them up
      compact->CleanupBatchBuffer();
      compact->CleanupMergedBuffer();
      // Buffer the key that triggers the mismatch in prefix
      if (ikey.type == kTypeValue &&
        (visible_at_tip || ikey.sequence > latest_snapshot)) {
        compact->BufferKeyValueSlices(key, value);
      } else {
        compact->BufferOtherKeyValueSlices(key, value);
      }
      backup_input->Next();
      if (!backup_input->Valid()) {
        // If this is the single last value, we need to merge it.
        if (compact->key_buf_.size() > 0) {
          CallCompactionFilterV2(compact, compaction_filter_v2);
        }
3068
        compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3069 3070

        status = ProcessKeyValueCompaction(
3071
            is_snapshot_supported,
D
Danny Guo 已提交
3072 3073 3074 3075 3076 3077 3078 3079 3080 3081 3082 3083 3084 3085 3086 3087 3088 3089 3090
            visible_at_tip,
            earliest_snapshot,
            latest_snapshot,
            deletion_state,
            bottommost_level,
            imm_micros,
            input.get(),
            compact,
            true,
            log_buffer);

        compact->CleanupBatchBuffer();
        compact->CleanupMergedBuffer();
      }
    }  // done processing all prefix batches
    // finish the last batch
    if (compact->key_buf_.size() > 0) {
      CallCompactionFilterV2(compact, compaction_filter_v2);
    }
3091
    compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3092
    status = ProcessKeyValueCompaction(
3093
        is_snapshot_supported,
D
Danny Guo 已提交
3094 3095 3096 3097 3098 3099 3100 3101 3102 3103 3104 3105 3106 3107
        visible_at_tip,
        earliest_snapshot,
        latest_snapshot,
        deletion_state,
        bottommost_level,
        imm_micros,
        input.get(),
        compact,
        true,
        log_buffer);
  }  // checking for compaction filter v2

  if (!compaction_filter_v2) {
    status = ProcessKeyValueCompaction(
3108
      is_snapshot_supported,
D
Danny Guo 已提交
3109 3110 3111 3112 3113 3114 3115 3116 3117 3118 3119 3120
      visible_at_tip,
      earliest_snapshot,
      latest_snapshot,
      deletion_state,
      bottommost_level,
      imm_micros,
      input.get(),
      compact,
      false,
      log_buffer);
  }

3121
  if (status.ok() && (shutting_down_.Acquire_Load() || cfd->IsDropped())) {
L
Lei Jin 已提交
3122
    status = Status::ShutdownInProgress(
3123
        "Database shutdown or Column family drop during compaction");
J
jorlow@chromium.org 已提交
3124
  }
3125
  if (status.ok() && compact->builder != nullptr) {
3126
    status = FinishCompactionOutputFile(compact, input.get());
J
jorlow@chromium.org 已提交
3127 3128 3129 3130
  }
  if (status.ok()) {
    status = input->status();
  }
3131
  input.reset();
J
jorlow@chromium.org 已提交
3132

3133 3134 3135
  if (!options_.disableDataSync) {
    db_directory_->Fsync();
  }
I
Igor Canadi 已提交
3136

L
Lei Jin 已提交
3137
  InternalStats::CompactionStats stats(1);
3138
  stats.micros = env_->NowMicros() - start_micros - imm_micros;
M
Mark Callaghan 已提交
3139 3140
  stats.files_in_leveln = compact->compaction->num_input_files(0);
  stats.files_in_levelnp1 = compact->compaction->num_input_files(1);
L
Lei Jin 已提交
3141
  MeasureTime(stats_, COMPACTION_TIME, stats.micros);
3142 3143

  int num_output_files = compact->outputs.size();
3144
  if (compact->builder != nullptr) {
P
Pascal Borreli 已提交
3145
    // An error occurred so ignore the last output.
3146 3147 3148 3149
    assert(num_output_files > 0);
    --num_output_files;
  }
  stats.files_out_levelnp1 = num_output_files;
M
Mark Callaghan 已提交
3150

3151
  for (int i = 0; i < compact->compaction->num_input_files(0); i++) {
3152
    stats.bytes_readn += compact->compaction->input(0, i)->fd.GetFileSize();
3153
  }
M
Mark Callaghan 已提交
3154

3155
  for (int i = 0; i < compact->compaction->num_input_files(1); i++) {
3156
    stats.bytes_readnp1 += compact->compaction->input(1, i)->fd.GetFileSize();
3157
  }
M
Mark Callaghan 已提交
3158

3159
  for (int i = 0; i < num_output_files; i++) {
3160 3161 3162
    stats.bytes_written += compact->outputs[i].file_size;
  }

3163 3164
  RecordCompactionIOStats();

I
Igor Canadi 已提交
3165
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
3166
  mutex_.Lock();
3167 3168
  cfd->internal_stats()->AddCompactionStats(
      compact->compaction->output_level(), stats);
J
jorlow@chromium.org 已提交
3169

3170 3171 3172 3173
  // if there were any unused file number (mostly in case of
  // compaction error), free up the entry from pending_putputs
  ReleaseCompactionUnusedFileNumbers(compact);

J
jorlow@chromium.org 已提交
3174
  if (status.ok()) {
3175
    status = InstallCompactionResults(compact, log_buffer);
I
Igor Canadi 已提交
3176
    InstallSuperVersion(cfd, deletion_state);
J
jorlow@chromium.org 已提交
3177
  }
3178
  Version::LevelSummaryStorage tmp;
3179 3180
  LogToBuffer(
      log_buffer,
I
Igor Canadi 已提交
3181
      "[%s] compacted to: %s, %.1f MB/sec, level %d, files in(%d, %d) out(%d) "
3182 3183
      "MB in(%.1f, %.1f) out(%.1f), read-write-amplify(%.1f) "
      "write-amplify(%.1f) %s\n",
I
Igor Canadi 已提交
3184
      cfd->GetName().c_str(), cfd->current()->LevelSummary(&tmp),
M
Mark Callaghan 已提交
3185
      (stats.bytes_readn + stats.bytes_readnp1 + stats.bytes_written) /
3186 3187 3188 3189
          (double)stats.micros,
      compact->compaction->output_level(), stats.files_in_leveln,
      stats.files_in_levelnp1, stats.files_out_levelnp1,
      stats.bytes_readn / 1048576.0, stats.bytes_readnp1 / 1048576.0,
M
Mark Callaghan 已提交
3190
      stats.bytes_written / 1048576.0,
3191
      (stats.bytes_written + stats.bytes_readnp1 + stats.bytes_readn) /
3192 3193
          (double)stats.bytes_readn,
      stats.bytes_written / (double)stats.bytes_readn,
3194
      status.ToString().c_str());
M
Mark Callaghan 已提交
3195

J
jorlow@chromium.org 已提交
3196 3197 3198
  return status;
}

3199 3200
namespace {
struct IterState {
3201 3202
  IterState(DBImpl* db, port::Mutex* mu, SuperVersion* super_version)
      : db(db), mu(mu), super_version(super_version) {}
3203 3204

  DBImpl* db;
3205
  port::Mutex* mu;
3206
  SuperVersion* super_version;
3207 3208 3209 3210
};

static void CleanupIteratorState(void* arg1, void* arg2) {
  IterState* state = reinterpret_cast<IterState*>(arg1);
3211

3212
  if (state->super_version->Unref()) {
3213 3214
    DBImpl::DeletionState deletion_state;

3215 3216 3217 3218 3219 3220
    state->mu->Lock();
    state->super_version->Cleanup();
    state->db->FindObsoleteFiles(deletion_state, false, true);
    state->mu->Unlock();

    delete state->super_version;
3221 3222 3223
    if (deletion_state.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(deletion_state);
    }
I
Igor Canadi 已提交
3224
  }
T
Tomislav Novak 已提交
3225

3226 3227
  delete state;
}
H
Hans Wennborg 已提交
3228
}  // namespace
3229

J
jorlow@chromium.org 已提交
3230
Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
3231
                                      ColumnFamilyData* cfd,
3232 3233 3234 3235 3236 3237 3238 3239 3240 3241 3242 3243 3244 3245 3246 3247 3248 3249 3250 3251 3252 3253 3254 3255 3256 3257 3258 3259
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
  if (arena != nullptr) {
    // Need to create internal iterator from the arena.
    MergeIteratorBuilder merge_iter_builder(&cfd->internal_comparator(), arena);
    // Collect iterator for mutable mem
    merge_iter_builder.AddIterator(
        super_version->mem->NewIterator(options, false, arena));
    // Collect all needed child iterators for immutable memtables
    super_version->imm->AddIterators(options, &merge_iter_builder);
    // Collect iterators for files in L0 - Ln
    super_version->current->AddIterators(options, storage_options_,
                                         &merge_iter_builder);
    internal_iter = merge_iter_builder.Finish();
  } else {
    // Need to create internal iterator using malloc.
    std::vector<Iterator*> iterator_list;
    // Collect iterator for mutable mem
    iterator_list.push_back(super_version->mem->NewIterator(options));
    // Collect all needed child iterators for immutable memtables
    super_version->imm->AddIterators(options, &iterator_list);
    // Collect iterators for files in L0 - Ln
    super_version->current->AddIterators(options, storage_options_,
                                         &iterator_list);
    internal_iter = NewMergingIterator(&cfd->internal_comparator(),
                                       &iterator_list[0], iterator_list.size());
  }
3260
  IterState* cleanup = new IterState(this, &mutex_, super_version);
3261
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
3262 3263 3264 3265

  return internal_iter;
}

3266 3267 3268 3269
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

J
jorlow@chromium.org 已提交
3270
Status DBImpl::Get(const ReadOptions& options,
3271
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
3272
                   std::string* value) {
3273
  return GetImpl(options, column_family, key, value);
3274 3275
}

I
Igor Canadi 已提交
3276 3277 3278
// DeletionState gets created and destructed outside of the lock -- we
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
3279
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
3280 3281 3282 3283 3284 3285
//
// However, if InstallSuperVersion() gets called twice with the same,
// deletion_state, we can't reuse the SuperVersion() that got malloced because
// first call already used it. In that rare case, we take a hit and create a
// new SuperVersion() inside of the mutex. We do similar thing
// for superversion_to_free
3286 3287
void DBImpl::InstallSuperVersion(ColumnFamilyData* cfd,
                                 DeletionState& deletion_state) {
3288
  mutex_.AssertHeld();
I
Igor Canadi 已提交
3289 3290 3291 3292
  // if new_superversion == nullptr, it means somebody already used it
  SuperVersion* new_superversion =
    (deletion_state.new_superversion != nullptr) ?
    deletion_state.new_superversion : new SuperVersion();
3293 3294
  SuperVersion* old_superversion =
      cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
3295
  deletion_state.new_superversion = nullptr;
3296
  deletion_state.superversions_to_free.push_back(old_superversion);
I
Igor Canadi 已提交
3297 3298
}

3299
Status DBImpl::GetImpl(const ReadOptions& options,
3300 3301
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
L
Lei Jin 已提交
3302
  StopWatch sw(env_, stats_, DB_GET);
L
Lei Jin 已提交
3303
  PERF_TIMER_AUTO(get_snapshot_time);
3304

3305 3306 3307
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3308
  SequenceNumber snapshot;
3309
  if (options.snapshot != nullptr) {
3310 3311 3312
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
3313
  }
3314

3315 3316
  // Acquire SuperVersion
  SuperVersion* sv = nullptr;
3317
  // TODO(ljin): consider using GetReferencedSuperVersion() directly
3318
  if (LIKELY(options_.allow_thread_local)) {
3319
    sv = cfd->GetThreadLocalSuperVersion(&mutex_);
3320 3321
  } else {
    mutex_.Lock();
3322
    sv = cfd->GetSuperVersion()->Ref();
3323 3324
    mutex_.Unlock();
  }
I
Igor Canadi 已提交
3325

3326
  // Prepare to store a list of merge operations if merge occurs.
3327
  MergeContext merge_context;
3328

3329
  Status s;
3330
  // First look in the memtable, then in the immutable memtable (if any).
3331
  // s is both in/out. When in, s could either be OK or MergeInProgress.
3332
  // merge_operands will contain the sequence of merges in the latter case.
3333
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
3334
  PERF_TIMER_STOP(get_snapshot_time);
I
Igor Canadi 已提交
3335
  if (sv->mem->Get(lkey, value, &s, merge_context, *cfd->options())) {
3336
    // Done
L
Lei Jin 已提交
3337
    RecordTick(stats_, MEMTABLE_HIT);
I
Igor Canadi 已提交
3338
  } else if (sv->imm->Get(lkey, value, &s, merge_context, *cfd->options())) {
3339
    // Done
L
Lei Jin 已提交
3340
    RecordTick(stats_, MEMTABLE_HIT);
3341
  } else {
L
Lei Jin 已提交
3342
    PERF_TIMER_START(get_from_output_files_time);
3343

I
Igor Canadi 已提交
3344
    sv->current->Get(options, lkey, value, &s, &merge_context, value_found);
L
Lei Jin 已提交
3345
    PERF_TIMER_STOP(get_from_output_files_time);
L
Lei Jin 已提交
3346
    RecordTick(stats_, MEMTABLE_MISS);
3347
  }
3348

L
Lei Jin 已提交
3349
  PERF_TIMER_START(get_post_process_time);
3350

3351
  bool unref_sv = true;
3352
  if (LIKELY(options_.allow_thread_local)) {
3353
    unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3354 3355 3356 3357
  }

  if (unref_sv) {
    // Release SuperVersion
3358
    if (sv->Unref()) {
I
Igor Canadi 已提交
3359
      mutex_.Lock();
3360
      sv->Cleanup();
I
Igor Canadi 已提交
3361
      mutex_.Unlock();
3362
      delete sv;
L
Lei Jin 已提交
3363
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
I
Igor Canadi 已提交
3364
    }
L
Lei Jin 已提交
3365
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
3366
  }
3367

L
Lei Jin 已提交
3368 3369
  RecordTick(stats_, NUMBER_KEYS_READ);
  RecordTick(stats_, BYTES_READ, value->size());
L
Lei Jin 已提交
3370
  PERF_TIMER_STOP(get_post_process_time);
3371
  return s;
J
jorlow@chromium.org 已提交
3372 3373
}

3374 3375
std::vector<Status> DBImpl::MultiGet(
    const ReadOptions& options,
3376
    const std::vector<ColumnFamilyHandle*>& column_family,
3377
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
3378

L
Lei Jin 已提交
3379
  StopWatch sw(env_, stats_, DB_MULTIGET);
L
Lei Jin 已提交
3380
  PERF_TIMER_AUTO(get_snapshot_time);
K
Kai Liu 已提交
3381

3382
  SequenceNumber snapshot;
3383

3384
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
3385
    ColumnFamilyData* cfd;
3386 3387 3388 3389 3390
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
3391 3392 3393 3394 3395 3396
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(cf);
    auto cfd = cfh->cfd();
    if (multiget_cf_data.find(cfd->GetID()) == multiget_cf_data.end()) {
      auto mgcfd = new MultiGetColumnFamilyData();
      mgcfd->cfd = cfd;
      multiget_cf_data.insert({cfd->GetID(), mgcfd});
3397 3398 3399
    }
  }

3400
  mutex_.Lock();
3401 3402 3403 3404 3405
  if (options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
  }
3406
  for (auto mgd_iter : multiget_cf_data) {
3407 3408
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
3409
  }
3410
  mutex_.Unlock();
3411

3412 3413
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
3414

3415
  // Note: this always resizes the values array
3416 3417 3418
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
3419 3420

  // Keep track of bytes that we read for statistics-recording later
3421
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
3422
  PERF_TIMER_STOP(get_snapshot_time);
3423 3424 3425 3426

  // For each of the given keys, apply the entire "get" process as follows:
  // First look in the memtable, then in the immutable memtable (if any).
  // s is both in/out. When in, s could either be OK or MergeInProgress.
3427
  // merge_operands will contain the sequence of merges in the latter case.
3428
  for (size_t i = 0; i < num_keys; ++i) {
3429
    merge_context.Clear();
3430
    Status& s = stat_list[i];
3431 3432 3433
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
3434 3435
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
3436 3437 3438
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
I
Igor Canadi 已提交
3439 3440
    auto cfd = mgd->cfd;
    if (super_version->mem->Get(lkey, value, &s, merge_context,
I
Igor Canadi 已提交
3441
                                *cfd->options())) {
3442
      // Done
3443
    } else if (super_version->imm->Get(lkey, value, &s, merge_context,
I
Igor Canadi 已提交
3444
                                       *cfd->options())) {
3445 3446
      // Done
    } else {
I
Igor Canadi 已提交
3447
      super_version->current->Get(options, lkey, value, &s, &merge_context);
3448 3449 3450
    }

    if (s.ok()) {
3451
      bytes_read += value->size();
3452 3453 3454 3455
    }
  }

  // Post processing (decrement reference counts and record statistics)
L
Lei Jin 已提交
3456
  PERF_TIMER_START(get_post_process_time);
3457 3458
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
3459
  // TODO(icanadi) do we need lock here or just around Cleanup()?
3460 3461 3462 3463 3464 3465
  mutex_.Lock();
  for (auto mgd_iter : multiget_cf_data) {
    auto mgd = mgd_iter.second;
    if (mgd->super_version->Unref()) {
      mgd->super_version->Cleanup();
      superversions_to_delete.push_back(mgd->super_version);
3466 3467
    }
  }
3468 3469 3470 3471 3472 3473 3474
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3475
  }
3476

L
Lei Jin 已提交
3477 3478 3479
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3480
  PERF_TIMER_STOP(get_post_process_time);
3481

3482
  return stat_list;
3483 3484
}

3485
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& options,
3486
                                  const std::string& column_family_name,
3487
                                  ColumnFamilyHandle** handle) {
I
Igor Canadi 已提交
3488 3489 3490
  *handle = nullptr;
  MutexLock l(&mutex_);

I
Igor Canadi 已提交
3491 3492
  if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
      nullptr) {
I
Igor Canadi 已提交
3493 3494
    return Status::InvalidArgument("Column family already exists");
  }
3495
  VersionEdit edit;
3496
  edit.AddColumnFamily(column_family_name);
3497 3498
  uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
  edit.SetColumnFamily(new_id);
I
Igor Canadi 已提交
3499
  edit.SetLogNumber(logfile_number_);
I
Igor Canadi 已提交
3500
  edit.SetComparatorName(options.comparator->Name());
3501

I
Igor Canadi 已提交
3502 3503
  // LogAndApply will both write the creation in MANIFEST and create
  // ColumnFamilyData object
3504 3505
  Status s = versions_->LogAndApply(nullptr, &edit, &mutex_,
                                    db_directory_.get(), false, &options);
3506
  if (s.ok()) {
3507
    single_column_family_mode_ = false;
3508 3509 3510
    auto cfd =
        versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
    assert(cfd != nullptr);
3511
    delete cfd->InstallSuperVersion(new SuperVersion(), &mutex_);
3512
    *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
I
Igor Canadi 已提交
3513
    Log(options_.info_log, "Created column family [%s] (ID %u)",
3514
        column_family_name.c_str(), (unsigned)cfd->GetID());
I
Igor Canadi 已提交
3515 3516
    max_total_in_memory_state_ += cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3517
  } else {
I
Igor Canadi 已提交
3518
    Log(options_.info_log, "Creating column family [%s] FAILED -- %s",
3519 3520
        column_family_name.c_str(), s.ToString().c_str());
  }
3521
  return s;
3522 3523
}

3524 3525 3526 3527
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3528 3529
    return Status::InvalidArgument("Can't drop default column family");
  }
3530

I
Igor Canadi 已提交
3531 3532
  VersionEdit edit;
  edit.DropColumnFamily();
3533 3534
  edit.SetColumnFamily(cfd->GetID());

3535
  Status s;
3536 3537 3538 3539 3540 3541 3542 3543
  {
    MutexLock l(&mutex_);
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
      s = versions_->LogAndApply(cfd, &edit, &mutex_);
    }
3544
  }
3545

3546
  if (s.ok()) {
I
Igor Canadi 已提交
3547
    assert(cfd->IsDropped());
I
Igor Canadi 已提交
3548 3549
    max_total_in_memory_state_ -= cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3550
    Log(options_.info_log, "Dropped column family with id %u\n", cfd->GetID());
3551
    // Flush the memtables. This will make all WAL files referencing dropped
I
Igor Canadi 已提交
3552
    // column family to be obsolete. They will be deleted once user deletes
3553 3554
    // column family handle
    Write(WriteOptions(), nullptr);  // ignore error
3555 3556 3557 3558 3559
  } else {
    Log(options_.info_log, "Dropping column family with id %u FAILED -- %s\n",
        cfd->GetID(), s.ToString().c_str());
  }

3560
  return s;
3561 3562
}

3563
bool DBImpl::KeyMayExist(const ReadOptions& options,
3564 3565
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3566
  if (value_found != nullptr) {
K
Kai Liu 已提交
3567 3568
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3569
  }
3570 3571
  ReadOptions roptions = options;
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3572
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3573 3574 3575 3576 3577

  // If options.block_cache != nullptr and the index block of the table didn't
  // not present in block_cache, the return value will be Status::Incomplete.
  // In this case, key may still exist in the table.
  return s.ok() || s.IsIncomplete();
3578 3579
}

3580
Iterator* DBImpl::NewIterator(const ReadOptions& options,
3581 3582 3583
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3584 3585

  if (options.tailing) {
I
Igor Canadi 已提交
3586 3587 3588 3589
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
L
Lei Jin 已提交
3590
    // TODO(ljin): remove tailing iterator
I
Igor Canadi 已提交
3591
    auto iter = new ForwardIterator(this, options, cfd);
3592 3593 3594
    return NewDBIterator(env_, *cfd->options(), cfd->user_comparator(), iter,
                         kMaxSequenceNumber);
// return new TailingIterator(env_, this, options, cfd);
I
Igor Canadi 已提交
3595
#endif
T
Tomislav Novak 已提交
3596
  } else {
3597 3598 3599 3600
    SequenceNumber latest_snapshot = versions_->LastSequence();
    SuperVersion* sv = nullptr;
    sv = cfd->GetReferencedSuperVersion(&mutex_);

I
Igor Canadi 已提交
3601 3602 3603 3604
    auto snapshot =
        options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_
            : latest_snapshot;
T
Tomislav Novak 已提交
3605

3606 3607 3608 3609 3610 3611 3612 3613 3614 3615 3616 3617 3618 3619 3620 3621 3622 3623 3624 3625 3626 3627 3628 3629 3630 3631 3632 3633 3634 3635 3636 3637 3638 3639 3640 3641 3642 3643 3644 3645 3646 3647 3648 3649 3650 3651 3652 3653 3654 3655
    // Try to generate a DB iterator tree in continuous memory area to be
    // cache friendly. Here is an example of result:
    // +-------------------------------+
    // |                               |
    // | ArenaWrappedDBIter            |
    // |  +                            |
    // |  +---> Inner Iterator   ------------+
    // |  |                            |     |
    // |  |    +-- -- -- -- -- -- -- --+     |
    // |  +--- | Arena                 |     |
    // |       |                       |     |
    // |          Allocated Memory:    |     |
    // |       |   +-------------------+     |
    // |       |   | DBIter            | <---+
    // |           |  +                |
    // |       |   |  +-> iter_  ------------+
    // |       |   |                   |     |
    // |       |   +-------------------+     |
    // |       |   | MergingIterator   | <---+
    // |           |  +                |
    // |       |   |  +->child iter1  ------------+
    // |       |   |  |                |          |
    // |           |  +->child iter2  ----------+ |
    // |       |   |  |                |        | |
    // |       |   |  +->child iter3  --------+ | |
    // |           |                   |      | | |
    // |       |   +-------------------+      | | |
    // |       |   | Iterator1         | <--------+
    // |       |   +-------------------+      | |
    // |       |   | Iterator2         | <------+
    // |       |   +-------------------+      |
    // |       |   | Iterator3         | <----+
    // |       |   +-------------------+
    // |       |                       |
    // +-------+-----------------------+
    //
    // ArenaWrappedDBIter inlines an arena area where all the iterartor in the
    // the iterator tree is allocated in the order of being accessed when
    // querying.
    // Laying out the iterators in the order of being accessed makes it more
    // likely that any iterator pointer is close to the iterator it points to so
    // that they are likely to be in the same cache line and/or page.
    ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
        env_, *cfd->options(), cfd->user_comparator(), snapshot);
    Iterator* internal_iter =
        NewInternalIterator(options, cfd, sv, db_iter->GetArena());
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
J
jorlow@chromium.org 已提交
3656 3657
}

3658 3659
Status DBImpl::NewIterators(
    const ReadOptions& options,
I
Igor Canadi 已提交
3660
    const std::vector<ColumnFamilyHandle*>& column_families,
3661
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3662 3663 3664 3665 3666 3667 3668 3669 3670 3671 3672 3673 3674 3675 3676 3677 3678
  iterators->clear();
  iterators->reserve(column_families.size());
  SequenceNumber latest_snapshot = 0;
  std::vector<SuperVersion*> super_versions;
  super_versions.reserve(column_families.size());

  if (!options.tailing) {
    mutex_.Lock();
    latest_snapshot = versions_->LastSequence();
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
      super_versions.push_back(cfd->GetSuperVersion()->Ref());
    }
    mutex_.Unlock();
  }

  if (options.tailing) {
I
Igor Canadi 已提交
3679 3680 3681 3682
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3683 3684
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
L
Lei Jin 已提交
3685
      iterators->push_back(new TailingIterator(env_, this, options, cfd));
I
Igor Canadi 已提交
3686
    }
I
Igor Canadi 已提交
3687
#endif
I
Igor Canadi 已提交
3688 3689 3690 3691 3692 3693 3694 3695 3696 3697 3698
  } else {
    for (size_t i = 0; i < column_families.size(); ++i) {
      auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_families[i]);
      auto cfd = cfh->cfd();

      auto snapshot =
          options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_
              : latest_snapshot;

      auto iter = NewInternalIterator(options, cfd, super_versions[i]);
L
Lei Jin 已提交
3699
      iter = NewDBIterator(env_, *cfd->options(),
I
Igor Canadi 已提交
3700 3701 3702 3703 3704 3705
                           cfd->user_comparator(), iter, snapshot);
      iterators->push_back(iter);
    }
  }

  return Status::OK();
3706 3707
}

3708 3709 3710 3711 3712 3713 3714 3715 3716
bool DBImpl::IsSnapshotSupported() const {
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (!cfd->mem()->IsSnapshotSupported()) {
      return false;
    }
  }
  return true;
}

J
jorlow@chromium.org 已提交
3717
const Snapshot* DBImpl::GetSnapshot() {
3718
  MutexLock l(&mutex_);
3719 3720
  // returns null if the underlying memtable does not support snapshot.
  if (!IsSnapshotSupported()) return nullptr;
3721
  return snapshots_.New(versions_->LastSequence());
J
jorlow@chromium.org 已提交
3722 3723 3724 3725
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
  MutexLock l(&mutex_);
3726
  snapshots_.Delete(reinterpret_cast<const SnapshotImpl*>(s));
J
jorlow@chromium.org 已提交
3727 3728 3729
}

// Convenience methods
3730 3731
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3732
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3733 3734
}

3735 3736 3737 3738
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  if (!cfh->cfd()->options()->merge_operator) {
3739 3740
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3741
    return DB::Merge(o, column_family, key, val);
3742 3743 3744
  }
}

3745
Status DBImpl::Delete(const WriteOptions& options,
3746
                      ColumnFamilyHandle* column_family, const Slice& key) {
3747
  return DB::Delete(options, column_family, key);
J
jorlow@chromium.org 已提交
3748 3749
}

3750
Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
L
Lei Jin 已提交
3751
  PERF_TIMER_AUTO(write_pre_and_post_process_time);
3752 3753 3754
  Writer w(&mutex_);
  w.batch = my_batch;
  w.sync = options.sync;
H
heyongqiang 已提交
3755
  w.disableWAL = options.disableWAL;
3756 3757 3758 3759 3760 3761 3762 3763 3764 3765
  w.in_batch_group = false;
  w.done = false;
  w.timeout_hint_us = options.timeout_hint_us;

  uint64_t expiration_time = 0;
  if (w.timeout_hint_us == 0) {
    w.timeout_hint_us = kNoTimeOut;
  } else {
    expiration_time = env_->NowMicros() + w.timeout_hint_us;
  }
3766
  w.done = false;
3767

I
Igor Canadi 已提交
3768
  mutex_.Lock();
3769 3770 3771 3772 3773
  // the following code block pushes the current writer "w" into the writer
  // queue "writers_" and wait until one of the following conditions met:
  // 1. the job of "w" has been done by some other writers.
  // 2. "w" becomes the first writer in "writers_"
  // 3. "w" timed-out.
3774
  writers_.push_back(&w);
3775 3776

  bool timed_out = false;
3777
  while (!w.done && &w != writers_.front()) {
3778 3779 3780 3781 3782 3783 3784 3785 3786 3787 3788 3789 3790
    if (expiration_time == 0) {
      w.cv.Wait();
    } else if (w.cv.TimedWait(expiration_time)) {
      if (w.in_batch_group) {
        // then it means the front writer is currently doing the
        // write on behalf of this "timed-out" writer.  Then it
        // should wait until the write completes.
        expiration_time = 0;
      } else {
        timed_out = true;
        break;
      }
    }
3791
  }
3792 3793

  if (!options.disableWAL) {
L
Lei Jin 已提交
3794
    RecordTick(stats_, WRITE_WITH_WAL);
3795 3796
    default_cf_internal_stats_->AddDBStats(
        InternalStats::WRITE_WITH_WAL, 1);
3797 3798
  }

3799
  if (w.done) {
3800 3801
    default_cf_internal_stats_->AddDBStats(
        InternalStats::WRITE_DONE_BY_OTHER, 1);
I
Igor Canadi 已提交
3802
    mutex_.Unlock();
L
Lei Jin 已提交
3803
    RecordTick(stats_, WRITE_DONE_BY_OTHER);
3804
    return w.status;
3805
  } else if (timed_out) {
3806
#ifndef NDEBUG
3807
    bool found = false;
3808
#endif
3809 3810 3811
    for (auto iter = writers_.begin(); iter != writers_.end(); iter++) {
      if (*iter == &w) {
        writers_.erase(iter);
3812
#ifndef NDEBUG
3813
        found = true;
3814
#endif
3815 3816 3817
        break;
      }
    }
3818
#ifndef NDEBUG
3819
    assert(found);
3820
#endif
3821 3822 3823 3824 3825 3826 3827
    // writers_.front() might still be in cond_wait without a time-out.
    // As a result, we need to signal it to wake it up.  Otherwise no
    // one else will wake him up, and RocksDB will hang.
    if (!writers_.empty()) {
      writers_.front()->cv.Signal();
    }
    mutex_.Unlock();
L
Lei Jin 已提交
3828
    RecordTick(stats_, WRITE_TIMEDOUT);
3829
    return Status::TimedOut();
3830
  } else {
L
Lei Jin 已提交
3831
    RecordTick(stats_, WRITE_DONE_BY_SELF);
3832 3833
    default_cf_internal_stats_->AddDBStats(
        InternalStats::WRITE_DONE_BY_SELF, 1);
3834 3835
  }

3836 3837 3838 3839
  // Once reaches this point, the current writer "w" will try to do its write
  // job.  It may also pick up some of the remaining writers in the "writers_"
  // when it finds suitable, and finish them in the same write batch.
  // This is how a write job could be done by the other writer.
3840 3841 3842
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

I
Igor Canadi 已提交
3843
  uint64_t flush_column_family_if_log_file = 0;
I
Igor Canadi 已提交
3844
  uint64_t max_total_wal_size = (options_.max_total_wal_size == 0)
3845
                                    ? 4 * max_total_in_memory_state_
I
Igor Canadi 已提交
3846
                                    : options_.max_total_wal_size;
3847
  if (UNLIKELY(!single_column_family_mode_) &&
3848
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3849 3850 3851 3852
      total_log_size_ > max_total_wal_size) {
    flush_column_family_if_log_file = alive_log_files_.begin()->number;
    alive_log_files_.begin()->getting_flushed = true;
    Log(options_.info_log,
3853 3854 3855
        "Flushing all column families with data in WAL number %" PRIu64
        ". Total log size is %" PRIu64 " while max_total_wal_size is %" PRIu64,
        flush_column_family_if_log_file, total_log_size_, max_total_wal_size);
I
Igor Canadi 已提交
3856 3857
  }

3858
  Status status;
3859 3860
  autovector<SuperVersion*> superversions_to_free;
  autovector<log::Writer*> logs_to_free;
3861 3862 3863

  if (LIKELY(single_column_family_mode_)) {
    // fast path
3864 3865 3866 3867
    status = MakeRoomForWrite(
        default_cf_handle_->cfd(), my_batch == nullptr,
        &superversions_to_free, &logs_to_free,
        expiration_time);
3868 3869 3870 3871 3872 3873 3874 3875 3876 3877
  } else {
    // refcounting cfd in iteration
    bool dead_cfd = false;
    for (auto cfd : *versions_->GetColumnFamilySet()) {
      cfd->Ref();
      bool force_flush =
          my_batch == nullptr ||
          (flush_column_family_if_log_file != 0 &&
           cfd->GetLogNumber() <= flush_column_family_if_log_file);
      // May temporarily unlock and wait.
3878 3879 3880
      status = MakeRoomForWrite(
          cfd, force_flush, &superversions_to_free, &logs_to_free,
          expiration_time);
3881 3882 3883 3884 3885 3886
      if (cfd->Unref()) {
        dead_cfd = true;
      }
      if (!status.ok()) {
        break;
      }
3887
    }
3888 3889
    if (dead_cfd) {
      versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
3890 3891
    }
  }
3892

D
dgrogan@chromium.org 已提交
3893
  uint64_t last_sequence = versions_->LastSequence();
3894
  Writer* last_writer = &w;
3895
  if (status.ok() && my_batch != nullptr) {  // nullptr batch is for compactions
3896 3897
    autovector<WriteBatch*> write_batch_group;
    BuildBatchGroup(&last_writer, &write_batch_group);
3898

3899 3900 3901
    // Add to log and apply to memtable.  We can release the lock
    // during this phase since &w is currently responsible for logging
    // and protects against concurrent loggers and concurrent writes
3902
    // into memtables
3903
    {
3904
      mutex_.Unlock();
3905 3906 3907 3908 3909 3910 3911 3912 3913 3914
      WriteBatch* updates = nullptr;
      if (write_batch_group.size() == 1) {
        updates = write_batch_group[0];
      } else {
        updates = &tmp_batch_;
        for (size_t i = 0; i < write_batch_group.size(); ++i) {
          WriteBatchInternal::Append(updates, write_batch_group[i]);
        }
      }

3915 3916 3917 3918
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
3919
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
3920
      // Record statistics
L
Lei Jin 已提交
3921 3922
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
      RecordTick(stats_, BYTES_WRITTEN, WriteBatchInternal::ByteSize(updates));
3923 3924
      if (options.disableWAL) {
        flush_on_destroy_ = true;
3925
      }
L
Lei Jin 已提交
3926
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3927

3928
      uint64_t log_size = 0;
3929
      if (!options.disableWAL) {
L
Lei Jin 已提交
3930
        PERF_TIMER_START(write_wal_time);
3931 3932
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
3933 3934
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
3935
        log_empty_ = false;
3936
        log_size = log_entry.size();
L
Lei Jin 已提交
3937 3938
        RecordTick(stats_, WAL_FILE_SYNCED);
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
H
heyongqiang 已提交
3939
        if (status.ok() && options.sync) {
3940
          if (options_.use_fsync) {
L
Lei Jin 已提交
3941
            StopWatch(env_, stats_, WAL_FILE_SYNC_MICROS);
3942
            status = log_->file()->Fsync();
3943
          } else {
L
Lei Jin 已提交
3944
            StopWatch(env_, stats_, WAL_FILE_SYNC_MICROS);
3945
            status = log_->file()->Sync();
3946
          }
H
heyongqiang 已提交
3947
        }
L
Lei Jin 已提交
3948
        PERF_TIMER_STOP(write_wal_time);
3949 3950
      }
      if (status.ok()) {
L
Lei Jin 已提交
3951
        PERF_TIMER_START(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
3952

3953
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3954
            updates, column_family_memtables_.get(), false, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
3955 3956 3957 3958 3959 3960 3961 3962
        // A non-OK status here indicates iteration failure (either in-memory
        // writebatch corruption (very bad), or the client specified invalid
        // column family).  This will later on trigger bg_error_.
        //
        // Note that existing logic was not sound. Any partial failure writing
        // into the memtable would result in a state that some write ops might
        // have succeeded in memtable but Status reports error for all writes.

L
Lei Jin 已提交
3963
        PERF_TIMER_STOP(write_memtable_time);
3964

L
Lei Jin 已提交
3965
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
3966
      }
L
Lei Jin 已提交
3967
      PERF_TIMER_START(write_pre_and_post_process_time);
3968 3969 3970
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
3971
      mutex_.Lock();
3972 3973 3974 3975 3976 3977 3978 3979 3980
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
      if (!options.disableWAL) {
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
3981
      if (status.ok()) {
3982
        versions_->SetLastSequence(last_sequence);
3983
      }
J
jorlow@chromium.org 已提交
3984 3985
    }
  }
3986 3987
  if (options_.paranoid_checks && !status.ok() &&
      !status.IsTimedOut() && bg_error_.ok()) {
I
Igor Canadi 已提交
3988 3989
    bg_error_ = status; // stop compaction & fail any further writes
  }
3990

3991 3992 3993
  // Pop out the current writer and all writers being pushed before the
  // current writer from the writer queue.
  while (!writers_.empty()) {
3994 3995 3996 3997 3998 3999
    Writer* ready = writers_.front();
    writers_.pop_front();
    if (ready != &w) {
      ready->status = status;
      ready->done = true;
      ready->cv.Signal();
4000
    }
4001 4002
    if (ready == last_writer) break;
  }
4003

4004 4005 4006
  // Notify new head of write queue
  if (!writers_.empty()) {
    writers_.front()->cv.Signal();
4007
  }
I
Igor Canadi 已提交
4008
  mutex_.Unlock();
4009

4010
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
4011
    RecordTick(stats_, WRITE_TIMEDOUT);
4012 4013
  }

4014 4015 4016 4017 4018 4019 4020
  for (auto& sv : superversions_to_free) {
    delete sv;
  }
  for (auto& log : logs_to_free) {
    delete log;
  }

L
Lei Jin 已提交
4021
  PERF_TIMER_STOP(write_pre_and_post_process_time);
J
jorlow@chromium.org 已提交
4022 4023 4024
  return status;
}

4025 4026 4027
// This function will be called only when the first writer succeeds.
// All writers in the to-be-built batch group will be processed.
//
4028
// REQUIRES: Writer list must be non-empty
4029
// REQUIRES: First writer must have a non-nullptr batch
4030 4031
void DBImpl::BuildBatchGroup(Writer** last_writer,
                             autovector<WriteBatch*>* write_batch_group) {
4032 4033
  assert(!writers_.empty());
  Writer* first = writers_.front();
4034
  assert(first->batch != nullptr);
4035 4036

  size_t size = WriteBatchInternal::ByteSize(first->batch);
4037
  write_batch_group->push_back(first->batch);
4038 4039 4040 4041 4042 4043 4044 4045 4046 4047 4048 4049 4050 4051 4052 4053 4054 4055 4056

  // Allow the group to grow up to a maximum size, but if the
  // original write is small, limit the growth so we do not slow
  // down the small write too much.
  size_t max_size = 1 << 20;
  if (size <= (128<<10)) {
    max_size = size + (128<<10);
  }

  *last_writer = first;
  std::deque<Writer*>::iterator iter = writers_.begin();
  ++iter;  // Advance past "first"
  for (; iter != writers_.end(); ++iter) {
    Writer* w = *iter;
    if (w->sync && !first->sync) {
      // Do not include a sync write into a batch handled by a non-sync write.
      break;
    }

H
heyongqiang 已提交
4057 4058 4059 4060 4061 4062
    if (!w->disableWAL && first->disableWAL) {
      // Do not include a write that needs WAL into a batch that has
      // WAL disabled.
      break;
    }

4063 4064 4065 4066 4067 4068
    if (w->timeout_hint_us < first->timeout_hint_us) {
      // Do not include those writes with shorter timeout.  Otherwise, we might
      // execute a write that should instead be aborted because of timeout.
      break;
    }

4069
    if (w->batch != nullptr) {
4070 4071 4072 4073 4074 4075
      size += WriteBatchInternal::ByteSize(w->batch);
      if (size > max_size) {
        // Do not make batch too big
        break;
      }

4076
      write_batch_group->push_back(w->batch);
4077
    }
4078
    w->in_batch_group = true;
4079 4080 4081 4082
    *last_writer = w;
  }
}

4083 4084 4085
// This function computes the amount of time in microseconds by which a write
// should be delayed based on the number of level-0 files according to the
// following formula:
J
Jim Paton 已提交
4086 4087 4088 4089
// if n < bottom, return 0;
// if n >= top, return 1000;
// otherwise, let r = (n - bottom) /
//                    (top - bottom)
4090 4091 4092 4093
//  and return r^2 * 1000.
// The goal of this formula is to gradually increase the rate at which writes
// are slowed. We also tried linear delay (r * 1000), but it seemed to do
// slightly worse. There is no other particular reason for choosing quadratic.
M
Mark Callaghan 已提交
4094
uint64_t DBImpl::SlowdownAmount(int n, double bottom, double top) {
4095
  uint64_t delay;
J
Jim Paton 已提交
4096
  if (n >= top) {
4097 4098
    delay = 1000;
  }
J
Jim Paton 已提交
4099
  else if (n < bottom) {
4100 4101 4102 4103
    delay = 0;
  }
  else {
    // If we are here, we know that:
J
Jim Paton 已提交
4104
    //   level0_start_slowdown <= n < level0_slowdown
4105
    // since the previous two conditions are false.
M
Mark Callaghan 已提交
4106 4107
    double how_much =
      (double) (n - bottom) /
J
Jim Paton 已提交
4108
              (top - bottom);
M
Mark Callaghan 已提交
4109
    delay = std::max(how_much * how_much * 1000, 100.0);
4110 4111 4112 4113 4114
  }
  assert(delay <= 1000);
  return delay;
}

4115
// REQUIRES: mutex_ is held
4116
// REQUIRES: this thread is currently at the front of the writer queue
4117 4118 4119
Status DBImpl::MakeRoomForWrite(
    ColumnFamilyData* cfd, bool force,
    autovector<SuperVersion*>* superversions_to_free,
4120 4121
    autovector<log::Writer*>* logs_to_free,
    uint64_t expiration_time) {
4122
  mutex_.AssertHeld();
4123
  assert(!writers_.empty());
4124
  bool allow_delay = !force;
J
Jim Paton 已提交
4125 4126
  bool allow_hard_rate_limit_delay = !force;
  bool allow_soft_rate_limit_delay = !force;
4127
  uint64_t rate_limit_delay_millis = 0;
4128
  Status s;
4129
  double score;
4130 4131 4132 4133
  // Once we schedule background work, we shouldn't schedule it again, since it
  // might generate a tight feedback loop, constantly scheduling more background
  // work, even if additional background work is not needed
  bool schedule_background_work = true;
4134
  bool has_timeout = (expiration_time > 0);
4135

4136 4137 4138 4139 4140
  while (true) {
    if (!bg_error_.ok()) {
      // Yield previous error
      s = bg_error_;
      break;
4141 4142 4143
    } else if (has_timeout && env_->NowMicros() > expiration_time) {
      s = Status::TimedOut();
      break;
4144
    } else if (allow_delay && cfd->NeedSlowdownForNumLevel0Files()) {
4145 4146 4147
      // We are getting close to hitting a hard limit on the number of
      // L0 files.  Rather than delaying a single write by several
      // seconds when we hit the hard limit, start delaying each
4148
      // individual write by 0-1ms to reduce latency variance.  Also,
4149 4150
      // this delay hands over some CPU to the compaction thread in
      // case it is sharing the same core as the writer.
4151
      uint64_t slowdown =
4152 4153 4154
          SlowdownAmount(cfd->current()->NumLevelFiles(0),
                         cfd->options()->level0_slowdown_writes_trigger,
                         cfd->options()->level0_stop_writes_trigger);
4155
      mutex_.Unlock();
4156
      uint64_t delayed;
J
Jim Paton 已提交
4157
      {
L
Lei Jin 已提交
4158
        StopWatch sw(env_, stats_, STALL_L0_SLOWDOWN_COUNT, &delayed);
4159
        env_->SleepForMicroseconds(slowdown);
J
Jim Paton 已提交
4160
      }
L
Lei Jin 已提交
4161
      RecordTick(stats_, STALL_L0_SLOWDOWN_MICROS, delayed);
4162 4163
      allow_delay = false;  // Do not delay a single write more than once
      mutex_.Lock();
4164 4165
      cfd->internal_stats()->AddCFStats(
          InternalStats::LEVEL0_SLOWDOWN, delayed);
4166
      delayed_writes_++;
4167
    } else if (!force && !cfd->mem()->ShouldFlush()) {
4168
      // There is room in current memtable
4169 4170 4171
      if (allow_delay) {
        DelayLoggingAndReset();
      }
4172
      break;
4173
    } else if (cfd->NeedWaitForNumMemtables()) {
4174
      // We have filled up the current memtable, but the previous
4175
      // ones are still being flushed, so we wait.
4176
      DelayLoggingAndReset();
I
Igor Canadi 已提交
4177 4178
      Log(options_.info_log, "[%s] wait for memtable flush...\n",
          cfd->GetName().c_str());
4179 4180 4181 4182
      if (schedule_background_work) {
        MaybeScheduleFlushOrCompaction();
        schedule_background_work = false;
      }
4183
      uint64_t stall;
J
Jim Paton 已提交
4184
      {
L
Lei Jin 已提交
4185
        StopWatch sw(env_, stats_, STALL_MEMTABLE_COMPACTION_COUNT, &stall);
4186 4187 4188 4189 4190
        if (!has_timeout) {
          bg_cv_.Wait();
        } else {
          bg_cv_.TimedWait(expiration_time);
        }
J
Jim Paton 已提交
4191
      }
L
Lei Jin 已提交
4192
      RecordTick(stats_, STALL_MEMTABLE_COMPACTION_MICROS, stall);
4193
      cfd->internal_stats()->AddCFStats(
4194
          InternalStats::MEMTABLE_COMPACTION, stall);
4195
    } else if (cfd->NeedWaitForNumLevel0Files()) {
4196
      DelayLoggingAndReset();
I
Igor Canadi 已提交
4197 4198
      Log(options_.info_log, "[%s] wait for fewer level0 files...\n",
          cfd->GetName().c_str());
4199
      uint64_t stall;
J
Jim Paton 已提交
4200
      {
L
Lei Jin 已提交
4201
        StopWatch sw(env_, stats_, STALL_L0_NUM_FILES_COUNT, &stall);
4202 4203 4204 4205 4206
        if (!has_timeout) {
          bg_cv_.Wait();
        } else {
          bg_cv_.TimedWait(expiration_time);
        }
J
Jim Paton 已提交
4207
      }
L
Lei Jin 已提交
4208
      RecordTick(stats_, STALL_L0_NUM_FILES_MICROS, stall);
4209 4210
      cfd->internal_stats()->AddCFStats(
          InternalStats::LEVEL0_NUM_FILES, stall);
4211
    } else if (allow_hard_rate_limit_delay && cfd->ExceedsHardRateLimit()) {
4212
      // Delay a write when the compaction score for any level is too large.
4213
      const int max_level = cfd->current()->MaxCompactionScoreLevel();
4214
      score = cfd->current()->MaxCompactionScore();
4215
      mutex_.Unlock();
4216
      uint64_t delayed;
J
Jim Paton 已提交
4217
      {
L
Lei Jin 已提交
4218
        StopWatch sw(env_, stats_, HARD_RATE_LIMIT_DELAY_COUNT, &delayed);
J
Jim Paton 已提交
4219 4220
        env_->SleepForMicroseconds(1000);
      }
4221
      // Make sure the following value doesn't round to zero.
4222 4223
      uint64_t rate_limit = std::max((delayed / 1000), (uint64_t) 1);
      rate_limit_delay_millis += rate_limit;
L
Lei Jin 已提交
4224
      RecordTick(stats_, RATE_LIMIT_DELAY_MILLIS, rate_limit);
4225
      if (cfd->options()->rate_limit_delay_max_milliseconds > 0 &&
J
Jim Paton 已提交
4226
          rate_limit_delay_millis >=
4227
              (unsigned)cfd->options()->rate_limit_delay_max_milliseconds) {
J
Jim Paton 已提交
4228
        allow_hard_rate_limit_delay = false;
4229
      }
4230
      mutex_.Lock();
4231
      cfd->internal_stats()->RecordLevelNSlowdown(max_level, delayed, false);
4232
    } else if (allow_soft_rate_limit_delay && cfd->ExceedsSoftRateLimit()) {
4233
      const int max_level = cfd->current()->MaxCompactionScoreLevel();
4234
      score = cfd->current()->MaxCompactionScore();
J
Jim Paton 已提交
4235 4236
      // Delay a write when the compaction score for any level is too large.
      // TODO: add statistics
4237 4238
      uint64_t slowdown = SlowdownAmount(score, cfd->options()->soft_rate_limit,
                                         cfd->options()->hard_rate_limit);
L
Lei Jin 已提交
4239
      uint64_t elapsed = 0;
J
Jim Paton 已提交
4240
      mutex_.Unlock();
J
Jim Paton 已提交
4241
      {
L
Lei Jin 已提交
4242
        StopWatch sw(env_, stats_, SOFT_RATE_LIMIT_DELAY_COUNT, &elapsed);
4243 4244
        env_->SleepForMicroseconds(slowdown);
        rate_limit_delay_millis += slowdown;
J
Jim Paton 已提交
4245
      }
J
Jim Paton 已提交
4246 4247
      allow_soft_rate_limit_delay = false;
      mutex_.Lock();
L
Lei Jin 已提交
4248
      cfd->internal_stats()->RecordLevelNSlowdown(max_level, elapsed, true);
4249
    } else {
4250
      unique_ptr<WritableFile> lfile;
S
sdong 已提交
4251
      log::Writer* new_log = nullptr;
K
kailiu 已提交
4252
      MemTable* new_mem = nullptr;
4253

4254
      // Attempt to switch to a new memtable and trigger flush of old.
4255
      // Do this without holding the dbmutex lock.
4256
      assert(versions_->PrevLogNumber() == 0);
I
Igor Canadi 已提交
4257 4258 4259
      bool creating_new_log = !log_empty_;
      uint64_t new_log_number =
          creating_new_log ? versions_->NewFileNumber() : logfile_number_;
I
Igor Canadi 已提交
4260
      SuperVersion* new_superversion = nullptr;
4261 4262 4263
      mutex_.Unlock();
      {
        DelayLoggingAndReset();
I
Igor Canadi 已提交
4264 4265 4266 4267 4268 4269 4270 4271 4272 4273 4274 4275 4276
        if (creating_new_log) {
          s = env_->NewWritableFile(
              LogFileName(options_.wal_dir, new_log_number), &lfile,
              env_->OptimizeForLogWrite(storage_options_));
          if (s.ok()) {
            // Our final size should be less than write_buffer_size
            // (compression, etc) but err on the side of caution.
            lfile->SetPreallocationBlockSize(1.1 *
                                             cfd->options()->write_buffer_size);
            new_log = new log::Writer(std::move(lfile));
          }
        }

4277
        if (s.ok()) {
4278
          new_mem = new MemTable(cfd->internal_comparator(), *cfd->options());
4279
          new_superversion = new SuperVersion();
4280 4281 4282
        }
      }
      mutex_.Lock();
4283
      if (!s.ok()) {
I
Igor Canadi 已提交
4284 4285
        // how do we fail if we're not creating new log?
        assert(creating_new_log);
H
heyongqiang 已提交
4286
        // Avoid chewing through file number space in a tight loop.
4287
        versions_->ReuseLogFileNumber(new_log_number);
4288
        assert(!new_mem);
4289
        assert(!new_log);
4290 4291
        break;
      }
I
Igor Canadi 已提交
4292 4293 4294
      if (creating_new_log) {
        logfile_number_ = new_log_number;
        assert(new_log != nullptr);
4295
        logs_to_free->push_back(log_.release());
I
Igor Canadi 已提交
4296 4297
        log_.reset(new_log);
        log_empty_ = true;
I
Igor Canadi 已提交
4298
        alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
4299 4300 4301 4302 4303 4304 4305 4306 4307 4308 4309
        for (auto cfd : *versions_->GetColumnFamilySet()) {
          // all this is just optimization to delete logs that
          // are no longer needed -- if CF is empty, that means it
          // doesn't need that particular log to stay alive, so we just
          // advance the log number. no need to persist this in the manifest
          if (cfd->mem()->GetFirstSequenceNumber() == 0 &&
              cfd->imm()->size() == 0) {
            cfd->SetLogNumber(logfile_number_);
          }
        }
      }
4310 4311
      cfd->mem()->SetNextLogNumber(logfile_number_);
      cfd->imm()->Add(cfd->mem());
4312
      if (force) {
4313
        cfd->imm()->FlushRequested();
4314
      }
4315 4316
      new_mem->Ref();
      cfd->SetMemtable(new_mem);
4317 4318 4319
      Log(options_.info_log,
          "[%s] New memtable created with log file: #%" PRIu64 "\n",
          cfd->GetName().c_str(), logfile_number_);
4320
      force = false;  // Do not force another compaction if have room
4321
      MaybeScheduleFlushOrCompaction();
4322 4323
      superversions_to_free->push_back(
          cfd->InstallSuperVersion(new_superversion, &mutex_));
4324 4325 4326 4327 4328
    }
  }
  return s;
}

I
Igor Canadi 已提交
4329
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
4330 4331 4332 4333 4334
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

4335 4336
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
4337
  auto version = cfd->current();
4338 4339 4340 4341 4342 4343 4344 4345 4346 4347 4348 4349
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

  // Decrement the ref count
  mutex_.Lock();
  version->Unref();
  mutex_.Unlock();

  return s;
}
I
Igor Canadi 已提交
4350
#endif  // ROCKSDB_LITE
4351

I
Igor Canadi 已提交
4352 4353 4354 4355
const std::string& DBImpl::GetName() const {
  return dbname_;
}

4356 4357 4358 4359
Env* DBImpl::GetEnv() const {
  return env_;
}

4360 4361
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
4362
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
4363 4364
}

4365
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
4366
                         const Slice& property, std::string* value) {
4367
  value->clear();
4368 4369
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4370
  DBPropertyType property_type = GetPropertyType(property);
J
jorlow@chromium.org 已提交
4371
  MutexLock l(&mutex_);
4372
  return cfd->internal_stats()->GetProperty(property_type, property, value);
J
jorlow@chromium.org 已提交
4373 4374
}

4375
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4376
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
4377 4378
  // TODO(opt): better implementation
  Version* v;
4379 4380
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
4381 4382
  {
    MutexLock l(&mutex_);
4383
    v = cfd->current();
4384
    v->Ref();
J
jorlow@chromium.org 已提交
4385 4386 4387 4388 4389 4390 4391 4392 4393 4394 4395 4396 4397 4398 4399 4400 4401
  }

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
    uint64_t start = versions_->ApproximateOffsetOf(v, k1);
    uint64_t limit = versions_->ApproximateOffsetOf(v, k2);
    sizes[i] = (limit >= start ? limit - start : 0);
  }

  {
    MutexLock l(&mutex_);
    v->Unref();
  }
}

4402 4403 4404 4405 4406 4407 4408
inline void DBImpl::DelayLoggingAndReset() {
  if (delayed_writes_ > 0) {
    Log(options_.info_log, "delayed %d write...\n", delayed_writes_ );
    delayed_writes_ = 0;
  }
}

I
Igor Canadi 已提交
4409 4410 4411 4412 4413
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
4414
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
4415 4416 4417 4418 4419 4420 4421 4422 4423 4424 4425 4426 4427 4428 4429 4430 4431 4432 4433 4434 4435 4436
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
  //  Get all sorted Wal Files.
  //  Do binary search and open files and find the seq number.

  std::unique_ptr<VectorLogPtr> wal_files(new VectorLogPtr);
  Status s = GetSortedWalFiles(*wal_files);
  if (!s.ok()) {
    return s;
  }

  s = RetainProbableWalFiles(*wal_files, seq);
  if (!s.ok()) {
    return s;
  }
  iter->reset(new TransactionLogIteratorImpl(options_.wal_dir, &options_,
                                             read_options, storage_options_,
                                             seq, std::move(wal_files), this));
  return (*iter)->status();
}

4437 4438 4439
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4440 4441 4442 4443
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
    Log(options_.info_log, "DeleteFile %s failed.\n", name.c_str());
4444 4445 4446
    return Status::InvalidArgument("Invalid file name");
  }

4447 4448 4449 4450
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4451 4452
      Log(options_.info_log, "DeleteFile %s failed - not archived log.\n",
          name.c_str());
4453 4454 4455 4456
      return Status::NotSupported("Delete only supported for archived logs");
    }
    status = env_->DeleteFile(options_.wal_dir + "/" + name.c_str());
    if (!status.ok()) {
4457 4458
      Log(options_.info_log, "DeleteFile %s failed -- %s.\n",
          name.c_str(), status.ToString().c_str());
4459 4460 4461 4462
    }
    return status;
  }

4463
  int level;
I
Igor Canadi 已提交
4464
  FileMetaData* metadata;
4465
  ColumnFamilyData* cfd;
4466
  VersionEdit edit;
K
Kai Liu 已提交
4467
  DeletionState deletion_state(true);
D
Dhruba Borthakur 已提交
4468 4469
  {
    MutexLock l(&mutex_);
4470
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4471
    if (!status.ok()) {
4472 4473
      Log(options_.info_log, "DeleteFile %s failed. File not found\n",
                             name.c_str());
D
Dhruba Borthakur 已提交
4474 4475
      return Status::InvalidArgument("File not found");
    }
I
Igor Canadi 已提交
4476
    assert((level > 0) && (level < cfd->NumberLevels()));
4477

D
Dhruba Borthakur 已提交
4478
    // If the file is being compacted no need to delete.
4479
    if (metadata->being_compacted) {
4480
      Log(options_.info_log,
4481
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
D
Dhruba Borthakur 已提交
4482
      return Status::OK();
4483 4484
    }

D
Dhruba Borthakur 已提交
4485 4486 4487
    // Only the files in the last level can be deleted externally.
    // This is to make sure that any deletion tombstones are not
    // lost. Check that the level passed is the last level.
I
Igor Canadi 已提交
4488
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
4489
      if (cfd->current()->NumLevelFiles(i) != 0) {
D
Dhruba Borthakur 已提交
4490
        Log(options_.info_log,
4491
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
D
Dhruba Borthakur 已提交
4492 4493 4494 4495
        return Status::InvalidArgument("File not in last level");
      }
    }
    edit.DeleteFile(level, number);
4496
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
I
Igor Canadi 已提交
4497
    if (status.ok()) {
4498
      InstallSuperVersion(cfd, deletion_state);
I
Igor Canadi 已提交
4499
    }
I
Igor Canadi 已提交
4500
    FindObsoleteFiles(deletion_state, false);
D
Dhruba Borthakur 已提交
4501
  } // lock released here
I
Igor Canadi 已提交
4502
  LogFlush(options_.info_log);
I
Igor Canadi 已提交
4503
  // remove files outside the db-lock
4504 4505 4506
  if (deletion_state.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(deletion_state);
  }
4507 4508 4509 4510 4511 4512
  {
    MutexLock l(&mutex_);
    // schedule flush if file deletion means we freed the space for flushes to
    // continue
    MaybeScheduleFlushOrCompaction();
  }
4513 4514 4515
  return status;
}

4516
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
4517
  MutexLock l(&mutex_);
4518
  versions_->GetLiveFilesMetaData(metadata);
4519
}
I
Igor Canadi 已提交
4520
#endif  // ROCKSDB_LITE
4521

I
Igor Canadi 已提交
4522 4523 4524 4525 4526 4527 4528
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

  std::string corruption_messages;
  for (const auto& md : metadata) {
4529 4530
    std::string file_path = md.db_path + "/" + md.name;

I
Igor Canadi 已提交
4531 4532 4533 4534 4535 4536
    uint64_t fsize = 0;
    Status s = env_->GetFileSize(file_path, &fsize);
    if (!s.ok()) {
      corruption_messages +=
          "Can't access " + md.name + ": " + s.ToString() + "\n";
    } else if (fsize != md.size) {
4537
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
4538 4539 4540 4541 4542 4543 4544 4545 4546 4547 4548 4549
                             ". Size recorded in manifest " +
                             std::to_string(md.size) + ", actual size " +
                             std::to_string(fsize) + "\n";
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

4550 4551 4552 4553 4554 4555 4556 4557 4558 4559 4560 4561 4562 4563 4564 4565 4566 4567 4568 4569 4570 4571 4572 4573 4574 4575 4576
Status DBImpl::GetDbIdentity(std::string& identity) {
  std::string idfilename = IdentityFileName(dbname_);
  unique_ptr<SequentialFile> idfile;
  const EnvOptions soptions;
  Status s = env_->NewSequentialFile(idfilename, &idfile, soptions);
  if (!s.ok()) {
    return s;
  }
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
  char buffer[file_size];
  Slice id;
  s = idfile->Read(file_size, &id, buffer);
  if (!s.ok()) {
    return s;
  }
  identity.assign(id.ToString());
  // If last character is '\n' remove it from identity
  if (identity.size() > 0 && identity.back() == '\n') {
    identity.pop_back();
  }
  return s;
}

J
jorlow@chromium.org 已提交
4577 4578
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
4579
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
4580
               const Slice& key, const Slice& value) {
4581 4582 4583 4584
  // Pre-allocate size of write batch conservatively.
  // 8 bytes are taken by header, 4 bytes for count, 1 byte for type,
  // and we allocate 11 extra bytes for key length, as well as value length.
  WriteBatch batch(key.size() + value.size() + 24);
4585
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
4586 4587 4588
  return Write(opt, &batch);
}

4589 4590
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
4591
  WriteBatch batch;
4592
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
4593 4594 4595
  return Write(opt, &batch);
}

4596 4597
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
4598
  WriteBatch batch;
4599
  batch.Merge(column_family, key, value);
4600 4601 4602
  return Write(opt, &batch);
}

4603 4604
// Default implementation -- returns not supported status
Status DB::CreateColumnFamily(const ColumnFamilyOptions& options,
4605
                              const std::string& column_family_name,
4606
                              ColumnFamilyHandle** handle) {
4607
  return Status::NotSupported("");
4608
}
4609
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4610
  return Status::NotSupported("");
4611 4612
}

J
jorlow@chromium.org 已提交
4613 4614
DB::~DB() { }

J
Jim Paton 已提交
4615
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4616 4617 4618 4619
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4620
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4621 4622 4623 4624 4625 4626 4627 4628 4629
  std::vector<ColumnFamilyHandle*> handles;
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
  if (s.ok()) {
    assert(handles.size() == 1);
    // i can delete the handle since DBImpl is always holding a reference to
    // default column family
    delete handles[0];
  }
  return s;
4630 4631
}

4632 4633
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4634
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4635
  if (db_options.db_paths.size() > 1) {
4636 4637 4638 4639 4640 4641 4642 4643 4644 4645 4646 4647
    for (auto& cfd : column_families) {
      if (cfd.options.compaction_style != kCompactionStyleUniversal) {
        return Status::NotSupported(
            "More than one DB paths are only supported in "
            "universal compaction style. ");
      }
    }

    if (db_options.db_paths.size() > 4) {
      return Status::NotSupported(
        "More than four DB paths are not supported yet. ");
    }
4648 4649
  }

4650
  *dbptr = nullptr;
4651
  handles->clear();
J
jorlow@chromium.org 已提交
4652

I
Igor Canadi 已提交
4653 4654 4655 4656 4657 4658 4659 4660
  size_t max_write_buffer_size = 0;
  for (auto cf : column_families) {
    max_write_buffer_size =
        std::max(max_write_buffer_size, cf.options.write_buffer_size);
    if (cf.options.block_cache != nullptr && cf.options.no_block_cache) {
      return Status::InvalidArgument(
          "no_block_cache is true while block_cache is not nullptr");
    }
4661
  }
4662

I
Igor Canadi 已提交
4663
  DBImpl* impl = new DBImpl(db_options, dbname);
4664
  Status s = impl->env_->CreateDirIfMissing(impl->options_.wal_dir);
4665
  if (s.ok()) {
4666 4667
    for (auto db_path : impl->options_.db_paths) {
      s = impl->env_->CreateDirIfMissing(db_path.path);
4668 4669 4670 4671 4672 4673
      if (!s.ok()) {
        break;
      }
    }
  }

4674 4675 4676 4677 4678 4679
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4680 4681 4682 4683
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4684
  impl->mutex_.Lock();
4685 4686
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4687
  if (s.ok()) {
4688
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4689
    unique_ptr<WritableFile> lfile;
4690
    EnvOptions soptions(db_options);
4691
    s = impl->options_.env->NewWritableFile(
I
Igor Canadi 已提交
4692
        LogFileName(impl->options_.wal_dir, new_log_number), &lfile,
I
Igor Canadi 已提交
4693
        impl->options_.env->OptimizeForLogWrite(soptions));
J
jorlow@chromium.org 已提交
4694
    if (s.ok()) {
4695
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4696
      impl->logfile_number_ = new_log_number;
4697
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
4698

4699 4700
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4701 4702
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
4703 4704 4705 4706 4707 4708 4709 4710 4711 4712 4713 4714 4715 4716 4717 4718 4719 4720 4721
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
        } else {
          if (db_options.create_missing_column_families) {
            // missing column family, create it
            ColumnFamilyHandle* handle;
            impl->mutex_.Unlock();
            s = impl->CreateColumnFamily(cf.options, cf.name, &handle);
            impl->mutex_.Lock();
            if (s.ok()) {
              handles->push_back(handle);
            } else {
              break;
            }
          } else {
            s = Status::InvalidArgument("Column family not found: ", cf.name);
            break;
          }
I
Igor Canadi 已提交
4722
        }
4723
      }
I
Igor Canadi 已提交
4724 4725
    }
    if (s.ok()) {
4726
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
4727
        delete cfd->InstallSuperVersion(new SuperVersion(), &impl->mutex_);
4728
      }
I
Igor Canadi 已提交
4729 4730
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4731
      impl->DeleteObsoleteFiles();
4732
      impl->MaybeScheduleFlushOrCompaction();
4733
      s = impl->db_directory_->Fsync();
J
jorlow@chromium.org 已提交
4734 4735
    }
  }
4736

I
Igor Canadi 已提交
4737 4738
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
4739 4740
      if (cfd->options()->compaction_style == kCompactionStyleUniversal ||
          cfd->options()->compaction_style == kCompactionStyleFIFO) {
I
Igor Canadi 已提交
4741 4742 4743 4744
        Version* current = cfd->current();
        for (int i = 1; i < current->NumberLevels(); ++i) {
          int num_files = current->NumLevelFiles(i);
          if (num_files > 0) {
I
Igor Canadi 已提交
4745 4746 4747
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
                "open with universal or FIFO compaction style.");
I
Igor Canadi 已提交
4748 4749 4750 4751
            break;
          }
        }
      }
4752 4753 4754 4755 4756 4757
      if (cfd->options()->merge_operator != nullptr &&
          !cfd->mem()->IsMergeOperatorSupported()) {
        s = Status::InvalidArgument(
            "The memtable of column family %s does not support merge operator "
            "its options.merge_operator is non-null", cfd->GetName().c_str());
      }
I
Igor Canadi 已提交
4758
      if (!s.ok()) {
4759 4760 4761 4762 4763
        break;
      }
    }
  }

4764 4765
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4766
  if (s.ok()) {
4767
    impl->opened_successfully_ = true;
J
jorlow@chromium.org 已提交
4768 4769
    *dbptr = impl;
  } else {
4770 4771 4772
    for (auto h : *handles) {
      delete h;
    }
4773
    handles->clear();
J
jorlow@chromium.org 已提交
4774 4775 4776 4777 4778
    delete impl;
  }
  return s;
}

4779 4780 4781
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4782
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4783 4784
}

4785 4786 4787
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4788
Status DestroyDB(const std::string& dbname, const Options& options) {
4789 4790 4791 4792 4793
  const InternalKeyComparator comparator(options.comparator);
  const InternalFilterPolicy filter_policy(options.filter_policy);
  const Options& soptions(SanitizeOptions(
    dbname, &comparator, &filter_policy, options));
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4794
  std::vector<std::string> filenames;
4795 4796
  std::vector<std::string> archiveFiles;

4797
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
4798 4799
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
4800 4801 4802 4803 4804 4805 4806

  if (dbname != soptions.wal_dir) {
    std::vector<std::string> logfilenames;
    env->GetChildren(soptions.wal_dir, &logfilenames);
    filenames.insert(filenames.end(), logfilenames.begin(), logfilenames.end());
    archivedir = ArchivalDirectory(soptions.wal_dir);
  }
4807

J
jorlow@chromium.org 已提交
4808 4809 4810 4811 4812
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
4813 4814
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4815 4816 4817
  if (result.ok()) {
    uint64_t number;
    FileType type;
D
dgrogan@chromium.org 已提交
4818
    for (size_t i = 0; i < filenames.size(); i++) {
4819
      if (ParseFileName(filenames[i], &number, &type) &&
4820
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4821 4822 4823
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
4824 4825
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
4826 4827 4828
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
4829 4830 4831 4832 4833
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4834

4835 4836
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
4837 4838 4839 4840 4841
      uint64_t number;
      FileType type;
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
4842
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
4843 4844 4845 4846 4847 4848 4849
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

4850
    env->GetChildren(archivedir, &archiveFiles);
4851 4852
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4853 4854
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4855
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4856 4857 4858 4859 4860
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4861
    // ignore case where no archival directory is present.
4862
    env->DeleteDir(archivedir);
4863

J
jorlow@chromium.org 已提交
4864
    env->UnlockFile(lock);  // Ignore error since state is already gone
4865
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4866
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4867
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4868 4869 4870 4871
  }
  return result;
}

4872 4873
//
// A global method that can dump out the build version
K
kailiu 已提交
4874
void DumpLeveldbBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4875 4876
#if !defined(IOS_CROSS_COMPILE)
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't generate util/build_version.cc
4877
  Log(log, "Git sha %s", rocksdb_build_git_sha);
4878
  Log(log, "Compile time %s %s",
4879
      rocksdb_build_compile_time, rocksdb_build_compile_date);
I
Igor Canadi 已提交
4880
#endif
4881 4882
}

4883
}  // namespace rocksdb