db_impl.cc 176.6 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"

L
liuhuahang 已提交
12
#ifndef __STDC_FORMAT_MACROS
I
Igor Canadi 已提交
13
#define __STDC_FORMAT_MACROS
L
liuhuahang 已提交
14 15
#endif

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

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

74
namespace rocksdb {
J
jorlow@chromium.org 已提交
75

76
const std::string kDefaultColumnFamilyName("default");
77

K
kailiu 已提交
78
void DumpLeveldbBuildVersion(Logger * log);
79

S
Stanislau Hlebik 已提交
80 81 82 83 84 85 86 87 88 89 90 91 92 93
struct DBImpl::WriteContext {
  autovector<SuperVersion*> superversions_to_free_;
  autovector<log::Writer*> logs_to_free_;

  ~WriteContext() {
    for (auto& sv : superversions_to_free_) {
      delete sv;
    }
    for (auto& log : logs_to_free_) {
      delete log;
    }
  }
};

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

97 98 99 100 101
  // 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 已提交
102 103 104 105

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

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

  uint64_t total_bytes;

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

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

127 128 129 130 131 132 133 134
  // 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;
  }

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

  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_;

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

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

  // 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;
186
    size_t total_size = key_str_buf_.size() + other_key_str_buf_.size();
D
Danny Guo 已提交
187 188 189 190 191
    combined_key_buf_.reserve(total_size);
    combined_value_buf_.reserve(total_size);

    while (i + j < total_size) {
      int comp_res = 0;
192 193 194
      if (i < key_str_buf_.size() && j < other_key_str_buf_.size()) {
        comp_res = comparator->Compare(key_str_buf_[i], other_key_str_buf_[j]);
      } else if (i >= key_str_buf_.size() && j < other_key_str_buf_.size()) {
D
Danny Guo 已提交
195
        comp_res = 1;
196
      } else if (j >= other_key_str_buf_.size() && i < key_str_buf_.size()) {
D
Danny Guo 已提交
197 198 199
        comp_res = -1;
      }
      if (comp_res > 0) {
200
        AddToCombinedKeyValueSlices(other_key_str_buf_[j], other_value_str_buf_[j]);
D
Danny Guo 已提交
201 202
        j++;
      } else if (comp_res < 0) {
203
        AddToCombinedKeyValueSlices(key_str_buf_[i], existing_value_str_buf_[i]);
D
Danny Guo 已提交
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
        i++;
      }
    }
  }

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

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

    other_key_str_buf_.clear();
    other_value_str_buf_.clear();
    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 已提交
234 235 236 237 238
};

Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
                        const Options& src) {
239
  auto db_options = SanitizeOptions(dbname, DBOptions(src));
240
  auto cf_options = SanitizeOptions(icmp, ColumnFamilyOptions(src));
241 242 243 244 245
  return Options(db_options, cf_options);
}

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

247 248
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
249
    ClipToRange(&result.max_open_files, 20, 1000000);
250
  }
251

252
  if (result.info_log == nullptr) {
K
Kai Liu 已提交
253 254
    Status s = CreateLoggerFromOptions(dbname, result.db_log_dir, src.env,
                                       result, &result.info_log);
J
jorlow@chromium.org 已提交
255 256
    if (!s.ok()) {
      // No place suitable for logging
257
      result.info_log = nullptr;
J
jorlow@chromium.org 已提交
258 259
    }
  }
260

261 262 263 264 265 266
  if (!result.rate_limiter) {
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

267 268 269 270
  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
271
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
272
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
273
  }
274

275
  if (result.db_paths.size() == 0) {
276
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
277 278
  }

J
jorlow@chromium.org 已提交
279 280 281
  return result;
}

282 283
namespace {

284
Status SanitizeDBOptionsByCFOptions(
285
    const DBOptions* db_opts,
286 287 288 289 290 291 292 293 294 295 296
    const std::vector<ColumnFamilyDescriptor>& column_families) {
  Status s;
  for (auto cf : column_families) {
    s = cf.options.table_factory->SanitizeDBOptions(db_opts);
    if (!s.ok()) {
      return s;
    }
  }
  return Status::OK();
}

297 298 299 300 301 302 303
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;

304
  if (options.compaction_style == kCompactionStyleUniversal) {
305 306 307 308
    can_compress =
        (options.compaction_options_universal.compression_size_percent < 0);
  } else {
    // For leveled compress when min_level_to_compress == 0.
309 310
    can_compress = options.compression_per_level.empty() ||
                   options.compression_per_level[0] != kNoCompression;
311 312 313 314 315 316 317 318
  }

  if (can_compress) {
    return options.compression;
  } else {
    return kNoCompression;
  }
}
319
}  // namespace
320

I
Igor Canadi 已提交
321
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
J
jorlow@chromium.org 已提交
322
    : env_(options.env),
H
heyongqiang 已提交
323
      dbname_(dbname),
324 325
      db_options_(SanitizeOptions(dbname, options)),
      stats_(db_options_.statistics.get()),
326
      db_lock_(nullptr),
H
Haobo Xu 已提交
327
      mutex_(options.use_adaptive_mutex),
328
      shutting_down_(nullptr),
J
jorlow@chromium.org 已提交
329
      bg_cv_(&mutex_),
330
      logfile_number_(0),
I
Igor Canadi 已提交
331
      log_empty_(true),
332
      default_cf_handle_(nullptr),
I
Igor Canadi 已提交
333 334
      total_log_size_(0),
      max_total_in_memory_state_(0),
335
      tmp_batch_(),
336
      bg_schedule_needed_(false),
337
      bg_compaction_scheduled_(0),
338
      bg_manual_only_(0),
339
      bg_flush_scheduled_(0),
340
      manual_compaction_(nullptr),
341
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
342
      delete_obsolete_files_last_run_(options.env->NowMicros()),
343
      purge_wal_files_last_run_(0),
344
      last_stats_dump_time_microsec_(0),
345
      default_interval_to_delete_obsolete_WAL_(600),
346
      flush_on_destroy_(false),
347
      delayed_writes_(0),
L
Lei Jin 已提交
348
      env_options_(options),
349
      bg_work_gate_closed_(false),
350 351
      refitting_level_(false),
      opened_successfully_(false) {
H
heyongqiang 已提交
352
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
353

J
jorlow@chromium.org 已提交
354
  // Reserve ten files or so for other uses and give the rest to TableCache.
355
  // Give a large number for setting of "infinite" open files.
356 357
  const int table_cache_size = (db_options_.max_open_files == -1) ?
        4194304 : db_options_.max_open_files - 10;
358 359
  // Reserve ten files or so for other uses and give the rest to TableCache.
  table_cache_ =
360 361
      NewLRUCache(table_cache_size, db_options_.table_cache_numshardbits,
                  db_options_.table_cache_remove_scan_count_limit);
362

I
Igor Canadi 已提交
363
  versions_.reset(
364
      new VersionSet(dbname_, &db_options_, env_options_, table_cache_.get()));
365 366
  column_family_memtables_.reset(
      new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet()));
367

368 369 370
  DumpLeveldbBuildVersion(db_options_.info_log.get());
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
371

372
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
373 374 375
}

DBImpl::~DBImpl() {
376 377 378 379 380 381 382 383
  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();
384
        cfd->Unref();
385 386
      }
    }
387
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
388
  }
I
Igor Canadi 已提交
389 390

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

I
Igor Canadi 已提交
396 397 398 399 400
  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();
401 402
  }

403
  if (db_options_.allow_thread_local) {
404 405 406 407 408 409 410 411 412 413 414 415 416 417
    // 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;
418 419 420
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
421 422 423
    }
  }

424
  // versions need to be destroyed before table_cache since it can hold
425 426
  // references to table_cache.
  versions_.reset();
427
  mutex_.Unlock();
I
Igor Canadi 已提交
428 429 430
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
431

432
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
433 434 435
}

Status DBImpl::NewDB() {
436
  VersionEdit new_db;
437
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
438 439 440
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

441
  Log(db_options_.info_log, "Creating manifest 1 \n");
J
jorlow@chromium.org 已提交
442
  const std::string manifest = DescriptorFileName(dbname_, 1);
443
  unique_ptr<WritableFile> file;
I
Igor Canadi 已提交
444
  Status s = env_->NewWritableFile(
L
Lei Jin 已提交
445
      manifest, &file, env_->OptimizeForManifestWrite(env_options_));
J
jorlow@chromium.org 已提交
446 447 448
  if (!s.ok()) {
    return s;
  }
449
  file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
J
jorlow@chromium.org 已提交
450
  {
451
    log::Writer log(std::move(file));
J
jorlow@chromium.org 已提交
452 453 454 455 456 457
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
458
    s = SetCurrentFile(env_, dbname_, 1, db_directory_.get());
J
jorlow@chromium.org 已提交
459 460 461 462 463 464 465
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
466
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
467 468
    // No change needed
  } else {
469
    Log(db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
470 471 472 473
    *s = Status::OK();
  }
}

474
const Status DBImpl::CreateArchivalDirectory() {
475 476
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
477 478 479 480 481
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

482
void DBImpl::PrintStatistics() {
483
  auto dbstats = db_options_.statistics.get();
484
  if (dbstats) {
485
    Log(db_options_.info_log,
486 487
        "STATISTCS:\n %s",
        dbstats->ToString().c_str());
488 489 490
  }
}

491
void DBImpl::MaybeDumpStats() {
492
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
493 494 495 496

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
497
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
498 499 500 501 502 503
      <= 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;
504

505 506 507 508 509 510
    bool tmp1 = false;
    bool tmp2 = false;
    DBPropertyType cf_property_type =
        GetPropertyType("rocksdb.cfstats", &tmp1, &tmp2);
    DBPropertyType db_property_type =
        GetPropertyType("rocksdb.dbstats", &tmp1, &tmp2);
H
Haobo Xu 已提交
511
    std::string stats;
512 513 514
    {
      MutexLock l(&mutex_);
      for (auto cfd : *versions_->GetColumnFamilySet()) {
515 516
        cfd->internal_stats()->GetStringProperty(cf_property_type,
                                                 "rocksdb.cfstats", &stats);
517
      }
518 519
      default_cf_internal_stats_->GetStringProperty(db_property_type,
                                                    "rocksdb.dbstats", &stats);
520
    }
521 522
    Log(db_options_.info_log, "------- DUMPING STATS -------");
    Log(db_options_.info_log, "%s", stats.c_str());
523

524
    PrintStatistics();
525 526 527
  }
}

528
// Returns the list of live files in 'sst_live' and the list
K
kailiu 已提交
529
// of all files in the filesystem in 'candidate_files'.
I
Igor Canadi 已提交
530 531
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
532
//  db_options_.delete_obsolete_files_period_micros
I
Igor Canadi 已提交
533 534 535 536
// force = true -- force the full scan
void DBImpl::FindObsoleteFiles(DeletionState& deletion_state,
                               bool force,
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
537 538
  mutex_.AssertHeld();

539
  // if deletion is disabled, do nothing
540
  if (disable_delete_obsolete_files_ > 0) {
541 542 543
    return;
  }

544 545 546 547 548
  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;
549
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
550 551 552 553
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
    if (delete_obsolete_files_last_run_ +
554
        db_options_.delete_obsolete_files_period_micros < now_micros) {
555 556 557 558 559
      doing_the_full_scan = true;
      delete_obsolete_files_last_run_ = now_micros;
    }
  }

I
Igor Canadi 已提交
560 561 562
  // get obsolete files
  versions_->GetObsoleteFiles(&deletion_state.sst_delete_files);

I
Igor Canadi 已提交
563 564
  // store the current filenum, lognum, etc
  deletion_state.manifest_file_number = versions_->ManifestFileNumber();
565 566
  deletion_state.pending_manifest_file_number =
      versions_->PendingManifestFileNumber();
567
  deletion_state.log_number = versions_->MinLogNumber();
I
Igor Canadi 已提交
568 569
  deletion_state.prev_log_number = versions_->PrevLogNumber();

570 571 572 573 574 575 576 577
  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
578 579 580 581 582
  for (auto pair : pending_outputs_) {
    deletion_state.sst_live.emplace_back(pair.first, pair.second, 0);
  }
  /*  deletion_state.sst_live.insert(pending_outputs_.begin(),
                                   pending_outputs_.end());*/
I
Igor Canadi 已提交
583 584
  versions_->AddLiveFiles(&deletion_state.sst_live);

585
  if (doing_the_full_scan) {
586 587
    for (uint32_t path_id = 0;
         path_id < db_options_.db_paths.size(); path_id++) {
588 589 590
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
591
      env_->GetChildren(db_options_.db_paths[path_id].path,
592
                        &files);  // Ignore errors
593 594 595 596
      for (std::string file : files) {
        deletion_state.candidate_files.emplace_back(file, path_id);
      }
    }
597 598

    //Add log files in wal_dir
599
    if (db_options_.wal_dir != dbname_) {
600
      std::vector<std::string> log_files;
601
      env_->GetChildren(db_options_.wal_dir, &log_files);  // Ignore errors
602
      for (std::string log_file : log_files) {
603 604 605
        deletion_state.candidate_files.emplace_back(log_file, 0);
      }
    }
606
    // Add info log files in db_log_dir
607
    if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) {
608
      std::vector<std::string> info_log_files;
609 610
      // Ignore errors
      env_->GetChildren(db_options_.db_log_dir, &info_log_files);
611 612 613 614
      for (std::string log_file : info_log_files) {
        deletion_state.candidate_files.emplace_back(log_file, 0);
      }
    }
615
  }
616 617
}

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

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

I
Igor Canadi 已提交
639 640 641 642
  // 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 已提交
643 644
    return;
  }
645

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

K
kailiu 已提交
653 654 655 656 657 658 659
  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 = "";
660
  for (auto file : state.sst_delete_files) {
661 662 663
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
664
    delete file;
I
Igor Canadi 已提交
665 666
  }

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

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

680
  std::vector<std::string> old_info_log_files;
681
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
682 683 684
  for (const auto& candidate_file : candidate_files) {
    std::string to_delete = candidate_file.file_name;
    uint32_t path_id = candidate_file.path_id;
K
kailiu 已提交
685 686 687
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
688
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
689 690
      continue;
    }
J
jorlow@chromium.org 已提交
691

K
kailiu 已提交
692 693 694 695 696 697 698 699
    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'
700
        // (can happen during manifest roll)
K
kailiu 已提交
701 702 703
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
704
        keep = (sst_live_map.find(number) != sst_live_map.end());
K
kailiu 已提交
705 706 707
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
708 709 710 711
        // 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
712
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
713
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
714 715 716 717
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
718
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
719
        }
K
kailiu 已提交
720 721 722 723 724 725 726 727 728 729 730 731 732
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

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

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

762
  // Delete old info log files.
763
  size_t old_info_log_file_count = old_info_log_files.size();
764
  if (old_info_log_file_count >= db_options_.keep_log_file_num) {
765
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
766
    size_t end = old_info_log_file_count - db_options_.keep_log_file_num;
767
    for (unsigned int i = 0; i <= end; i++) {
768
      std::string& to_delete = old_info_log_files.at(i);
769 770 771
      std::string full_path_to_delete = (db_options_.db_log_dir.empty() ?
           dbname_ : db_options_.db_log_dir) + "/" + to_delete;
      Log(db_options_.info_log, "Delete info log file %s\n",
772 773
          full_path_to_delete.c_str());
      Status s = env_->DeleteFile(full_path_to_delete);
774
      if (!s.ok()) {
775
        Log(db_options_.info_log, "Delete info log file %s FAILED -- %s\n",
776 777
            to_delete.c_str(), s.ToString().c_str());
      }
H
heyongqiang 已提交
778 779
    }
  }
780
  PurgeObsoleteWALFiles();
781
  LogFlush(db_options_.info_log);
D
Dhruba Borthakur 已提交
782 783 784 785 786
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
  DeletionState deletion_state;
I
Igor Canadi 已提交
787
  FindObsoleteFiles(deletion_state, true);
788 789 790
  if (deletion_state.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(deletion_state);
  }
791 792
}

I
Igor Canadi 已提交
793
#ifndef ROCKSDB_LITE
794 795 796 797 798 799 800 801
// 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
802
void DBImpl::PurgeObsoleteWALFiles() {
803 804
  bool const ttl_enabled = db_options_.WAL_ttl_seconds > 0;
  bool const size_limit_enabled =  db_options_.WAL_size_limit_MB > 0;
805 806 807 808
  if (!ttl_enabled && !size_limit_enabled) {
    return;
  }

809 810
  int64_t current_time;
  Status s = env_->GetCurrentTime(&current_time);
811
  if (!s.ok()) {
812 813
    Log(db_options_.info_log, "Can't get current time: %s",
        s.ToString().c_str());
814 815 816 817 818
    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) ?
819
    db_options_.WAL_ttl_seconds / 2 : default_interval_to_delete_obsolete_WAL_;
820

821 822 823 824 825 826
  if (purge_wal_files_last_run_ + time_to_check > now_seconds) {
    return;
  }

  purge_wal_files_last_run_ = now_seconds;

827
  std::string archival_dir = ArchivalDirectory(db_options_.wal_dir);
828 829 830
  std::vector<std::string> files;
  s = env_->GetChildren(archival_dir, &files);
  if (!s.ok()) {
831 832
    Log(db_options_.info_log, "Can't get archive files: %s",
        s.ToString().c_str());
833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849
    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()) {
850
          Log(db_options_.info_log, "Can't get file mod time: %s: %s",
851 852 853
              file_path.c_str(), s.ToString().c_str());
          continue;
        }
854
        if (now_seconds - file_m_time > db_options_.WAL_ttl_seconds) {
855 856
          Status const s = env_->DeleteFile(file_path);
          if (!s.ok()) {
857
            Log(db_options_.info_log, "Can't delete file: %s: %s",
858 859
                file_path.c_str(), s.ToString().c_str());
            continue;
I
Igor Canadi 已提交
860 861 862
          } else {
            MutexLock l(&read_first_record_cache_mutex_);
            read_first_record_cache_.erase(number);
863 864
          }
          continue;
865
        }
866 867 868 869 870 871
      }

      if (size_limit_enabled) {
        uint64_t file_size;
        Status const s = env_->GetFileSize(file_path, &file_size);
        if (!s.ok()) {
872
          Log(db_options_.info_log, "Can't get file size: %s: %s",
873 874 875 876 877 878 879 880 881
              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()) {
882
              Log(db_options_.info_log, "Can't delete file: %s: %s",
883 884
                  file_path.c_str(), s.ToString().c_str());
              continue;
I
Igor Canadi 已提交
885 886 887
            } else {
              MutexLock l(&read_first_record_cache_mutex_);
              read_first_record_cache_.erase(number);
888 889 890 891 892 893 894 895 896 897 898
            }
          }
        }
      }
    }
  }

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

