db_impl.cc 165.0 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/stop_watch.h"
69
#include "util/sync_point.h"
J
jorlow@chromium.org 已提交
70

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

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

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

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

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

J
jorlow@chromium.org 已提交
89 90 91
struct DBImpl::CompactionState {
  Compaction* const compaction;

92 93 94 95 96
  // 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 已提交
97 98 99 100 101 102

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

  // State kept for output being generated
109 110
  unique_ptr<WritableFile> outfile;
  unique_ptr<TableBuilder> builder;
J
jorlow@chromium.org 已提交
111 112 113 114 115 116 117 118 119

  uint64_t total_bytes;

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

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

121 122 123 124 125 126 127 128
  // 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;
  }

129
  // Create a client visible context of this compaction
D
Danny Guo 已提交
130 131
  CompactionFilterContext GetFilterContext() {
    CompactionFilterContext context;
132
    context.is_full_compaction = compaction->IsFullCompaction();
133
    context.is_manual_compaction = compaction->IsManualCompaction();
134 135
    return context;
  }
D
Danny Guo 已提交
136 137 138 139 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

  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 已提交
253 254
};

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

J
jorlow@chromium.org 已提交
264 265
Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
S
Sanjay Ghemawat 已提交
266
                        const InternalFilterPolicy* ipolicy,
J
jorlow@chromium.org 已提交
267
                        const Options& src) {
268 269 270 271 272 273 274
  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;
275 276
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
277
    ClipToRange(&result.max_open_files, 20, 1000000);
278
  }
279

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

  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
293
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
294
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
295
  }
296

J
jorlow@chromium.org 已提交
297 298 299
  return result;
}

S
Siying Dong 已提交
300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320
CompressionType GetCompressionType(const Options& options, int level,
                                   const bool enable_compression) {
  if (!enable_compression) {
    // disable compression
    return kNoCompression;
  }
  // If the use has specified a different compression level for each level,
  // then pick the compresison for that level.
  if (!options.compression_per_level.empty()) {
    const int n = options.compression_per_level.size() - 1;
    // It is possible for level_ to be -1; in that case, we use level
    // 0's compression.  This occurs mostly in backwards compatibility
    // situations when the builder doesn't know what level the file
    // belongs to.  Likewise, if level_ is beyond the end of the
    // specified compression levels, use the last value.
    return options.compression_per_level[std::max(0, std::min(level, n))];
  } else {
    return options.compression;
  }
}

321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342
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;

  if  (options.compaction_style == kCompactionStyleUniversal) {
    can_compress =
        (options.compaction_options_universal.compression_size_percent < 0);
  } else {
    // For leveled compress when min_level_to_compress == 0.
    can_compress = (GetCompressionType(options, 0, true) != kNoCompression);
  }

  if (can_compress) {
    return options.compression;
  } else {
    return kNoCompression;
  }
}

I
Igor Canadi 已提交
343
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
J
jorlow@chromium.org 已提交
344
    : env_(options.env),
H
heyongqiang 已提交
345
      dbname_(dbname),
I
Igor Canadi 已提交
346
      options_(SanitizeOptions(dbname, options)),
347
      db_lock_(nullptr),
H
Haobo Xu 已提交
348
      mutex_(options.use_adaptive_mutex),
349
      shutting_down_(nullptr),
J
jorlow@chromium.org 已提交
350
      bg_cv_(&mutex_),
351
      logfile_number_(0),
I
Igor Canadi 已提交
352
      log_empty_(true),
353
      default_cf_handle_(nullptr),
I
Igor Canadi 已提交
354 355
      total_log_size_(0),
      max_total_in_memory_state_(0),
356
      tmp_batch_(),
357
      bg_schedule_needed_(false),
358
      bg_compaction_scheduled_(0),
359
      bg_manual_only_(0),
360
      bg_flush_scheduled_(0),
361
      bg_logstats_scheduled_(false),
362 363
      manual_compaction_(nullptr),
      logger_(nullptr),
364
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
365
      delete_obsolete_files_last_run_(options.env->NowMicros()),
366
      purge_wal_files_last_run_(0),
367
      last_stats_dump_time_microsec_(0),
368
      default_interval_to_delete_obsolete_WAL_(600),
369
      flush_on_destroy_(false),
370
      delayed_writes_(0),
371 372
      storage_options_(options),
      bg_work_gate_closed_(false),
373 374
      refitting_level_(false),
      opened_successfully_(false) {
H
heyongqiang 已提交
375
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
376

J
jorlow@chromium.org 已提交
377
  // Reserve ten files or so for other uses and give the rest to TableCache.
378 379
  // Give a large number for setting of "infinite" open files.
  const int table_cache_size =
380 381 382 383 384 385
      (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 已提交
386 387
  versions_.reset(
      new VersionSet(dbname_, &options_, storage_options_, table_cache_.get()));
388 389
  column_family_memtables_.reset(
      new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet()));
390

K
kailiu 已提交
391
  DumpLeveldbBuildVersion(options_.info_log.get());
392
  options_.Dump(options_.info_log.get());
393

394
  char name[100];
K
kailiu 已提交
395 396
  Status s = env_->GetHostName(name, 100L);
  if (s.ok()) {
397 398 399 400 401 402
    host_name_ = name;
  } else {
    Log(options_.info_log, "Can't get hostname, use localhost as host name.");
    host_name_ = "localhost";
  }
  last_log_ts = 0;
403

I
Igor Canadi 已提交
404
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
405 406 407
}

DBImpl::~DBImpl() {
408 409 410 411 412 413 414 415
  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();
416
        cfd->Unref();
417 418
      }
    }
419
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
420
  }
I
Igor Canadi 已提交
421 422

  // Wait for background work to finish
423
  shutting_down_.Release_Store(this);  // Any non-nullptr value is ok
424 425 426
  while (bg_compaction_scheduled_ ||
         bg_flush_scheduled_ ||
         bg_logstats_scheduled_) {
H
hans@chromium.org 已提交
427
    bg_cv_.Wait();
J
jorlow@chromium.org 已提交
428
  }
429

I
Igor Canadi 已提交
430 431 432 433 434
  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();
435 436
  }

437 438 439 440 441 442 443 444 445 446 447 448 449 450 451
  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;
452 453 454
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
455 456 457
    }
  }

458
  // versions need to be destroyed before table_cache since it can hold
459 460
  // references to table_cache.
  versions_.reset();
461
  mutex_.Unlock();
I
Igor Canadi 已提交
462 463 464
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
465

I
Igor Canadi 已提交
466
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
467 468 469
}

Status DBImpl::NewDB() {
470
  VersionEdit new_db;
471
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
472 473 474 475
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

  const std::string manifest = DescriptorFileName(dbname_, 1);
476
  unique_ptr<WritableFile> file;
I
Igor Canadi 已提交
477 478
  Status s = env_->NewWritableFile(
      manifest, &file, env_->OptimizeForManifestWrite(storage_options_));
J
jorlow@chromium.org 已提交
479 480 481
  if (!s.ok()) {
    return s;
  }
482
  file->SetPreallocationBlockSize(options_.manifest_preallocation_size);
J
jorlow@chromium.org 已提交
483
  {
484
    log::Writer log(std::move(file));
J
jorlow@chromium.org 已提交
485 486 487 488 489 490
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
491
    s = SetCurrentFile(env_, dbname_, 1, db_directory_.get());
J
jorlow@chromium.org 已提交
492 493 494 495 496 497 498 499 500 501
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

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

507
const Status DBImpl::CreateArchivalDirectory() {
508
  if (options_.WAL_ttl_seconds > 0 || options_.WAL_size_limit_MB > 0) {
509
    std::string archivalPath = ArchivalDirectory(options_.wal_dir);
510 511 512 513 514
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

515
void DBImpl::PrintStatistics() {
516
  auto dbstats = options_.statistics.get();
517 518
  if (dbstats) {
    Log(options_.info_log,
519 520
        "STATISTCS:\n %s",
        dbstats->ToString().c_str());
521 522 523
  }
}

524
void DBImpl::MaybeDumpStats() {
H
Haobo Xu 已提交
525 526 527 528 529 530 531 532 533 534 535 536 537
  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;
    std::string stats;
538
    GetProperty("rocksdb.stats", &stats);
H
Haobo Xu 已提交
539
    Log(options_.info_log, "%s", stats.c_str());
540
    PrintStatistics();
541 542 543
  }
}

544
// Returns the list of live files in 'sst_live' and the list
K
kailiu 已提交
545
// of all files in the filesystem in 'candidate_files'.
I
Igor Canadi 已提交
546 547 548 549 550 551 552
// 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 已提交
553 554
  mutex_.AssertHeld();

555
  // if deletion is disabled, do nothing
556
  if (disable_delete_obsolete_files_ > 0) {
557 558 559
    return;
  }

560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575
  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 已提交
576 577 578
  // get obsolete files
  versions_->GetObsoleteFiles(&deletion_state.sst_delete_files);

I
Igor Canadi 已提交
579 580
  // store the current filenum, lognum, etc
  deletion_state.manifest_file_number = versions_->ManifestFileNumber();
581 582
  deletion_state.pending_manifest_file_number =
      versions_->PendingManifestFileNumber();
583
  deletion_state.log_number = versions_->MinLogNumber();
I
Igor Canadi 已提交
584 585
  deletion_state.prev_log_number = versions_->PrevLogNumber();

586 587 588 589 590 591 592 593
  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
I
Igor Canadi 已提交
594 595 596 597
  deletion_state.sst_live.assign(pending_outputs_.begin(),
                                 pending_outputs_.end());
  versions_->AddLiveFiles(&deletion_state.sst_live);

598
  if (doing_the_full_scan) {
K
kailiu 已提交
599 600 601 602 603
    // set of all files in the directory. We'll exclude files that are still
    // alive in the subsequent processings.
    env_->GetChildren(
        dbname_, &deletion_state.candidate_files
    ); // Ignore errors
604 605 606 607 608

    //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
K
kailiu 已提交
609 610
      deletion_state.candidate_files.insert(
        deletion_state.candidate_files.end(),
611 612 613
        log_files.begin(),
        log_files.end()
      );
614
    }
615
  }
616 617
}

D
Dhruba Borthakur 已提交
618
// Diffs the files listed in filenames and those that do not
I
Igor Canadi 已提交
619
// belong to live files are posibly removed. Also, removes all the
620
// files in sst_delete_files and log_delete_files.
621
// It is not necessary to hold the mutex when invoking this method.
D
Dhruba Borthakur 已提交
622
void DBImpl::PurgeObsoleteFiles(DeletionState& state) {
623 624
  // we'd better have sth to delete
  assert(state.HaveSomethingToDelete());
625

I
Igor Canadi 已提交
626 627 628 629
  // 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 已提交
630 631
    return;
  }
632

633 634
  // Now, convert live list to an unordered set, WITHOUT mutex held;
  // set is slow.
635 636
  std::unordered_set<uint64_t> sst_live(state.sst_live.begin(),
                                        state.sst_live.end());
I
Igor Canadi 已提交
637

K
kailiu 已提交
638 639 640 641 642 643 644
  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 = "";
645
  for (auto file : state.sst_delete_files) {
K
kailiu 已提交
646 647 648
    candidate_files.push_back(
        TableFileName(kDumbDbName, file->number).substr(1)
    );
649
    delete file;
I
Igor Canadi 已提交
650 651
  }

K
kailiu 已提交
652 653
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
654
      candidate_files.push_back(LogFileName(kDumbDbName, file_num).substr(1));
I
Igor Canadi 已提交
655 656
    }
  }
657

K
kailiu 已提交
658
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
659
  // file twice
K
kailiu 已提交
660
  sort(candidate_files.begin(), candidate_files.end());
661 662
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
663

664 665
  std::vector<std::string> old_info_log_files;

K
kailiu 已提交
666 667 668 669 670 671 672
  for (const auto& to_delete : candidate_files) {
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
    if (!ParseFileName(to_delete, &number, &type)) {
      continue;
    }
J
jorlow@chromium.org 已提交
673

K
kailiu 已提交
674 675 676 677 678 679 680 681
    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'
682
        // (can happen during manifest roll)
K
kailiu 已提交
683 684 685 686 687 688 689
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
        keep = (sst_live.find(number) != sst_live.end());
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
690 691 692 693 694 695
        // 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
        keep = (sst_live.find(number) != sst_live.end()) ||
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
696 697 698 699
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
700
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
701
        }
K
kailiu 已提交
702 703 704 705 706 707 708 709 710 711 712 713 714 715 716
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

    if (type == kTableFile) {
      // evict from cache
717
      TableCache::Evict(table_cache_.get(), number);
K
kailiu 已提交
718
    }
719

K
kailiu 已提交
720 721 722 723
    std::string fname = ((type == kLogFile) ? options_.wal_dir : dbname_) +
        "/" + to_delete;
    if (type == kLogFile &&
        (options_.WAL_ttl_seconds > 0 || options_.WAL_size_limit_MB > 0)) {
724
      auto archived_log_name = ArchivedLogFileName(options_.wal_dir, number);
725 726
      // The sync point below is used in (DBTest,TransactionLogIteratorRace)
      TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:1");
727
      Status s = env_->RenameFile(fname, archived_log_name);
728 729
      // The sync point below is used in (DBTest,TransactionLogIteratorRace)
      TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:2");
730 731 732
      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 已提交
733 734
    } else {
      Status s = env_->DeleteFile(fname);
735 736 737
      Log(options_.info_log, "Delete %s type=%d #%lu -- %s\n",
          fname.c_str(), type, (unsigned long)number,
          s.ToString().c_str());
J
jorlow@chromium.org 已提交
738 739
    }
  }
H
heyongqiang 已提交
740

741
  // Delete old info log files.
742
  size_t old_info_log_file_count = old_info_log_files.size();
K
Kai Liu 已提交
743 744
  // NOTE: Currently we only support log purge when options_.db_log_dir is
  // located in `dbname` directory.
745
  if (old_info_log_file_count >= options_.keep_log_file_num &&
K
Kai Liu 已提交
746
      options_.db_log_dir.empty()) {
747 748
    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;
749
    for (unsigned int i = 0; i <= end; i++) {
750 751 752 753 754 755 756
      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 已提交
757 758
    }
  }
759
  PurgeObsoleteWALFiles();
I
Igor Canadi 已提交
760
  LogFlush(options_.info_log);
D
Dhruba Borthakur 已提交
761 762 763 764 765
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
  DeletionState deletion_state;
I
Igor Canadi 已提交
766
  FindObsoleteFiles(deletion_state, true);
767 768 769
  if (deletion_state.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(deletion_state);
  }
770 771
}

I
Igor Canadi 已提交
772
#ifndef ROCKSDB_LITE
773 774 775 776 777 778 779 780
// 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
781
void DBImpl::PurgeObsoleteWALFiles() {
782 783 784 785 786 787
  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;
  }

788 789
  int64_t current_time;
  Status s = env_->GetCurrentTime(&current_time);
790 791 792 793 794 795 796 797
  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_;
798

799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836
  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 已提交
837 838 839
          } else {
            MutexLock l(&read_first_record_cache_mutex_);
            read_first_record_cache_.erase(number);
840 841
          }
          continue;
842
        }
843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861
      }

      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 已提交
862 863 864
            } else {
              MutexLock l(&read_first_record_cache_mutex_);
              read_first_record_cache_.erase(number);
865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883
            }
          }
        }
      }
    }
  }

  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;
884
  GetSortedWalsOfType(archival_dir, archived_logs, kArchivedLogFile);
885 886 887 888 889 890 891 892 893 894 895 896 897 898

  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 已提交
899 900 901
    } else {
      MutexLock l(&read_first_record_cache_mutex_);
      read_first_record_cache_.erase(archived_logs[i]->LogNumber());
902 903
    }
  }
D
Dhruba Borthakur 已提交
904 905
}

906 907 908 909 910 911 912 913 914 915
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 已提交
916 917 918 919 920 921 922 923 924 925 926 927 928
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 已提交
929 930
      SequenceNumber sequence;
      Status s = ReadFirstRecord(log_type, number, &sequence);
I
Igor Canadi 已提交
931 932 933
      if (!s.ok()) {
        return s;
      }
I
Igor Canadi 已提交
934 935 936 937
      if (sequence == 0) {
        // empty file
        continue;
      }