899
  size_t const files_keep_num = db_options_.WAL_size_limit_MB *
900 901 902 903 904 905 906
    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;
907
  GetSortedWalsOfType(archival_dir, archived_logs, kArchivedLogFile);
908 909

  if (files_del_num > archived_logs.size()) {
910
    Log(db_options_.info_log, "Trying to delete more archived log files than "
911 912 913 914 915 916 917 918
        "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()) {
919
      Log(db_options_.info_log, "Can't delete file: %s: %s",
920 921
          file_path.c_str(), s.ToString().c_str());
      continue;
I
Igor Canadi 已提交
922 923 924
    } else {
      MutexLock l(&read_first_record_cache_mutex_);
      read_first_record_cache_.erase(archived_logs[i]->LogNumber());
925 926
    }
  }
D
Dhruba Borthakur 已提交
927 928
}

929 930 931 932 933 934 935 936 937 938
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 已提交
939 940 941 942 943 944 945 946 947 948 949 950 951
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 已提交
952 953
      SequenceNumber sequence;
      Status s = ReadFirstRecord(log_type, number, &sequence);
I
Igor Canadi 已提交
954 955 956
      if (!s.ok()) {
        return s;
      }
I
Igor Canadi 已提交
957 958 959 960
      if (sequence == 0) {
        // empty file
        continue;
      }
I
Igor Canadi 已提交
961

962 963 964 965 966 967
      // 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 已提交
968 969
      uint64_t size_bytes;
      s = env_->GetFileSize(LogFileName(path, number), &size_bytes);
970 971 972 973 974
      // 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 已提交
975 976 977 978 979
      if (!s.ok()) {
        return s;
      }

      log_files.push_back(std::move(unique_ptr<LogFile>(
I
Igor Canadi 已提交
980
          new LogFileImpl(number, log_type, sequence, size_bytes))));
I
Igor Canadi 已提交
981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004
    }
  }
  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 已提交
1005 1006
  // end could be -ve.
  size_t start_index = std::max(static_cast<int64_t>(0), end);
I
Igor Canadi 已提交
1007 1008 1009 1010 1011 1012
  // 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 已提交
1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025
                               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 已提交
1026
  if (type == kAliveLogFile) {
1027
    std::string fname = LogFileName(db_options_.wal_dir, number);
I
Igor Canadi 已提交
1028 1029 1030 1031
    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 已提交
1032
    }
I
Igor Canadi 已提交
1033
  }
I
Igor Canadi 已提交
1034

I
Igor Canadi 已提交
1035
  if (type == kArchivedLogFile || !s.ok()) {
I
Igor Canadi 已提交
1036
    //  check if the file got moved to archive.
1037 1038
    std::string archived_file =
        ArchivedLogFileName(db_options_.wal_dir, number);
I
Igor Canadi 已提交
1039
    s = ReadFirstLine(archived_file, sequence);
I
Igor Canadi 已提交
1040
  }
I
Igor Canadi 已提交
1041 1042 1043 1044 1045 1046

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

I
Igor Canadi 已提交
1049 1050
// the function returns status.ok() and sequence == 0 if the file exists, but is
// empty
I
Igor Canadi 已提交
1051
Status DBImpl::ReadFirstLine(const std::string& fname,
I
Igor Canadi 已提交
1052
                             SequenceNumber* sequence) {
I
Igor Canadi 已提交
1053 1054 1055 1056 1057 1058
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
    Logger* info_log;
    const char* fname;

    Status* status;
1059
    bool ignore_error;  // true if db_options_.paranoid_checks==false
I
Igor Canadi 已提交
1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071
    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;
L
Lei Jin 已提交
1072
  Status status = env_->NewSequentialFile(fname, &file, env_options_);
I
Igor Canadi 已提交
1073 1074 1075 1076 1077 1078 1079

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

  LogReporter reporter;
  reporter.env = env_;
1080
  reporter.info_log = db_options_.info_log.get();
I
Igor Canadi 已提交
1081 1082
  reporter.fname = fname.c_str();
  reporter.status = &status;
1083
  reporter.ignore_error = !db_options_.paranoid_checks;
I
Igor Canadi 已提交
1084 1085 1086 1087 1088 1089
  log::Reader reader(std::move(file), &reporter, true /*checksum*/,
                     0 /*initial_offset*/);
  std::string scratch;
  Slice record;

  if (reader.ReadRecord(&record, &scratch) &&
1090
      (status.ok() || !db_options_.paranoid_checks)) {
I
Igor Canadi 已提交
1091 1092 1093 1094 1095
    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 已提交
1096 1097 1098
      WriteBatch batch;
      WriteBatchInternal::SetContents(&batch, record);
      *sequence = WriteBatchInternal::Sequence(&batch);
I
Igor Canadi 已提交
1099 1100 1101 1102
      return Status::OK();
    }
  }

I
Igor Canadi 已提交
1103 1104 1105
  // 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 已提交
1106 1107 1108 1109 1110
  return status;
}

#endif  // ROCKSDB_LITE

1111
Status DBImpl::Recover(
1112 1113
    const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
    bool error_if_log_file_exist) {
J
jorlow@chromium.org 已提交
1114 1115
  mutex_.AssertHeld();

1116
  bool is_new_db = false;
1117
  assert(db_lock_ == nullptr);
I
Igor Canadi 已提交
1118
  if (!read_only) {
1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130
    // 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;
    }

1131
    for (auto& db_path : db_options_.db_paths) {
1132
      s = env_->CreateDirIfMissing(db_path.path);
1133 1134 1135 1136 1137
      if (!s.ok()) {
        return s;
      }
    }

1138 1139 1140 1141 1142
    s = env_->NewDirectory(dbname_, &db_directory_);
    if (!s.ok()) {
      return s;
    }

1143
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
1144 1145 1146
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
1147

1148
    if (!env_->FileExists(CurrentFileName(dbname_))) {
1149
      if (db_options_.create_if_missing) {
1150
        s = NewDB();
1151
        is_new_db = true;
1152 1153 1154 1155 1156 1157
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
1158 1159
      }
    } else {
1160
      if (db_options_.error_if_exists) {
1161 1162 1163
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
1164
    }
M
Mayank Agarwal 已提交
1165 1166 1167 1168 1169 1170 1171
    // 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 已提交
1172 1173
  }

1174
  Status s = versions_->Recover(column_families, read_only);
1175
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
1176 1177
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
1178 1179
  if (s.ok()) {
    SequenceNumber max_sequence(0);
1180 1181
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
1182
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
1183 1184
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
1185 1186 1187 1188 1189 1190 1191

    // 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
1192
    // produced by an older version of rocksdb.
1193
    const uint64_t min_log = versions_->MinLogNumber();
1194 1195
    const uint64_t prev_log = versions_->PrevLogNumber();
    std::vector<std::string> filenames;
1196
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
1197 1198
    if (!s.ok()) {
      return s;
1199
    }
K
kailiu 已提交
1200

1201 1202
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
1203 1204
      uint64_t number;
      FileType type;
1205 1206 1207 1208 1209 1210 1211 1212 1213
      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);
        }
1214
      }
J
jorlow@chromium.org 已提交
1215
    }
1216

H
heyongqiang 已提交
1217 1218 1219 1220 1221 1222
    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");
    }

1223 1224
    // Recover in the order in which the logs were generated
    std::sort(logs.begin(), logs.end());
K
kailiu 已提交
1225
    for (const auto& log : logs) {
1226 1227 1228
      // 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 已提交
1229
      versions_->MarkFileNumberUsed(log);
K
Kai Liu 已提交
1230
      s = RecoverLogFile(log, &max_sequence, read_only);
1231
    }
L
Lei Jin 已提交
1232
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
1233 1234
  }

I
Igor Canadi 已提交
1235 1236 1237 1238 1239
  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 已提交
1240 1241 1242
  return s;
}

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

  mutex_.AssertHeld();

1261
  std::unordered_map<int, VersionEdit> version_edits;
1262
  // no need to refcount because iteration is under mutex
1263 1264
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1265 1266
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1267
  }
I
Igor Canadi 已提交
1268

J
jorlow@chromium.org 已提交
1269
  // Open the log file
1270
  std::string fname = LogFileName(db_options_.wal_dir, log_number);
1271
  unique_ptr<SequentialFile> file;
L
Lei Jin 已提交
1272
  Status status = env_->NewSequentialFile(fname, &file, env_options_);
J
jorlow@chromium.org 已提交
1273 1274 1275 1276 1277 1278 1279 1280
  if (!status.ok()) {
    MaybeIgnoreError(&status);
    return status;
  }

  // Create the log reader.
  LogReporter reporter;
  reporter.env = env_;
1281
  reporter.info_log = db_options_.info_log.get();
J
jorlow@chromium.org 已提交
1282
  reporter.fname = fname.c_str();
1283 1284 1285
  reporter.status = (db_options_.paranoid_checks &&
                     !db_options_.skip_log_error_on_recovery ? &status
                                                             : nullptr);
J
jorlow@chromium.org 已提交
1286 1287 1288 1289
  // 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).
1290
  log::Reader reader(std::move(file), &reporter, true/*checksum*/,
1291
                     0/*initial_offset*/);
1292
  Log(db_options_.info_log, "Recovering log #%" PRIu64 "", log_number);
J
jorlow@chromium.org 已提交
1293 1294 1295 1296 1297

  // Read all the records and add to a memtable
  std::string scratch;
  Slice record;
  WriteBatch batch;
I
Igor Canadi 已提交
1298
  while (reader.ReadRecord(&record, &scratch)) {
J
jorlow@chromium.org 已提交
1299
    if (record.size() < 12) {
I
Igor Canadi 已提交
1300 1301
      reporter.Corruption(record.size(),
                          Status::Corruption("log record too small"));
J
jorlow@chromium.org 已提交
1302 1303 1304 1305
      continue;
    }
    WriteBatchInternal::SetContents(&batch, record);

I
Igor Canadi 已提交
1306 1307 1308 1309 1310
    // If column family was not found, it might mean that the WAL write
    // batch references to the column family that was dropped after the
    // insert. We don't want to fail the whole write batch in that case -- we
    // just ignore the update. That's why we set ignore missing column families
    // to true
1311
    status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
1312 1313
        &batch, column_family_memtables_.get(),
        true /* ignore missing column families */, log_number);
1314

J
jorlow@chromium.org 已提交
1315 1316
    MaybeIgnoreError(&status);
    if (!status.ok()) {
I
Igor Canadi 已提交
1317
      return status;
J
jorlow@chromium.org 已提交
1318 1319 1320 1321 1322 1323 1324 1325
    }
    const SequenceNumber last_seq =
        WriteBatchInternal::Sequence(&batch) +
        WriteBatchInternal::Count(&batch) - 1;
    if (last_seq > *max_sequence) {
      *max_sequence = last_seq;
    }

1326
    if (!read_only) {
1327 1328
      // no need to refcount since client still doesn't have access
      // to the DB and can not drop column families while we iterate
1329
      for (auto cfd : *versions_->GetColumnFamilySet()) {
1330
        if (cfd->mem()->ShouldFlush()) {
1331
          // If this asserts, it means that InsertInto failed in
1332 1333
          // filtering updates to already-flushed column families
          assert(cfd->GetLogNumber() <= log_number);
1334
          auto iter = version_edits.find(cfd->GetID());
1335 1336
          assert(iter != version_edits.end());
          VersionEdit* edit = &iter->second;
1337
          status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
1338 1339 1340 1341 1342 1343 1344 1345
          // 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 已提交
1346 1347 1348 1349
      }
    }
  }

1350 1351 1352 1353
  if (versions_->LastSequence() < *max_sequence) {
    versions_->SetLastSequence(*max_sequence);
  }

1354
  if (!read_only) {
1355 1356
    // no need to refcount since client still doesn't have access
    // to the DB and can not drop column families while we iterate
1357
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1358
      auto iter = version_edits.find(cfd->GetID());
1359 1360 1361
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

1362 1363 1364 1365
      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
1366
        // (in WriteBatch::InsertInto)
1367 1368 1369 1370 1371 1372 1373
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1374
        status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
1375
      }
1376 1377 1378 1379 1380
      // we still want to clear the memtable, even if the recovery failed
      cfd->CreateNewMemtable();
      if (!status.ok()) {
        return status;
      }
J
jorlow@chromium.org 已提交
1381

1382 1383 1384 1385 1386 1387 1388
      // 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);
1389 1390 1391
      // 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 已提交
1392
      // log number
1393
      versions_->MarkFileNumberUsed(log_number + 1);
1394 1395 1396 1397 1398
      status = versions_->LogAndApply(cfd, edit, &mutex_);
      if (!status.ok()) {
        return status;
      }
    }
1399
  }
I
Igor Canadi 已提交
1400

J
jorlow@chromium.org 已提交
1401 1402 1403
  return status;
}

1404 1405
Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
                                           VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1406
  mutex_.AssertHeld();
1407
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1408
  FileMetaData meta;
1409 1410
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
  pending_outputs_[meta.fd.GetNumber()] = 0;  // path 0 for level 0 file.
1411 1412
  ReadOptions ro;
  ro.total_order_seek = true;
1413
  Arena arena;
1414 1415
  Status s;
  {
1416 1417 1418 1419
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
    const SequenceNumber newest_snapshot = snapshots_.GetNewest();
    const SequenceNumber earliest_seqno_in_memtable =
        mem->GetFirstSequenceNumber();
1420
    Log(db_options_.info_log, "[%s] Level-0 table #%" PRIu64 ": started",
1421
        cfd->GetName().c_str(), meta.fd.GetNumber());
1422

1423 1424 1425 1426 1427 1428 1429
    {
      mutex_.Unlock();
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
          iter.get(), &meta, cfd->internal_comparator(), newest_snapshot,
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->options()),
          cfd->options()->compression_opts, Env::IO_HIGH);
1430
      LogFlush(db_options_.info_log);
1431 1432
      mutex_.Lock();
    }
1433

1434
    Log(db_options_.info_log,
1435 1436 1437 1438
        "[%s] Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
        s.ToString().c_str());
  }
1439
  pending_outputs_.erase(meta.fd.GetNumber());
1440 1441 1442 1443

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1444
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1445 1446 1447
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1448 1449
  }

L
Lei Jin 已提交
1450
  InternalStats::CompactionStats stats(1);
1451
  stats.micros = env_->NowMicros() - start_micros;
1452
  stats.bytes_written = meta.fd.GetFileSize();
M
Mark Callaghan 已提交
1453
  stats.files_out_levelnp1 = 1;
1454
  cfd->internal_stats()->AddCompactionStats(level, stats);
1455 1456
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1457
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1458 1459 1460
  return s;
}

1461
Status DBImpl::WriteLevel0Table(ColumnFamilyData* cfd,
1462
                                autovector<MemTable*>& mems, VersionEdit* edit,
1463
                                uint64_t* filenumber, LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
1464
  mutex_.AssertHeld();
1465 1466
  const uint64_t start_micros = env_->NowMicros();
  FileMetaData meta;
1467 1468

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

1472 1473
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
1474
    mems[0]->GetFirstSequenceNumber();
1475
  Version* base = cfd->current();
1476
  base->Ref();          // it is likely that we do not need this reference
1477 1478 1479
  Status s;
  {
    mutex_.Unlock();
H
Haobo Xu 已提交
1480
    log_buffer->FlushBufferToLog();
K
Kai Liu 已提交
1481
    std::vector<Iterator*> memtables;
1482 1483
    ReadOptions ro;
    ro.total_order_seek = true;
1484
    Arena arena;
1485
    for (MemTable* m : mems) {
1486
      Log(db_options_.info_log,
1487 1488
          "[%s] Flushing memtable with next log file: %" PRIu64 "\n",
          cfd->GetName().c_str(), m->GetNextLogNumber());
1489 1490 1491 1492 1493 1494
      memtables.push_back(m->NewIterator(ro, &arena));
    }
    {
      ScopedArenaIterator iter(NewMergingIterator(&cfd->internal_comparator(),
                                                  &memtables[0],
                                                  memtables.size(), &arena));
1495 1496
      Log(db_options_.info_log,
           "[%s] Level-0 flush table #%" PRIu64 ": started",
1497 1498 1499 1500 1501 1502 1503
          cfd->GetName().c_str(), meta.fd.GetNumber());

      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
          iter.get(), &meta, cfd->internal_comparator(), newest_snapshot,
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->options()),
          cfd->options()->compression_opts, Env::IO_HIGH);
1504
      LogFlush(db_options_.info_log);
1505
    }
1506
    Log(db_options_.info_log,
1507
        "[%s] Level-0 flush table #%" PRIu64 ": %" PRIu64 " bytes %s",
F
Feng Zhu 已提交
1508
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
1509
        s.ToString().c_str());
I
Igor Canadi 已提交
1510

1511
    if (!db_options_.disableDataSync) {
1512 1513
      db_directory_->Fsync();
    }
1514 1515
    mutex_.Lock();
  }
1516 1517
  base->Unref();

1518
  // re-acquire the most current version
1519
  base = cfd->current();
1520 1521 1522 1523 1524 1525

  // 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 已提交
1526
  // committed so that other threads can recognize this file as a
1527 1528 1529 1530 1531 1532
  // 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;
1533
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1534 1535 1536 1537 1538 1539
    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.
1540
    if (base != nullptr && db_options_.max_background_compactions <= 1 &&
1541
        cfd->options()->compaction_style == kCompactionStyleLevel) {
1542 1543
      level = base->PickLevelForMemTableOutput(min_user_key, max_user_key);
    }
1544 1545 1546
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1547 1548
  }

L
Lei Jin 已提交
1549
  InternalStats::CompactionStats stats(1);
1550
  stats.micros = env_->NowMicros() - start_micros;
1551
  stats.bytes_written = meta.fd.GetFileSize();
1552
  cfd->internal_stats()->AddCompactionStats(level, stats);
1553 1554
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1555
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
1556 1557 1558
  return s;
}

1559 1560
Status DBImpl::FlushMemTableToOutputFile(ColumnFamilyData* cfd,
                                         bool* madeProgress,
H
Haobo Xu 已提交
1561 1562
                                         DeletionState& deletion_state,
                                         LogBuffer* log_buffer) {
1563
  mutex_.AssertHeld();
1564
  assert(cfd->imm()->size() != 0);
1565
  assert(cfd->imm()->IsFlushPending());
1566 1567 1568

  // Save the contents of the earliest memtable as a new Table
  uint64_t file_number;
K
Kai Liu 已提交
1569
  autovector<MemTable*> mems;
1570
  cfd->imm()->PickMemtablesToFlush(&mems);
1571
  if (mems.empty()) {
I
Igor Canadi 已提交
1572 1573
    LogToBuffer(log_buffer, "[%s] Nothing in memtable to flush",
                cfd->GetName().c_str());
L
Lei Jin 已提交
1574
    return Status::OK();
1575 1576 1577
  }

  // record the logfile_number_ before we release the mutex
1578 1579 1580
  // 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.
1581
  MemTable* m = mems[0];
1582 1583
  VersionEdit* edit = m->GetEdits();
  edit->SetPrevLogNumber(0);
1584 1585
  // SetLogNumber(log_num) indicates logs with number smaller than log_num
  // will no longer be picked up for recovery.
1586 1587
  edit->SetLogNumber(mems.back()->GetNextLogNumber());
  edit->SetColumnFamily(cfd->GetID());
1588

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

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

L
Lei Jin 已提交
1597
  if (!s.ok()) {
1598
    cfd->imm()->RollbackMemtableFlush(mems, file_number, &pending_outputs_);
1599 1600
  } else {
    // Replace immutable memtable with the generated Table
1601
    s = cfd->imm()->InstallMemtableFlushResults(
1602
        cfd, mems, versions_.get(), &mutex_, db_options_.info_log.get(),
1603
        file_number, &pending_outputs_, &deletion_state.memtables_to_free,
1604
        db_directory_.get(), log_buffer);
1605
  }
J
jorlow@chromium.org 已提交
1606 1607

  if (s.ok()) {
1608
    InstallSuperVersion(cfd, deletion_state);
1609 1610 1611
    if (madeProgress) {
      *madeProgress = 1;
    }
I
Igor Canadi 已提交
1612 1613 1614
    Version::LevelSummaryStorage tmp;
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
                cfd->current()->LevelSummary(&tmp));
1615

1616
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1617
      // add to deletion state
1618
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1619 1620 1621 1622
             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;
1623 1624
        alive_log_files_.pop_front();
      }
1625
    }
J
jorlow@chromium.org 已提交
1626
  }
1627

1628
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1629 1630 1631 1632 1633
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1634
  RecordFlushIOStats();
J
jorlow@chromium.org 已提交
1635 1636 1637
  return s;
}

1638
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1639
                            const Slice* begin, const Slice* end,
1640 1641
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
1642
  if (target_path_id >= db_options_.db_paths.size()) {
1643 1644 1645
    return Status::InvalidArgument("Invalid target path ID");
  }

1646 1647
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1648 1649

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1650
  if (!s.ok()) {
1651
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1652 1653 1654
    return s;
  }

I
Igor Canadi 已提交
1655
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1656 1657
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1658 1659
    Version* base = cfd->current();
    for (int level = 1; level < cfd->NumberLevels(); level++) {
G
Gabor Cselle 已提交
1660 1661 1662 1663 1664
      if (base->OverlapInLevel(level, begin, end)) {
        max_level_with_files = level;
      }
    }
  }
1665 1666
  for (int level = 0; level <= max_level_with_files; level++) {
    // in case the compaction is unversal or if we're compacting the
1667 1668 1669
    // bottom-most level, the output level will be the same as input one.
    // level 0 can never be the bottommost level (i.e. if all files are in level
    // 0, we will compact to level 1)
I
Igor Canadi 已提交
1670
    if (cfd->options()->compaction_style == kCompactionStyleUniversal ||
I
Igor Canadi 已提交
1671
        cfd->options()->compaction_style == kCompactionStyleFIFO ||
1672
        (level == max_level_with_files && level > 0)) {
1673
      s = RunManualCompaction(cfd, level, level, target_path_id, begin, end);
1674
    } else {
1675 1676
      s = RunManualCompaction(cfd, level, level + 1, target_path_id, begin,
                              end);
L
Lei Jin 已提交
1677 1678
    }
    if (!s.ok()) {
1679
      LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1680
      return s;
1681
    }
G
Gabor Cselle 已提交
1682
  }
1683 1684

  if (reduce_level) {
I
Igor Canadi 已提交
1685
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1686
  }
1687
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1688

1689 1690 1691 1692 1693 1694 1695
  {
    MutexLock l(&mutex_);
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1696
  return s;
1697 1698 1699
}

// return the same level if it cannot be moved
I
Igor Canadi 已提交
1700
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd, int level) {
1701
  mutex_.AssertHeld();
I
Igor Canadi 已提交
1702
  Version* current = cfd->current();
1703
  int minimum_level = level;
1704
  for (int i = level - 1; i > 0; --i) {
1705
    // stop if level i is not empty
1706
    if (current->NumLevelFiles(i) > 0) break;
1707
    // stop if level i is too small (cannot fit the level files)
I
Igor Canadi 已提交
1708
    if (cfd->compaction_picker()->MaxBytesForLevel(i) <
1709 1710 1711
        current->NumLevelBytes(level)) {
      break;
    }
1712 1713 1714 1715 1716 1717

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1721
  SuperVersion* superversion_to_free = nullptr;
1722
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1723 1724

  mutex_.Lock();
1725 1726 1727

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1728
    mutex_.Unlock();
1729
    Log(db_options_.info_log, "ReFitLevel: another thread is refitting");
I
Igor Canadi 已提交
1730
    delete new_superversion;
L
Lei Jin 已提交
1731
    return Status::NotSupported("another thread is refitting");
1732 1733 1734 1735 1736
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1737
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1738
    Log(db_options_.info_log,
1739 1740
        "RefitLevel: waiting for background threads to stop: %d %d",
        bg_compaction_scheduled_, bg_flush_scheduled_);
1741 1742 1743 1744
    bg_cv_.Wait();
  }

  // move to a smaller level
1745 1746
  int to_level = target_level;
  if (target_level < 0) {
I
Igor Canadi 已提交
1747
    to_level = FindMinimumEmptyLevelFitting(cfd, level);
1748
  }
1749 1750 1751

  assert(to_level <= level);

L
Lei Jin 已提交
1752
  Status status;
1753
  if (to_level < level) {
1754 1755
    Log(db_options_.info_log, "[%s] Before refitting:\n%s",
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1756

1757
    VersionEdit edit;
I
Igor Canadi 已提交
1758 1759
    edit.SetColumnFamily(cfd->GetID());
    for (const auto& f : cfd->current()->files_[level]) {
1760
      edit.DeleteFile(level, f->fd.GetNumber());
1761 1762 1763
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1764
    }
1765
    Log(db_options_.info_log, "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1766
        cfd->GetName().c_str(), edit.DebugString().data());
1767

I
Igor Canadi 已提交
1768
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
1769
    superversion_to_free = cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
1770
    new_superversion = nullptr;
1771

1772
    Log(db_options_.info_log, "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1773
        status.ToString().data());
1774 1775

    if (status.ok()) {
1776
      Log(db_options_.info_log, "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1777
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1778 1779 1780 1781 1782
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1783 1784 1785 1786

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1787
  return status;
G
Gabor Cselle 已提交
1788 1789
}

1790 1791 1792
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1793 1794
}

1795 1796 1797
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->max_mem_compaction_level;
1798 1799
}

1800 1801 1802
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->level0_stop_writes_trigger;
1803 1804
}

1805
Status DBImpl::Flush(const FlushOptions& options,
1806 1807 1808
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return FlushMemTable(cfh->cfd(), options);
H
heyongqiang 已提交
1809 1810
}

1811
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1812 1813 1814
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1815
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1816 1817
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1818
  assert(input_level >= 0);
1819

G
Gabor Cselle 已提交
1820 1821
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1822
  ManualCompaction manual;
I
Igor Canadi 已提交
1823
  manual.cfd = cfd;
1824 1825
  manual.input_level = input_level;
  manual.output_level = output_level;
1826
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
1827
  manual.done = false;
1828
  manual.in_progress = false;
1829 1830 1831
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
I
Igor Canadi 已提交
1832 1833
      cfd->options()->compaction_style == kCompactionStyleUniversal ||
      cfd->options()->compaction_style == kCompactionStyleFIFO) {
1834
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1835 1836 1837 1838
  } else {
    begin_storage = InternalKey(*begin, kMaxSequenceNumber, kValueTypeForSeek);
    manual.begin = &begin_storage;
  }
1839
  if (end == nullptr ||
I
Igor Canadi 已提交
1840 1841
      cfd->options()->compaction_style == kCompactionStyleUniversal ||
      cfd->options()->compaction_style == kCompactionStyleFIFO) {
1842
    manual.end = nullptr;
G
Gabor Cselle 已提交
1843 1844 1845 1846 1847 1848
  } else {
    end_storage = InternalKey(*end, 0, static_cast<ValueType>(0));
    manual.end = &end_storage;
  }

  MutexLock l(&mutex_);
1849

1850 1851 1852 1853 1854 1855
  // 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
1856
  // RunManualCompaction(), i.e. during that time no other compaction will
1857 1858 1859
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1860
  // RunManualCompaction() from getting to the second while loop below.
1861 1862 1863 1864 1865
  // 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) {
1866
    Log(db_options_.info_log,
I
Igor Canadi 已提交
1867 1868 1869
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1870 1871
    bg_cv_.Wait();
  }
1872

1873
  Log(db_options_.info_log, "[%s] Manual compaction starting",
I
Igor Canadi 已提交
1874
      cfd->GetName().c_str());
1875

1876 1877 1878 1879
  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 已提交
1880
      bg_cv_.Wait();
1881 1882
    } else {
      manual_compaction_ = &manual;
1883 1884 1885
      assert(bg_compaction_scheduled_ == 0);
      bg_compaction_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
G
Gabor Cselle 已提交
1886
    }
H
hans@chromium.org 已提交
1887
  }
1888

1889 1890 1891
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1892
  return manual.status;
J
jorlow@chromium.org 已提交
1893 1894
}

1895 1896
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
                             const FlushOptions& options) {
S
Stanislau Hlebik 已提交
1897 1898 1899 1900 1901 1902 1903 1904
  Writer w(&mutex_);
  w.batch = nullptr;
  w.sync = false;
  w.disableWAL = false;
  w.in_batch_group = false;
  w.done = false;
  w.timeout_hint_us = kNoTimeOut;

S
Stanislau Hlebik 已提交
1905 1906 1907 1908 1909 1910 1911 1912 1913 1914 1915 1916 1917 1918 1919 1920 1921
  Status s;
  {
    WriteContext context;
    MutexLock guard_lock(&mutex_);
    s = BeginWrite(&w, 0);
    assert(s.ok() && !w.done);  // No timeout and nobody should do our job

    // SetNewMemtableAndNewLogFile() will release and reacquire mutex
    // during execution
    s = SetNewMemtableAndNewLogFile(cfd, &context);
    cfd->imm()->FlushRequested();
    MaybeScheduleFlushOrCompaction();

    assert(!writers_.empty());
    assert(writers_.front() == &w);
    EndWrite(&w, &w, s);
  }
S
Stanislau Hlebik 已提交
1922 1923


H
heyongqiang 已提交
1924
  if (s.ok() && options.wait) {
1925
    // Wait until the compaction completes
1926
    s = WaitForFlushMemTable(cfd);
1927 1928
  }
  return s;
J
jorlow@chromium.org 已提交
1929 1930
}

1931
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1932 1933 1934
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1935
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1936 1937
    bg_cv_.Wait();
  }
1938
  if (!bg_error_.ok()) {
1939 1940 1941
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1942 1943
}

1944
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1945
  mutex_.AssertHeld();
1946
  bg_schedule_needed_ = false;
1947 1948
  if (bg_work_gate_closed_) {
    // gate closed for backgrond work
J
jorlow@chromium.org 已提交
1949 1950 1951
  } else if (shutting_down_.Acquire_Load()) {
    // DB is being deleted; no more background compactions
  } else {
1952
    bool is_flush_pending = false;
1953
    // no need to refcount since we're under a mutex
1954
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1955
      if (cfd->imm()->IsFlushPending()) {
1956 1957 1958
        is_flush_pending = true;
      }
    }
1959
    if (is_flush_pending) {
1960
      // memtable flush needed
1961
      if (bg_flush_scheduled_ < db_options_.max_background_flushes) {
1962 1963
        bg_flush_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH);
1964
      } else if (db_options_.max_background_flushes > 0) {
1965
        bg_schedule_needed_ = true;
1966
      }
1967
    }
1968
    bool is_compaction_needed = false;
1969
    // no need to refcount since we're under a mutex
1970
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1971
      if (cfd->current()->NeedsCompaction()) {
1972 1973 1974 1975
        is_compaction_needed = true;
        break;
      }
    }
1976

1977 1978
    // Schedule BGWorkCompaction if there's a compaction pending (or a memtable
    // flush, but the HIGH pool is not enabled)
1979 1980 1981 1982
    // Do it only if max_background_compactions hasn't been reached and
    // bg_manual_only_ == 0
    if (!bg_manual_only_ &&
        (is_compaction_needed ||
1983 1984
         (is_flush_pending && db_options_.max_background_flushes == 0))) {
      if (bg_compaction_scheduled_ < db_options_.max_background_compactions) {
1985 1986 1987 1988 1989
        bg_compaction_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
      } else {
        bg_schedule_needed_ = true;
      }
1990 1991 1992 1993
    }
  }
}

1994
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
1995
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
1996 1997 1998 1999
  IOSTATS_RESET(bytes_written);
}

void DBImpl::RecordCompactionIOStats() {
L
Lei Jin 已提交
2000
  RecordTick(stats_, COMPACT_READ_BYTES, IOSTATS(bytes_read));
2001
  IOSTATS_RESET(bytes_read);
L
Lei Jin 已提交
2002
  RecordTick(stats_, COMPACT_WRITE_BYTES, IOSTATS(bytes_written));
2003 2004 2005
  IOSTATS_RESET(bytes_written);
}

2006
void DBImpl::BGWorkFlush(void* db) {
2007
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
2008 2009 2010 2011
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
2012
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
2013 2014 2015
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
2016
Status DBImpl::BackgroundFlush(bool* madeProgress,
H
Haobo Xu 已提交
2017 2018
                               DeletionState& deletion_state,
                               LogBuffer* log_buffer) {
2019 2020 2021 2022 2023 2024
  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
2025
  for (auto cfd : *versions_->GetColumnFamilySet()) {
2026 2027 2028
    cfd->Ref();
    Status flush_status;
    while (flush_status.ok() && cfd->imm()->IsFlushPending()) {
2029 2030
      LogToBuffer(
          log_buffer,
2031
          "BackgroundCallFlush doing FlushMemTableToOutputFile with column "
I
Igor Canadi 已提交
2032 2033
          "family [%s], flush slots available %d",
          cfd->GetName().c_str(),
2034
          db_options_.max_background_flushes - bg_flush_scheduled_);
2035 2036
      flush_status = FlushMemTableToOutputFile(cfd, madeProgress,
                                               deletion_state, log_buffer);
2037 2038 2039
    }
    if (call_status.ok() && !flush_status.ok()) {
      call_status = flush_status;
2040
    }
2041
    cfd->Unref();
J
jorlow@chromium.org 已提交
2042
  }
2043
  versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
2044
  return call_status;
J
jorlow@chromium.org 已提交
2045 2046
}

2047
void DBImpl::BackgroundCallFlush() {
2048
  bool madeProgress = false;
K
Kai Liu 已提交
2049
  DeletionState deletion_state(true);
2050 2051
  assert(bg_flush_scheduled_);

2052
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2053 2054 2055 2056 2057 2058 2059 2060 2061 2062 2063
  {
    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.
2064 2065
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
H
Haobo Xu 已提交
2066 2067
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
2068
        Log(db_options_.info_log,
2069 2070 2071
            "Waiting after background flush error: %s"
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
H
Haobo Xu 已提交
2072
        log_buffer.FlushBufferToLog();
2073
        LogFlush(db_options_.info_log);
H
Haobo Xu 已提交
2074 2075 2076 2077 2078 2079 2080 2081 2082
        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
2083
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2084
      mutex_.Unlock();
2085 2086 2087 2088 2089
      // 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 已提交
2090
      log_buffer.FlushBufferToLog();
2091 2092 2093
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
2094 2095
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2096

H
Haobo Xu 已提交
2097
    bg_flush_scheduled_--;
2098 2099 2100 2101 2102
    // 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 已提交
2103 2104
      MaybeScheduleFlushOrCompaction();
    }
I
Igor Canadi 已提交
2105
    RecordFlushIOStats();
H
Haobo Xu 已提交
2106
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2107 2108 2109 2110
    // 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.
2111
  }
J
jorlow@chromium.org 已提交
2112 2113
}

2114
void DBImpl::BackgroundCallCompaction() {
2115
  bool madeProgress = false;
K
Kai Liu 已提交
2116
  DeletionState deletion_state(true);
H
Haobo Xu 已提交
2117 2118

  MaybeDumpStats();
2119
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2120 2121 2122 2123 2124 2125 2126 2127 2128 2129 2130
  {
    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.
2131 2132
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2133 2134
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
H
Haobo Xu 已提交
2135
        log_buffer.FlushBufferToLog();
2136
        Log(db_options_.info_log,
2137 2138 2139
            "Waiting after background compaction error: %s, "
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
2140
        LogFlush(db_options_.info_log);
2141 2142 2143 2144
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }
H
Haobo Xu 已提交
2145

2146 2147 2148 2149 2150 2151 2152
    // 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
2153
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2154
      mutex_.Unlock();
2155 2156 2157 2158 2159
      // 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 已提交
2160
      log_buffer.FlushBufferToLog();
2161 2162 2163
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
2164 2165
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2166

2167
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2168

2169 2170
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2171 2172 2173
    // Previous compaction may have produced too many files in a level,
    // So reschedule another compaction if we made progress in the
    // last compaction.
2174 2175 2176 2177 2178 2179
    //
    // 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_) {
2180 2181
      MaybeScheduleFlushOrCompaction();
    }
2182 2183 2184 2185 2186 2187 2188 2189 2190
    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 已提交
2191 2192 2193 2194
    // 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.
2195
  }
J
jorlow@chromium.org 已提交
2196 2197
}