I
Igor Canadi 已提交
938

939 940 941 942 943 944
      // 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 已提交
945 946
      uint64_t size_bytes;
      s = env_->GetFileSize(LogFileName(path, number), &size_bytes);
947 948 949 950 951
      // 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 已提交
952 953 954 955 956
      if (!s.ok()) {
        return s;
      }

      log_files.push_back(std::move(unique_ptr<LogFile>(
I
Igor Canadi 已提交
957
          new LogFileImpl(number, log_type, sequence, size_bytes))));
I
Igor Canadi 已提交
958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981
    }
  }
  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 已提交
982 983
  // end could be -ve.
  size_t start_index = std::max(static_cast<int64_t>(0), end);
I
Igor Canadi 已提交
984 985 986 987 988 989
  // 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 已提交
990 991 992 993 994 995 996 997 998 999 1000 1001 1002
                               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 已提交
1003 1004
  if (type == kAliveLogFile) {
    std::string fname = LogFileName(options_.wal_dir, number);
I
Igor Canadi 已提交
1005 1006 1007 1008
    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 已提交
1009
    }
I
Igor Canadi 已提交
1010
  }
I
Igor Canadi 已提交
1011

I
Igor Canadi 已提交
1012
  if (type == kArchivedLogFile || !s.ok()) {
I
Igor Canadi 已提交
1013 1014
    //  check if the file got moved to archive.
    std::string archived_file = ArchivedLogFileName(options_.wal_dir, number);
I
Igor Canadi 已提交
1015
    s = ReadFirstLine(archived_file, sequence);
I
Igor Canadi 已提交
1016
  }
I
Igor Canadi 已提交
1017 1018 1019 1020 1021 1022

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

I
Igor Canadi 已提交
1025 1026
// the function returns status.ok() and sequence == 0 if the file exists, but is
// empty
I
Igor Canadi 已提交
1027
Status DBImpl::ReadFirstLine(const std::string& fname,
I
Igor Canadi 已提交
1028
                             SequenceNumber* sequence) {
I
Igor Canadi 已提交
1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 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
  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 已提交
1072 1073 1074
      WriteBatch batch;
      WriteBatchInternal::SetContents(&batch, record);
      *sequence = WriteBatchInternal::Sequence(&batch);
I
Igor Canadi 已提交
1075 1076 1077 1078
      return Status::OK();
    }
  }

I
Igor Canadi 已提交
1079 1080 1081
  // 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 已提交
1082 1083 1084 1085 1086
  return status;
}

#endif  // ROCKSDB_LITE

1087
Status DBImpl::Recover(
1088 1089
    const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
    bool error_if_log_file_exist) {
J
jorlow@chromium.org 已提交
1090 1091
  mutex_.AssertHeld();

1092
  bool is_new_db = false;
1093
  assert(db_lock_ == nullptr);
I
Igor Canadi 已提交
1094
  if (!read_only) {
1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106
    // 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;
    }

1107 1108 1109 1110 1111
    s = env_->NewDirectory(dbname_, &db_directory_);
    if (!s.ok()) {
      return s;
    }

1112
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
1113 1114 1115
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
1116

1117 1118
    if (!env_->FileExists(CurrentFileName(dbname_))) {
      if (options_.create_if_missing) {
1119
        // TODO: add merge_operator name check
1120
        s = NewDB();
1121
        is_new_db = true;
1122 1123 1124 1125 1126 1127
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
1128 1129
      }
    } else {
1130 1131 1132 1133
      if (options_.error_if_exists) {
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
1134
    }
M
Mayank Agarwal 已提交
1135 1136 1137 1138 1139 1140 1141
    // 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 已提交
1142 1143
  }

1144
  Status s = versions_->Recover(column_families, read_only);
I
Igor Canadi 已提交
1145 1146 1147
  if (options_.paranoid_checks && s.ok()) {
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
1148 1149
  if (s.ok()) {
    SequenceNumber max_sequence(0);
1150 1151
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
1152 1153 1154 1155 1156 1157 1158

    // 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
1159
    // produced by an older version of rocksdb.
1160
    const uint64_t min_log = versions_->MinLogNumber();
1161 1162
    const uint64_t prev_log = versions_->PrevLogNumber();
    std::vector<std::string> filenames;
1163
    s = env_->GetChildren(options_.wal_dir, &filenames);
1164 1165
    if (!s.ok()) {
      return s;
1166
    }
K
kailiu 已提交
1167

1168 1169
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
1170 1171
      uint64_t number;
      FileType type;
1172 1173 1174 1175 1176 1177 1178 1179 1180
      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);
        }
1181
      }
J
jorlow@chromium.org 已提交
1182
    }
1183

H
heyongqiang 已提交
1184 1185 1186 1187 1188 1189
    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");
    }

1190 1191
    // Recover in the order in which the logs were generated
    std::sort(logs.begin(), logs.end());
K
kailiu 已提交
1192
    for (const auto& log : logs) {
1193 1194 1195
      // 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 已提交
1196
      versions_->MarkFileNumberUsed(log);
K
Kai Liu 已提交
1197
      s = RecoverLogFile(log, &max_sequence, read_only);
1198
    }
I
Igor Canadi 已提交
1199 1200
    SetTickerCount(options_.statistics.get(), SEQUENCE_NUMBER,
                   versions_->LastSequence());
J
jorlow@chromium.org 已提交
1201 1202
  }

I
Igor Canadi 已提交
1203 1204 1205 1206 1207
  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 已提交
1208 1209 1210
  return s;
}

I
Igor Canadi 已提交
1211 1212
Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
                              bool read_only) {
J
jorlow@chromium.org 已提交
1213 1214
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
1215
    Logger* info_log;
J
jorlow@chromium.org 已提交
1216
    const char* fname;
1217 1218
    Status* status;  // nullptr if options_.paranoid_checks==false or
                     //            options_.skip_log_error_on_recovery==true
J
jorlow@chromium.org 已提交
1219
    virtual void Corruption(size_t bytes, const Status& s) {
1220
      Log(info_log, "%s%s: dropping %d bytes; %s",
1221
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
1222
          fname, static_cast<int>(bytes), s.ToString().c_str());
1223
      if (this->status != nullptr && this->status->ok()) *this->status = s;
J
jorlow@chromium.org 已提交
1224 1225 1226 1227 1228
    }
  };

  mutex_.AssertHeld();

1229
  std::unordered_map<int, VersionEdit> version_edits;
1230
  // no need to refcount because iteration is under mutex
1231 1232
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1233 1234
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1235
  }
I
Igor Canadi 已提交
1236

J
jorlow@chromium.org 已提交
1237
  // Open the log file
1238
  std::string fname = LogFileName(options_.wal_dir, log_number);
1239
  unique_ptr<SequentialFile> file;
1240
  Status status = env_->NewSequentialFile(fname, &file, storage_options_);
J
jorlow@chromium.org 已提交
1241 1242 1243 1244 1245 1246 1247 1248
  if (!status.ok()) {
    MaybeIgnoreError(&status);
    return status;
  }

  // Create the log reader.
  LogReporter reporter;
  reporter.env = env_;
1249
  reporter.info_log = options_.info_log.get();
J
jorlow@chromium.org 已提交
1250
  reporter.fname = fname.c_str();
1251 1252
  reporter.status = (options_.paranoid_checks &&
                     !options_.skip_log_error_on_recovery ? &status : nullptr);
J
jorlow@chromium.org 已提交
1253 1254 1255 1256
  // 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).
1257
  log::Reader reader(std::move(file), &reporter, true/*checksum*/,
1258
                     0/*initial_offset*/);
K
Kai Liu 已提交
1259 1260
  Log(options_.info_log, "Recovering log #%lu",
      (unsigned long) log_number);
J
jorlow@chromium.org 已提交
1261 1262 1263 1264 1265

  // Read all the records and add to a memtable
  std::string scratch;
  Slice record;
  WriteBatch batch;
I
Igor Canadi 已提交
1266
  while (reader.ReadRecord(&record, &scratch)) {
J
jorlow@chromium.org 已提交
1267 1268 1269 1270 1271 1272 1273
    if (record.size() < 12) {
      reporter.Corruption(
          record.size(), Status::Corruption("log record too small"));
      continue;
    }
    WriteBatchInternal::SetContents(&batch, record);

1274
    status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
1275
        &batch, column_family_memtables_.get(), true, log_number);
1276

J
jorlow@chromium.org 已提交
1277 1278
    MaybeIgnoreError(&status);
    if (!status.ok()) {
I
Igor Canadi 已提交
1279
      return status;
J
jorlow@chromium.org 已提交
1280 1281 1282 1283 1284 1285 1286 1287
    }
    const SequenceNumber last_seq =
        WriteBatchInternal::Sequence(&batch) +
        WriteBatchInternal::Count(&batch) - 1;
    if (last_seq > *max_sequence) {
      *max_sequence = last_seq;
    }

1288
    if (!read_only) {
1289 1290
      // no need to refcount since client still doesn't have access
      // to the DB and can not drop column families while we iterate
1291
      for (auto cfd : *versions_->GetColumnFamilySet()) {
1292
        if (cfd->mem()->ShouldFlush()) {
1293
          // If this asserts, it means that InsertInto failed in
1294 1295
          // filtering updates to already-flushed column families
          assert(cfd->GetLogNumber() <= log_number);
1296
          auto iter = version_edits.find(cfd->GetID());
1297 1298
          assert(iter != version_edits.end());
          VersionEdit* edit = &iter->second;
1299
          status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
1300 1301 1302 1303 1304 1305 1306 1307
          // 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 已提交
1308 1309 1310 1311
      }
    }
  }

1312 1313 1314 1315
  if (versions_->LastSequence() < *max_sequence) {
    versions_->SetLastSequence(*max_sequence);
  }

1316
  if (!read_only) {
1317 1318
    // no need to refcount since client still doesn't have access
    // to the DB and can not drop column families while we iterate
1319
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1320
      auto iter = version_edits.find(cfd->GetID());
1321 1322 1323
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

1324 1325 1326 1327
      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
1328
        // (in WriteBatch::InsertInto)
1329 1330 1331 1332 1333 1334 1335
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1336
        status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
1337
      }
1338 1339 1340 1341 1342
      // we still want to clear the memtable, even if the recovery failed
      cfd->CreateNewMemtable();
      if (!status.ok()) {
        return status;
      }
J
jorlow@chromium.org 已提交
1343

1344 1345 1346 1347 1348 1349 1350
      // 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);
1351 1352 1353
      // 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 已提交
1354
      // log number
1355
      versions_->MarkFileNumberUsed(log_number + 1);
1356 1357 1358 1359 1360
      status = versions_->LogAndApply(cfd, edit, &mutex_);
      if (!status.ok()) {
        return status;
      }
    }
1361
  }
I
Igor Canadi 已提交
1362

J
jorlow@chromium.org 已提交
1363 1364 1365
  return status;
}

1366 1367
Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
                                           VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1368
  mutex_.AssertHeld();
1369
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1370 1371 1372
  FileMetaData meta;
  meta.number = versions_->NewFileNumber();
  pending_outputs_.insert(meta.number);
1373
  Iterator* iter = mem->NewIterator(ReadOptions(), true);
1374 1375 1376
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
    mem->GetFirstSequenceNumber();
I
Igor Canadi 已提交
1377 1378
  Log(options_.info_log, "[%s] Level-0 table #%lu: started",
      cfd->GetName().c_str(), (unsigned long)meta.number);
1379 1380 1381 1382

  Status s;
  {
    mutex_.Unlock();
I
Igor Canadi 已提交
1383
    s = BuildTable(dbname_, env_, *cfd->options(), storage_options_,
1384
                   cfd->table_cache(), iter, &meta, cfd->internal_comparator(),
1385
                   newest_snapshot, earliest_seqno_in_memtable,
I
Igor Canadi 已提交
1386
                   GetCompressionFlush(*cfd->options()));
I
Igor Canadi 已提交
1387
    LogFlush(options_.info_log);
1388 1389 1390
    mutex_.Lock();
  }

I
Igor Canadi 已提交
1391 1392 1393
  Log(options_.info_log, "[%s] Level-0 table #%lu: %lu bytes %s",
      cfd->GetName().c_str(), (unsigned long)meta.number,
      (unsigned long)meta.file_size, s.ToString().c_str());
J
jorlow@chromium.org 已提交
1394
  delete iter;
1395

1396
  pending_outputs_.erase(meta.number);
1397 1398 1399 1400 1401 1402

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
  if (s.ok() && meta.file_size > 0) {
    edit->AddFile(level, meta.number, meta.file_size,
1403 1404
                  meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1405 1406
  }

I
Igor Canadi 已提交
1407
  InternalStats::CompactionStats stats;
1408 1409
  stats.micros = env_->NowMicros() - start_micros;
  stats.bytes_written = meta.file_size;
M
Mark Callaghan 已提交
1410
  stats.files_out_levelnp1 = 1;
1411
  cfd->internal_stats()->AddCompactionStats(level, stats);
1412
  RecordTick(options_.statistics.get(), COMPACT_WRITE_BYTES, meta.file_size);
J
jorlow@chromium.org 已提交
1413 1414 1415
  return s;
}

1416
Status DBImpl::WriteLevel0Table(ColumnFamilyData* cfd,
1417
                                autovector<MemTable*>& mems, VersionEdit* edit,
1418
                                uint64_t* filenumber, LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
1419
  mutex_.AssertHeld();
1420 1421 1422 1423 1424
  const uint64_t start_micros = env_->NowMicros();
  FileMetaData meta;
  meta.number = versions_->NewFileNumber();
  *filenumber = meta.number;
  pending_outputs_.insert(meta.number);
1425

1426 1427
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
1428
    mems[0]->GetFirstSequenceNumber();
1429
  Version* base = cfd->current();
1430
  base->Ref();          // it is likely that we do not need this reference
1431 1432 1433
  Status s;
  {
    mutex_.Unlock();
H
Haobo Xu 已提交
1434
    log_buffer->FlushBufferToLog();
K
Kai Liu 已提交
1435
    std::vector<Iterator*> memtables;
1436
    for (MemTable* m : mems) {
I
Igor Canadi 已提交
1437 1438
      Log(options_.info_log, "[%s] Flushing memtable with next log file: %lu\n",
          cfd->GetName().c_str(), (unsigned long)m->GetNextLogNumber());
1439
      memtables.push_back(m->NewIterator(ReadOptions(), true));
1440
    }
I
Igor Canadi 已提交
1441
    Iterator* iter = NewMergingIterator(&cfd->internal_comparator(),
1442
                                        &memtables[0], memtables.size());
I
Igor Canadi 已提交
1443 1444
    Log(options_.info_log, "[%s] Level-0 flush table #%lu: started",
        cfd->GetName().c_str(), (unsigned long)meta.number);
1445

I
Igor Canadi 已提交
1446
    s = BuildTable(dbname_, env_, *cfd->options(), storage_options_,
1447
                   cfd->table_cache(), iter, &meta, cfd->internal_comparator(),
1448
                   newest_snapshot, earliest_seqno_in_memtable,
I
Igor Canadi 已提交
1449
                   GetCompressionFlush(*cfd->options()));
I
Igor Canadi 已提交
1450
    LogFlush(options_.info_log);
1451
    delete iter;
I
Igor Canadi 已提交
1452 1453 1454 1455
    Log(options_.info_log, "[%s] Level-0 flush table #%lu: %lu bytes %s",
        cfd->GetName().c_str(), (unsigned long)meta.number,
        (unsigned long)meta.file_size, s.ToString().c_str());

1456 1457 1458
    if (!options_.disableDataSync) {
      db_directory_->Fsync();
    }
1459 1460
    mutex_.Lock();
  }
1461 1462
  base->Unref();

1463
  // re-acquire the most current version
1464
  base = cfd->current();
1465 1466 1467 1468 1469 1470

  // 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 已提交
1471
  // committed so that other threads can recognize this file as a
1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483 1484
  // 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;
  if (s.ok() && meta.file_size > 0) {
    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.
1485
    if (base != nullptr && options_.max_background_compactions <= 1 &&
1486
        cfd->options()->compaction_style == kCompactionStyleLevel) {
1487 1488 1489
      level = base->PickLevelForMemTableOutput(min_user_key, max_user_key);
    }
    edit->AddFile(level, meta.number, meta.file_size,
1490 1491
                  meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1492 1493
  }

I
Igor Canadi 已提交
1494
  InternalStats::CompactionStats stats;
1495 1496
  stats.micros = env_->NowMicros() - start_micros;
  stats.bytes_written = meta.file_size;
1497
  cfd->internal_stats()->AddCompactionStats(level, stats);
1498
  RecordTick(options_.statistics.get(), COMPACT_WRITE_BYTES, meta.file_size);
1499 1500 1501
  return s;
}

1502 1503
Status DBImpl::FlushMemTableToOutputFile(ColumnFamilyData* cfd,
                                         bool* madeProgress,
H
Haobo Xu 已提交
1504 1505
                                         DeletionState& deletion_state,
                                         LogBuffer* log_buffer) {
1506
  mutex_.AssertHeld();
1507
  assert(cfd->imm()->size() != 0);
1508
  assert(cfd->imm()->IsFlushPending());
1509 1510 1511

  // Save the contents of the earliest memtable as a new Table
  uint64_t file_number;
K
Kai Liu 已提交
1512
  autovector<MemTable*> mems;
1513
  cfd->imm()->PickMemtablesToFlush(&mems);
1514
  if (mems.empty()) {
I
Igor Canadi 已提交
1515 1516
    LogToBuffer(log_buffer, "[%s] Nothing in memtable to flush",
                cfd->GetName().c_str());
L
Lei Jin 已提交
1517
    return Status::OK();
1518 1519 1520
  }

  // record the logfile_number_ before we release the mutex
1521 1522 1523
  // 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.
1524
  MemTable* m = mems[0];
1525 1526
  VersionEdit* edit = m->GetEdits();
  edit->SetPrevLogNumber(0);
1527 1528
  // SetLogNumber(log_num) indicates logs with number smaller than log_num
  // will no longer be picked up for recovery.
1529 1530
  edit->SetLogNumber(mems.back()->GetNextLogNumber());
  edit->SetColumnFamily(cfd->GetID());
1531

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

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

L
Lei Jin 已提交
1540
  if (!s.ok()) {
1541
    cfd->imm()->RollbackMemtableFlush(mems, file_number, &pending_outputs_);
1542 1543
  } else {
    // Replace immutable memtable with the generated Table
1544 1545 1546
    s = cfd->imm()->InstallMemtableFlushResults(
        cfd, mems, versions_.get(), &mutex_, options_.info_log.get(),
        file_number, pending_outputs_, &deletion_state.memtables_to_free,
1547
        db_directory_.get(), log_buffer);
1548
  }
J
jorlow@chromium.org 已提交
1549 1550

  if (s.ok()) {
1551
    InstallSuperVersion(cfd, deletion_state);
1552 1553 1554
    if (madeProgress) {
      *madeProgress = 1;
    }
I
Igor Canadi 已提交
1555 1556 1557
    Version::LevelSummaryStorage tmp;
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
                cfd->current()->LevelSummary(&tmp));
1558

1559
    MaybeScheduleLogDBDeployStats();
I
Igor Canadi 已提交
1560

1561
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1562
      // add to deletion state
1563
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1564 1565 1566 1567
             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;
1568 1569
        alive_log_files_.pop_front();
      }
1570
    }