A
Abhishek Kona 已提交
2198
Status DBImpl::BackgroundCompaction(bool* madeProgress,
2199 2200
                                    DeletionState& deletion_state,
                                    LogBuffer* log_buffer) {
2201
  *madeProgress = false;
J
jorlow@chromium.org 已提交
2202
  mutex_.AssertHeld();
2203

2204 2205
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2206 2207 2208 2209

  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2210 2211 2212 2213
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2214 2215 2216 2217 2218 2219 2220 2221 2222 2223
  }

  // 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",
2224
          db_options_.max_background_compactions - bg_compaction_scheduled_);
2225 2226 2227
      cfd->Ref();
      flush_stat = FlushMemTableToOutputFile(cfd, madeProgress, deletion_state,
                                             log_buffer);
2228
      cfd->Unref();
2229 2230 2231 2232 2233 2234 2235
      if (!flush_stat.ok()) {
        if (is_manual) {
          manual_compaction_->status = flush_stat;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
2236
        return flush_stat;
2237 2238 2239 2240 2241
      }
    }
  }

  unique_ptr<Compaction> c;
2242 2243
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2244
  if (is_manual) {
G
Gabor Cselle 已提交
2245
    ManualCompaction* m = manual_compaction_;
2246
    assert(m->in_progress);
2247 2248 2249
    c.reset(m->cfd->CompactRange(m->input_level, m->output_level,
                                 m->output_path_id, m->begin, m->end,
                                 &manual_end));
2250
    if (!c) {
2251
      m->done = true;
G
Gabor Cselle 已提交
2252
    }
I
Igor Canadi 已提交
2253 2254 2255 2256 2257 2258 2259 2260 2261
    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 已提交
2262
  } else {
2263
    // no need to refcount in iteration since it's always under a mutex
I
Igor Canadi 已提交
2264
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
2265
      if (!cfd->options()->disable_auto_compactions) {
2266
        c.reset(cfd->PickCompaction(log_buffer));
I
Igor Canadi 已提交
2267 2268
        if (c != nullptr) {
          // update statistics
L
Lei Jin 已提交
2269
          MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
I
Igor Canadi 已提交
2270 2271 2272
                      c->inputs(0)->size());
          break;
        }
I
Igor Canadi 已提交
2273 2274
      }
    }
J
jorlow@chromium.org 已提交
2275 2276 2277
  }

  Status status;
2278
  if (!c) {
H
hans@chromium.org 已提交
2279
    // Nothing to do
2280
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2281 2282 2283 2284 2285 2286 2287 2288
  } 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)) {
2289
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2290 2291 2292 2293 2294 2295 2296 2297 2298
    }
    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 已提交
2299
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
2300
    // Move file to next level
2301
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2302
    FileMetaData* f = c->input(0, 0);
2303
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
2304 2305 2306
    c->edit()->AddFile(c->level() + 1, f->fd.GetNumber(), f->fd.GetPathId(),
                       f->fd.GetFileSize(), f->smallest, f->largest,
                       f->smallest_seqno, f->largest_seqno);
I
Igor Canadi 已提交
2307
    status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_,
2308
                                    db_directory_.get());
I
Igor Canadi 已提交
2309
    InstallSuperVersion(c->column_family_data(), deletion_state);
2310

2311
    Version::LevelSummaryStorage tmp;
2312 2313 2314 2315 2316 2317
    LogToBuffer(
        log_buffer, "[%s] Moved #%lld to level-%d %lld bytes %s: %s\n",
        c->column_family_data()->GetName().c_str(),
        static_cast<unsigned long long>(f->fd.GetNumber()), c->level() + 1,
        static_cast<unsigned long long>(f->fd.GetFileSize()),
        status.ToString().c_str(), c->input_version()->LevelSummary(&tmp));
I
Igor Canadi 已提交
2318
    c->ReleaseCompactionFiles(status);
2319
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2320
  } else {
2321
    MaybeScheduleFlushOrCompaction(); // do more compaction work in parallel.
2322
    CompactionState* compact = new CompactionState(c.get());
H
Haobo Xu 已提交
2323
    status = DoCompactionWork(compact, deletion_state, log_buffer);
2324
    CleanupCompaction(compact, status);
I
Igor Canadi 已提交
2325
    c->ReleaseCompactionFiles(status);
2326
    c->ReleaseInputs();
2327
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2328
  }
2329
  c.reset();
J
jorlow@chromium.org 已提交
2330 2331 2332

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2333
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2334 2335
    // Ignore compaction errors found during shutting down
  } else {
2336
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2337
        status.ToString().c_str());
2338
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2339 2340 2341
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2342 2343

  if (is_manual) {
G
Gabor Cselle 已提交
2344
    ManualCompaction* m = manual_compaction_;
2345
    if (!status.ok()) {
L
Lei Jin 已提交
2346
      m->status = status;
2347 2348
      m->done = true;
    }
2349 2350 2351 2352 2353 2354 2355 2356 2357
    // 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.
2358 2359 2360 2361 2362
    //
    // 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) {
2363 2364
      m->done = true;
    }
G
Gabor Cselle 已提交
2365 2366 2367
    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 已提交
2368
      // Universal and FIFO compactions should always compact the whole range
I
Igor Canadi 已提交
2369
      assert(m->cfd->options()->compaction_style != kCompactionStyleUniversal);
I
Igor Canadi 已提交
2370
      assert(m->cfd->options()->compaction_style != kCompactionStyleFIFO);
2371
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2372 2373
      m->begin = &m->tmp_storage;
    }
2374
    m->in_progress = false; // not being processed anymore
2375
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2376
  }
2377
  return status;
J
jorlow@chromium.org 已提交
2378 2379
}

2380
void DBImpl::CleanupCompaction(CompactionState* compact, Status status) {
J
jorlow@chromium.org 已提交
2381
  mutex_.AssertHeld();
2382
  if (compact->builder != nullptr) {
J
jorlow@chromium.org 已提交
2383 2384
    // May happen if we get a shutdown call in the middle of compaction
    compact->builder->Abandon();
2385
    compact->builder.reset();
J
jorlow@chromium.org 已提交
2386
  } else {
2387
    assert(compact->outfile == nullptr);
J
jorlow@chromium.org 已提交
2388
  }
D
dgrogan@chromium.org 已提交
2389
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2390 2391
    const CompactionState::Output& out = compact->outputs[i];
    pending_outputs_.erase(out.number);
2392 2393 2394 2395

    // 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 已提交
2396
      TableCache::Evict(table_cache_.get(), out.number);
2397
    }
J
jorlow@chromium.org 已提交
2398 2399 2400 2401
  }
  delete compact;
}

2402
// Allocate the file numbers for the output file. We allocate as
2403
// many output file numbers as there are files in level+1 (at least one)
2404 2405 2406
// Insert them into pending_outputs so that they do not get deleted.
void DBImpl::AllocateCompactionOutputFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2407 2408
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
2409
  int filesNeeded = compact->compaction->num_input_files(1);
2410
  for (int i = 0; i < std::max(filesNeeded, 1); i++) {
2411
    uint64_t file_number = versions_->NewFileNumber();
2412
    pending_outputs_[file_number] = compact->compaction->GetOutputPathId();
2413 2414 2415 2416 2417 2418 2419
    compact->allocated_file_numbers.push_back(file_number);
  }
}

// Frees up unused file number.
void DBImpl::ReleaseCompactionUnusedFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2420
  for (const auto file_number : compact->allocated_file_numbers) {
2421 2422 2423 2424
    pending_outputs_.erase(file_number);
  }
}

J
jorlow@chromium.org 已提交
2425
Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
2426 2427
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
J
jorlow@chromium.org 已提交
2428
  uint64_t file_number;
2429 2430 2431 2432 2433 2434 2435
  // 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 已提交
2436 2437
    mutex_.Lock();
    file_number = versions_->NewFileNumber();
2438
    pending_outputs_[file_number] = compact->compaction->GetOutputPathId();
J
jorlow@chromium.org 已提交
2439 2440
    mutex_.Unlock();
  }
2441 2442
  CompactionState::Output out;
  out.number = file_number;
2443
  out.path_id = compact->compaction->GetOutputPathId();
2444 2445
  out.smallest.Clear();
  out.largest.Clear();
2446
  out.smallest_seqno = out.largest_seqno = 0;
2447
  compact->outputs.push_back(out);
J
jorlow@chromium.org 已提交
2448 2449

  // Make the output file
2450
  std::string fname = TableFileName(db_options_.db_paths, file_number,
2451
                                    compact->compaction->GetOutputPathId());
L
Lei Jin 已提交
2452
  Status s = env_->NewWritableFile(fname, &compact->outfile, env_options_);
2453

J
jorlow@chromium.org 已提交
2454
  if (s.ok()) {
L
Lei Jin 已提交
2455
    compact->outfile->SetIOPriority(Env::IO_LOW);
2456
    compact->outfile->SetPreallocationBlockSize(
2457
        compact->compaction->OutputFilePreallocationSize());
2458

2459
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
2460
    compact->builder.reset(NewTableBuilder(
L
Lei Jin 已提交
2461 2462 2463
        *cfd->ioptions(), cfd->internal_comparator(), compact->outfile.get(),
        compact->compaction->OutputCompressionType(),
        cfd->options()->compression_opts));
J
jorlow@chromium.org 已提交
2464
  }
2465
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
2466 2467 2468 2469 2470
  return s;
}

Status DBImpl::FinishCompactionOutputFile(CompactionState* compact,
                                          Iterator* input) {
2471
  assert(compact != nullptr);
2472
  assert(compact->outfile);
2473
  assert(compact->builder != nullptr);
J
jorlow@chromium.org 已提交
2474 2475

  const uint64_t output_number = compact->current_output()->number;
2476
  const uint32_t output_path_id = compact->current_output()->path_id;
J
jorlow@chromium.org 已提交
2477 2478 2479 2480 2481 2482 2483 2484 2485 2486 2487 2488 2489
  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;
2490
  compact->builder.reset();
J
jorlow@chromium.org 已提交
2491 2492

  // Finish and check for file errors
2493 2494
  if (s.ok() && !db_options_.disableDataSync) {
    if (db_options_.use_fsync) {
L
Lei Jin 已提交
2495
      StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
2496 2497
      s = compact->outfile->Fsync();
    } else {
L
Lei Jin 已提交
2498
      StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
2499 2500
      s = compact->outfile->Sync();
    }
J
jorlow@chromium.org 已提交
2501 2502 2503 2504
  }
  if (s.ok()) {
    s = compact->outfile->Close();
  }
2505
  compact->outfile.reset();
J
jorlow@chromium.org 已提交
2506 2507 2508

  if (s.ok() && current_entries > 0) {
    // Verify that the table is usable
I
Igor Canadi 已提交
2509
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
2510
    FileDescriptor fd(output_number, output_path_id, current_bytes);
I
Igor Canadi 已提交
2511
    Iterator* iter = cfd->table_cache()->NewIterator(
L
Lei Jin 已提交
2512
        ReadOptions(), env_options_, cfd->internal_comparator(), fd);
J
jorlow@chromium.org 已提交
2513 2514 2515
    s = iter->status();
    delete iter;
    if (s.ok()) {
2516
      Log(db_options_.info_log, "[%s] Generated table #%" PRIu64 ": %" PRIu64
2517 2518 2519
                             " keys, %" PRIu64 " bytes",
          cfd->GetName().c_str(), output_number, current_entries,
          current_bytes);
J
jorlow@chromium.org 已提交
2520 2521 2522 2523 2524 2525
    }
  }
  return s;
}


2526 2527
Status DBImpl::InstallCompactionResults(CompactionState* compact,
                                        LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
2528
  mutex_.AssertHeld();
2529 2530 2531 2532 2533

  // 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.
2534
  if (!versions_->VerifyCompactionFileConsistency(compact->compaction)) {
2535
    Log(db_options_.info_log, "[%s] Compaction %d@%d + %d@%d files aborted",
I
Igor Canadi 已提交
2536 2537 2538 2539
        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 已提交
2540
    return Status::Corruption("Compaction input files inconsistent");
2541 2542
  }

I
Igor Canadi 已提交
2543 2544
  LogToBuffer(log_buffer, "[%s] Compacted %d@%d + %d@%d files => %lld bytes",
              compact->compaction->column_family_data()->GetName().c_str(),
2545 2546 2547 2548 2549
              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 已提交
2550 2551 2552

  // Add compaction outputs
  compact->compaction->AddInputDeletions(compact->compaction->edit());
D
dgrogan@chromium.org 已提交
2553
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2554
    const CompactionState::Output& out = compact->outputs[i];
2555 2556 2557 2558
    compact->compaction->edit()->AddFile(compact->compaction->output_level(),
                                         out.number, out.path_id, out.file_size,
                                         out.smallest, out.largest,
                                         out.smallest_seqno, out.largest_seqno);
J
jorlow@chromium.org 已提交
2559
  }
I
Igor Canadi 已提交
2560 2561 2562
  return versions_->LogAndApply(compact->compaction->column_family_data(),
                                compact->compaction->edit(), &mutex_,
                                db_directory_.get());
J
jorlow@chromium.org 已提交
2563 2564
}

2565 2566 2567 2568 2569 2570 2571
// 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(
2572 2573
  SequenceNumber in, std::vector<SequenceNumber>& snapshots,
  SequenceNumber* prev_snapshot) {
2574
  SequenceNumber prev __attribute__((unused)) = 0;
2575 2576 2577
  for (const auto cur : snapshots) {
    assert(prev <= cur);
    if (cur >= in) {
2578
      *prev_snapshot = prev;
2579
      return cur;
2580
    }
2581 2582
    prev = cur; // assignment
    assert(prev);
2583
  }
2584
  Log(db_options_.info_log,
2585 2586
      "Looking for seqid %" PRIu64 " but maxseqid is %" PRIu64 "", in,
      snapshots[snapshots.size() - 1]);
2587 2588 2589 2590
  assert(0);
  return 0;
}

I
Igor Canadi 已提交
2591 2592 2593
uint64_t DBImpl::CallFlushDuringCompaction(ColumnFamilyData* cfd,
                                           DeletionState& deletion_state,
                                           LogBuffer* log_buffer) {
2594
  if (db_options_.max_background_flushes > 0) {
2595 2596 2597
    // flush thread will take care of this
    return 0;
  }
I
Igor Canadi 已提交
2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613
  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 已提交
2614
Status DBImpl::ProcessKeyValueCompaction(
2615
    bool is_snapshot_supported,
D
Danny Guo 已提交
2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626
    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 已提交
2627
  Status status;
D
Danny Guo 已提交
2628
  std::string compaction_filter_value;
J
jorlow@chromium.org 已提交
2629
  ParsedInternalKey ikey;
2630
  IterKey current_user_key;
J
jorlow@chromium.org 已提交
2631
  bool has_current_user_key = false;
2632
  IterKey delete_key;
2633 2634
  SequenceNumber last_sequence_for_key __attribute__((unused)) =
    kMaxSequenceNumber;
2635
  SequenceNumber visible_in_snapshot = kMaxSequenceNumber;
2636
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
I
Igor Canadi 已提交
2637 2638
  MergeHelper merge(
      cfd->user_comparator(), cfd->options()->merge_operator.get(),
2639
      db_options_.info_log.get(), cfd->options()->min_partial_merge_operands,
2640
      false /* internal key corruption is expected */);
I
Igor Canadi 已提交
2641
  auto compaction_filter = cfd->options()->compaction_filter;
2642 2643
  std::unique_ptr<CompactionFilter> compaction_filter_from_factory = nullptr;
  if (!compaction_filter) {
2644
    auto context = compact->GetFilterContextV1();
2645
    compaction_filter_from_factory =
I
Igor Canadi 已提交
2646 2647
        cfd->options()->compaction_filter_factory->CreateCompactionFilter(
            context);
2648 2649
    compaction_filter = compaction_filter_from_factory.get();
  }
2650

2651 2652 2653 2654
  int64_t key_drop_user = 0;
  int64_t key_drop_newer_entry = 0;
  int64_t key_drop_obsolete = 0;
  int64_t loop_cnt = 0;
2655 2656
  while (input->Valid() && !shutting_down_.Acquire_Load() &&
         !cfd->IsDropped()) {
2657 2658 2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673
    if (++loop_cnt > 1000) {
      if (key_drop_user > 0) {
        RecordTick(stats_, COMPACTION_KEY_DROP_USER, key_drop_user);
        key_drop_user = 0;
      }
      if (key_drop_newer_entry > 0) {
        RecordTick(stats_, COMPACTION_KEY_DROP_NEWER_ENTRY,
                   key_drop_newer_entry);
        key_drop_newer_entry = 0;
      }
      if (key_drop_obsolete > 0) {
        RecordTick(stats_, COMPACTION_KEY_DROP_OBSOLETE, key_drop_obsolete);
        key_drop_obsolete = 0;
      }
      RecordCompactionIOStats();
      loop_cnt = 0;
    }
2674 2675 2676 2677
    // 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 已提交
2678
    imm_micros += CallFlushDuringCompaction(cfd, deletion_state, log_buffer);
2679

D
Danny Guo 已提交
2680 2681 2682 2683 2684 2685 2686 2687 2688 2689 2690 2691 2692 2693 2694 2695 2696 2697 2698 2699 2700
    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 已提交
2701

2702
    if (compact->compaction->ShouldStopBefore(key) &&
2703
        compact->builder != nullptr) {
D
Danny Guo 已提交
2704
      status = FinishCompactionOutputFile(compact, input);
2705 2706 2707 2708 2709 2710
      if (!status.ok()) {
        break;
      }
    }

    // Handle key/value, add to state, etc.
J
jorlow@chromium.org 已提交
2711
    bool drop = false;
2712
    bool current_entry_is_merging = false;
J
jorlow@chromium.org 已提交
2713 2714
    if (!ParseInternalKey(key, &ikey)) {
      // Do not hide error keys
2715 2716
      // 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.
2717
      current_user_key.Clear();
J
jorlow@chromium.org 已提交
2718 2719
      has_current_user_key = false;
      last_sequence_for_key = kMaxSequenceNumber;
2720
      visible_in_snapshot = kMaxSequenceNumber;
J
jorlow@chromium.org 已提交
2721 2722
    } else {
      if (!has_current_user_key ||
2723
          cfd->user_comparator()->Compare(ikey.user_key,
2724
                                          current_user_key.GetKey()) != 0) {
J
jorlow@chromium.org 已提交
2725
        // First occurrence of this user key
L
Lei Jin 已提交
2726
        current_user_key.SetKey(ikey.user_key);
J
jorlow@chromium.org 已提交
2727 2728
        has_current_user_key = true;
        last_sequence_for_key = kMaxSequenceNumber;
2729
        visible_in_snapshot = kMaxSequenceNumber;
H
Haobo Xu 已提交
2730
        // apply the compaction filter to the first occurrence of the user key
D
Danny Guo 已提交
2731
        if (compaction_filter && !is_compaction_v2 &&
H
Haobo Xu 已提交
2732 2733 2734 2735 2736 2737 2738 2739 2740
            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 已提交
2741 2742 2743
          bool to_delete = compaction_filter->Filter(
              compact->compaction->level(), ikey.user_key, value,
              &compaction_filter_value, &value_changed);
H
Haobo Xu 已提交
2744
          if (to_delete) {
2745 2746 2747
            // make a copy of the original key and convert it to a delete
            delete_key.SetInternalKey(ExtractUserKey(key), ikey.sequence,
                                      kTypeDeletion);
H
Haobo Xu 已提交
2748
            // anchor the key again
2749
            key = delete_key.GetKey();
H
Haobo Xu 已提交
2750 2751 2752 2753
            // needed because ikey is backed by key
            ParseInternalKey(key, &ikey);
            // no value associated with delete
            value.clear();
2754
            ++key_drop_user;
H
Haobo Xu 已提交
2755 2756 2757 2758
          } else if (value_changed) {
            value = compaction_filter_value;
          }
        }
J
jorlow@chromium.org 已提交
2759 2760
      }

2761 2762 2763
      // 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.
2764
      SequenceNumber prev_snapshot = 0; // 0 means no previous snapshot
2765 2766 2767 2768
      SequenceNumber visible = visible_at_tip ? visible_at_tip :
        is_snapshot_supported ?  findEarliestVisibleSnapshot(ikey.sequence,
                                  compact->existing_snapshots, &prev_snapshot)
                              : 0;
2769 2770 2771 2772 2773

      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 已提交
2774
        // Hidden by an newer entry for same user key
2775
        // TODO: why not > ?
2776
        assert(last_sequence_for_key >= ikey.sequence);
J
jorlow@chromium.org 已提交
2777
        drop = true;    // (A)
2778
        ++key_drop_newer_entry;
J
jorlow@chromium.org 已提交
2779
      } else if (ikey.type == kTypeDeletion &&
D
Danny Guo 已提交
2780
          ikey.sequence <= earliest_snapshot &&
2781
          compact->compaction->KeyNotExistsBeyondOutputLevel(ikey.user_key)) {
J
jorlow@chromium.org 已提交
2782 2783 2784 2785 2786 2787 2788 2789
        // 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;
2790
        ++key_drop_obsolete;
2791
      } else if (ikey.type == kTypeMerge) {
2792 2793 2794 2795 2796 2797
        if (!merge.HasOperator()) {
          LogToBuffer(log_buffer, "Options::merge_operator is null.");
          status = Status::InvalidArgument(
              "merge_operator is not properly initialized.");
          break;
        }
2798 2799 2800 2801 2802 2803
        // 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 已提交
2804 2805
        int steps = 0;
        merge.MergeUntil(input, prev_snapshot, bottommost_level,
2806
            db_options_.statistics.get(), &steps);
D
Danny Guo 已提交
2807 2808 2809
        // Skip the Merge ops
        combined_idx = combined_idx - 1 + steps;

2810 2811 2812 2813 2814 2815 2816 2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827
        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 已提交
2828 2829 2830
      }

      last_sequence_for_key = ikey.sequence;
2831
      visible_in_snapshot = visible;
J
jorlow@chromium.org 已提交
2832 2833 2834
    }

    if (!drop) {
2835 2836 2837 2838 2839 2840 2841 2842 2843 2844 2845 2846 2847 2848 2849 2850
      // 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);
2851
      }
2852

2853 2854 2855 2856 2857 2858 2859 2860 2861 2862 2863
      // 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.
2864
        if (bottommost_level && ikey.sequence < earliest_snapshot &&
2865 2866 2867 2868 2869 2870 2871 2872 2873 2874 2875
            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
2876

2877 2878 2879 2880 2881 2882 2883
        // Open output file if necessary
        if (compact->builder == nullptr) {
          status = OpenCompactionOutputFile(compact);
          if (!status.ok()) {
            break;
          }
        }
2884 2885

        SequenceNumber seqno = GetInternalKeySeqno(newkey);
2886 2887
        if (compact->builder->NumEntries() == 0) {
          compact->current_output()->smallest.DecodeFrom(newkey);
2888 2889 2890 2891
          compact->current_output()->smallest_seqno = seqno;
        } else {
          compact->current_output()->smallest_seqno =
            std::min(compact->current_output()->smallest_seqno, seqno);
2892 2893 2894
        }
        compact->current_output()->largest.DecodeFrom(newkey);
        compact->builder->Add(newkey, value);
2895 2896
        compact->current_output()->largest_seqno =
          std::max(compact->current_output()->largest_seqno, seqno);
2897 2898 2899 2900

        // Close output file if it is big enough
        if (compact->builder->FileSize() >=
            compact->compaction->MaxOutputFileSize()) {
D
Danny Guo 已提交
2901
          status = FinishCompactionOutputFile(compact, input);
2902 2903 2904
          if (!status.ok()) {
            break;
          }
J
jorlow@chromium.org 已提交
2905 2906
        }

2907 2908 2909 2910 2911 2912 2913 2914 2915 2916 2917 2918 2919 2920 2921 2922 2923 2924 2925 2926
        // 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 已提交
2927 2928 2929 2930 2931
          break;
        }
      }
    }

2932
    // MergeUntil has moved input to the next entry
2933
    if (!current_entry_is_merging) {
2934 2935
      input->Next();
    }
J
jorlow@chromium.org 已提交
2936
  }
2937 2938 2939 2940 2941 2942 2943 2944 2945
  if (key_drop_user > 0) {
    RecordTick(stats_, COMPACTION_KEY_DROP_USER, key_drop_user);
  }
  if (key_drop_newer_entry > 0) {
    RecordTick(stats_, COMPACTION_KEY_DROP_NEWER_ENTRY, key_drop_newer_entry);
  }
  if (key_drop_obsolete > 0) {
    RecordTick(stats_, COMPACTION_KEY_DROP_OBSOLETE, key_drop_obsolete);
  }
2946 2947
  RecordCompactionIOStats();

D
Danny Guo 已提交
2948 2949 2950 2951 2952 2953 2954 2955 2956
  return status;
}

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

2957
  // Assemble slice vectors for user keys and existing values.
2958
  // We also keep track of our parsed internal key structs because
2959 2960 2961
  // we may need to access the sequence number in the event that
  // keys are garbage collected during the filter process.
  std::vector<ParsedInternalKey> ikey_buf;
D
Danny Guo 已提交
2962
  std::vector<Slice> user_key_buf;
2963 2964 2965 2966 2967 2968 2969 2970 2971 2972
  std::vector<Slice> existing_value_buf;

  for (const auto& key : compact->key_str_buf_) {
    ParsedInternalKey ikey;
    ParseInternalKey(Slice(key), &ikey);
    ikey_buf.emplace_back(ikey);
    user_key_buf.emplace_back(ikey.user_key);
  }
  for (const auto& value : compact->existing_value_str_buf_) {
    existing_value_buf.emplace_back(Slice(value));
D
Danny Guo 已提交
2973 2974 2975 2976 2977 2978 2979 2980 2981
  }

  // 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(),
2982
      user_key_buf, existing_value_buf,
D
Danny Guo 已提交
2983 2984 2985 2986 2987 2988
      &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() ==
2989
      compact->key_str_buf_.size());
D
Danny Guo 已提交
2990
  assert(compact->to_delete_buf_.size() ==
2991
      compact->existing_value_str_buf_.size());
D
Danny Guo 已提交
2992 2993 2994 2995 2996 2997 2998 2999 3000
  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],
3001 3002 3003
                        compact->key_str_buf_[i].size(),
                        ikey_buf[i].sequence,
                        kTypeDeletion);
D
Danny Guo 已提交
3004 3005

      // no value associated with delete
3006
      compact->existing_value_str_buf_[i].clear();
L
Lei Jin 已提交
3007
      RecordTick(stats_, COMPACTION_KEY_DROP_USER);
D
Danny Guo 已提交
3008
    } else if (compact->value_changed_buf_[i]) {
3009 3010
      compact->existing_value_str_buf_[i] =
          compact->new_value_buf_[new_value_idx++];
D
Danny Guo 已提交
3011 3012 3013 3014 3015 3016 3017 3018 3019 3020
    }
  }  // for
}

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

F
Feng Zhu 已提交
3023 3024
  // Generate file_levels_ for compaction berfore making Iterator
  compact->compaction->GenerateFileLevels();
D
Danny Guo 已提交
3025
  int64_t imm_micros = 0;  // Micros spent doing imm_ compactions
3026
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
3027 3028
  LogToBuffer(
      log_buffer,
I
Igor Canadi 已提交
3029 3030
      "[%s] Compacting %d@%d + %d@%d files, score %.2f slots available %d",
      cfd->GetName().c_str(), compact->compaction->num_input_files(0),
3031 3032
      compact->compaction->level(), compact->compaction->num_input_files(1),
      compact->compaction->output_level(), compact->compaction->score(),
3033
      db_options_.max_background_compactions - bg_compaction_scheduled_);
D
Danny Guo 已提交
3034 3035
  char scratch[2345];
  compact->compaction->Summary(scratch, sizeof(scratch));
I
Igor Canadi 已提交
3036 3037
  LogToBuffer(log_buffer, "[%s] Compaction start summary: %s\n",
              cfd->GetName().c_str(), scratch);
D
Danny Guo 已提交
3038

3039
  assert(cfd->current()->NumLevelFiles(compact->compaction->level()) > 0);
D
Danny Guo 已提交
3040 3041 3042 3043 3044 3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062 3063 3064
  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);

3065
  bool is_snapshot_supported = IsSnapshotSupported();
D
Danny Guo 已提交
3066 3067
  // Release mutex while we're actually doing the compaction work
  mutex_.Unlock();
3068
  log_buffer->FlushBufferToLog();
D
Danny Guo 已提交
3069 3070 3071 3072 3073 3074 3075 3076 3077 3078 3079 3080 3081 3082

  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 =
3083 3084
      cfd->options()->compaction_filter_factory_v2->CreateCompactionFilterV2(
          context);