J
jorlow@chromium.org 已提交
1571
  }
1572 1573 1574 1575 1576 1577 1578

  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;
  }
J
jorlow@chromium.org 已提交
1579 1580 1581
  return s;
}

1582
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1583 1584
                            const Slice* begin, const Slice* end,
                            bool reduce_level, int target_level) {
1585 1586
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1587 1588

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1589 1590 1591 1592 1593
  if (!s.ok()) {
    LogFlush(options_.info_log);
    return s;
  }

I
Igor Canadi 已提交
1594
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1595 1596
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1597 1598
    Version* base = cfd->current();
    for (int level = 1; level < cfd->NumberLevels(); level++) {
G
Gabor Cselle 已提交
1599 1600 1601 1602 1603
      if (base->OverlapInLevel(level, begin, end)) {
        max_level_with_files = level;
      }
    }
  }
1604 1605 1606
  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 已提交
1607
    if (cfd->options()->compaction_style == kCompactionStyleUniversal ||
I
Igor Canadi 已提交
1608
        cfd->options()->compaction_style == kCompactionStyleFIFO ||
1609
        level == max_level_with_files) {
I
Igor Canadi 已提交
1610
      s = RunManualCompaction(cfd, level, level, begin, end);
1611
    } else {
I
Igor Canadi 已提交
1612
      s = RunManualCompaction(cfd, level, level + 1, begin, end);
L
Lei Jin 已提交
1613 1614 1615 1616
    }
    if (!s.ok()) {
      LogFlush(options_.info_log);
      return s;
1617
    }
G
Gabor Cselle 已提交
1618
  }
1619 1620

  if (reduce_level) {
I
Igor Canadi 已提交
1621
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1622
  }
I
Igor Canadi 已提交
1623
  LogFlush(options_.info_log);
L
Lei Jin 已提交
1624 1625

  return s;
1626 1627 1628
}

// return the same level if it cannot be moved
I
Igor Canadi 已提交
1629
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd, int level) {
1630
  mutex_.AssertHeld();
I
Igor Canadi 已提交
1631
  Version* current = cfd->current();
1632
  int minimum_level = level;
1633
  for (int i = level - 1; i > 0; --i) {
1634
    // stop if level i is not empty
1635
    if (current->NumLevelFiles(i) > 0) break;
1636
    // stop if level i is too small (cannot fit the level files)
I
Igor Canadi 已提交
1637
    if (cfd->compaction_picker()->MaxBytesForLevel(i) <
1638 1639 1640
        current->NumLevelBytes(level)) {
      break;
    }
1641 1642 1643 1644 1645 1646

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1650
  SuperVersion* superversion_to_free = nullptr;
1651
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1652 1653

  mutex_.Lock();
1654 1655 1656

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1657
    mutex_.Unlock();
1658
    Log(options_.info_log, "ReFitLevel: another thread is refitting");
I
Igor Canadi 已提交
1659
    delete new_superversion;
L
Lei Jin 已提交
1660
    return Status::NotSupported("another thread is refitting");
1661 1662 1663 1664 1665
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1666
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1667
    Log(options_.info_log,
1668 1669
        "RefitLevel: waiting for background threads to stop: %d %d",
        bg_compaction_scheduled_, bg_flush_scheduled_);
1670 1671 1672 1673
    bg_cv_.Wait();
  }

  // move to a smaller level
1674 1675
  int to_level = target_level;
  if (target_level < 0) {
I
Igor Canadi 已提交
1676
    to_level = FindMinimumEmptyLevelFitting(cfd, level);
1677
  }
1678 1679 1680

  assert(to_level <= level);

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

1686
    VersionEdit edit;
I
Igor Canadi 已提交
1687 1688
    edit.SetColumnFamily(cfd->GetID());
    for (const auto& f : cfd->current()->files_[level]) {
1689
      edit.DeleteFile(level, f->number);
1690 1691
      edit.AddFile(to_level, f->number, f->file_size, f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1692
    }
I
Igor Canadi 已提交
1693 1694
    Log(options_.info_log, "[%s] Apply version edit:\n%s",
        cfd->GetName().c_str(), edit.DebugString().data());
1695

I
Igor Canadi 已提交
1696
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
1697
    superversion_to_free = cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
1698
    new_superversion = nullptr;
1699

I
Igor Canadi 已提交
1700 1701
    Log(options_.info_log, "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
        status.ToString().data());
1702 1703

    if (status.ok()) {
I
Igor Canadi 已提交
1704 1705
      Log(options_.info_log, "[%s] After refitting:\n%s",
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1706 1707 1708 1709 1710
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1711 1712 1713 1714

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1715
  return status;
G
Gabor Cselle 已提交
1716 1717
}

1718 1719 1720
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1721 1722
}

1723 1724 1725
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->max_mem_compaction_level;
1726 1727
}

1728 1729 1730
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->level0_stop_writes_trigger;
1731 1732
}

1733
Status DBImpl::Flush(const FlushOptions& options,
1734 1735 1736
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return FlushMemTable(cfh->cfd(), options);
H
heyongqiang 已提交
1737 1738
}

1739
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1740 1741 1742
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1743 1744
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
                                   int output_level, const Slice* begin,
L
Lei Jin 已提交
1745
                                   const Slice* end) {
1746
  assert(input_level >= 0);
1747

G
Gabor Cselle 已提交
1748 1749
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1750
  ManualCompaction manual;
I
Igor Canadi 已提交
1751
  manual.cfd = cfd;
1752 1753
  manual.input_level = input_level;
  manual.output_level = output_level;
G
Gabor Cselle 已提交
1754
  manual.done = false;
1755
  manual.in_progress = false;
1756 1757 1758
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
I
Igor Canadi 已提交
1759 1760
      cfd->options()->compaction_style == kCompactionStyleUniversal ||
      cfd->options()->compaction_style == kCompactionStyleFIFO) {
1761
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1762 1763 1764 1765
  } else {
    begin_storage = InternalKey(*begin, kMaxSequenceNumber, kValueTypeForSeek);
    manual.begin = &begin_storage;
  }
1766
  if (end == nullptr ||
I
Igor Canadi 已提交
1767 1768
      cfd->options()->compaction_style == kCompactionStyleUniversal ||
      cfd->options()->compaction_style == kCompactionStyleFIFO) {
1769
    manual.end = nullptr;
G
Gabor Cselle 已提交
1770 1771 1772 1773 1774 1775
  } else {
    end_storage = InternalKey(*end, 0, static_cast<ValueType>(0));
    manual.end = &end_storage;
  }

  MutexLock l(&mutex_);
1776

1777 1778 1779 1780 1781 1782
  // 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
1783
  // RunManualCompaction(), i.e. during that time no other compaction will
1784 1785 1786
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1787
  // RunManualCompaction() from getting to the second while loop below.
1788 1789 1790 1791 1792 1793
  // 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 已提交
1794 1795 1796
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1797 1798
    bg_cv_.Wait();
  }
1799

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

1803 1804 1805 1806
  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 已提交
1807
      bg_cv_.Wait();
1808 1809 1810
    } else {
      manual_compaction_ = &manual;
      MaybeScheduleFlushOrCompaction();
G
Gabor Cselle 已提交
1811
    }
H
hans@chromium.org 已提交
1812
  }
1813

1814 1815 1816
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1817
  return manual.status;
J
jorlow@chromium.org 已提交
1818 1819
}

1820 1821
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
                             const FlushOptions& options) {
1822 1823
  // nullptr batch means just wait for earlier writes to be done
  Status s = Write(WriteOptions(), nullptr);
H
heyongqiang 已提交
1824
  if (s.ok() && options.wait) {
1825
    // Wait until the compaction completes
1826
    s = WaitForFlushMemTable(cfd);
1827 1828
  }
  return s;
J
jorlow@chromium.org 已提交
1829 1830
}

1831
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1832 1833 1834
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1835
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1836 1837
    bg_cv_.Wait();
  }
1838
  if (!bg_error_.ok()) {
1839 1840 1841
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1842 1843
}

1844
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1845
  mutex_.AssertHeld();
1846
  bg_schedule_needed_ = false;
1847 1848
  if (bg_work_gate_closed_) {
    // gate closed for backgrond work
J
jorlow@chromium.org 已提交
1849 1850 1851
  } else if (shutting_down_.Acquire_Load()) {
    // DB is being deleted; no more background compactions
  } else {
1852
    bool is_flush_pending = false;
1853
    // no need to refcount since we're under a mutex
1854
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1855
      if (cfd->imm()->IsFlushPending()) {
1856 1857 1858
        is_flush_pending = true;
      }
    }
1859
    if (is_flush_pending) {
1860
      // memtable flush needed
1861 1862 1863
      if (bg_flush_scheduled_ < options_.max_background_flushes) {
        bg_flush_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH);
1864
      } else if (options_.max_background_flushes > 0) {
1865
        bg_schedule_needed_ = true;
1866
      }
1867
    }
1868
    bool is_compaction_needed = false;
1869
    // no need to refcount since we're under a mutex
1870
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1871
      if (cfd->current()->NeedsCompaction()) {
1872 1873 1874 1875
        is_compaction_needed = true;
        break;
      }
    }
1876

1877 1878
    // Schedule BGWorkCompaction if there's a compaction pending (or a memtable
    // flush, but the HIGH pool is not enabled)
1879
    // Do it only if max_background_compactions hasn't been reached and, in case
1880
    // bg_manual_only_ > 0, if it's a manual compaction.
1881 1882
    if ((manual_compaction_ || is_compaction_needed ||
         (is_flush_pending && options_.max_background_flushes == 0)) &&
1883
        (!bg_manual_only_ || manual_compaction_)) {
1884 1885 1886 1887 1888 1889
      if (bg_compaction_scheduled_ < options_.max_background_compactions) {
        bg_compaction_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
      } else {
        bg_schedule_needed_ = true;
      }
1890 1891 1892 1893 1894 1895 1896 1897 1898 1899 1900 1901
    }
  }
}

void DBImpl::BGWorkFlush(void* db) {
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
1902
Status DBImpl::BackgroundFlush(bool* madeProgress,
H
Haobo Xu 已提交
1903 1904
                               DeletionState& deletion_state,
                               LogBuffer* log_buffer) {
1905 1906 1907 1908 1909 1910
  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
1911
  for (auto cfd : *versions_->GetColumnFamilySet()) {
1912 1913 1914
    cfd->Ref();
    Status flush_status;
    while (flush_status.ok() && cfd->imm()->IsFlushPending()) {
1915 1916
      LogToBuffer(
          log_buffer,
1917
          "BackgroundCallFlush doing FlushMemTableToOutputFile with column "
I
Igor Canadi 已提交
1918 1919 1920
          "family [%s], flush slots available %d",
          cfd->GetName().c_str(),
          options_.max_background_flushes - bg_flush_scheduled_);
1921 1922
      flush_status = FlushMemTableToOutputFile(cfd, madeProgress,
                                               deletion_state, log_buffer);
1923 1924 1925
    }
    if (call_status.ok() && !flush_status.ok()) {
      call_status = flush_status;
1926
    }
1927
    cfd->Unref();
J
jorlow@chromium.org 已提交
1928
  }
1929
  versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
1930
  return call_status;
J
jorlow@chromium.org 已提交
1931 1932
}

1933
void DBImpl::BackgroundCallFlush() {
1934
  bool madeProgress = false;
K
Kai Liu 已提交
1935
  DeletionState deletion_state(true);
1936 1937
  assert(bg_flush_scheduled_);

1938
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, options_.info_log.get());
H
Haobo Xu 已提交
1939 1940 1941 1942 1943 1944 1945 1946 1947 1948 1949
  {
    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.
1950 1951 1952
        uint64_t error_cnt = default_cf_handle_->cfd()
                                 ->internal_stats()
                                 ->BumpAndGetBackgroundErrorCount();
H
Haobo Xu 已提交
1953 1954
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
1955 1956 1957 1958
        Log(options_.info_log,
            "Waiting after background flush error: %s"
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
H
Haobo Xu 已提交
1959 1960 1961 1962 1963 1964 1965 1966 1967 1968 1969
        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
1970
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
1971
      mutex_.Unlock();
1972 1973 1974 1975 1976
      // 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 已提交
1977
      log_buffer.FlushBufferToLog();
1978 1979 1980
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
1981 1982
      mutex_.Lock();
    }
I
Igor Canadi 已提交
1983

H
Haobo Xu 已提交
1984
    bg_flush_scheduled_--;
1985 1986 1987 1988 1989
    // 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 已提交
1990 1991 1992
      MaybeScheduleFlushOrCompaction();
    }
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
1993 1994 1995 1996
    // 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.
1997
  }
J
jorlow@chromium.org 已提交
1998 1999
}

2000
void DBImpl::BackgroundCallCompaction() {
2001
  bool madeProgress = false;
K
Kai Liu 已提交
2002
  DeletionState deletion_state(true);
H
Haobo Xu 已提交
2003 2004

  MaybeDumpStats();
2005
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, options_.info_log.get());
2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016
  {
    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.
2017 2018 2019
        uint64_t error_cnt = default_cf_handle_->cfd()
                                 ->internal_stats()
                                 ->BumpAndGetBackgroundErrorCount();
2020 2021
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
H
Haobo Xu 已提交
2022
        log_buffer.FlushBufferToLog();
2023 2024 2025 2026
        Log(options_.info_log,
            "Waiting after background compaction error: %s, "
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
2027 2028 2029 2030 2031
        LogFlush(options_.info_log);
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }
H
Haobo Xu 已提交
2032

2033 2034 2035 2036 2037 2038 2039
    // 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
2040
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2041
      mutex_.Unlock();
2042 2043 2044 2045 2046
      // 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 已提交
2047
      log_buffer.FlushBufferToLog();
2048 2049 2050
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
2051 2052
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2053

2054
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2055

2056
    MaybeScheduleLogDBDeployStats();
2057

2058 2059
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2060 2061 2062
    // Previous compaction may have produced too many files in a level,
    // So reschedule another compaction if we made progress in the
    // last compaction.
2063 2064 2065 2066 2067 2068
    //
    // 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_) {
2069 2070
      MaybeScheduleFlushOrCompaction();
    }
2071 2072 2073 2074 2075 2076 2077 2078 2079
    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 已提交
2080 2081 2082 2083
    // 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.
2084
  }
J
jorlow@chromium.org 已提交
2085 2086
}

A
Abhishek Kona 已提交
2087
Status DBImpl::BackgroundCompaction(bool* madeProgress,
2088 2089
                                    DeletionState& deletion_state,
                                    LogBuffer* log_buffer) {
2090
  *madeProgress = false;
J
jorlow@chromium.org 已提交
2091
  mutex_.AssertHeld();
2092

2093 2094
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2095 2096 2097 2098 2099 2100 2101 2102 2103 2104 2105 2106 2107 2108 2109 2110 2111 2112

  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);
2113
      cfd->Unref();
2114 2115 2116 2117 2118 2119 2120
      if (!flush_stat.ok()) {
        if (is_manual) {
          manual_compaction_->status = flush_stat;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
2121
        return flush_stat;
2122 2123 2124 2125 2126
      }
    }
  }

  unique_ptr<Compaction> c;
2127 2128
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2129
  if (is_manual) {
G
Gabor Cselle 已提交
2130
    ManualCompaction* m = manual_compaction_;
2131
    assert(m->in_progress);
I
Igor Canadi 已提交
2132 2133
    c.reset(m->cfd->CompactRange(m->input_level, m->output_level, m->begin,
                                 m->end, &manual_end));
2134
    if (!c) {
2135
      m->done = true;
G
Gabor Cselle 已提交
2136
    }
I
Igor Canadi 已提交
2137 2138 2139 2140 2141 2142 2143 2144 2145
    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 已提交
2146
  } else {
2147
    // no need to refcount in iteration since it's always under a mutex
I
Igor Canadi 已提交
2148
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
2149
      if (!cfd->options()->disable_auto_compactions) {
2150
        c.reset(cfd->PickCompaction(log_buffer));
I
Igor Canadi 已提交
2151 2152 2153 2154 2155 2156
        if (c != nullptr) {
          // update statistics
          MeasureTime(options_.statistics.get(), NUM_FILES_IN_SINGLE_COMPACTION,
                      c->inputs(0)->size());
          break;
        }
I
Igor Canadi 已提交
2157 2158
      }
    }
J
jorlow@chromium.org 已提交
2159 2160 2161
  }

  Status status;
2162
  if (!c) {
H
hans@chromium.org 已提交
2163
    // Nothing to do
2164
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2165 2166 2167 2168 2169 2170 2171 2172 2173 2174 2175 2176 2177 2178 2179 2180 2181 2182
  } 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)) {
      c->edit()->DeleteFile(c->level(), f->number);
    }
    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 已提交
2183
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
2184
    // Move file to next level
2185
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2186 2187 2188
    FileMetaData* f = c->input(0, 0);
    c->edit()->DeleteFile(c->level(), f->number);
    c->edit()->AddFile(c->level() + 1, f->number, f->file_size,
2189 2190
                       f->smallest, f->largest,
                       f->smallest_seqno, f->largest_seqno);
I
Igor Canadi 已提交
2191
    status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_,
2192
                                    db_directory_.get());
I
Igor Canadi 已提交
2193
    InstallSuperVersion(c->column_family_data(), deletion_state);
2194

2195
    Version::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
2196 2197 2198 2199 2200 2201
    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->number), c->level() + 1,
                static_cast<unsigned long long>(f->file_size),
                status.ToString().c_str(),
                c->input_version()->LevelSummary(&tmp));
I
Igor Canadi 已提交
2202
    c->ReleaseCompactionFiles(status);
2203
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2204
  } else {
2205
    MaybeScheduleFlushOrCompaction(); // do more compaction work in parallel.
2206
    CompactionState* compact = new CompactionState(c.get());
H
Haobo Xu 已提交
2207
    status = DoCompactionWork(compact, deletion_state, log_buffer);
2208
    CleanupCompaction(compact, status);
I
Igor Canadi 已提交
2209
    c->ReleaseCompactionFiles(status);
2210
    c->ReleaseInputs();
2211
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2212
  }
2213
  c.reset();
J
jorlow@chromium.org 已提交
2214 2215 2216 2217 2218 2219

  if (status.ok()) {
    // Done
  } else if (shutting_down_.Acquire_Load()) {
    // Ignore compaction errors found during shutting down
  } else {
2220
    Log(InfoLogLevel::WARN_LEVEL, options_.info_log, "Compaction error: %s",
2221
        status.ToString().c_str());
J
jorlow@chromium.org 已提交
2222 2223 2224 2225
    if (options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2226 2227

  if (is_manual) {
G
Gabor Cselle 已提交
2228
    ManualCompaction* m = manual_compaction_;
2229
    if (!status.ok()) {
L
Lei Jin 已提交
2230
      m->status = status;
2231 2232
      m->done = true;
    }
2233 2234 2235 2236 2237 2238 2239 2240 2241
    // 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.
2242 2243 2244 2245 2246
    //
    // 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) {
2247 2248
      m->done = true;
    }
G
Gabor Cselle 已提交
2249 2250 2251
    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 已提交
2252
      // Universal and FIFO compactions should always compact the whole range
I
Igor Canadi 已提交
2253
      assert(m->cfd->options()->compaction_style != kCompactionStyleUniversal);
I
Igor Canadi 已提交
2254
      assert(m->cfd->options()->compaction_style != kCompactionStyleFIFO);
2255
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2256 2257
      m->begin = &m->tmp_storage;
    }
2258
    m->in_progress = false; // not being processed anymore
2259
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2260
  }
2261
  return status;
J
jorlow@chromium.org 已提交
2262 2263
}

2264
void DBImpl::CleanupCompaction(CompactionState* compact, Status status) {
J
jorlow@chromium.org 已提交
2265
  mutex_.AssertHeld();
2266
  if (compact->builder != nullptr) {
J
jorlow@chromium.org 已提交
2267 2268
    // May happen if we get a shutdown call in the middle of compaction
    compact->builder->Abandon();
2269
    compact->builder.reset();
J
jorlow@chromium.org 已提交
2270
  } else {
2271
    assert(compact->outfile == nullptr);
J
jorlow@chromium.org 已提交
2272
  }
D
dgrogan@chromium.org 已提交
2273
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2274 2275
    const CompactionState::Output& out = compact->outputs[i];
    pending_outputs_.erase(out.number);
2276 2277 2278 2279

    // 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 已提交
2280
      TableCache::Evict(table_cache_.get(), out.number);
2281
    }
J
jorlow@chromium.org 已提交
2282 2283 2284 2285
  }
  delete compact;
}

2286
// Allocate the file numbers for the output file. We allocate as
2287
// many output file numbers as there are files in level+1 (at least one)
2288 2289 2290
// Insert them into pending_outputs so that they do not get deleted.
void DBImpl::AllocateCompactionOutputFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2291 2292
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
2293
  int filesNeeded = compact->compaction->num_input_files(1);
2294
  for (int i = 0; i < std::max(filesNeeded, 1); i++) {
2295 2296 2297 2298 2299 2300 2301 2302 2303
    uint64_t file_number = versions_->NewFileNumber();
    pending_outputs_.insert(file_number);
    compact->allocated_file_numbers.push_back(file_number);
  }
}

// Frees up unused file number.
void DBImpl::ReleaseCompactionUnusedFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2304
  for (const auto file_number : compact->allocated_file_numbers) {
2305 2306 2307 2308
    pending_outputs_.erase(file_number);
  }
}

J
jorlow@chromium.org 已提交
2309
Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
2310 2311
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
J
jorlow@chromium.org 已提交
2312
  uint64_t file_number;
2313 2314 2315 2316 2317 2318 2319
  // 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 已提交
2320 2321 2322 2323 2324
    mutex_.Lock();
    file_number = versions_->NewFileNumber();
    pending_outputs_.insert(file_number);
    mutex_.Unlock();
  }
2325 2326 2327 2328
  CompactionState::Output out;
  out.number = file_number;
  out.smallest.Clear();
  out.largest.Clear();
2329
  out.smallest_seqno = out.largest_seqno = 0;
2330
  compact->outputs.push_back(out);
J
jorlow@chromium.org 已提交
2331 2332 2333

  // Make the output file
  std::string fname = TableFileName(dbname_, file_number);
2334
  Status s = env_->NewWritableFile(fname, &compact->outfile, storage_options_);
2335

J
jorlow@chromium.org 已提交
2336
  if (s.ok()) {
2337
    compact->outfile->SetPreallocationBlockSize(
2338
        compact->compaction->OutputFilePreallocationSize());
2339

2340
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
I
Igor Canadi 已提交
2341 2342 2343
    CompressionType compression_type =
        GetCompressionType(*cfd->options(), compact->compaction->output_level(),
                           compact->compaction->enable_compression());
S
Siying Dong 已提交
2344

2345
    compact->builder.reset(
I
Igor Canadi 已提交
2346
        NewTableBuilder(*cfd->options(), cfd->internal_comparator(),
2347
                        compact->outfile.get(), compression_type));
J
jorlow@chromium.org 已提交
2348
  }
I
Igor Canadi 已提交
2349
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
2350 2351 2352 2353 2354
  return s;
}

Status DBImpl::FinishCompactionOutputFile(CompactionState* compact,
                                          Iterator* input) {
2355
  assert(compact != nullptr);
2356
  assert(compact->outfile);
2357
  assert(compact->builder != nullptr);
J
jorlow@chromium.org 已提交
2358 2359 2360 2361 2362 2363 2364 2365 2366 2367 2368 2369 2370 2371 2372

  const uint64_t output_number = compact->current_output()->number;
  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;
2373
  compact->builder.reset();
J
jorlow@chromium.org 已提交
2374 2375

  // Finish and check for file errors
2376
  if (s.ok() && !options_.disableDataSync) {
2377
    if (options_.use_fsync) {
2378
      StopWatch sw(env_, options_.statistics.get(),
2379
                   COMPACTION_OUTFILE_SYNC_MICROS, false);
2380 2381
      s = compact->outfile->Fsync();
    } else {
2382
      StopWatch sw(env_, options_.statistics.get(),
2383
                   COMPACTION_OUTFILE_SYNC_MICROS, false);
2384 2385
      s = compact->outfile->Sync();
    }
J
jorlow@chromium.org 已提交
2386 2387 2388 2389
  }
  if (s.ok()) {
    s = compact->outfile->Close();
  }
2390
  compact->outfile.reset();
J
jorlow@chromium.org 已提交
2391 2392 2393

  if (s.ok() && current_entries > 0) {
    // Verify that the table is usable
I
Igor Canadi 已提交
2394
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
2395
    FileMetaData meta(output_number, current_bytes);
I
Igor Canadi 已提交
2396
    Iterator* iter = cfd->table_cache()->NewIterator(
2397
        ReadOptions(), storage_options_, cfd->internal_comparator(), meta);
J
jorlow@chromium.org 已提交
2398 2399 2400
    s = iter->status();
    delete iter;
    if (s.ok()) {
I
Igor Canadi 已提交
2401 2402 2403
      Log(options_.info_log, "[%s] Generated table #%lu: %lu keys, %lu bytes",
          cfd->GetName().c_str(), (unsigned long)output_number,
          (unsigned long)current_entries, (unsigned long)current_bytes);
J
jorlow@chromium.org 已提交
2404 2405 2406 2407 2408 2409
    }
  }
  return s;
}


2410 2411
Status DBImpl::InstallCompactionResults(CompactionState* compact,
                                        LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
2412
  mutex_.AssertHeld();
2413 2414 2415 2416 2417

  // 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.
2418
  if (!versions_->VerifyCompactionFileConsistency(compact->compaction)) {
I
Igor Canadi 已提交
2419 2420 2421 2422 2423
    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 已提交
2424
    return Status::Corruption("Compaction input files inconsistent");
2425 2426
  }

I
Igor Canadi 已提交
2427 2428
  LogToBuffer(log_buffer, "[%s] Compacted %d@%d + %d@%d files => %lld bytes",
              compact->compaction->column_family_data()->GetName().c_str(),
2429 2430 2431 2432 2433
              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 已提交
2434 2435 2436

  // Add compaction outputs
  compact->compaction->AddInputDeletions(compact->compaction->edit());
D
dgrogan@chromium.org 已提交
2437
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2438 2439
    const CompactionState::Output& out = compact->outputs[i];
    compact->compaction->edit()->AddFile(
2440 2441
        compact->compaction->output_level(), out.number, out.file_size,
        out.smallest, out.largest, out.smallest_seqno, out.largest_seqno);
J
jorlow@chromium.org 已提交
2442
  }
I
Igor Canadi 已提交
2443 2444 2445
  return versions_->LogAndApply(compact->compaction->column_family_data(),
                                compact->compaction->edit(), &mutex_,
                                db_directory_.get());
J
jorlow@chromium.org 已提交
2446 2447
}

2448 2449 2450 2451 2452 2453 2454
// 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(
2455 2456
  SequenceNumber in, std::vector<SequenceNumber>& snapshots,
  SequenceNumber* prev_snapshot) {
2457 2458 2459
  if (!IsSnapshotSupported()) {
    return 0;
  }
2460
  SequenceNumber prev __attribute__((unused)) = 0;
2461 2462 2463
  for (const auto cur : snapshots) {
    assert(prev <= cur);
    if (cur >= in) {
2464
      *prev_snapshot = prev;
2465
      return cur;
2466
    }
2467 2468
    prev = cur; // assignment
    assert(prev);
2469 2470
  }
  Log(options_.info_log,
K
Kai Liu 已提交
2471 2472 2473
      "Looking for seqid %lu but maxseqid is %lu",
      (unsigned long)in,
      (unsigned long)snapshots[snapshots.size()-1]);
2474 2475 2476 2477
  assert(0);
  return 0;
}