D
Danny Guo 已提交
3085 3086 3087 3088 3089 3090 3091 3092 3093 3094 3095
  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) {
3096 3097
    while (backup_input->Valid() && !shutting_down_.Acquire_Load() &&
           !cfd->IsDropped()) {
I
Igor Canadi 已提交
3098 3099 3100 3101 3102 3103
      // 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 已提交
3104 3105 3106 3107 3108
      Slice key = backup_input->key();
      Slice value = backup_input->value();

      if (!ParseInternalKey(key, &ikey)) {
        // log error
3109
        Log(db_options_.info_log, "[%s] Failed to parse key: %s",
I
Igor Canadi 已提交
3110
            cfd->GetName().c_str(), key.ToString().c_str());
D
Danny Guo 已提交
3111 3112
        continue;
      } else {
3113 3114 3115 3116 3117 3118 3119
        const SliceTransform* transformer =
            cfd->options()->compaction_filter_factory_v2->GetPrefixExtractor();
        const auto key_prefix = transformer->Transform(ikey.user_key);
        if (!prefix_initialized) {
          compact->cur_prefix_ = key_prefix.ToString();
          prefix_initialized = true;
        }
D
Danny Guo 已提交
3120
        // If the prefix remains the same, keep buffering
3121
        if (key_prefix.compare(Slice(compact->cur_prefix_)) == 0) {
D
Danny Guo 已提交
3122 3123 3124 3125 3126 3127 3128 3129 3130 3131 3132 3133 3134 3135 3136 3137 3138
          // 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
3139
          if (compact->key_str_buf_.size() > 0) {
D
Danny Guo 已提交
3140 3141
            CallCompactionFilterV2(compact, compaction_filter_v2);
          }
3142
          compact->cur_prefix_ = key_prefix.ToString();
D
Danny Guo 已提交
3143 3144 3145 3146
        }
      }

      // Merge this batch of data (values + ineligible keys)
3147
      compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3148 3149 3150 3151

      // Done buffering for the current prefix. Spit it out to disk
      // Now just iterate through all the kv-pairs
      status = ProcessKeyValueCompaction(
3152
          is_snapshot_supported,
D
Danny Guo 已提交
3153 3154 3155 3156 3157 3158 3159 3160 3161 3162 3163 3164 3165 3166 3167 3168 3169 3170 3171 3172 3173 3174 3175 3176 3177 3178 3179 3180 3181
          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.
3182
        if (compact->key_str_buf_.size() > 0) {
D
Danny Guo 已提交
3183 3184
          CallCompactionFilterV2(compact, compaction_filter_v2);
        }
3185
        compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3186 3187

        status = ProcessKeyValueCompaction(
3188
            is_snapshot_supported,
D
Danny Guo 已提交
3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204
            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
3205
    if (compact->key_str_buf_.size() > 0) {
D
Danny Guo 已提交
3206 3207
      CallCompactionFilterV2(compact, compaction_filter_v2);
    }
3208
    compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3209
    status = ProcessKeyValueCompaction(
3210
        is_snapshot_supported,
D
Danny Guo 已提交
3211 3212 3213 3214 3215 3216 3217 3218 3219 3220 3221 3222 3223 3224
        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(
3225
      is_snapshot_supported,
D
Danny Guo 已提交
3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237
      visible_at_tip,
      earliest_snapshot,
      latest_snapshot,
      deletion_state,
      bottommost_level,
      imm_micros,
      input.get(),
      compact,
      false,
      log_buffer);
  }

3238
  if (status.ok() && (shutting_down_.Acquire_Load() || cfd->IsDropped())) {
L
Lei Jin 已提交
3239
    status = Status::ShutdownInProgress(
3240
        "Database shutdown or Column family drop during compaction");
J
jorlow@chromium.org 已提交
3241
  }
3242
  if (status.ok() && compact->builder != nullptr) {
3243
    status = FinishCompactionOutputFile(compact, input.get());
J
jorlow@chromium.org 已提交
3244 3245 3246 3247
  }
  if (status.ok()) {
    status = input->status();
  }
3248
  input.reset();
J
jorlow@chromium.org 已提交
3249

3250
  if (!db_options_.disableDataSync) {
3251 3252
    db_directory_->Fsync();
  }
I
Igor Canadi 已提交
3253

L
Lei Jin 已提交
3254
  InternalStats::CompactionStats stats(1);
3255
  stats.micros = env_->NowMicros() - start_micros - imm_micros;
M
Mark Callaghan 已提交
3256 3257
  stats.files_in_leveln = compact->compaction->num_input_files(0);
  stats.files_in_levelnp1 = compact->compaction->num_input_files(1);
L
Lei Jin 已提交
3258
  MeasureTime(stats_, COMPACTION_TIME, stats.micros);
3259 3260

  int num_output_files = compact->outputs.size();
3261
  if (compact->builder != nullptr) {
P
Pascal Borreli 已提交
3262
    // An error occurred so ignore the last output.
3263 3264 3265 3266
    assert(num_output_files > 0);
    --num_output_files;
  }
  stats.files_out_levelnp1 = num_output_files;
M
Mark Callaghan 已提交
3267

3268
  for (int i = 0; i < compact->compaction->num_input_files(0); i++) {
3269
    stats.bytes_readn += compact->compaction->input(0, i)->fd.GetFileSize();
3270
  }
M
Mark Callaghan 已提交
3271

3272
  for (int i = 0; i < compact->compaction->num_input_files(1); i++) {
3273
    stats.bytes_readnp1 += compact->compaction->input(1, i)->fd.GetFileSize();
3274
  }
M
Mark Callaghan 已提交
3275

3276
  for (int i = 0; i < num_output_files; i++) {
3277 3278 3279
    stats.bytes_written += compact->outputs[i].file_size;
  }

3280 3281
  RecordCompactionIOStats();

3282
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
3283
  mutex_.Lock();
3284 3285
  cfd->internal_stats()->AddCompactionStats(
      compact->compaction->output_level(), stats);
J
jorlow@chromium.org 已提交
3286

3287 3288 3289 3290
  // 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 已提交
3291
  if (status.ok()) {
3292
    status = InstallCompactionResults(compact, log_buffer);
I
Igor Canadi 已提交
3293
    InstallSuperVersion(cfd, deletion_state);
J
jorlow@chromium.org 已提交
3294
  }
3295
  Version::LevelSummaryStorage tmp;
3296 3297
  LogToBuffer(
      log_buffer,
I
Igor Canadi 已提交
3298
      "[%s] compacted to: %s, %.1f MB/sec, level %d, files in(%d, %d) out(%d) "
3299 3300
      "MB in(%.1f, %.1f) out(%.1f), read-write-amplify(%.1f) "
      "write-amplify(%.1f) %s\n",
I
Igor Canadi 已提交
3301
      cfd->GetName().c_str(), cfd->current()->LevelSummary(&tmp),
M
Mark Callaghan 已提交
3302
      (stats.bytes_readn + stats.bytes_readnp1 + stats.bytes_written) /
3303 3304 3305 3306
          (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 已提交
3307
      stats.bytes_written / 1048576.0,
3308
      (stats.bytes_written + stats.bytes_readnp1 + stats.bytes_readn) /
3309 3310
          (double)stats.bytes_readn,
      stats.bytes_written / (double)stats.bytes_readn,
3311
      status.ToString().c_str());
M
Mark Callaghan 已提交
3312

J
jorlow@chromium.org 已提交
3313 3314 3315
  return status;
}

3316 3317
namespace {
struct IterState {
3318 3319
  IterState(DBImpl* db, port::Mutex* mu, SuperVersion* super_version)
      : db(db), mu(mu), super_version(super_version) {}
3320 3321

  DBImpl* db;
3322
  port::Mutex* mu;
3323
  SuperVersion* super_version;
3324 3325 3326 3327
};

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

3329
  if (state->super_version->Unref()) {
3330 3331
    DBImpl::DeletionState deletion_state;

3332 3333 3334 3335 3336 3337
    state->mu->Lock();
    state->super_version->Cleanup();
    state->db->FindObsoleteFiles(deletion_state, false, true);
    state->mu->Unlock();

    delete state->super_version;
3338 3339 3340
    if (deletion_state.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(deletion_state);
    }
I
Igor Canadi 已提交
3341
  }
T
Tomislav Novak 已提交
3342

3343 3344
  delete state;
}
H
Hans Wennborg 已提交
3345
}  // namespace
3346

J
jorlow@chromium.org 已提交
3347
Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
3348
                                      ColumnFamilyData* cfd,
3349 3350 3351
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
3352 3353 3354 3355 3356 3357 3358 3359 3360 3361 3362 3363
  assert(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, 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, env_options_,
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
3364
  IterState* cleanup = new IterState(this, &mutex_, super_version);
3365
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
3366 3367 3368 3369

  return internal_iter;
}

3370 3371 3372 3373
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

J
jorlow@chromium.org 已提交
3374
Status DBImpl::Get(const ReadOptions& options,
3375
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
3376
                   std::string* value) {
3377
  return GetImpl(options, column_family, key, value);
3378 3379
}

I
Igor Canadi 已提交
3380 3381 3382
// DeletionState gets created and destructed outside of the lock -- we
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
3383
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
3384 3385 3386 3387 3388 3389
//
// 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
3390 3391
void DBImpl::InstallSuperVersion(ColumnFamilyData* cfd,
                                 DeletionState& deletion_state) {
3392
  mutex_.AssertHeld();
I
Igor Canadi 已提交
3393 3394 3395 3396
  // if new_superversion == nullptr, it means somebody already used it
  SuperVersion* new_superversion =
    (deletion_state.new_superversion != nullptr) ?
    deletion_state.new_superversion : new SuperVersion();
3397 3398
  SuperVersion* old_superversion =
      cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
3399
  deletion_state.new_superversion = nullptr;
3400
  deletion_state.superversions_to_free.push_back(old_superversion);
I
Igor Canadi 已提交
3401 3402
}

3403
Status DBImpl::GetImpl(const ReadOptions& options,
3404 3405
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
L
Lei Jin 已提交
3406
  StopWatch sw(env_, stats_, DB_GET);
3407
  PERF_TIMER_GUARD(get_snapshot_time);
3408

3409 3410 3411
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3412
  SequenceNumber snapshot;
3413
  if (options.snapshot != nullptr) {
3414 3415 3416
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
3417
  }
3418

3419
  // Acquire SuperVersion
3420
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
I
Igor Canadi 已提交
3421

3422
  // Prepare to store a list of merge operations if merge occurs.
3423
  MergeContext merge_context;
3424

3425
  Status s;
3426
  // First look in the memtable, then in the immutable memtable (if any).
3427
  // s is both in/out. When in, s could either be OK or MergeInProgress.
3428
  // merge_operands will contain the sequence of merges in the latter case.
3429
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
3430
  PERF_TIMER_STOP(get_snapshot_time);
3431

I
Igor Canadi 已提交
3432
  if (sv->mem->Get(lkey, value, &s, merge_context, *cfd->options())) {
3433
    // Done
L
Lei Jin 已提交
3434
    RecordTick(stats_, MEMTABLE_HIT);
I
Igor Canadi 已提交
3435
  } else if (sv->imm->Get(lkey, value, &s, merge_context, *cfd->options())) {
3436
    // Done
L
Lei Jin 已提交
3437
    RecordTick(stats_, MEMTABLE_HIT);
3438
  } else {
3439
    PERF_TIMER_GUARD(get_from_output_files_time);
I
Igor Canadi 已提交
3440
    sv->current->Get(options, lkey, value, &s, &merge_context, value_found);
L
Lei Jin 已提交
3441
    RecordTick(stats_, MEMTABLE_MISS);
3442
  }
3443

3444 3445
  {
    PERF_TIMER_GUARD(get_post_process_time);
3446

3447
    ReturnAndCleanupSuperVersion(cfd, sv);
3448

3449 3450 3451
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
3452
  return s;
J
jorlow@chromium.org 已提交
3453 3454
}

3455 3456
std::vector<Status> DBImpl::MultiGet(
    const ReadOptions& options,
3457
    const std::vector<ColumnFamilyHandle*>& column_family,
3458
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
3459

L
Lei Jin 已提交
3460
  StopWatch sw(env_, stats_, DB_MULTIGET);
3461
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
3462

3463
  SequenceNumber snapshot;
3464

3465
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
3466
    ColumnFamilyData* cfd;
3467 3468 3469 3470 3471
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
3472 3473 3474 3475 3476 3477
    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});
3478 3479 3480
    }
  }

3481
  mutex_.Lock();
3482 3483 3484 3485 3486
  if (options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
  }
3487
  for (auto mgd_iter : multiget_cf_data) {
3488 3489
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
3490
  }
3491
  mutex_.Unlock();
3492

3493 3494
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
3495

3496
  // Note: this always resizes the values array
3497 3498 3499
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
3500 3501

  // Keep track of bytes that we read for statistics-recording later
3502
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
3503
  PERF_TIMER_STOP(get_snapshot_time);
3504 3505 3506 3507

  // 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.
3508
  // merge_operands will contain the sequence of merges in the latter case.
3509
  for (size_t i = 0; i < num_keys; ++i) {
3510
    merge_context.Clear();
3511
    Status& s = stat_list[i];
3512 3513 3514
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
3515 3516
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
3517 3518 3519
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
I
Igor Canadi 已提交
3520 3521
    auto cfd = mgd->cfd;
    if (super_version->mem->Get(lkey, value, &s, merge_context,
I
Igor Canadi 已提交
3522
                                *cfd->options())) {
3523
      // Done
3524
    } else if (super_version->imm->Get(lkey, value, &s, merge_context,
I
Igor Canadi 已提交
3525
                                       *cfd->options())) {
3526 3527
      // Done
    } else {
I
Igor Canadi 已提交
3528
      super_version->current->Get(options, lkey, value, &s, &merge_context);
3529 3530 3531
    }

    if (s.ok()) {
3532
      bytes_read += value->size();
3533 3534 3535 3536
    }
  }

  // Post processing (decrement reference counts and record statistics)
3537
  PERF_TIMER_GUARD(get_post_process_time);
3538 3539
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
3540
  // TODO(icanadi) do we need lock here or just around Cleanup()?
3541 3542 3543 3544 3545 3546
  mutex_.Lock();
  for (auto mgd_iter : multiget_cf_data) {
    auto mgd = mgd_iter.second;
    if (mgd->super_version->Unref()) {
      mgd->super_version->Cleanup();
      superversions_to_delete.push_back(mgd->super_version);
3547 3548
    }
  }
3549 3550 3551 3552 3553 3554 3555
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3556
  }
3557

L
Lei Jin 已提交
3558 3559 3560
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3561
  PERF_TIMER_STOP(get_post_process_time);
3562

3563
  return stat_list;
3564 3565
}

3566
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& options,
3567
                                  const std::string& column_family_name,
3568
                                  ColumnFamilyHandle** handle) {
I
Igor Canadi 已提交
3569 3570 3571
  *handle = nullptr;
  MutexLock l(&mutex_);

I
Igor Canadi 已提交
3572 3573
  if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
      nullptr) {
I
Igor Canadi 已提交
3574 3575
    return Status::InvalidArgument("Column family already exists");
  }
3576
  VersionEdit edit;
3577
  edit.AddColumnFamily(column_family_name);
3578 3579
  uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
  edit.SetColumnFamily(new_id);
I
Igor Canadi 已提交
3580
  edit.SetLogNumber(logfile_number_);
I
Igor Canadi 已提交
3581
  edit.SetComparatorName(options.comparator->Name());
3582

I
Igor Canadi 已提交
3583 3584
  // LogAndApply will both write the creation in MANIFEST and create
  // ColumnFamilyData object
3585 3586
  Status s = versions_->LogAndApply(nullptr, &edit, &mutex_,
                                    db_directory_.get(), false, &options);
3587
  if (s.ok()) {
3588
    single_column_family_mode_ = false;
3589 3590 3591
    auto cfd =
        versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
    assert(cfd != nullptr);
3592
    delete cfd->InstallSuperVersion(new SuperVersion(), &mutex_);
3593
    *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
3594
    Log(db_options_.info_log, "Created column family [%s] (ID %u)",
3595
        column_family_name.c_str(), (unsigned)cfd->GetID());
I
Igor Canadi 已提交
3596 3597
    max_total_in_memory_state_ += cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3598
  } else {
3599
    Log(db_options_.info_log, "Creating column family [%s] FAILED -- %s",
3600 3601
        column_family_name.c_str(), s.ToString().c_str());
  }
3602
  return s;
3603 3604
}

3605 3606 3607 3608
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3609 3610
    return Status::InvalidArgument("Can't drop default column family");
  }
3611

I
Igor Canadi 已提交
3612 3613
  VersionEdit edit;
  edit.DropColumnFamily();
3614 3615
  edit.SetColumnFamily(cfd->GetID());

3616 3617 3618 3619 3620 3621 3622 3623
  Writer w(&mutex_);
  w.batch = nullptr;
  w.sync = false;
  w.disableWAL = false;
  w.in_batch_group = false;
  w.done = false;
  w.timeout_hint_us = kNoTimeOut;

3624
  Status s;
3625 3626 3627 3628 3629 3630
  {
    MutexLock l(&mutex_);
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
3631 3632 3633
      // we drop column family from a single write thread
      s = BeginWrite(&w, 0);
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
3634
      s = versions_->LogAndApply(cfd, &edit, &mutex_);
3635
      EndWrite(&w, &w, s);
3636
    }
3637
  }
3638

3639
  if (s.ok()) {
I
Igor Canadi 已提交
3640
    assert(cfd->IsDropped());
I
Igor Canadi 已提交
3641 3642
    max_total_in_memory_state_ -= cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3643 3644
    Log(db_options_.info_log, "Dropped column family with id %u\n",
        cfd->GetID());
3645
  } else {
3646 3647
    Log(db_options_.info_log,
        "Dropping column family with id %u FAILED -- %s\n",
3648 3649 3650
        cfd->GetID(), s.ToString().c_str());
  }

3651
  return s;
3652 3653
}

3654
bool DBImpl::KeyMayExist(const ReadOptions& options,
3655 3656
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3657
  if (value_found != nullptr) {
K
Kai Liu 已提交
3658 3659
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3660
  }
3661 3662
  ReadOptions roptions = options;
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3663
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3664

3665
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
3666 3667 3668
  // 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();
3669 3670
}

3671
Iterator* DBImpl::NewIterator(const ReadOptions& options,
3672 3673 3674
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3675 3676

  if (options.tailing) {
I
Igor Canadi 已提交
3677 3678 3679 3680
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
L
Lei Jin 已提交
3681
    // TODO(ljin): remove tailing iterator
I
Igor Canadi 已提交
3682
    auto iter = new ForwardIterator(this, options, cfd);
3683
    return NewDBIterator(env_, *cfd->options(), cfd->user_comparator(), iter,
3684
                         kMaxSequenceNumber, options.iterate_upper_bound);
3685
// return new TailingIterator(env_, this, options, cfd);
I
Igor Canadi 已提交
3686
#endif
T
Tomislav Novak 已提交
3687
  } else {
3688 3689 3690 3691
    SequenceNumber latest_snapshot = versions_->LastSequence();
    SuperVersion* sv = nullptr;
    sv = cfd->GetReferencedSuperVersion(&mutex_);

I
Igor Canadi 已提交
3692 3693 3694 3695
    auto snapshot =
        options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_
            : latest_snapshot;
T
Tomislav Novak 已提交
3696

3697 3698 3699 3700 3701 3702 3703 3704 3705 3706 3707 3708 3709 3710 3711 3712 3713 3714 3715 3716 3717 3718 3719 3720 3721 3722 3723 3724 3725 3726 3727 3728 3729 3730 3731 3732 3733 3734 3735 3736 3737 3738 3739
    // 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(
3740 3741 3742
        env_, *cfd->options(), cfd->user_comparator(),
        snapshot, options.iterate_upper_bound);

3743 3744 3745 3746 3747 3748
    Iterator* internal_iter =
        NewInternalIterator(options, cfd, sv, db_iter->GetArena());
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
J
jorlow@chromium.org 已提交
3749 3750
}

3751 3752
Status DBImpl::NewIterators(
    const ReadOptions& options,
I
Igor Canadi 已提交
3753
    const std::vector<ColumnFamilyHandle*>& column_families,
3754
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3755 3756 3757 3758 3759 3760 3761 3762 3763 3764 3765 3766 3767 3768 3769 3770 3771
  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 已提交
3772 3773 3774 3775
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3776 3777
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
L
Lei Jin 已提交
3778 3779 3780 3781
      auto iter = new ForwardIterator(this, options, cfd);
      iterators->push_back(
          NewDBIterator(env_, *cfd->options(), cfd->user_comparator(), iter,
                        kMaxSequenceNumber));
I
Igor Canadi 已提交
3782
    }
I
Igor Canadi 已提交
3783
#endif
I
Igor Canadi 已提交
3784 3785 3786 3787 3788 3789 3790 3791 3792 3793
  } 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;

3794 3795 3796 3797 3798 3799
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
          env_, *cfd->options(), cfd->user_comparator(), snapshot);
      Iterator* internal_iter = NewInternalIterator(
          options, cfd, super_versions[i], db_iter->GetArena());
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3800 3801 3802 3803
    }
  }

  return Status::OK();
3804 3805
}

3806 3807 3808 3809 3810 3811 3812 3813 3814
bool DBImpl::IsSnapshotSupported() const {
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (!cfd->mem()->IsSnapshotSupported()) {
      return false;
    }
  }
  return true;
}

J
jorlow@chromium.org 已提交
3815
const Snapshot* DBImpl::GetSnapshot() {
3816
  MutexLock l(&mutex_);
3817 3818
  // returns null if the underlying memtable does not support snapshot.
  if (!IsSnapshotSupported()) return nullptr;
3819
  return snapshots_.New(versions_->LastSequence());
J
jorlow@chromium.org 已提交
3820 3821 3822 3823
}

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

// Convenience methods
3828 3829
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3830
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3831 3832
}

3833 3834 3835 3836
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) {
3837 3838
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3839
    return DB::Merge(o, column_family, key, val);
3840 3841 3842
  }
}

3843
Status DBImpl::Delete(const WriteOptions& options,
3844
                      ColumnFamilyHandle* column_family, const Slice& key) {
3845
  return DB::Delete(options, column_family, key);
J
jorlow@chromium.org 已提交
3846 3847
}

S
Stanislau Hlebik 已提交
3848 3849
// REQUIRES: mutex_ is held
Status DBImpl::BeginWrite(Writer* w, uint64_t expiration_time) {
3850 3851 3852 3853 3854
  // the following code block pushes the current writer "w" into the writer
  // queue "writers_" and wait until one of the following conditions met:
  // 1. the job of "w" has been done by some other writers.
  // 2. "w" becomes the first writer in "writers_"
  // 3. "w" timed-out.
S
Stanislau Hlebik 已提交
3855 3856
  mutex_.AssertHeld();
  writers_.push_back(w);
3857 3858

  bool timed_out = false;
S
Stanislau Hlebik 已提交
3859
  while (!w->done && w != writers_.front()) {
3860
    if (expiration_time == 0) {
S
Stanislau Hlebik 已提交
3861 3862 3863
      w->cv.Wait();
    } else if (w->cv.TimedWait(expiration_time)) {
      if (w->in_batch_group) {
3864 3865 3866 3867 3868 3869 3870 3871 3872
        // then it means the front writer is currently doing the
        // write on behalf of this "timed-out" writer.  Then it
        // should wait until the write completes.
        expiration_time = 0;
      } else {
        timed_out = true;
        break;
      }
    }
3873
  }
3874

S
Stanislau Hlebik 已提交
3875
  if (timed_out) {
3876
#ifndef NDEBUG
3877
    bool found = false;
3878
#endif
3879
    for (auto iter = writers_.begin(); iter != writers_.end(); iter++) {
S
Stanislau Hlebik 已提交
3880
      if (*iter == w) {
3881
        writers_.erase(iter);
3882
#ifndef NDEBUG
3883
        found = true;
3884
#endif
3885 3886 3887
        break;
      }
    }
3888
#ifndef NDEBUG
3889
    assert(found);
3890
#endif
3891 3892 3893 3894 3895 3896
    // writers_.front() might still be in cond_wait without a time-out.
    // As a result, we need to signal it to wake it up.  Otherwise no
    // one else will wake him up, and RocksDB will hang.
    if (!writers_.empty()) {
      writers_.front()->cv.Signal();
    }
S
Stanislau Hlebik 已提交
3897 3898 3899 3900 3901 3902 3903 3904 3905 3906 3907 3908 3909 3910 3911 3912 3913 3914 3915 3916 3917 3918 3919 3920 3921 3922 3923 3924 3925 3926 3927
    return Status::TimedOut();
  }
  return Status::OK();
}