I
Igor Canadi 已提交
2478 2479 2480 2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491 2492 2493 2494 2495 2496
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 已提交
2497 2498 2499 2500 2501 2502 2503 2504 2505 2506 2507 2508
Status DBImpl::ProcessKeyValueCompaction(
    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 已提交
2509
  Status status;
D
Danny Guo 已提交
2510
  std::string compaction_filter_value;
J
jorlow@chromium.org 已提交
2511
  ParsedInternalKey ikey;
2512
  IterKey current_user_key;
J
jorlow@chromium.org 已提交
2513
  bool has_current_user_key = false;
2514
  IterKey delete_key;
2515 2516
  SequenceNumber last_sequence_for_key __attribute__((unused)) =
    kMaxSequenceNumber;
2517
  SequenceNumber visible_in_snapshot = kMaxSequenceNumber;
2518
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
I
Igor Canadi 已提交
2519 2520
  MergeHelper merge(
      cfd->user_comparator(), cfd->options()->merge_operator.get(),
2521 2522
      options_.info_log.get(), cfd->options()->min_partial_merge_operands,
      false /* internal key corruption is expected */);
I
Igor Canadi 已提交
2523
  auto compaction_filter = cfd->options()->compaction_filter;
2524 2525
  std::unique_ptr<CompactionFilter> compaction_filter_from_factory = nullptr;
  if (!compaction_filter) {
2526
    auto context = compact->GetFilterContextV1();
2527
    compaction_filter_from_factory =
I
Igor Canadi 已提交
2528 2529
        cfd->options()->compaction_filter_factory->CreateCompactionFilter(
            context);
2530 2531
    compaction_filter = compaction_filter_from_factory.get();
  }
2532

2533 2534 2535 2536 2537 2538
  while (input->Valid() && !shutting_down_.Acquire_Load() &&
         !cfd->IsDropped()) {
    // 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 已提交
2539
    imm_micros += CallFlushDuringCompaction(cfd, deletion_state, log_buffer);
2540

D
Danny Guo 已提交
2541 2542 2543 2544 2545 2546 2547 2548 2549 2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561
    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 已提交
2562

2563
    if (compact->compaction->ShouldStopBefore(key) &&
2564
        compact->builder != nullptr) {
D
Danny Guo 已提交
2565
      status = FinishCompactionOutputFile(compact, input);
2566 2567 2568 2569 2570 2571
      if (!status.ok()) {
        break;
      }
    }

    // Handle key/value, add to state, etc.
J
jorlow@chromium.org 已提交
2572
    bool drop = false;
2573
    bool current_entry_is_merging = false;
J
jorlow@chromium.org 已提交
2574 2575
    if (!ParseInternalKey(key, &ikey)) {
      // Do not hide error keys
2576 2577
      // 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.
2578
      current_user_key.Clear();
J
jorlow@chromium.org 已提交
2579 2580
      has_current_user_key = false;
      last_sequence_for_key = kMaxSequenceNumber;
2581
      visible_in_snapshot = kMaxSequenceNumber;
J
jorlow@chromium.org 已提交
2582 2583
    } else {
      if (!has_current_user_key ||
2584
          cfd->user_comparator()->Compare(ikey.user_key,
2585
                                          current_user_key.GetKey()) != 0) {
J
jorlow@chromium.org 已提交
2586
        // First occurrence of this user key
L
Lei Jin 已提交
2587
        current_user_key.SetKey(ikey.user_key);
J
jorlow@chromium.org 已提交
2588 2589
        has_current_user_key = true;
        last_sequence_for_key = kMaxSequenceNumber;
2590
        visible_in_snapshot = kMaxSequenceNumber;
H
Haobo Xu 已提交
2591
        // apply the compaction filter to the first occurrence of the user key
D
Danny Guo 已提交
2592
        if (compaction_filter && !is_compaction_v2 &&
H
Haobo Xu 已提交
2593 2594 2595 2596 2597 2598 2599 2600 2601
            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 已提交
2602 2603 2604
          bool to_delete = compaction_filter->Filter(
              compact->compaction->level(), ikey.user_key, value,
              &compaction_filter_value, &value_changed);
H
Haobo Xu 已提交
2605
          if (to_delete) {
2606 2607 2608
            // make a copy of the original key and convert it to a delete
            delete_key.SetInternalKey(ExtractUserKey(key), ikey.sequence,
                                      kTypeDeletion);
H
Haobo Xu 已提交
2609
            // anchor the key again
2610
            key = delete_key.GetKey();
H
Haobo Xu 已提交
2611 2612 2613 2614
            // needed because ikey is backed by key
            ParseInternalKey(key, &ikey);
            // no value associated with delete
            value.clear();
2615
            RecordTick(options_.statistics.get(), COMPACTION_KEY_DROP_USER);
H
Haobo Xu 已提交
2616 2617 2618 2619
          } else if (value_changed) {
            value = compaction_filter_value;
          }
        }
J
jorlow@chromium.org 已提交
2620 2621
      }

2622 2623 2624
      // 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.
2625 2626 2627 2628
      SequenceNumber prev_snapshot = 0; // 0 means no previous snapshot
      SequenceNumber visible = visible_at_tip ?
        visible_at_tip :
        findEarliestVisibleSnapshot(ikey.sequence,
D
Danny Guo 已提交
2629 2630
            compact->existing_snapshots,
            &prev_snapshot);
2631 2632 2633 2634 2635

      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 已提交
2636
        // Hidden by an newer entry for same user key
2637
        // TODO: why not > ?
2638
        assert(last_sequence_for_key >= ikey.sequence);
J
jorlow@chromium.org 已提交
2639
        drop = true;    // (A)
2640
        RecordTick(options_.statistics.get(), COMPACTION_KEY_DROP_NEWER_ENTRY);
J
jorlow@chromium.org 已提交
2641
      } else if (ikey.type == kTypeDeletion &&
D
Danny Guo 已提交
2642 2643
          ikey.sequence <= earliest_snapshot &&
          compact->compaction->IsBaseLevelForKey(ikey.user_key)) {
J
jorlow@chromium.org 已提交
2644 2645 2646 2647 2648 2649 2650 2651
        // 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;
2652
        RecordTick(options_.statistics.get(), COMPACTION_KEY_DROP_OBSOLETE);
2653 2654 2655 2656 2657 2658 2659
      } 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 已提交
2660 2661 2662 2663 2664 2665
        int steps = 0;
        merge.MergeUntil(input, prev_snapshot, bottommost_level,
            options_.statistics.get(), &steps);
        // Skip the Merge ops
        combined_idx = combined_idx - 1 + steps;

2666 2667 2668 2669 2670 2671 2672 2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683
        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 已提交
2684 2685 2686
      }

      last_sequence_for_key = ikey.sequence;
2687
      visible_in_snapshot = visible;
J
jorlow@chromium.org 已提交
2688 2689 2690
    }

    if (!drop) {
2691 2692 2693 2694 2695 2696 2697 2698 2699 2700 2701 2702 2703 2704 2705 2706
      // 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);
2707
      }
2708

2709 2710 2711 2712 2713 2714 2715 2716 2717 2718 2719
      // 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.
2720
        if (bottommost_level && ikey.sequence < earliest_snapshot &&
2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731
            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
2732

2733 2734 2735 2736 2737 2738 2739
        // Open output file if necessary
        if (compact->builder == nullptr) {
          status = OpenCompactionOutputFile(compact);
          if (!status.ok()) {
            break;
          }
        }
2740 2741

        SequenceNumber seqno = GetInternalKeySeqno(newkey);
2742 2743
        if (compact->builder->NumEntries() == 0) {
          compact->current_output()->smallest.DecodeFrom(newkey);
2744 2745 2746 2747
          compact->current_output()->smallest_seqno = seqno;
        } else {
          compact->current_output()->smallest_seqno =
            std::min(compact->current_output()->smallest_seqno, seqno);
2748 2749 2750
        }
        compact->current_output()->largest.DecodeFrom(newkey);
        compact->builder->Add(newkey, value);
2751 2752
        compact->current_output()->largest_seqno =
          std::max(compact->current_output()->largest_seqno, seqno);
2753 2754 2755 2756

        // Close output file if it is big enough
        if (compact->builder->FileSize() >=
            compact->compaction->MaxOutputFileSize()) {
D
Danny Guo 已提交
2757
          status = FinishCompactionOutputFile(compact, input);
2758 2759 2760
          if (!status.ok()) {
            break;
          }
J
jorlow@chromium.org 已提交
2761 2762
        }

2763 2764 2765 2766 2767 2768 2769 2770 2771 2772 2773 2774 2775 2776 2777 2778 2779 2780 2781 2782
        // 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 已提交
2783 2784 2785 2786 2787
          break;
        }
      }
    }

2788
    // MergeUntil has moved input to the next entry
2789
    if (!current_entry_is_merging) {
2790 2791
      input->Next();
    }
J
jorlow@chromium.org 已提交
2792 2793
  }

D
Danny Guo 已提交
2794 2795 2796 2797 2798 2799 2800 2801 2802 2803 2804 2805 2806 2807 2808 2809 2810 2811 2812 2813 2814 2815 2816 2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828 2829 2830 2831 2832 2833 2834 2835 2836 2837 2838 2839 2840 2841 2842 2843 2844 2845 2846 2847 2848 2849 2850 2851 2852 2853
  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();
      RecordTick(options_.statistics.get(), COMPACTION_KEY_DROP_USER);
    } 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();
2854
  bool prefix_initialized = false;
D
Danny Guo 已提交
2855 2856

  int64_t imm_micros = 0;  // Micros spent doing imm_ compactions
2857
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
2858 2859
  LogToBuffer(
      log_buffer,
I
Igor Canadi 已提交
2860 2861
      "[%s] Compacting %d@%d + %d@%d files, score %.2f slots available %d",
      cfd->GetName().c_str(), compact->compaction->num_input_files(0),
2862 2863
      compact->compaction->level(), compact->compaction->num_input_files(1),
      compact->compaction->output_level(), compact->compaction->score(),
D
Danny Guo 已提交
2864 2865 2866
      options_.max_background_compactions - bg_compaction_scheduled_);
  char scratch[2345];
  compact->compaction->Summary(scratch, sizeof(scratch));
I
Igor Canadi 已提交
2867 2868
  LogToBuffer(log_buffer, "[%s] Compaction start summary: %s\n",
              cfd->GetName().c_str(), scratch);
D
Danny Guo 已提交
2869

2870
  assert(cfd->current()->NumLevelFiles(compact->compaction->level()) > 0);
D
Danny Guo 已提交
2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886 2887 2888 2889 2890 2891 2892 2893 2894 2895 2896 2897
  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);

  // Release mutex while we're actually doing the compaction work
  mutex_.Unlock();
2898
  log_buffer->FlushBufferToLog();
D
Danny Guo 已提交
2899 2900 2901 2902 2903 2904 2905 2906 2907 2908 2909 2910 2911 2912

  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 =
2913 2914
      cfd->options()->compaction_filter_factory_v2->CreateCompactionFilterV2(
          context);
D
Danny Guo 已提交
2915 2916 2917 2918 2919 2920 2921 2922 2923 2924 2925
  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) {
2926 2927
    while (backup_input->Valid() && !shutting_down_.Acquire_Load() &&
           !cfd->IsDropped()) {
I
Igor Canadi 已提交
2928 2929 2930 2931 2932 2933
      // 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 已提交
2934 2935 2936 2937
      Slice key = backup_input->key();
      Slice value = backup_input->value();

      const SliceTransform* transformer =
2938
          cfd->options()->compaction_filter_factory_v2->GetPrefixExtractor();
2939
      const auto key_prefix = transformer->Transform(key);
2940
      if (!prefix_initialized) {
2941
        compact->cur_prefix_ = key_prefix.ToString();
2942
        prefix_initialized = true;
D
Danny Guo 已提交
2943 2944 2945
      }
      if (!ParseInternalKey(key, &ikey)) {
        // log error
I
Igor Canadi 已提交
2946 2947
        Log(options_.info_log, "[%s] Failed to parse key: %s",
            cfd->GetName().c_str(), key.ToString().c_str());
D
Danny Guo 已提交
2948 2949 2950
        continue;
      } else {
        // If the prefix remains the same, keep buffering
2951
        if (key_prefix.compare(Slice(compact->cur_prefix_)) == 0) {
D
Danny Guo 已提交
2952 2953 2954 2955 2956 2957 2958 2959 2960 2961 2962 2963 2964 2965 2966 2967 2968 2969 2970 2971
          // 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);
          }
2972
          compact->cur_prefix_ = key_prefix.ToString();
D
Danny Guo 已提交
2973 2974 2975 2976
        }
      }

      // Merge this batch of data (values + ineligible keys)
2977
      compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
2978 2979 2980 2981 2982 2983 2984 2985 2986 2987 2988 2989 2990 2991 2992 2993 2994 2995 2996 2997 2998 2999 3000 3001 3002 3003 3004 3005 3006 3007 3008 3009 3010 3011 3012 3013

      // Done buffering for the current prefix. Spit it out to disk
      // Now just iterate through all the kv-pairs
      status = ProcessKeyValueCompaction(
          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);
        }
3014
        compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3015 3016 3017 3018 3019 3020 3021 3022 3023 3024 3025 3026 3027 3028 3029 3030 3031 3032 3033 3034 3035

        status = ProcessKeyValueCompaction(
            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);
    }
3036
    compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
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
    status = ProcessKeyValueCompaction(
        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(
      visible_at_tip,
      earliest_snapshot,
      latest_snapshot,
      deletion_state,
      bottommost_level,
      imm_micros,
      input.get(),
      compact,
      false,
      log_buffer);
  }

3064
  if (status.ok() && (shutting_down_.Acquire_Load() || cfd->IsDropped())) {
L
Lei Jin 已提交
3065
    status = Status::ShutdownInProgress(
3066
        "Database shutdown or Column family drop during compaction");
J
jorlow@chromium.org 已提交
3067
  }
3068
  if (status.ok() && compact->builder != nullptr) {
3069
    status = FinishCompactionOutputFile(compact, input.get());
J
jorlow@chromium.org 已提交
3070 3071 3072 3073
  }
  if (status.ok()) {
    status = input->status();
  }
3074
  input.reset();
J
jorlow@chromium.org 已提交
3075

3076 3077 3078
  if (!options_.disableDataSync) {
    db_directory_->Fsync();
  }
I
Igor Canadi 已提交
3079 3080

  InternalStats::CompactionStats stats;
3081
  stats.micros = env_->NowMicros() - start_micros - imm_micros;
I
Igor Canadi 已提交
3082
  MeasureTime(options_.statistics.get(), COMPACTION_TIME, stats.micros);
M
Mark Callaghan 已提交
3083 3084
  stats.files_in_leveln = compact->compaction->num_input_files(0);
  stats.files_in_levelnp1 = compact->compaction->num_input_files(1);
3085 3086

  int num_output_files = compact->outputs.size();
3087
  if (compact->builder != nullptr) {
P
Pascal Borreli 已提交
3088
    // An error occurred so ignore the last output.
3089 3090 3091 3092
    assert(num_output_files > 0);
    --num_output_files;
  }
  stats.files_out_levelnp1 = num_output_files;
M
Mark Callaghan 已提交
3093

3094
  for (int i = 0; i < compact->compaction->num_input_files(0); i++) {
M
Mark Callaghan 已提交
3095
    stats.bytes_readn += compact->compaction->input(0, i)->file_size;
3096 3097 3098
    RecordTick(options_.statistics.get(), COMPACT_READ_BYTES,
               compact->compaction->input(0, i)->file_size);
  }
M
Mark Callaghan 已提交
3099

3100
  for (int i = 0; i < compact->compaction->num_input_files(1); i++) {
M
Mark Callaghan 已提交
3101
    stats.bytes_readnp1 += compact->compaction->input(1, i)->file_size;
3102 3103 3104
    RecordTick(options_.statistics.get(), COMPACT_READ_BYTES,
               compact->compaction->input(1, i)->file_size);
  }
M
Mark Callaghan 已提交
3105

3106
  for (int i = 0; i < num_output_files; i++) {
3107
    stats.bytes_written += compact->outputs[i].file_size;
3108 3109
    RecordTick(options_.statistics.get(), COMPACT_WRITE_BYTES,
               compact->outputs[i].file_size);
3110 3111
  }

I
Igor Canadi 已提交
3112
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
3113
  mutex_.Lock();
3114 3115
  cfd->internal_stats()->AddCompactionStats(compact->compaction->output_level(),
                                            stats);
J
jorlow@chromium.org 已提交
3116

3117 3118 3119 3120
  // 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 已提交
3121
  if (status.ok()) {
3122
    status = InstallCompactionResults(compact, log_buffer);
I
Igor Canadi 已提交
3123
    InstallSuperVersion(cfd, deletion_state);
J
jorlow@chromium.org 已提交
3124
  }
3125
  Version::LevelSummaryStorage tmp;
3126 3127
  LogToBuffer(
      log_buffer,
I
Igor Canadi 已提交
3128
      "[%s] compacted to: %s, %.1f MB/sec, level %d, files in(%d, %d) out(%d) "
3129 3130
      "MB in(%.1f, %.1f) out(%.1f), read-write-amplify(%.1f) "
      "write-amplify(%.1f) %s\n",
I
Igor Canadi 已提交
3131
      cfd->GetName().c_str(), cfd->current()->LevelSummary(&tmp),
M
Mark Callaghan 已提交
3132
      (stats.bytes_readn + stats.bytes_readnp1 + stats.bytes_written) /
3133 3134 3135 3136
          (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 已提交
3137
      stats.bytes_written / 1048576.0,
3138
      (stats.bytes_written + stats.bytes_readnp1 + stats.bytes_readn) /
3139 3140
          (double)stats.bytes_readn,
      stats.bytes_written / (double)stats.bytes_readn,
3141
      status.ToString().c_str());
M
Mark Callaghan 已提交
3142

J
jorlow@chromium.org 已提交
3143 3144 3145
  return status;
}

3146 3147
namespace {
struct IterState {
3148 3149
  IterState(DBImpl* db, port::Mutex* mu, SuperVersion* super_version)
      : db(db), mu(mu), super_version(super_version) {}
3150 3151

  DBImpl* db;
3152
  port::Mutex* mu;
3153
  SuperVersion* super_version;
3154 3155 3156 3157
};

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

3159
  if (state->super_version->Unref()) {
3160 3161
    DBImpl::DeletionState deletion_state;

3162 3163 3164 3165 3166 3167
    state->mu->Lock();
    state->super_version->Cleanup();
    state->db->FindObsoleteFiles(deletion_state, false, true);
    state->mu->Unlock();

    delete state->super_version;
3168 3169 3170
    if (deletion_state.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(deletion_state);
    }
I
Igor Canadi 已提交
3171
  }
T
Tomislav Novak 已提交
3172

3173 3174
  delete state;
}
H
Hans Wennborg 已提交
3175
}  // namespace
3176

J
jorlow@chromium.org 已提交
3177
Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
3178
                                      ColumnFamilyData* cfd,
3179 3180 3181 3182 3183 3184 3185 3186 3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204 3205 3206
                                      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());
  }
3207
  IterState* cleanup = new IterState(this, &mutex_, super_version);
3208
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
3209 3210 3211 3212

  return internal_iter;
}

3213 3214 3215 3216
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

J
jorlow@chromium.org 已提交
3217
Status DBImpl::Get(const ReadOptions& options,
3218
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
3219
                   std::string* value) {
3220
  return GetImpl(options, column_family, key, value);
3221 3222
}

I
Igor Canadi 已提交
3223 3224 3225
// DeletionState gets created and destructed outside of the lock -- we
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
3226
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
3227 3228 3229 3230 3231 3232
//
// 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
3233 3234
void DBImpl::InstallSuperVersion(ColumnFamilyData* cfd,
                                 DeletionState& deletion_state) {
3235
  mutex_.AssertHeld();
I
Igor Canadi 已提交
3236 3237 3238 3239
  // if new_superversion == nullptr, it means somebody already used it
  SuperVersion* new_superversion =
    (deletion_state.new_superversion != nullptr) ?
    deletion_state.new_superversion : new SuperVersion();
3240 3241
  SuperVersion* old_superversion =
      cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
3242
  deletion_state.new_superversion = nullptr;
3243
  deletion_state.superversions_to_free.push_back(old_superversion);
I
Igor Canadi 已提交
3244 3245
}

3246
Status DBImpl::GetImpl(const ReadOptions& options,
3247 3248
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
3249
  StopWatch sw(env_, options_.statistics.get(), DB_GET, false);
L
Lei Jin 已提交
3250
  PERF_TIMER_AUTO(get_snapshot_time);
3251

3252 3253 3254
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3255
  SequenceNumber snapshot;
3256
  if (options.snapshot != nullptr) {
3257 3258 3259
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
3260
  }
3261

3262 3263
  // Acquire SuperVersion
  SuperVersion* sv = nullptr;
3264
  // TODO(ljin): consider using GetReferencedSuperVersion() directly
3265
  if (LIKELY(options_.allow_thread_local)) {
3266
    sv = cfd->GetThreadLocalSuperVersion(&mutex_);
3267 3268
  } else {
    mutex_.Lock();
3269
    sv = cfd->GetSuperVersion()->Ref();
3270 3271
    mutex_.Unlock();
  }
I
Igor Canadi 已提交
3272

3273
  bool have_stat_update = false;
3274
  Version::GetStats stats;
3275

3276
  // Prepare to store a list of merge operations if merge occurs.
3277
  MergeContext merge_context;
3278

3279
  Status s;
3280
  // First look in the memtable, then in the immutable memtable (if any).
3281
  // s is both in/out. When in, s could either be OK or MergeInProgress.
3282
  // merge_operands will contain the sequence of merges in the latter case.
3283
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
3284
  PERF_TIMER_STOP(get_snapshot_time);
I
Igor Canadi 已提交
3285
  if (sv->mem->Get(lkey, value, &s, merge_context, *cfd->options())) {
3286
    // Done
3287
    RecordTick(options_.statistics.get(), MEMTABLE_HIT);
I
Igor Canadi 已提交
3288
  } else if (sv->imm->Get(lkey, value, &s, merge_context, *cfd->options())) {
3289
    // Done
3290
    RecordTick(options_.statistics.get(), MEMTABLE_HIT);
3291
  } else {
L
Lei Jin 已提交
3292
    PERF_TIMER_START(get_from_output_files_time);
3293

3294
    sv->current->Get(options, lkey, value, &s, &merge_context, &stats,
3295
                     value_found);
3296
    have_stat_update = true;
L
Lei Jin 已提交
3297
    PERF_TIMER_STOP(get_from_output_files_time);
3298
    RecordTick(options_.statistics.get(), MEMTABLE_MISS);
3299
  }
3300

L
Lei Jin 已提交
3301
  PERF_TIMER_START(get_post_process_time);
3302

I
Igor Canadi 已提交
3303
  if (!cfd->options()->disable_seek_compaction && have_stat_update) {
I
Igor Canadi 已提交
3304
    mutex_.Lock();
3305
    if (sv->current->UpdateStats(stats)) {
I
Igor Canadi 已提交
3306 3307 3308
      MaybeScheduleFlushOrCompaction();
    }
    mutex_.Unlock();
3309 3310
  }

3311
  bool unref_sv = true;
3312
  if (LIKELY(options_.allow_thread_local)) {
3313
    unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3314 3315 3316 3317
  }

  if (unref_sv) {
    // Release SuperVersion
3318
    if (sv->Unref()) {
I
Igor Canadi 已提交
3319
      mutex_.Lock();
3320
      sv->Cleanup();
I
Igor Canadi 已提交
3321
      mutex_.Unlock();
3322
      delete sv;
3323
      RecordTick(options_.statistics.get(), NUMBER_SUPERVERSION_CLEANUPS);
I
Igor Canadi 已提交
3324
    }
3325
    RecordTick(options_.statistics.get(), NUMBER_SUPERVERSION_RELEASES);
3326
  }
3327

3328 3329
  RecordTick(options_.statistics.get(), NUMBER_KEYS_READ);
  RecordTick(options_.statistics.get(), BYTES_READ, value->size());
L
Lei Jin 已提交
3330
  PERF_TIMER_STOP(get_post_process_time);
3331
  return s;
J
jorlow@chromium.org 已提交
3332 3333
}

3334 3335
std::vector<Status> DBImpl::MultiGet(
    const ReadOptions& options,
3336
    const std::vector<ColumnFamilyHandle*>& column_family,
3337
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
3338

3339
  StopWatch sw(env_, options_.statistics.get(), DB_MULTIGET, false);
L
Lei Jin 已提交
3340
  PERF_TIMER_AUTO(get_snapshot_time);
K
Kai Liu 已提交
3341

3342
  SequenceNumber snapshot;
3343

3344
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
3345
    ColumnFamilyData* cfd;
3346 3347 3348 3349 3350 3351 3352
    SuperVersion* super_version;
    Version::GetStats stats;
    bool have_stat_update = false;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
3353 3354 3355 3356 3357 3358
    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});
3359 3360 3361
    }
  }

3362
  mutex_.Lock();
3363 3364 3365 3366 3367
  if (options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
  }
3368
  for (auto mgd_iter : multiget_cf_data) {
3369 3370
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
3371
  }
3372
  mutex_.Unlock();
3373

3374 3375
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
3376

3377
  // Note: this always resizes the values array
3378 3379 3380
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
3381 3382

  // Keep track of bytes that we read for statistics-recording later
3383
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
3384
  PERF_TIMER_STOP(get_snapshot_time);
3385 3386 3387 3388

  // 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.
3389
  // merge_operands will contain the sequence of merges in the latter case.
3390
  for (size_t i = 0; i < num_keys; ++i) {
3391
    merge_context.Clear();
3392
    Status& s = stat_list[i];
3393 3394 3395
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
3396 3397
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
3398 3399 3400
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
I
Igor Canadi 已提交
3401 3402
    auto cfd = mgd->cfd;
    if (super_version->mem->Get(lkey, value, &s, merge_context,
I
Igor Canadi 已提交
3403
                                *cfd->options())) {
3404
      // Done
3405
    } else if (super_version->imm->Get(lkey, value, &s, merge_context,
I
Igor Canadi 已提交
3406
                                       *cfd->options())) {
3407 3408
      // Done
    } else {
3409
      super_version->current->Get(options, lkey, value, &s, &merge_context,
3410
                                  &mgd->stats);
3411
      mgd->have_stat_update = true;
3412 3413 3414
    }

    if (s.ok()) {
3415
      bytes_read += value->size();
3416 3417 3418 3419
    }
  }

  // Post processing (decrement reference counts and record statistics)
L
Lei Jin 已提交
3420
  PERF_TIMER_START(get_post_process_time);
3421 3422 3423 3424 3425 3426
  autovector<SuperVersion*> superversions_to_delete;

  bool schedule_flush_or_compaction = false;
  mutex_.Lock();
  for (auto mgd_iter : multiget_cf_data) {
    auto mgd = mgd_iter.second;
I
Igor Canadi 已提交
3427 3428
    auto cfd = mgd->cfd;
    if (!cfd->options()->disable_seek_compaction && mgd->have_stat_update) {
3429 3430 3431
      if (mgd->super_version->current->UpdateStats(mgd->stats)) {
        schedule_flush_or_compaction = true;
      }
3432
    }
3433 3434 3435
    if (mgd->super_version->Unref()) {
      mgd->super_version->Cleanup();
      superversions_to_delete.push_back(mgd->super_version);
3436 3437
    }
  }
3438 3439 3440 3441 3442 3443 3444 3445 3446 3447
  if (schedule_flush_or_compaction) {
    MaybeScheduleFlushOrCompaction();
  }
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3448
  }
3449

3450
  RecordTick(options_.statistics.get(), NUMBER_MULTIGET_CALLS);
3451 3452
  RecordTick(options_.statistics.get(), NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(options_.statistics.get(), NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3453
  PERF_TIMER_STOP(get_post_process_time);
3454

3455
  return stat_list;
3456 3457
}

3458
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& options,
3459
                                  const std::string& column_family_name,
3460
                                  ColumnFamilyHandle** handle) {
I
Igor Canadi 已提交
3461 3462 3463
  *handle = nullptr;
  MutexLock l(&mutex_);

I
Igor Canadi 已提交
3464 3465
  if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
      nullptr) {
I
Igor Canadi 已提交
3466 3467
    return Status::InvalidArgument("Column family already exists");
  }
3468
  VersionEdit edit;
3469
  edit.AddColumnFamily(column_family_name);
3470 3471
  uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
  edit.SetColumnFamily(new_id);
I
Igor Canadi 已提交
3472
  edit.SetLogNumber(logfile_number_);
I
Igor Canadi 已提交
3473
  edit.SetComparatorName(options.comparator->Name());
3474

I
Igor Canadi 已提交
3475 3476
  // LogAndApply will both write the creation in MANIFEST and create
  // ColumnFamilyData object
3477 3478
  Status s = versions_->LogAndApply(nullptr, &edit, &mutex_,
                                    db_directory_.get(), false, &options);
3479
  if (s.ok()) {
3480 3481 3482
    auto cfd =
        versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
    assert(cfd != nullptr);
3483
    delete cfd->InstallSuperVersion(new SuperVersion(), &mutex_);
3484
    *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
I
Igor Canadi 已提交
3485
    Log(options_.info_log, "Created column family [%s] (ID %u)",
3486
        column_family_name.c_str(), (unsigned)cfd->GetID());
I
Igor Canadi 已提交
3487 3488
    max_total_in_memory_state_ += cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3489
  } else {
I
Igor Canadi 已提交
3490
    Log(options_.info_log, "Creating column family [%s] FAILED -- %s",
3491 3492
        column_family_name.c_str(), s.ToString().c_str());
  }
3493
  return s;
3494 3495
}

3496 3497 3498 3499
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3500 3501
    return Status::InvalidArgument("Can't drop default column family");
  }
3502

I
Igor Canadi 已提交
3503 3504
  VersionEdit edit;
  edit.DropColumnFamily();
3505 3506
  edit.SetColumnFamily(cfd->GetID());

3507
  Status s;
3508 3509 3510 3511 3512 3513 3514 3515
  {
    MutexLock l(&mutex_);
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
      s = versions_->LogAndApply(cfd, &edit, &mutex_);
    }
3516
  }
3517

3518
  if (s.ok()) {
I
Igor Canadi 已提交
3519
    assert(cfd->IsDropped());
I
Igor Canadi 已提交
3520 3521
    max_total_in_memory_state_ -= cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3522
    Log(options_.info_log, "Dropped column family with id %u\n", cfd->GetID());
3523
    // Flush the memtables. This will make all WAL files referencing dropped
I
Igor Canadi 已提交
3524
    // column family to be obsolete. They will be deleted once user deletes
3525 3526
    // column family handle
    Write(WriteOptions(), nullptr);  // ignore error
3527 3528 3529 3530 3531
  } else {
    Log(options_.info_log, "Dropping column family with id %u FAILED -- %s\n",
        cfd->GetID(), s.ToString().c_str());
  }

3532
  return s;
3533 3534
}

3535
bool DBImpl::KeyMayExist(const ReadOptions& options,
3536 3537
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3538
  if (value_found != nullptr) {
K
Kai Liu 已提交
3539 3540
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3541
  }
3542 3543
  ReadOptions roptions = options;
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3544
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3545 3546 3547 3548 3549

  // 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();
3550 3551
}

3552
Iterator* DBImpl::NewIterator(const ReadOptions& options,
3553 3554 3555
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3556 3557

  if (options.tailing) {
I
Igor Canadi 已提交
3558 3559 3560 3561
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
L
Lei Jin 已提交
3562
    // TODO(ljin): remove tailing iterator
I
Igor Canadi 已提交
3563
    auto iter = new ForwardIterator(this, options, cfd);
3564 3565 3566
    return NewDBIterator(env_, *cfd->options(), cfd->user_comparator(), iter,
                         kMaxSequenceNumber);
// return new TailingIterator(env_, this, options, cfd);
I
Igor Canadi 已提交
3567
#endif
T
Tomislav Novak 已提交
3568
  } else {
3569 3570 3571 3572
    SequenceNumber latest_snapshot = versions_->LastSequence();
    SuperVersion* sv = nullptr;
    sv = cfd->GetReferencedSuperVersion(&mutex_);

I
Igor Canadi 已提交
3573 3574 3575 3576
    auto snapshot =
        options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_
            : latest_snapshot;
T
Tomislav Novak 已提交
3577

3578 3579 3580 3581 3582 3583 3584 3585 3586 3587 3588 3589 3590 3591 3592 3593 3594 3595 3596 3597 3598 3599 3600 3601 3602 3603 3604 3605 3606 3607 3608 3609 3610 3611 3612 3613 3614 3615 3616 3617 3618 3619 3620 3621 3622 3623 3624 3625 3626 3627
    // 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 已提交
3628 3629
}

3630 3631
Status DBImpl::NewIterators(
    const ReadOptions& options,
I
Igor Canadi 已提交
3632
    const std::vector<ColumnFamilyHandle*>& column_families,
3633
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3634 3635 3636 3637 3638 3639 3640 3641 3642 3643 3644 3645 3646 3647 3648 3649 3650
  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 已提交
3651 3652 3653 3654
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3655 3656
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
L
Lei Jin 已提交
3657
      iterators->push_back(new TailingIterator(env_, this, options, cfd));
I
Igor Canadi 已提交
3658
    }
I
Igor Canadi 已提交
3659
#endif
I
Igor Canadi 已提交
3660 3661 3662 3663 3664 3665 3666 3667 3668 3669 3670
  } 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 已提交
3671
      iter = NewDBIterator(env_, *cfd->options(),
I
Igor Canadi 已提交
3672 3673 3674 3675 3676 3677
                           cfd->user_comparator(), iter, snapshot);
      iterators->push_back(iter);
    }
  }

  return Status::OK();
3678 3679
}