// REQUIRES: mutex_ is held
void DBImpl::EndWrite(Writer* w, Writer* last_writer, Status status) {
  // Pop out the current writer and all writers being pushed before the
  // current writer from the writer queue.
  mutex_.AssertHeld();
  while (!writers_.empty()) {
    Writer* ready = writers_.front();
    writers_.pop_front();
    if (ready != w) {
      ready->status = status;
      ready->done = true;
      ready->cv.Signal();
    }
    if (ready == last_writer) break;
  }

  // Notify new head of write queue
  if (!writers_.empty()) {
    writers_.front()->cv.Signal();
  }
}

Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
3928
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
S
Stanislau Hlebik 已提交
3929 3930 3931 3932 3933 3934 3935 3936 3937 3938 3939 3940 3941 3942 3943 3944 3945 3946 3947 3948 3949 3950 3951 3952 3953
  Writer w(&mutex_);
  w.batch = my_batch;
  w.sync = options.sync;
  w.disableWAL = options.disableWAL;
  w.in_batch_group = false;
  w.done = false;
  w.timeout_hint_us = options.timeout_hint_us;

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

  if (!options.disableWAL) {
    RecordTick(stats_, WRITE_WITH_WAL);
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_WITH_WAL, 1);
  }

  WriteContext context;
  mutex_.Lock();
  Status status = BeginWrite(&w, expiration_time);
  assert(status.ok() || status.IsTimedOut());
  if (status.IsTimedOut()) {
3954
    mutex_.Unlock();
L
Lei Jin 已提交
3955
    RecordTick(stats_, WRITE_TIMEDOUT);
3956
    return Status::TimedOut();
3957
  }
S
Stanislau Hlebik 已提交
3958 3959 3960 3961 3962 3963 3964 3965 3966 3967
  if (w.done) {  // write was done by someone else
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_OTHER,
                                           1);
    mutex_.Unlock();
    RecordTick(stats_, WRITE_DONE_BY_OTHER);
    return w.status;
  }

  RecordTick(stats_, WRITE_DONE_BY_SELF);
  default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_SELF, 1);
3968

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

I
Igor Canadi 已提交
3976
  uint64_t flush_column_family_if_log_file = 0;
3977
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
3978
                                    ? 4 * max_total_in_memory_state_
3979
                                    : db_options_.max_total_wal_size;
3980
  if (UNLIKELY(!single_column_family_mode_) &&
3981
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3982 3983 3984
      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;
3985
    Log(db_options_.info_log,
3986 3987 3988
        "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 已提交
3989 3990
  }

3991 3992
  if (LIKELY(single_column_family_mode_)) {
    // fast path
S
Stanislau Hlebik 已提交
3993 3994
    status = MakeRoomForWrite(default_cf_handle_->cfd(),
                              &context, expiration_time);
3995 3996 3997 3998 3999
  } else {
    // refcounting cfd in iteration
    bool dead_cfd = false;
    for (auto cfd : *versions_->GetColumnFamilySet()) {
      cfd->Ref();
S
Stanislau Hlebik 已提交
4000 4001 4002 4003 4004 4005 4006 4007 4008 4009 4010 4011
      if (flush_column_family_if_log_file != 0 &&
          cfd->GetLogNumber() <= flush_column_family_if_log_file) {
        // log size excedded limit and we need to do flush
        // SetNewMemtableAndNewLogFie may temporarily unlock and wait
        status = SetNewMemtableAndNewLogFile(cfd, &context);
        cfd->imm()->FlushRequested();
        MaybeScheduleFlushOrCompaction();
      } else {
        // May temporarily unlock and wait.
        status = MakeRoomForWrite(cfd, &context, expiration_time);
      }

4012 4013 4014 4015 4016 4017
      if (cfd->Unref()) {
        dead_cfd = true;
      }
      if (!status.ok()) {
        break;
      }
4018
    }
4019 4020
    if (dead_cfd) {
      versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
4021 4022
    }
  }
4023

D
dgrogan@chromium.org 已提交
4024
  uint64_t last_sequence = versions_->LastSequence();
4025
  Writer* last_writer = &w;
S
Stanislau Hlebik 已提交
4026
  if (status.ok()) {
4027 4028
    autovector<WriteBatch*> write_batch_group;
    BuildBatchGroup(&last_writer, &write_batch_group);
4029

4030 4031 4032
    // 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
4033
    // into memtables
4034
    {
4035
      mutex_.Unlock();
4036 4037 4038 4039 4040 4041 4042 4043 4044 4045
      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]);
        }
      }

4046 4047 4048 4049
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
4050
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
4051
      // Record statistics
L
Lei Jin 已提交
4052 4053
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
      RecordTick(stats_, BYTES_WRITTEN, WriteBatchInternal::ByteSize(updates));
4054 4055
      if (options.disableWAL) {
        flush_on_destroy_ = true;
4056
      }
L
Lei Jin 已提交
4057
      PERF_TIMER_STOP(write_pre_and_post_process_time);
4058

4059
      uint64_t log_size = 0;
4060
      if (!options.disableWAL) {
4061
        PERF_TIMER_GUARD(write_wal_time);
4062 4063
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
4064 4065
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
4066
        log_empty_ = false;
4067
        log_size = log_entry.size();
L
Lei Jin 已提交
4068 4069
        RecordTick(stats_, WAL_FILE_SYNCED);
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
H
heyongqiang 已提交
4070
        if (status.ok() && options.sync) {
4071
          if (db_options_.use_fsync) {
L
Lei Jin 已提交
4072
            StopWatch(env_, stats_, WAL_FILE_SYNC_MICROS);
4073
            status = log_->file()->Fsync();
4074
          } else {
L
Lei Jin 已提交
4075
            StopWatch(env_, stats_, WAL_FILE_SYNC_MICROS);
4076
            status = log_->file()->Sync();
4077
          }
H
heyongqiang 已提交
4078
        }
4079 4080
      }
      if (status.ok()) {
4081
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
4082

4083
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
4084 4085
            updates, column_family_memtables_.get(),
            options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
4086 4087 4088 4089 4090 4091 4092 4093
        // A non-OK status here indicates iteration failure (either in-memory
        // writebatch corruption (very bad), or the client specified invalid
        // column family).  This will later on trigger bg_error_.
        //
        // Note that existing logic was not sound. Any partial failure writing
        // into the memtable would result in a state that some write ops might
        // have succeeded in memtable but Status reports error for all writes.

L
Lei Jin 已提交
4094
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
4095
      }
L
Lei Jin 已提交
4096
      PERF_TIMER_START(write_pre_and_post_process_time);
4097 4098 4099
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
4100
      mutex_.Lock();
4101 4102 4103 4104 4105 4106 4107 4108 4109
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
      if (!options.disableWAL) {
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
4110
      if (status.ok()) {
4111
        versions_->SetLastSequence(last_sequence);
4112
      }
J
jorlow@chromium.org 已提交
4113 4114
    }
  }
4115
  if (db_options_.paranoid_checks && !status.ok() &&
4116
      !status.IsTimedOut() && bg_error_.ok()) {
I
Igor Canadi 已提交
4117 4118
    bg_error_ = status; // stop compaction & fail any further writes
  }
4119

S
Stanislau Hlebik 已提交
4120
  EndWrite(&w, last_writer, status);
I
Igor Canadi 已提交
4121
  mutex_.Unlock();
4122

4123
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
4124
    RecordTick(stats_, WRITE_TIMEDOUT);
4125 4126
  }

J
jorlow@chromium.org 已提交
4127 4128 4129
  return status;
}

4130 4131 4132
// This function will be called only when the first writer succeeds.
// All writers in the to-be-built batch group will be processed.
//
4133
// REQUIRES: Writer list must be non-empty
4134
// REQUIRES: First writer must have a non-nullptr batch
4135 4136
void DBImpl::BuildBatchGroup(Writer** last_writer,
                             autovector<WriteBatch*>* write_batch_group) {
4137 4138
  assert(!writers_.empty());
  Writer* first = writers_.front();
4139
  assert(first->batch != nullptr);
4140 4141

  size_t size = WriteBatchInternal::ByteSize(first->batch);
4142
  write_batch_group->push_back(first->batch);
4143 4144 4145 4146 4147 4148 4149 4150 4151 4152 4153 4154 4155 4156 4157 4158 4159 4160 4161

  // 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 已提交
4162 4163 4164 4165 4166 4167
    if (!w->disableWAL && first->disableWAL) {
      // Do not include a write that needs WAL into a batch that has
      // WAL disabled.
      break;
    }

4168 4169 4170 4171 4172 4173
    if (w->timeout_hint_us < first->timeout_hint_us) {
      // Do not include those writes with shorter timeout.  Otherwise, we might
      // execute a write that should instead be aborted because of timeout.
      break;
    }

4174 4175 4176 4177 4178
    if (w->batch == nullptr) {
      // Do not include those writes with nullptr batch. Those are not writes,
      // those are something else. They want to be alone
      break;
    }
4179

4180 4181 4182 4183
    size += WriteBatchInternal::ByteSize(w->batch);
    if (size > max_size) {
      // Do not make batch too big
      break;
4184
    }
4185 4186

    write_batch_group->push_back(w->batch);
4187
    w->in_batch_group = true;
4188 4189 4190 4191
    *last_writer = w;
  }
}

4192 4193 4194
// 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 已提交
4195 4196 4197 4198
// if n < bottom, return 0;
// if n >= top, return 1000;
// otherwise, let r = (n - bottom) /
//                    (top - bottom)
4199 4200 4201 4202
//  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 已提交
4203
uint64_t DBImpl::SlowdownAmount(int n, double bottom, double top) {
4204
  uint64_t delay;
J
Jim Paton 已提交
4205
  if (n >= top) {
4206 4207
    delay = 1000;
  }
J
Jim Paton 已提交
4208
  else if (n < bottom) {
4209 4210 4211 4212
    delay = 0;
  }
  else {
    // If we are here, we know that:
J
Jim Paton 已提交
4213
    //   level0_start_slowdown <= n < level0_slowdown
4214
    // since the previous two conditions are false.
M
Mark Callaghan 已提交
4215 4216
    double how_much =
      (double) (n - bottom) /
J
Jim Paton 已提交
4217
              (top - bottom);
M
Mark Callaghan 已提交
4218
    delay = std::max(how_much * how_much * 1000, 100.0);
4219 4220 4221 4222 4223
  }
  assert(delay <= 1000);
  return delay;
}

4224
// REQUIRES: mutex_ is held
4225
// REQUIRES: this thread is currently at the front of the writer queue
S
Stanislau Hlebik 已提交
4226 4227 4228
Status DBImpl::MakeRoomForWrite(ColumnFamilyData* cfd,
                                WriteContext* context,
                                uint64_t expiration_time) {
4229
  mutex_.AssertHeld();
4230
  assert(!writers_.empty());
S
Stanislau Hlebik 已提交
4231 4232 4233
  bool allow_delay = true;
  bool allow_hard_rate_limit_delay = true;
  bool allow_soft_rate_limit_delay = true;
4234
  uint64_t rate_limit_delay_millis = 0;
4235
  Status s;
4236
  double score;
4237 4238 4239 4240
  // 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;
4241
  bool has_timeout = (expiration_time > 0);
4242

4243 4244 4245 4246 4247
  while (true) {
    if (!bg_error_.ok()) {
      // Yield previous error
      s = bg_error_;
      break;
4248 4249 4250
    } else if (has_timeout && env_->NowMicros() > expiration_time) {
      s = Status::TimedOut();
      break;
4251
    } else if (allow_delay && cfd->NeedSlowdownForNumLevel0Files()) {
4252 4253 4254
      // 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
4255
      // individual write by 0-1ms to reduce latency variance.  Also,
4256 4257
      // this delay hands over some CPU to the compaction thread in
      // case it is sharing the same core as the writer.
4258
      uint64_t slowdown =
4259 4260 4261
          SlowdownAmount(cfd->current()->NumLevelFiles(0),
                         cfd->options()->level0_slowdown_writes_trigger,
                         cfd->options()->level0_stop_writes_trigger);
4262
      mutex_.Unlock();
4263
      uint64_t delayed;
J
Jim Paton 已提交
4264
      {
L
Lei Jin 已提交
4265
        StopWatch sw(env_, stats_, STALL_L0_SLOWDOWN_COUNT, &delayed);
4266
        env_->SleepForMicroseconds(slowdown);
J
Jim Paton 已提交
4267
      }
L
Lei Jin 已提交
4268
      RecordTick(stats_, STALL_L0_SLOWDOWN_MICROS, delayed);
4269 4270
      allow_delay = false;  // Do not delay a single write more than once
      mutex_.Lock();
4271 4272
      cfd->internal_stats()->AddCFStats(
          InternalStats::LEVEL0_SLOWDOWN, delayed);
4273
      delayed_writes_++;
S
Stanislau Hlebik 已提交
4274
    } else if (!cfd->mem()->ShouldFlush()) {
4275
      // There is room in current memtable
4276 4277 4278
      if (allow_delay) {
        DelayLoggingAndReset();
      }
4279
      break;
4280
    } else if (cfd->NeedWaitForNumMemtables()) {
4281
      // We have filled up the current memtable, but the previous
4282
      // ones are still being flushed, so we wait.
4283
      DelayLoggingAndReset();
4284
      Log(db_options_.info_log, "[%s] wait for memtable flush...\n",
I
Igor Canadi 已提交
4285
          cfd->GetName().c_str());
4286 4287 4288 4289
      if (schedule_background_work) {
        MaybeScheduleFlushOrCompaction();
        schedule_background_work = false;
      }
4290
      uint64_t stall;
J
Jim Paton 已提交
4291
      {
L
Lei Jin 已提交
4292
        StopWatch sw(env_, stats_, STALL_MEMTABLE_COMPACTION_COUNT, &stall);
4293 4294 4295 4296 4297
        if (!has_timeout) {
          bg_cv_.Wait();
        } else {
          bg_cv_.TimedWait(expiration_time);
        }
J
Jim Paton 已提交
4298
      }
L
Lei Jin 已提交
4299
      RecordTick(stats_, STALL_MEMTABLE_COMPACTION_MICROS, stall);
4300
      cfd->internal_stats()->AddCFStats(
4301
          InternalStats::MEMTABLE_COMPACTION, stall);
4302
    } else if (cfd->NeedWaitForNumLevel0Files()) {
4303
      DelayLoggingAndReset();
4304
      Log(db_options_.info_log, "[%s] wait for fewer level0 files...\n",
I
Igor Canadi 已提交
4305
          cfd->GetName().c_str());
4306
      uint64_t stall;
J
Jim Paton 已提交
4307
      {
L
Lei Jin 已提交
4308
        StopWatch sw(env_, stats_, STALL_L0_NUM_FILES_COUNT, &stall);
4309 4310 4311 4312 4313
        if (!has_timeout) {
          bg_cv_.Wait();
        } else {
          bg_cv_.TimedWait(expiration_time);
        }
J
Jim Paton 已提交
4314
      }
L
Lei Jin 已提交
4315
      RecordTick(stats_, STALL_L0_NUM_FILES_MICROS, stall);
4316 4317
      cfd->internal_stats()->AddCFStats(
          InternalStats::LEVEL0_NUM_FILES, stall);
4318
    } else if (allow_hard_rate_limit_delay && cfd->ExceedsHardRateLimit()) {
4319
      // Delay a write when the compaction score for any level is too large.
4320
      const int max_level = cfd->current()->MaxCompactionScoreLevel();
4321
      score = cfd->current()->MaxCompactionScore();
4322
      mutex_.Unlock();
4323
      uint64_t delayed;
J
Jim Paton 已提交
4324
      {
L
Lei Jin 已提交
4325
        StopWatch sw(env_, stats_, HARD_RATE_LIMIT_DELAY_COUNT, &delayed);
J
Jim Paton 已提交
4326 4327
        env_->SleepForMicroseconds(1000);
      }
4328
      // Make sure the following value doesn't round to zero.
4329 4330
      uint64_t rate_limit = std::max((delayed / 1000), (uint64_t) 1);
      rate_limit_delay_millis += rate_limit;
L
Lei Jin 已提交
4331
      RecordTick(stats_, RATE_LIMIT_DELAY_MILLIS, rate_limit);
4332
      if (cfd->options()->rate_limit_delay_max_milliseconds > 0 &&
J
Jim Paton 已提交
4333
          rate_limit_delay_millis >=
4334
              (unsigned)cfd->options()->rate_limit_delay_max_milliseconds) {
J
Jim Paton 已提交
4335
        allow_hard_rate_limit_delay = false;
4336
      }
4337
      mutex_.Lock();
4338
      cfd->internal_stats()->RecordLevelNSlowdown(max_level, delayed, false);
4339
    } else if (allow_soft_rate_limit_delay && cfd->ExceedsSoftRateLimit()) {
4340
      const int max_level = cfd->current()->MaxCompactionScoreLevel();
4341
      score = cfd->current()->MaxCompactionScore();
J
Jim Paton 已提交
4342 4343
      // Delay a write when the compaction score for any level is too large.
      // TODO: add statistics
4344 4345
      uint64_t slowdown = SlowdownAmount(score, cfd->options()->soft_rate_limit,
                                         cfd->options()->hard_rate_limit);
L
Lei Jin 已提交
4346
      uint64_t elapsed = 0;
J
Jim Paton 已提交
4347
      mutex_.Unlock();
J
Jim Paton 已提交
4348
      {
L
Lei Jin 已提交
4349
        StopWatch sw(env_, stats_, SOFT_RATE_LIMIT_DELAY_COUNT, &elapsed);
4350 4351
        env_->SleepForMicroseconds(slowdown);
        rate_limit_delay_millis += slowdown;
J
Jim Paton 已提交
4352
      }
J
Jim Paton 已提交
4353 4354
      allow_soft_rate_limit_delay = false;
      mutex_.Lock();
L
Lei Jin 已提交
4355
      cfd->internal_stats()->RecordLevelNSlowdown(max_level, elapsed, true);
4356
    } else {
S
Stanislau Hlebik 已提交
4357
      s = SetNewMemtableAndNewLogFile(cfd, context);
4358 4359 4360
      if (!s.ok()) {
        break;
      }
S
Stanislau Hlebik 已提交
4361 4362 4363 4364 4365 4366 4367 4368 4369 4370 4371 4372 4373 4374 4375 4376 4377 4378 4379 4380 4381 4382 4383 4384 4385 4386 4387
      MaybeScheduleFlushOrCompaction();
    }
  }
  return s;
}

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd,
                                           WriteContext* context) {
  mutex_.AssertHeld();
  unique_ptr<WritableFile> lfile;
  log::Writer* new_log = nullptr;
  MemTable* new_mem = nullptr;

  // Attempt to switch to a new memtable and trigger flush of old.
  // Do this without holding the dbmutex lock.
  assert(versions_->PrevLogNumber() == 0);
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
  mutex_.Unlock();
  Status s;
  {
    DelayLoggingAndReset();
    if (creating_new_log) {
4388 4389 4390
      s = env_->NewWritableFile(
          LogFileName(db_options_.wal_dir, new_log_number),
          &lfile, env_->OptimizeForLogWrite(env_options_));
S
Stanislau Hlebik 已提交
4391 4392 4393 4394 4395 4396
      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));
I
Igor Canadi 已提交
4397
      }