3680 3681 3682 3683 3684 3685 3686 3687 3688
bool DBImpl::IsSnapshotSupported() const {
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (!cfd->mem()->IsSnapshotSupported()) {
      return false;
    }
  }
  return true;
}

J
jorlow@chromium.org 已提交
3689
const Snapshot* DBImpl::GetSnapshot() {
3690 3691
  // returns null if the underlying memtable does not support snapshot.
  if (!IsSnapshotSupported()) return nullptr;
J
jorlow@chromium.org 已提交
3692
  MutexLock l(&mutex_);
3693
  return snapshots_.New(versions_->LastSequence());
J
jorlow@chromium.org 已提交
3694 3695 3696 3697
}

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

// Convenience methods
3702 3703
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3704
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3705 3706
}

3707 3708 3709 3710
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) {
3711 3712
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3713
    return DB::Merge(o, column_family, key, val);
3714 3715 3716
  }
}

3717
Status DBImpl::Delete(const WriteOptions& options,
3718
                      ColumnFamilyHandle* column_family, const Slice& key) {
3719
  return DB::Delete(options, column_family, key);
J
jorlow@chromium.org 已提交
3720 3721
}

3722
Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
L
Lei Jin 已提交
3723
  PERF_TIMER_AUTO(write_pre_and_post_process_time);
3724 3725 3726
  Writer w(&mutex_);
  w.batch = my_batch;
  w.sync = options.sync;
H
heyongqiang 已提交
3727
  w.disableWAL = options.disableWAL;
3728
  w.done = false;
3729

3730
  StopWatch sw(env_, options_.statistics.get(), DB_WRITE, false);
I
Igor Canadi 已提交
3731
  mutex_.Lock();
3732 3733 3734 3735
  writers_.push_back(&w);
  while (!w.done && &w != writers_.front()) {
    w.cv.Wait();
  }
3736 3737 3738 3739 3740

  if (!options.disableWAL) {
    RecordTick(options_.statistics.get(), WRITE_WITH_WAL, 1);
  }

3741
  if (w.done) {
I
Igor Canadi 已提交
3742
    mutex_.Unlock();
3743
    RecordTick(options_.statistics.get(), WRITE_DONE_BY_OTHER, 1);
3744
    return w.status;
3745 3746
  } else {
    RecordTick(options_.statistics.get(), WRITE_DONE_BY_SELF, 1);
3747 3748
  }

I
Igor Canadi 已提交
3749
  uint64_t flush_column_family_if_log_file = 0;
I
Igor Canadi 已提交
3750
  uint64_t max_total_wal_size = (options_.max_total_wal_size == 0)
3751
                                    ? 4 * max_total_in_memory_state_
I
Igor Canadi 已提交
3752
                                    : options_.max_total_wal_size;
3753 3754
  if (versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1 &&
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3755 3756 3757 3758
      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,
3759 3760 3761
        "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 已提交
3762 3763
  }

3764
  Status status;
3765
  // refcounting cfd in iteration
3766
  bool dead_cfd = false;
3767 3768
  autovector<SuperVersion*> superversions_to_free;
  autovector<log::Writer*> logs_to_free;
3769
  for (auto cfd : *versions_->GetColumnFamilySet()) {
3770
    cfd->Ref();
I
Igor Canadi 已提交
3771 3772 3773
    bool force_flush = my_batch == nullptr ||
                       (flush_column_family_if_log_file != 0 &&
                        cfd->GetLogNumber() <= flush_column_family_if_log_file);
3774
    // May temporarily unlock and wait.
I
Igor Canadi 已提交
3775
    status = MakeRoomForWrite(cfd, force_flush, &superversions_to_free,
3776
                              &logs_to_free);
3777
    if (cfd->Unref()) {
3778
      dead_cfd = true;
3779
    }
3780 3781 3782 3783
    if (!status.ok()) {
      break;
    }
  }
3784 3785
  if (dead_cfd) {
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
3786
  }
3787

D
dgrogan@chromium.org 已提交
3788
  uint64_t last_sequence = versions_->LastSequence();
3789
  Writer* last_writer = &w;
3790
  if (status.ok() && my_batch != nullptr) {  // nullptr batch is for compactions
3791 3792
    autovector<WriteBatch*> write_batch_group;
    BuildBatchGroup(&last_writer, &write_batch_group);
3793

3794 3795 3796
    // 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
3797
    // into memtables
3798
    {
3799
      mutex_.Unlock();
3800 3801 3802 3803 3804 3805 3806 3807 3808 3809
      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]);
        }
      }

3810 3811 3812 3813 3814
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
      // Record statistics
3815 3816 3817
      RecordTick(options_.statistics.get(),
                 NUMBER_KEYS_WRITTEN, my_batch_count);
      RecordTick(options_.statistics.get(),
3818 3819
                 BYTES_WRITTEN,
                 WriteBatchInternal::ByteSize(updates));
3820 3821
      if (options.disableWAL) {
        flush_on_destroy_ = true;
3822
      }
L
Lei Jin 已提交
3823
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3824 3825

      if (!options.disableWAL) {
L
Lei Jin 已提交
3826
        PERF_TIMER_START(write_wal_time);
3827 3828
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
3829 3830
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
3831
        log_empty_ = false;
3832 3833
        RecordTick(options_.statistics.get(), WAL_FILE_SYNCED, 1);
        RecordTick(options_.statistics.get(), WAL_FILE_BYTES, log_entry.size());
H
heyongqiang 已提交
3834
        if (status.ok() && options.sync) {
3835
          if (options_.use_fsync) {
3836
            StopWatch(env_, options_.statistics.get(), WAL_FILE_SYNC_MICROS);
3837
            status = log_->file()->Fsync();
3838
          } else {
3839
            StopWatch(env_, options_.statistics.get(), WAL_FILE_SYNC_MICROS);
3840
            status = log_->file()->Sync();
3841
          }
H
heyongqiang 已提交
3842
        }
L
Lei Jin 已提交
3843
        PERF_TIMER_STOP(write_wal_time);
3844 3845
      }
      if (status.ok()) {
L
Lei Jin 已提交
3846
        PERF_TIMER_START(write_memtable_time);
3847
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3848
            updates, column_family_memtables_.get(), false, 0, this, false);
L
Lei Jin 已提交
3849
        PERF_TIMER_STOP(write_memtable_time);
3850

3851
        if (!status.ok()) {
3852 3853 3854
          // Iteration failed (either in-memory writebatch corruption (very
          // bad), or the client specified invalid column family). Return
          // failure.
3855 3856 3857
          // 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.
3858
          return status;
3859
        }
I
Igor Canadi 已提交
3860 3861
        SetTickerCount(options_.statistics.get(), SEQUENCE_NUMBER,
                       last_sequence);
3862
      }
L
Lei Jin 已提交
3863
      PERF_TIMER_START(write_pre_and_post_process_time);
3864
      if (updates == &tmp_batch_) tmp_batch_.Clear();
3865 3866
      mutex_.Lock();
      if (status.ok()) {
3867
        versions_->SetLastSequence(last_sequence);
3868
      }
J
jorlow@chromium.org 已提交
3869 3870
    }
  }
I
Igor Canadi 已提交
3871 3872 3873
  if (options_.paranoid_checks && !status.ok() && bg_error_.ok()) {
    bg_error_ = status; // stop compaction & fail any further writes
  }
3874

3875 3876 3877 3878 3879 3880 3881
  while (true) {
    Writer* ready = writers_.front();
    writers_.pop_front();
    if (ready != &w) {
      ready->status = status;
      ready->done = true;
      ready->cv.Signal();
3882
    }
3883 3884
    if (ready == last_writer) break;
  }
3885

3886 3887 3888
  // Notify new head of write queue
  if (!writers_.empty()) {
    writers_.front()->cv.Signal();
3889
  }
I
Igor Canadi 已提交
3890
  mutex_.Unlock();
3891 3892 3893 3894 3895 3896 3897 3898

  for (auto& sv : superversions_to_free) {
    delete sv;
  }
  for (auto& log : logs_to_free) {
    delete log;
  }

L
Lei Jin 已提交
3899
  PERF_TIMER_STOP(write_pre_and_post_process_time);
J
jorlow@chromium.org 已提交
3900 3901 3902
  return status;
}

3903
// REQUIRES: Writer list must be non-empty
3904
// REQUIRES: First writer must have a non-nullptr batch
3905 3906
void DBImpl::BuildBatchGroup(Writer** last_writer,
                             autovector<WriteBatch*>* write_batch_group) {
3907 3908
  assert(!writers_.empty());
  Writer* first = writers_.front();
3909
  assert(first->batch != nullptr);
3910 3911

  size_t size = WriteBatchInternal::ByteSize(first->batch);
3912
  write_batch_group->push_back(first->batch);
3913 3914 3915 3916 3917 3918 3919 3920 3921 3922 3923 3924 3925 3926 3927 3928 3929 3930 3931

  // 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 已提交
3932 3933 3934 3935 3936 3937
    if (!w->disableWAL && first->disableWAL) {
      // Do not include a write that needs WAL into a batch that has
      // WAL disabled.
      break;
    }

3938
    if (w->batch != nullptr) {
3939 3940 3941 3942 3943 3944
      size += WriteBatchInternal::ByteSize(w->batch);
      if (size > max_size) {
        // Do not make batch too big
        break;
      }

3945
      write_batch_group->push_back(w->batch);
3946 3947 3948 3949 3950
    }
    *last_writer = w;
  }
}

3951 3952 3953
// 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 已提交
3954 3955 3956 3957
// if n < bottom, return 0;
// if n >= top, return 1000;
// otherwise, let r = (n - bottom) /
//                    (top - bottom)
3958 3959 3960 3961
//  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 已提交
3962
uint64_t DBImpl::SlowdownAmount(int n, double bottom, double top) {
3963
  uint64_t delay;
J
Jim Paton 已提交
3964
  if (n >= top) {
3965 3966
    delay = 1000;
  }
J
Jim Paton 已提交
3967
  else if (n < bottom) {
3968 3969 3970 3971
    delay = 0;
  }
  else {
    // If we are here, we know that:
J
Jim Paton 已提交
3972
    //   level0_start_slowdown <= n < level0_slowdown
3973
    // since the previous two conditions are false.
M
Mark Callaghan 已提交
3974 3975
    double how_much =
      (double) (n - bottom) /
J
Jim Paton 已提交
3976
              (top - bottom);
M
Mark Callaghan 已提交
3977
    delay = std::max(how_much * how_much * 1000, 100.0);
3978 3979 3980 3981 3982
  }
  assert(delay <= 1000);
  return delay;
}

3983
// REQUIRES: mutex_ is held
3984
// REQUIRES: this thread is currently at the front of the writer queue
3985 3986 3987 3988
Status DBImpl::MakeRoomForWrite(
    ColumnFamilyData* cfd, bool force,
    autovector<SuperVersion*>* superversions_to_free,
    autovector<log::Writer*>* logs_to_free) {
3989
  mutex_.AssertHeld();
3990
  assert(!writers_.empty());
3991
  bool allow_delay = !force;
J
Jim Paton 已提交
3992 3993
  bool allow_hard_rate_limit_delay = !force;
  bool allow_soft_rate_limit_delay = !force;
3994
  uint64_t rate_limit_delay_millis = 0;
3995
  Status s;
3996
  double score;
3997 3998 3999 4000
  // 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;
4001

4002 4003 4004 4005 4006
  while (true) {
    if (!bg_error_.ok()) {
      // Yield previous error
      s = bg_error_;
      break;
4007
    } else if (allow_delay && cfd->NeedSlowdownForNumLevel0Files()) {
4008 4009 4010
      // 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
4011
      // individual write by 0-1ms to reduce latency variance.  Also,
4012 4013
      // this delay hands over some CPU to the compaction thread in
      // case it is sharing the same core as the writer.
4014
      uint64_t slowdown =
4015 4016 4017
          SlowdownAmount(cfd->current()->NumLevelFiles(0),
                         cfd->options()->level0_slowdown_writes_trigger,
                         cfd->options()->level0_stop_writes_trigger);
4018
      mutex_.Unlock();
4019
      uint64_t delayed;
J
Jim Paton 已提交
4020
      {
4021
        StopWatch sw(env_, options_.statistics.get(), STALL_L0_SLOWDOWN_COUNT);
4022
        env_->SleepForMicroseconds(slowdown);
4023
        delayed = sw.ElapsedMicros();
J
Jim Paton 已提交
4024
      }
4025
      RecordTick(options_.statistics.get(), STALL_L0_SLOWDOWN_MICROS, delayed);
4026 4027
      cfd->internal_stats()->RecordWriteStall(InternalStats::LEVEL0_SLOWDOWN,
                                              delayed);
4028 4029
      allow_delay = false;  // Do not delay a single write more than once
      mutex_.Lock();
4030
      delayed_writes_++;
4031
    } else if (!force && !cfd->mem()->ShouldFlush()) {
4032
      // There is room in current memtable
4033 4034 4035
      if (allow_delay) {
        DelayLoggingAndReset();
      }
4036
      break;
4037 4038
    } else if (cfd->imm()->size() ==
               cfd->options()->max_write_buffer_number - 1) {
4039
      // We have filled up the current memtable, but the previous
4040
      // ones are still being flushed, so we wait.
4041
      DelayLoggingAndReset();
I
Igor Canadi 已提交
4042 4043
      Log(options_.info_log, "[%s] wait for memtable flush...\n",
          cfd->GetName().c_str());
4044 4045 4046 4047
      if (schedule_background_work) {
        MaybeScheduleFlushOrCompaction();
        schedule_background_work = false;
      }
4048
      uint64_t stall;
J
Jim Paton 已提交
4049
      {
4050
        StopWatch sw(env_, options_.statistics.get(),
4051
                     STALL_MEMTABLE_COMPACTION_COUNT);
J
Jim Paton 已提交
4052
        bg_cv_.Wait();
4053
        stall = sw.ElapsedMicros();
J
Jim Paton 已提交
4054
      }
4055 4056
      RecordTick(options_.statistics.get(),
                 STALL_MEMTABLE_COMPACTION_MICROS, stall);
4057 4058
      cfd->internal_stats()->RecordWriteStall(
          InternalStats::MEMTABLE_COMPACTION, stall);
4059 4060
    } else if (cfd->current()->NumLevelFiles(0) >=
               cfd->options()->level0_stop_writes_trigger) {
4061
      // There are too many level-0 files.
4062
      DelayLoggingAndReset();
I
Igor Canadi 已提交
4063 4064
      Log(options_.info_log, "[%s] wait for fewer level0 files...\n",
          cfd->GetName().c_str());
4065
      uint64_t stall;
J
Jim Paton 已提交
4066
      {
4067 4068
        StopWatch sw(env_, options_.statistics.get(),
                     STALL_L0_NUM_FILES_COUNT);
J
Jim Paton 已提交
4069
        bg_cv_.Wait();
4070
        stall = sw.ElapsedMicros();
J
Jim Paton 已提交
4071
      }
4072
      RecordTick(options_.statistics.get(), STALL_L0_NUM_FILES_MICROS, stall);
4073 4074
      cfd->internal_stats()->RecordWriteStall(InternalStats::LEVEL0_NUM_FILES,
                                              stall);
I
Igor Canadi 已提交
4075 4076
    } else if (allow_hard_rate_limit_delay &&
               cfd->options()->hard_rate_limit > 1.0 &&
4077 4078
               (score = cfd->current()->MaxCompactionScore()) >
                   cfd->options()->hard_rate_limit) {
4079
      // Delay a write when the compaction score for any level is too large.
4080
      int max_level = cfd->current()->MaxCompactionScoreLevel();
4081
      mutex_.Unlock();
4082
      uint64_t delayed;
J
Jim Paton 已提交
4083
      {
4084 4085
        StopWatch sw(env_, options_.statistics.get(),
                     HARD_RATE_LIMIT_DELAY_COUNT);
J
Jim Paton 已提交
4086
        env_->SleepForMicroseconds(1000);
4087
        delayed = sw.ElapsedMicros();
J
Jim Paton 已提交
4088
      }
4089
      cfd->internal_stats()->RecordLevelNSlowdown(max_level, delayed);
4090
      // Make sure the following value doesn't round to zero.
4091 4092
      uint64_t rate_limit = std::max((delayed / 1000), (uint64_t) 1);
      rate_limit_delay_millis += rate_limit;
4093 4094
      RecordTick(options_.statistics.get(),
                 RATE_LIMIT_DELAY_MILLIS, rate_limit);
4095
      if (cfd->options()->rate_limit_delay_max_milliseconds > 0 &&
J
Jim Paton 已提交
4096
          rate_limit_delay_millis >=
4097
              (unsigned)cfd->options()->rate_limit_delay_max_milliseconds) {
J
Jim Paton 已提交
4098
        allow_hard_rate_limit_delay = false;
4099
      }
4100
      mutex_.Lock();
4101 4102 4103 4104
    } else if (allow_soft_rate_limit_delay &&
               cfd->options()->soft_rate_limit > 0.0 &&
               (score = cfd->current()->MaxCompactionScore()) >
                   cfd->options()->soft_rate_limit) {
J
Jim Paton 已提交
4105 4106 4107
      // Delay a write when the compaction score for any level is too large.
      // TODO: add statistics
      mutex_.Unlock();
J
Jim Paton 已提交
4108
      {
4109 4110
        StopWatch sw(env_, options_.statistics.get(),
                     SOFT_RATE_LIMIT_DELAY_COUNT);
4111 4112 4113
        env_->SleepForMicroseconds(
            SlowdownAmount(score, cfd->options()->soft_rate_limit,
                           cfd->options()->hard_rate_limit));
J
Jim Paton 已提交
4114 4115
        rate_limit_delay_millis += sw.ElapsedMicros();
      }
J
Jim Paton 已提交
4116 4117
      allow_soft_rate_limit_delay = false;
      mutex_.Lock();
4118

4119
    } else {
4120
      unique_ptr<WritableFile> lfile;
S
sdong 已提交
4121
      log::Writer* new_log = nullptr;
K
kailiu 已提交
4122
      MemTable* new_mem = nullptr;
4123

4124
      // Attempt to switch to a new memtable and trigger flush of old.
4125
      // Do this without holding the dbmutex lock.
4126
      assert(versions_->PrevLogNumber() == 0);
I
Igor Canadi 已提交
4127 4128 4129
      bool creating_new_log = !log_empty_;
      uint64_t new_log_number =
          creating_new_log ? versions_->NewFileNumber() : logfile_number_;
I
Igor Canadi 已提交
4130
      SuperVersion* new_superversion = nullptr;
4131 4132 4133
      mutex_.Unlock();
      {
        DelayLoggingAndReset();
I
Igor Canadi 已提交
4134 4135 4136 4137 4138 4139 4140 4141 4142 4143 4144 4145 4146
        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));
          }
        }