S
Stanislau Hlebik 已提交
4398 4399 4400 4401 4402 4403 4404 4405 4406 4407 4408 4409 4410 4411 4412 4413 4414 4415 4416 4417 4418 4419 4420 4421 4422 4423 4424 4425 4426 4427 4428 4429
    }

    if (s.ok()) {
      new_mem = new MemTable(cfd->internal_comparator(), *cfd->options());
      new_superversion = new SuperVersion();
    }
  }
  mutex_.Lock();
  if (!s.ok()) {
    // how do we fail if we're not creating new log?
    assert(creating_new_log);
    // Avoid chewing through file number space in a tight loop.
    versions_->ReuseLogFileNumber(new_log_number);
    assert(!new_mem);
    assert(!new_log);
    return s;
  }
  if (creating_new_log) {
    logfile_number_ = new_log_number;
    assert(new_log != nullptr);
    context->logs_to_free_.push_back(log_.release());
    log_.reset(new_log);
    log_empty_ = true;
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
    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_);
4430
      }
4431 4432
    }
  }
S
Stanislau Hlebik 已提交
4433 4434 4435 4436
  cfd->mem()->SetNextLogNumber(logfile_number_);
  cfd->imm()->Add(cfd->mem());
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
4437
  Log(db_options_.info_log,
S
Stanislau Hlebik 已提交
4438 4439 4440 4441
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), logfile_number_);
  context->superversions_to_free_.push_back(
      cfd->InstallSuperVersion(new_superversion, &mutex_));
4442 4443 4444
  return s;
}

I
Igor Canadi 已提交
4445
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
4446 4447 4448 4449 4450
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

4451 4452
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
4453
  auto version = cfd->current();
4454 4455 4456 4457 4458 4459 4460 4461 4462 4463 4464 4465
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
4466
#endif  // ROCKSDB_LITE
4467

I
Igor Canadi 已提交
4468 4469 4470 4471
const std::string& DBImpl::GetName() const {
  return dbname_;
}

4472 4473 4474 4475
Env* DBImpl::GetEnv() const {
  return env_;
}

4476 4477
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
4478
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
4479 4480
}

4481
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
4482
                         const Slice& property, std::string* value) {
4483 4484
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4485 4486 4487
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

4488
  value->clear();
4489 4490 4491 4492 4493 4494 4495 4496 4497 4498 4499 4500 4501 4502 4503
  if (is_int_property) {
    uint64_t int_value;
    bool ret_value = GetIntPropertyInternal(column_family, property_type,
                                            need_out_of_mutex, &int_value);
    if (ret_value) {
      *value = std::to_string(int_value);
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
    MutexLock l(&mutex_);
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
4504 4505
}

4506 4507
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
4508 4509
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4510 4511 4512 4513 4514 4515 4516 4517 4518 4519 4520 4521
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);
  if (!is_int_property) {
    return false;
  }
  return GetIntPropertyInternal(column_family, property_type, need_out_of_mutex,
                                value);
}

bool DBImpl::GetIntPropertyInternal(ColumnFamilyHandle* column_family,
                                    DBPropertyType property_type,
                                    bool need_out_of_mutex, uint64_t* value) {
4522 4523
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4524 4525 4526

  if (!need_out_of_mutex) {
    MutexLock l(&mutex_);
4527
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
4528 4529 4530 4531 4532 4533 4534 4535 4536 4537 4538 4539 4540 4541
  } else {
    SuperVersion* sv = GetAndRefSuperVersion(cfd);

    bool ret = cfd->internal_stats()->GetIntPropertyOutOfMutex(
        property_type, sv->current, value);

    ReturnAndCleanupSuperVersion(cfd, sv);

    return ret;
  }
}

SuperVersion* DBImpl::GetAndRefSuperVersion(ColumnFamilyData* cfd) {
  // TODO(ljin): consider using GetReferencedSuperVersion() directly
4542
  if (LIKELY(db_options_.allow_thread_local)) {
4543 4544 4545 4546 4547 4548 4549 4550 4551 4552
    return cfd->GetThreadLocalSuperVersion(&mutex_);
  } else {
    MutexLock l(&mutex_);
    return cfd->GetSuperVersion()->Ref();
  }
}

void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
  bool unref_sv = true;
4553
  if (LIKELY(db_options_.allow_thread_local)) {
4554 4555 4556 4557 4558 4559 4560 4561 4562 4563 4564 4565 4566 4567 4568
    unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
  }

  if (unref_sv) {
    // Release SuperVersion
    if (sv->Unref()) {
      {
        MutexLock l(&mutex_);
        sv->Cleanup();
      }
      delete sv;
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
    }
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
  }
4569 4570
}

4571
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4572
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
4573 4574
  // TODO(opt): better implementation
  Version* v;
4575 4576
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
4577 4578
  {
    MutexLock l(&mutex_);
4579
    v = cfd->current();
4580
    v->Ref();
J
jorlow@chromium.org 已提交
4581 4582 4583 4584 4585 4586 4587 4588 4589 4590 4591 4592 4593 4594 4595 4596 4597
  }

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

4598 4599
inline void DBImpl::DelayLoggingAndReset() {
  if (delayed_writes_ > 0) {
4600
    Log(db_options_.info_log, "delayed %d write...\n", delayed_writes_);
4601 4602 4603 4604
    delayed_writes_ = 0;
  }
}

I
Igor Canadi 已提交
4605 4606 4607 4608 4609
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
4610
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
4611 4612 4613 4614 4615 4616 4617 4618 4619 4620 4621 4622 4623 4624 4625 4626
  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;
  }
4627
  iter->reset(new TransactionLogIteratorImpl(db_options_.wal_dir, &db_options_,
L
Lei Jin 已提交
4628
                                             read_options, env_options_,
I
Igor Canadi 已提交
4629 4630 4631 4632
                                             seq, std::move(wal_files), this));
  return (*iter)->status();
}

4633 4634 4635
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4636 4637 4638
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
4639
    Log(db_options_.info_log, "DeleteFile %s failed.\n", name.c_str());
4640 4641 4642
    return Status::InvalidArgument("Invalid file name");
  }

4643 4644 4645 4646
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4647
      Log(db_options_.info_log, "DeleteFile %s failed - not archived log.\n",
4648
          name.c_str());
4649 4650
      return Status::NotSupported("Delete only supported for archived logs");
    }
4651
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
4652
    if (!status.ok()) {
4653
      Log(db_options_.info_log, "DeleteFile %s failed -- %s.\n",
4654
          name.c_str(), status.ToString().c_str());
4655 4656 4657 4658
    }
    return status;
  }

4659
  int level;
I
Igor Canadi 已提交
4660
  FileMetaData* metadata;
4661
  ColumnFamilyData* cfd;
4662
  VersionEdit edit;
K
Kai Liu 已提交
4663
  DeletionState deletion_state(true);
D
Dhruba Borthakur 已提交
4664 4665
  {
    MutexLock l(&mutex_);
4666
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4667
    if (!status.ok()) {
4668
      Log(db_options_.info_log, "DeleteFile %s failed. File not found\n",
4669
                             name.c_str());
D
Dhruba Borthakur 已提交
4670 4671
      return Status::InvalidArgument("File not found");
    }
I
Igor Canadi 已提交
4672
    assert((level > 0) && (level < cfd->NumberLevels()));
4673

D
Dhruba Borthakur 已提交
4674
    // If the file is being compacted no need to delete.
4675
    if (metadata->being_compacted) {
4676
      Log(db_options_.info_log,
4677
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
D
Dhruba Borthakur 已提交
4678
      return Status::OK();
4679 4680
    }

D
Dhruba Borthakur 已提交
4681 4682 4683
    // 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 已提交
4684
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
4685
      if (cfd->current()->NumLevelFiles(i) != 0) {
4686
        Log(db_options_.info_log,
4687
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
D
Dhruba Borthakur 已提交
4688 4689 4690 4691
        return Status::InvalidArgument("File not in last level");
      }
    }
    edit.DeleteFile(level, number);
4692
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
I
Igor Canadi 已提交
4693
    if (status.ok()) {
4694
      InstallSuperVersion(cfd, deletion_state);
I
Igor Canadi 已提交
4695
    }
I
Igor Canadi 已提交
4696
    FindObsoleteFiles(deletion_state, false);
D
Dhruba Borthakur 已提交
4697
  } // lock released here
4698
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
4699
  // remove files outside the db-lock
4700 4701 4702
  if (deletion_state.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(deletion_state);
  }
4703 4704 4705 4706 4707 4708
  {
    MutexLock l(&mutex_);
    // schedule flush if file deletion means we freed the space for flushes to
    // continue
    MaybeScheduleFlushOrCompaction();
  }
4709 4710 4711
  return status;
}

4712
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
4713
  MutexLock l(&mutex_);
4714
  versions_->GetLiveFilesMetaData(metadata);
4715
}
I
Igor Canadi 已提交
4716
#endif  // ROCKSDB_LITE
4717

I
Igor Canadi 已提交
4718 4719 4720 4721 4722 4723 4724
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
4727 4728 4729 4730 4731 4732
    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) {
4733
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
4734 4735 4736 4737 4738 4739 4740 4741 4742 4743 4744 4745
                             ". 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);
  }
}

4746 4747 4748 4749 4750 4751 4752 4753 4754 4755 4756 4757 4758 4759 4760 4761 4762 4763 4764 4765 4766 4767 4768 4769 4770 4771 4772
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 已提交
4773 4774
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
4775
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
4776
               const Slice& key, const Slice& value) {
4777 4778 4779 4780
  // 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);
4781
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
4782 4783 4784
  return Write(opt, &batch);
}

4785 4786
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
4787
  WriteBatch batch;
4788
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
4789 4790 4791
  return Write(opt, &batch);
}

4792 4793
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
4794
  WriteBatch batch;
4795
  batch.Merge(column_family, key, value);
4796 4797 4798
  return Write(opt, &batch);
}

4799 4800
// Default implementation -- returns not supported status
Status DB::CreateColumnFamily(const ColumnFamilyOptions& options,
4801
                              const std::string& column_family_name,
4802
                              ColumnFamilyHandle** handle) {
4803
  return Status::NotSupported("");
4804
}
4805
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4806
  return Status::NotSupported("");
4807 4808
}

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

J
Jim Paton 已提交
4811
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4812 4813 4814 4815
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4816
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4817
  std::vector<ColumnFamilyHandle*> handles;
4818
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
4819 4820 4821 4822 4823 4824 4825
  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;
4826 4827
}

4828 4829
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4830
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4831 4832 4833 4834
  Status s = SanitizeDBOptionsByCFOptions(&db_options, column_families);
  if (!s.ok()) {
    return s;
  }
4835
  if (db_options.db_paths.size() > 1) {
4836 4837 4838 4839 4840 4841 4842 4843 4844 4845 4846 4847
    for (auto& cfd : column_families) {
      if (cfd.options.compaction_style != kCompactionStyleUniversal) {
        return Status::NotSupported(
            "More than one DB paths are only supported in "
            "universal compaction style. ");
      }
    }

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

4850
  *dbptr = nullptr;
4851
  handles->clear();
J
jorlow@chromium.org 已提交
4852

I
Igor Canadi 已提交
4853 4854 4855 4856
  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);
4857
  }
4858

I
Igor Canadi 已提交
4859
  DBImpl* impl = new DBImpl(db_options, dbname);
4860
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
4861
  if (s.ok()) {
4862
    for (auto db_path : impl->db_options_.db_paths) {
4863
      s = impl->env_->CreateDirIfMissing(db_path.path);
4864 4865 4866 4867 4868 4869
      if (!s.ok()) {
        break;
      }
    }
  }

4870 4871 4872 4873 4874 4875
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4876 4877 4878 4879
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4880
  impl->mutex_.Lock();
4881 4882
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4883
  if (s.ok()) {
4884
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4885
    unique_ptr<WritableFile> lfile;
4886
    EnvOptions soptions(db_options);
4887 4888 4889
    s = impl->db_options_.env->NewWritableFile(
        LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
        impl->db_options_.env->OptimizeForLogWrite(soptions));
J
jorlow@chromium.org 已提交
4890
    if (s.ok()) {
4891
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4892
      impl->logfile_number_ = new_log_number;
4893
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
4894

4895 4896
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4897 4898
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
4899 4900 4901 4902 4903 4904 4905 4906 4907 4908 4909 4910 4911 4912 4913 4914 4915 4916 4917
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
        } else {
          if (db_options.create_missing_column_families) {
            // missing column family, create it
            ColumnFamilyHandle* handle;
            impl->mutex_.Unlock();
            s = impl->CreateColumnFamily(cf.options, cf.name, &handle);
            impl->mutex_.Lock();
            if (s.ok()) {
              handles->push_back(handle);
            } else {
              break;
            }
          } else {
            s = Status::InvalidArgument("Column family not found: ", cf.name);
            break;
          }
I
Igor Canadi 已提交
4918
        }
4919
      }
I
Igor Canadi 已提交
4920 4921
    }
    if (s.ok()) {
4922
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
4923
        delete cfd->InstallSuperVersion(new SuperVersion(), &impl->mutex_);
4924
      }
I
Igor Canadi 已提交
4925 4926
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4927
      impl->DeleteObsoleteFiles();
4928
      impl->MaybeScheduleFlushOrCompaction();
4929
      s = impl->db_directory_->Fsync();
J
jorlow@chromium.org 已提交
4930 4931
    }
  }
4932

I
Igor Canadi 已提交
4933 4934
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
4935 4936
      if (cfd->options()->compaction_style == kCompactionStyleUniversal ||
          cfd->options()->compaction_style == kCompactionStyleFIFO) {
I
Igor Canadi 已提交
4937 4938 4939 4940
        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 已提交
4941 4942 4943
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
                "open with universal or FIFO compaction style.");
I
Igor Canadi 已提交
4944 4945 4946 4947
            break;
          }
        }
      }
4948 4949 4950 4951 4952 4953
      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 已提交
4954
      if (!s.ok()) {
4955 4956 4957 4958 4959
        break;
      }
    }
  }

4960 4961
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4962
  if (s.ok()) {
4963
    impl->opened_successfully_ = true;
J
jorlow@chromium.org 已提交
4964 4965
    *dbptr = impl;
  } else {
4966 4967 4968
    for (auto h : *handles) {
      delete h;
    }
4969
    handles->clear();
J
jorlow@chromium.org 已提交
4970 4971 4972 4973 4974
    delete impl;
  }
  return s;
}

4975 4976 4977
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4978
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4979 4980
}

4981 4982 4983
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4984
Status DestroyDB(const std::string& dbname, const Options& options) {
4985
  const InternalKeyComparator comparator(options.comparator);
4986
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
4987
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4988
  std::vector<std::string> filenames;
4989 4990
  std::vector<std::string> archiveFiles;

4991
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
4992 4993
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
4994 4995 4996 4997 4998 4999 5000

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

J
jorlow@chromium.org 已提交
5002 5003 5004 5005 5006
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
5007 5008
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
5009 5010 5011
  if (result.ok()) {
    uint64_t number;
    FileType type;
5012
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
5013
    for (size_t i = 0; i < filenames.size(); i++) {
5014
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
5015
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
5016 5017 5018
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
5019 5020
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
5021 5022 5023
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
5024 5025 5026 5027 5028
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
5029

5030 5031
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
5032 5033 5034 5035 5036
      uint64_t number;
      FileType type;
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
5037
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
5038 5039 5040 5041 5042 5043 5044
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

5045
    env->GetChildren(archivedir, &archiveFiles);
5046 5047
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
5048 5049
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
5050
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
5051 5052 5053 5054 5055
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
5056
    // ignore case where no archival directory is present.
5057
    env->DeleteDir(archivedir);
5058

J
jorlow@chromium.org 已提交
5059
    env->UnlockFile(lock);  // Ignore error since state is already gone
5060
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
5061
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
5062
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
5063 5064 5065 5066
  }
  return result;
}

5067 5068
//
// A global method that can dump out the build version
K
kailiu 已提交
5069
void DumpLeveldbBuildVersion(Logger * log) {
I
Igor Canadi 已提交
5070 5071
#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
5072
  Log(log, "Git sha %s", rocksdb_build_git_sha);
5073
  Log(log, "Compile time %s %s",
5074
      rocksdb_build_compile_time, rocksdb_build_compile_date);
I
Igor Canadi 已提交
5075
#endif
5076 5077
}

5078
}  // namespace rocksdb