4147
        if (s.ok()) {
4148
          new_mem = new MemTable(cfd->internal_comparator(), *cfd->options());
4149
          new_superversion = new SuperVersion();
4150 4151 4152
        }
      }
      mutex_.Lock();
4153
      if (!s.ok()) {
I
Igor Canadi 已提交
4154 4155
        // how do we fail if we're not creating new log?
        assert(creating_new_log);
H
heyongqiang 已提交
4156
        // Avoid chewing through file number space in a tight loop.
4157
        versions_->ReuseFileNumber(new_log_number);
4158
        assert(!new_mem);
4159
        assert(!new_log);
4160 4161
        break;
      }
I
Igor Canadi 已提交
4162 4163 4164
      if (creating_new_log) {
        logfile_number_ = new_log_number;
        assert(new_log != nullptr);
4165
        logs_to_free->push_back(log_.release());
I
Igor Canadi 已提交
4166 4167
        log_.reset(new_log);
        log_empty_ = true;
I
Igor Canadi 已提交
4168
        alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
4169 4170 4171 4172 4173 4174 4175 4176 4177 4178 4179
        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_);
          }
        }
      }
4180 4181
      cfd->mem()->SetNextLogNumber(logfile_number_);
      cfd->imm()->Add(cfd->mem());
4182
      if (force) {
4183
        cfd->imm()->FlushRequested();
4184
      }
4185 4186
      new_mem->Ref();
      cfd->SetMemtable(new_mem);
I
Igor Canadi 已提交
4187 4188
      Log(options_.info_log, "[%s] New memtable created with log file: #%lu\n",
          cfd->GetName().c_str(), (unsigned long)logfile_number_);
4189
      force = false;  // Do not force another compaction if have room
4190
      MaybeScheduleFlushOrCompaction();
4191 4192
      superversions_to_free->push_back(
          cfd->InstallSuperVersion(new_superversion, &mutex_));
4193 4194 4195 4196 4197
    }
  }
  return s;
}

I
Igor Canadi 已提交
4198
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
4199 4200 4201 4202 4203
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

4204 4205
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
4206
  auto version = cfd->current();
4207 4208 4209 4210 4211 4212 4213 4214 4215 4216 4217 4218
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
4219
#endif  // ROCKSDB_LITE
4220

I
Igor Canadi 已提交
4221 4222 4223 4224
const std::string& DBImpl::GetName() const {
  return dbname_;
}

4225 4226 4227 4228
Env* DBImpl::GetEnv() const {
  return env_;
}

4229 4230
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
4231
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
4232 4233
}

4234
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
4235
                         const Slice& property, std::string* value) {
4236
  value->clear();
4237 4238
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4239
  DBPropertyType property_type = GetPropertyType(property);
J
jorlow@chromium.org 已提交
4240
  MutexLock l(&mutex_);
4241 4242
  return cfd->internal_stats()->GetProperty(property_type, property, value,
                                            cfd);
J
jorlow@chromium.org 已提交
4243 4244
}

4245
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4246
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
4247 4248
  // TODO(opt): better implementation
  Version* v;
4249 4250
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
4251 4252
  {
    MutexLock l(&mutex_);
4253
    v = cfd->current();
4254
    v->Ref();
J
jorlow@chromium.org 已提交
4255 4256 4257 4258 4259 4260 4261 4262 4263 4264 4265 4266 4267 4268 4269 4270 4271
  }

  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();
  }
}

4272 4273 4274 4275 4276 4277 4278
inline void DBImpl::DelayLoggingAndReset() {
  if (delayed_writes_ > 0) {
    Log(options_.info_log, "delayed %d write...\n", delayed_writes_ );
    delayed_writes_ = 0;
  }
}

I
Igor Canadi 已提交
4279 4280 4281 4282 4283 4284 4285 4286 4287 4288 4289 4290 4291 4292 4293 4294 4295 4296 4297 4298 4299 4300 4301 4302 4303 4304 4305 4306
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

  RecordTick(options_.statistics.get(), GET_UPDATES_SINCE_CALLS);
  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();
}

4307 4308 4309
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4310 4311 4312 4313
  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());
4314 4315 4316
    return Status::InvalidArgument("Invalid file name");
  }

4317 4318 4319 4320
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4321 4322
      Log(options_.info_log, "DeleteFile %s failed - not archived log.\n",
          name.c_str());
4323 4324 4325 4326
      return Status::NotSupported("Delete only supported for archived logs");
    }
    status = env_->DeleteFile(options_.wal_dir + "/" + name.c_str());
    if (!status.ok()) {
4327 4328
      Log(options_.info_log, "DeleteFile %s failed -- %s.\n",
          name.c_str(), status.ToString().c_str());
4329 4330 4331 4332
    }
    return status;
  }

4333
  int level;
I
Igor Canadi 已提交
4334
  FileMetaData* metadata;
4335
  ColumnFamilyData* cfd;
4336
  VersionEdit edit;
K
Kai Liu 已提交
4337
  DeletionState deletion_state(true);
D
Dhruba Borthakur 已提交
4338 4339
  {
    MutexLock l(&mutex_);
4340
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4341
    if (!status.ok()) {
4342 4343
      Log(options_.info_log, "DeleteFile %s failed. File not found\n",
                             name.c_str());
D
Dhruba Borthakur 已提交
4344 4345
      return Status::InvalidArgument("File not found");
    }
I
Igor Canadi 已提交
4346
    assert((level > 0) && (level < cfd->NumberLevels()));
4347

D
Dhruba Borthakur 已提交
4348
    // If the file is being compacted no need to delete.
4349
    if (metadata->being_compacted) {
4350
      Log(options_.info_log,
4351
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
D
Dhruba Borthakur 已提交
4352
      return Status::OK();
4353 4354
    }

D
Dhruba Borthakur 已提交
4355 4356 4357
    // 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 已提交
4358
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
4359
      if (cfd->current()->NumLevelFiles(i) != 0) {
D
Dhruba Borthakur 已提交
4360
        Log(options_.info_log,
4361
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
D
Dhruba Borthakur 已提交
4362 4363 4364 4365
        return Status::InvalidArgument("File not in last level");
      }
    }
    edit.DeleteFile(level, number);
4366
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
I
Igor Canadi 已提交
4367
    if (status.ok()) {
4368
      InstallSuperVersion(cfd, deletion_state);
I
Igor Canadi 已提交
4369
    }
I
Igor Canadi 已提交
4370
    FindObsoleteFiles(deletion_state, false);
D
Dhruba Borthakur 已提交
4371
  } // lock released here
I
Igor Canadi 已提交
4372
  LogFlush(options_.info_log);
I
Igor Canadi 已提交
4373
  // remove files outside the db-lock
4374 4375 4376
  if (deletion_state.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(deletion_state);
  }
4377 4378 4379 4380 4381 4382
  {
    MutexLock l(&mutex_);
    // schedule flush if file deletion means we freed the space for flushes to
    // continue
    MaybeScheduleFlushOrCompaction();
  }
4383 4384 4385
  return status;
}

4386
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
4387
  MutexLock l(&mutex_);
4388
  versions_->GetLiveFilesMetaData(metadata);
4389
}
I
Igor Canadi 已提交
4390
#endif  // ROCKSDB_LITE
4391

I
Igor Canadi 已提交
4392 4393 4394 4395 4396 4397 4398 4399 4400 4401 4402 4403 4404 4405 4406 4407 4408 4409 4410 4411 4412 4413 4414 4415 4416 4417 4418
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

  std::string corruption_messages;
  for (const auto& md : metadata) {
    std::string file_path = dbname_ + md.name;
    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) {
      corruption_messages += "Sst file size mismatch: " + md.name +
                             ". 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);
  }
}

4419 4420 4421 4422 4423 4424 4425 4426 4427 4428 4429 4430 4431 4432 4433 4434 4435 4436 4437 4438 4439 4440 4441 4442 4443 4444 4445
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 已提交
4446 4447
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
4448
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
4449
               const Slice& key, const Slice& value) {
4450 4451 4452 4453
  // 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);
4454
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
4455 4456 4457
  return Write(opt, &batch);
}

4458 4459
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
4460
  WriteBatch batch;
4461
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
4462 4463 4464
  return Write(opt, &batch);
}

4465 4466
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
4467
  WriteBatch batch;
4468
  batch.Merge(column_family, key, value);
4469 4470 4471
  return Write(opt, &batch);
}

4472 4473
// Default implementation -- returns not supported status
Status DB::CreateColumnFamily(const ColumnFamilyOptions& options,
4474
                              const std::string& column_family_name,
4475
                              ColumnFamilyHandle** handle) {
4476
  return Status::NotSupported("");
4477
}
4478
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4479
  return Status::NotSupported("");
4480 4481
}

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

J
Jim Paton 已提交
4484
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4485 4486 4487 4488
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4489
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4490 4491 4492 4493 4494 4495 4496 4497 4498
  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;
4499 4500
}

4501 4502
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4503
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4504
  *dbptr = nullptr;
4505
  handles->clear();
J
jorlow@chromium.org 已提交
4506

I
Igor Canadi 已提交
4507 4508 4509 4510 4511 4512 4513 4514
  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");
    }
4515
  }
4516

I
Igor Canadi 已提交
4517
  DBImpl* impl = new DBImpl(db_options, dbname);
4518 4519 4520 4521 4522 4523 4524
  Status s = impl->env_->CreateDirIfMissing(impl->options_.wal_dir);
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4525 4526 4527 4528
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4529
  impl->mutex_.Lock();
4530 4531
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4532
  if (s.ok()) {
4533
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4534
    unique_ptr<WritableFile> lfile;
4535
    EnvOptions soptions(db_options);
4536
    s = impl->options_.env->NewWritableFile(
I
Igor Canadi 已提交
4537
        LogFileName(impl->options_.wal_dir, new_log_number), &lfile,
I
Igor Canadi 已提交
4538
        impl->options_.env->OptimizeForLogWrite(soptions));
J
jorlow@chromium.org 已提交
4539
    if (s.ok()) {
4540
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4541
      impl->logfile_number_ = new_log_number;
4542
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
4543

4544 4545
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4546 4547 4548
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
        if (cfd == nullptr) {
I
Igor Canadi 已提交
4549 4550 4551
          s = Status::InvalidArgument("Column family not found: ", cf.name);
          break;
        }
4552 4553
        handles->push_back(
            new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
4554
      }
I
Igor Canadi 已提交
4555 4556
    }
    if (s.ok()) {
4557
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
4558
        delete cfd->InstallSuperVersion(new SuperVersion(), &impl->mutex_);
4559
      }
I
Igor Canadi 已提交
4560 4561
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4562
      impl->DeleteObsoleteFiles();
4563
      impl->MaybeScheduleFlushOrCompaction();
4564
      impl->MaybeScheduleLogDBDeployStats();
4565
      s = impl->db_directory_->Fsync();
J
jorlow@chromium.org 已提交
4566 4567
    }
  }
4568

I
Igor Canadi 已提交
4569 4570
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
4571 4572
      if (cfd->options()->compaction_style == kCompactionStyleUniversal ||
          cfd->options()->compaction_style == kCompactionStyleFIFO) {
I
Igor Canadi 已提交
4573 4574 4575 4576
        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 已提交
4577 4578 4579
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
                "open with universal or FIFO compaction style.");
I
Igor Canadi 已提交
4580 4581 4582 4583
            break;
          }
        }
      }
4584 4585 4586 4587 4588 4589
      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 已提交
4590
      if (!s.ok()) {
4591 4592 4593 4594 4595
        break;
      }
    }
  }

4596 4597
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4598
  if (s.ok()) {
4599
    impl->opened_successfully_ = true;
J
jorlow@chromium.org 已提交
4600 4601
    *dbptr = impl;
  } else {
4602 4603 4604
    for (auto h : *handles) {
      delete h;
    }
4605
    handles->clear();
J
jorlow@chromium.org 已提交
4606 4607 4608 4609 4610
    delete impl;
  }
  return s;
}

4611 4612 4613
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4614
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4615 4616
}

4617 4618 4619
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4620
Status DestroyDB(const std::string& dbname, const Options& options) {
4621 4622 4623 4624 4625
  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 已提交
4626
  std::vector<std::string> filenames;
4627 4628
  std::vector<std::string> archiveFiles;

4629
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
4630 4631
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
4632 4633 4634 4635 4636 4637 4638

  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);
  }
4639

J
jorlow@chromium.org 已提交
4640 4641 4642 4643 4644
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
4645 4646
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4647 4648 4649
  if (result.ok()) {
    uint64_t number;
    FileType type;
D
dgrogan@chromium.org 已提交
4650
    for (size_t i = 0; i < filenames.size(); i++) {
4651
      if (ParseFileName(filenames[i], &number, &type) &&
4652
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4653 4654 4655
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
4656 4657
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
4658 4659 4660
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
4661 4662 4663 4664 4665
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4666

4667
    env->GetChildren(archivedir, &archiveFiles);
4668 4669
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4670 4671
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4672
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4673 4674 4675 4676 4677
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4678
    // ignore case where no archival directory is present.
4679
    env->DeleteDir(archivedir);
4680

J
jorlow@chromium.org 已提交
4681
    env->UnlockFile(lock);  // Ignore error since state is already gone
4682
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4683
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4684
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4685 4686 4687 4688
  }
  return result;
}

4689 4690
//
// A global method that can dump out the build version
K
kailiu 已提交
4691
void DumpLeveldbBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4692 4693
#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
4694
  Log(log, "Git sha %s", rocksdb_build_git_sha);
4695
  Log(log, "Compile time %s %s",
4696
      rocksdb_build_compile_time, rocksdb_build_compile_date);
I
Igor Canadi 已提交
4697
#endif
4698 4699
}

4700
}  // namespace rocksdb