db_impl.cc 168.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
CompressionType GetCompressionFlush(const ImmutableCFOptions& ioptions) {
298 299 300 301 302 303
  // 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 (ioptions.compaction_style == kCompactionStyleUniversal) {
305
    can_compress =
306
        (ioptions.compaction_options_universal.compression_size_percent < 0);
307 308
  } else {
    // For leveled compress when min_level_to_compress == 0.
309 310
    can_compress = ioptions.compression_per_level.empty() ||
                   ioptions.compression_per_level[0] != kNoCompression;
311 312 313
  }

  if (can_compress) {
314
    return ioptions.compression;
315 316 317 318
  } 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),
L
Lei Jin 已提交
347
      env_options_(options),
348
      bg_work_gate_closed_(false),
349 350
      refitting_level_(false),
      opened_successfully_(false) {
H
heyongqiang 已提交
351
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
352

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

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

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

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

DBImpl::~DBImpl() {
375 376 377
  mutex_.Lock();
  if (flush_on_destroy_) {
    for (auto cfd : *versions_->GetColumnFamilySet()) {
378
      if (!cfd->mem()->IsEmpty()) {
379 380 381 382
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
383
        cfd->Unref();
384 385
      }
    }
386
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
387
  }
I
Igor Canadi 已提交
388 389

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

I
Igor Canadi 已提交
395 396
  flush_scheduler_.Clear();

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

I
Igor Canadi 已提交
404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419
  // 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;
    if (deletion_state.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(deletion_state);
420 421 422
    }
  }

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

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

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

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

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

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

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

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

  const uint64_t now_micros = env_->NowMicros();

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

504 505 506 507 508 509
    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 已提交
510
    std::string stats;
511 512 513
    {
      MutexLock l(&mutex_);
      for (auto cfd : *versions_->GetColumnFamilySet()) {
514 515
        cfd->internal_stats()->GetStringProperty(cf_property_type,
                                                 "rocksdb.cfstats", &stats);
516
      }
517 518
      default_cf_internal_stats_->GetStringProperty(db_property_type,
                                                    "rocksdb.dbstats", &stats);
519
    }
520 521
    Log(db_options_.info_log, "------- DUMPING STATS -------");
    Log(db_options_.info_log, "%s", stats.c_str());
522

523
    PrintStatistics();
524 525 526
  }
}

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

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

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

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

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

569 570 571 572 573 574 575 576
  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
577 578 579 580 581
  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 已提交
582 583
  versions_->AddLiveFiles(&deletion_state.sst_live);

584
  if (doing_the_full_scan) {
585 586
    for (uint32_t path_id = 0;
         path_id < db_options_.db_paths.size(); path_id++) {
587 588 589
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
590
      env_->GetChildren(db_options_.db_paths[path_id].path,
591
                        &files);  // Ignore errors
592
      for (std::string file : files) {
I
Igor Canadi 已提交
593 594
        // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
        deletion_state.candidate_files.emplace_back("/" + file, path_id);
595 596
      }
    }
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");
    }

S
Stanislau Hlebik 已提交
1223 1224 1225 1226 1227 1228 1229
    if (!logs.empty()) {
      // Recover in the order in which the logs were generated
      std::sort(logs.begin(), logs.end());
      s = RecoverLogFiles(logs, &max_sequence, read_only);
      if (!s.ok()) {
        // Clear memtables if recovery failed
        for (auto cfd : *versions_->GetColumnFamilySet()) {
1230 1231
          cfd->CreateNewMemtable(MemTableOptions(
              *cfd->GetLatestMutableCFOptions(), *cfd->options()));
S
Stanislau Hlebik 已提交
1232 1233
        }
      }
1234
    }
L
Lei Jin 已提交
1235
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
1236 1237
  }

I
Igor Canadi 已提交
1238 1239 1240 1241 1242
  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 已提交
1243 1244 1245
  return s;
}

S
Stanislau Hlebik 已提交
1246 1247 1248
// REQUIRES: log_numbers are sorted in ascending order
Status DBImpl::RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
                               SequenceNumber* max_sequence, bool read_only) {
J
jorlow@chromium.org 已提交
1249 1250
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
1251
    Logger* info_log;
J
jorlow@chromium.org 已提交
1252
    const char* fname;
1253 1254
    Status* status;  // nullptr if db_options_.paranoid_checks==false or
                     //            db_options_.skip_log_error_on_recovery==true
J
jorlow@chromium.org 已提交
1255
    virtual void Corruption(size_t bytes, const Status& s) {
1256
      Log(info_log, "%s%s: dropping %d bytes; %s",
1257
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
1258
          fname, static_cast<int>(bytes), s.ToString().c_str());
1259
      if (this->status != nullptr && this->status->ok()) *this->status = s;
J
jorlow@chromium.org 已提交
1260 1261 1262 1263
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
1264
  Status status;
1265
  std::unordered_map<int, VersionEdit> version_edits;
1266
  // no need to refcount because iteration is under mutex
1267 1268
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1269 1270
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1271
  }
I
Igor Canadi 已提交
1272

S
Stanislau Hlebik 已提交
1273 1274 1275 1276 1277 1278 1279 1280 1281
  for (auto log_number : log_numbers) {
    // 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.
    versions_->MarkFileNumberUsed(log_number);
    // Open the log file
    std::string fname = LogFileName(db_options_.wal_dir, log_number);
    unique_ptr<SequentialFile> file;
    status = env_->NewSequentialFile(fname, &file, env_options_);
J
jorlow@chromium.org 已提交
1282
    if (!status.ok()) {
S
Stanislau Hlebik 已提交
1283 1284 1285 1286 1287 1288 1289 1290
      MaybeIgnoreError(&status);
      if (!status.ok()) {
        return status;
      } else {
        // Fail with one log file, but that's ok.
        // Try next one.
        continue;
      }
J
jorlow@chromium.org 已提交
1291 1292
    }

S
Stanislau Hlebik 已提交
1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340
    // Create the log reader.
    LogReporter reporter;
    reporter.env = env_;
    reporter.info_log = db_options_.info_log.get();
    reporter.fname = fname.c_str();
    reporter.status =
        (db_options_.paranoid_checks && !db_options_.skip_log_error_on_recovery
             ? &status
             : nullptr);
    // 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).
    log::Reader reader(std::move(file), &reporter, true /*checksum*/,
                       0 /*initial_offset*/);
    Log(db_options_.info_log, "Recovering log #%" PRIu64 "", log_number);

    // Read all the records and add to a memtable
    std::string scratch;
    Slice record;
    WriteBatch batch;
    while (reader.ReadRecord(&record, &scratch)) {
      if (record.size() < 12) {
        reporter.Corruption(record.size(),
                            Status::Corruption("log record too small"));
        continue;
      }
      WriteBatchInternal::SetContents(&batch, record);

      // 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
      status = WriteBatchInternal::InsertInto(
          &batch, column_family_memtables_.get(), true, log_number);

      MaybeIgnoreError(&status);
      if (!status.ok()) {
        return status;
      }
      const SequenceNumber last_seq = WriteBatchInternal::Sequence(&batch) +
                                      WriteBatchInternal::Count(&batch) - 1;
      if (last_seq > *max_sequence) {
        *max_sequence = last_seq;
      }

      if (!read_only) {
I
Igor Canadi 已提交
1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357
        // we can do this because this is called before client has access to the
        // DB and there is only a single thread operating on DB
        ColumnFamilyData* cfd;

        while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
          cfd->Unref();
          // If this asserts, it means that InsertInto failed in
          // filtering updates to already-flushed column families
          assert(cfd->GetLogNumber() <= log_number);
          auto iter = version_edits.find(cfd->GetID());
          assert(iter != version_edits.end());
          VersionEdit* edit = &iter->second;
          status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
1358
          }
1359 1360
          cfd->CreateNewMemtable(MemTableOptions(
              *cfd->GetLatestMutableCFOptions(), *cfd->options()));
1361
        }
J
jorlow@chromium.org 已提交
1362 1363 1364
      }
    }

I
Igor Canadi 已提交
1365
    flush_scheduler_.Clear();
S
Stanislau Hlebik 已提交
1366 1367 1368
    if (versions_->LastSequence() < *max_sequence) {
      versions_->SetLastSequence(*max_sequence);
    }
1369 1370
  }

1371
  if (!read_only) {
1372 1373
    // no need to refcount since client still doesn't have access
    // to the DB and can not drop column families while we iterate
S
Stanislau Hlebik 已提交
1374
    auto max_log_number = log_numbers.back();
1375
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1376
      auto iter = version_edits.find(cfd->GetID());
1377 1378 1379
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
1380
      if (cfd->GetLogNumber() > max_log_number) {
1381
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
1382
        // from all logs. Memtable has to be empty because
1383
        // we filter the updates based on log_number
1384
        // (in WriteBatch::InsertInto)
1385 1386 1387 1388 1389 1390 1391
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1392
        status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
1393 1394 1395 1396
        if (!status.ok()) {
          // Recovery failed
          break;
        }
1397 1398
        cfd->CreateNewMemtable(MemTableOptions(
            *cfd->GetLatestMutableCFOptions(), *cfd->options()));
1399
      }
J
jorlow@chromium.org 已提交
1400

1401
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
1402
      // writing log_number in the manifest means that any log file
1403 1404
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
1405 1406 1407
      // Since we already recovered max_log_number, we want all logs
      // with numbers `<= max_log_number` (includes this one) to be ignored
      edit->SetLogNumber(max_log_number + 1);
1408 1409 1410
      // 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 已提交
1411
      // log number
S
Stanislau Hlebik 已提交
1412
      versions_->MarkFileNumberUsed(max_log_number + 1);
1413 1414
      status = versions_->LogAndApply(cfd, edit, &mutex_);
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1415 1416
        // Recovery failed
        break;
1417 1418
      }
    }
1419
  }
I
Igor Canadi 已提交
1420

J
jorlow@chromium.org 已提交
1421 1422 1423
  return status;
}

1424 1425
Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
                                           VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1426
  mutex_.AssertHeld();
1427
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1428
  FileMetaData meta;
1429 1430
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
  pending_outputs_[meta.fd.GetNumber()] = 0;  // path 0 for level 0 file.
1431 1432
  ReadOptions ro;
  ro.total_order_seek = true;
1433
  Arena arena;
1434 1435
  Status s;
  {
1436 1437 1438 1439
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
    const SequenceNumber newest_snapshot = snapshots_.GetNewest();
    const SequenceNumber earliest_seqno_in_memtable =
        mem->GetFirstSequenceNumber();
1440
    Log(db_options_.info_log, "[%s] Level-0 table #%" PRIu64 ": started",
1441
        cfd->GetName().c_str(), meta.fd.GetNumber());
1442

1443 1444 1445 1446 1447
    {
      mutex_.Unlock();
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
          iter.get(), &meta, cfd->internal_comparator(), newest_snapshot,
1448 1449
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
          cfd->ioptions()->compression_opts, Env::IO_HIGH);
1450
      LogFlush(db_options_.info_log);
1451 1452
      mutex_.Lock();
    }
1453

1454
    Log(db_options_.info_log,
1455 1456 1457 1458
        "[%s] Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
        s.ToString().c_str());
  }
1459
  pending_outputs_.erase(meta.fd.GetNumber());
1460 1461 1462 1463

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1464
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1465 1466 1467
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1468 1469
  }

L
Lei Jin 已提交
1470
  InternalStats::CompactionStats stats(1);
1471
  stats.micros = env_->NowMicros() - start_micros;
1472
  stats.bytes_written = meta.fd.GetFileSize();
M
Mark Callaghan 已提交
1473
  stats.files_out_levelnp1 = 1;
1474
  cfd->internal_stats()->AddCompactionStats(level, stats);
1475 1476
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1477
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1478 1479 1480
  return s;
}

1481
Status DBImpl::WriteLevel0Table(ColumnFamilyData* cfd,
1482
                                autovector<MemTable*>& mems, VersionEdit* edit,
1483
                                uint64_t* filenumber, LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
1484
  mutex_.AssertHeld();
1485 1486
  const uint64_t start_micros = env_->NowMicros();
  FileMetaData meta;
1487 1488

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

1492 1493
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
1494
    mems[0]->GetFirstSequenceNumber();
1495
  Version* base = cfd->current();
1496
  base->Ref();          // it is likely that we do not need this reference
1497 1498 1499
  Status s;
  {
    mutex_.Unlock();
H
Haobo Xu 已提交
1500
    log_buffer->FlushBufferToLog();
K
Kai Liu 已提交
1501
    std::vector<Iterator*> memtables;
1502 1503
    ReadOptions ro;
    ro.total_order_seek = true;
1504
    Arena arena;
1505
    for (MemTable* m : mems) {
1506
      Log(db_options_.info_log,
1507 1508
          "[%s] Flushing memtable with next log file: %" PRIu64 "\n",
          cfd->GetName().c_str(), m->GetNextLogNumber());
1509 1510 1511 1512 1513 1514
      memtables.push_back(m->NewIterator(ro, &arena));
    }
    {
      ScopedArenaIterator iter(NewMergingIterator(&cfd->internal_comparator(),
                                                  &memtables[0],
                                                  memtables.size(), &arena));
1515 1516
      Log(db_options_.info_log,
           "[%s] Level-0 flush table #%" PRIu64 ": started",
1517 1518 1519 1520 1521
          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,
1522 1523
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
          cfd->ioptions()->compression_opts, Env::IO_HIGH);
1524
      LogFlush(db_options_.info_log);
1525
    }
1526
    Log(db_options_.info_log,
1527
        "[%s] Level-0 flush table #%" PRIu64 ": %" PRIu64 " bytes %s",
F
Feng Zhu 已提交
1528
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
1529
        s.ToString().c_str());
I
Igor Canadi 已提交
1530

1531
    if (!db_options_.disableDataSync) {
1532 1533
      db_directory_->Fsync();
    }
1534 1535
    mutex_.Lock();
  }
1536 1537
  base->Unref();

1538
  // re-acquire the most current version
1539
  base = cfd->current();
1540 1541 1542 1543 1544 1545

  // 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 已提交
1546
  // committed so that other threads can recognize this file as a
1547 1548 1549 1550 1551 1552
  // 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;
1553
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1554 1555 1556 1557 1558 1559
    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.
1560
    if (base != nullptr && db_options_.max_background_compactions <= 1 &&
I
Igor Canadi 已提交
1561
        db_options_.max_background_flushes == 0 &&
1562
        cfd->ioptions()->compaction_style == kCompactionStyleLevel) {
1563 1564
      level = base->PickLevelForMemTableOutput(min_user_key, max_user_key);
    }
1565 1566 1567
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1568 1569
  }

L
Lei Jin 已提交
1570
  InternalStats::CompactionStats stats(1);
1571
  stats.micros = env_->NowMicros() - start_micros;
1572
  stats.bytes_written = meta.fd.GetFileSize();
1573
  cfd->internal_stats()->AddCompactionStats(level, stats);
1574 1575
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1576
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
1577 1578 1579
  return s;
}

1580 1581
Status DBImpl::FlushMemTableToOutputFile(ColumnFamilyData* cfd,
                                         bool* madeProgress,
H
Haobo Xu 已提交
1582 1583
                                         DeletionState& deletion_state,
                                         LogBuffer* log_buffer) {
1584
  mutex_.AssertHeld();
1585
  assert(cfd->imm()->size() != 0);
1586
  assert(cfd->imm()->IsFlushPending());
1587 1588 1589

  // Save the contents of the earliest memtable as a new Table
  uint64_t file_number;
K
Kai Liu 已提交
1590
  autovector<MemTable*> mems;
1591
  cfd->imm()->PickMemtablesToFlush(&mems);
1592
  if (mems.empty()) {
I
Igor Canadi 已提交
1593 1594
    LogToBuffer(log_buffer, "[%s] Nothing in memtable to flush",
                cfd->GetName().c_str());
L
Lei Jin 已提交
1595
    return Status::OK();
1596 1597 1598
  }

  // record the logfile_number_ before we release the mutex
1599 1600 1601
  // 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.
1602
  MemTable* m = mems[0];
1603 1604
  VersionEdit* edit = m->GetEdits();
  edit->SetPrevLogNumber(0);
1605 1606
  // SetLogNumber(log_num) indicates logs with number smaller than log_num
  // will no longer be picked up for recovery.
1607 1608
  edit->SetLogNumber(mems.back()->GetNextLogNumber());
  edit->SetColumnFamily(cfd->GetID());
1609

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

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

L
Lei Jin 已提交
1618
  if (!s.ok()) {
1619
    cfd->imm()->RollbackMemtableFlush(mems, file_number, &pending_outputs_);
1620 1621
  } else {
    // Replace immutable memtable with the generated Table
1622
    s = cfd->imm()->InstallMemtableFlushResults(
1623
        cfd, mems, versions_.get(), &mutex_, db_options_.info_log.get(),
1624
        file_number, &pending_outputs_, &deletion_state.memtables_to_free,
1625
        db_directory_.get(), log_buffer);
1626
  }
J
jorlow@chromium.org 已提交
1627 1628

  if (s.ok()) {
1629
    // Use latest MutableCFOptions
1630
    InstallSuperVersion(cfd, deletion_state);
1631 1632 1633
    if (madeProgress) {
      *madeProgress = 1;
    }
I
Igor Canadi 已提交
1634 1635 1636
    Version::LevelSummaryStorage tmp;
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
                cfd->current()->LevelSummary(&tmp));
1637

1638
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1639
      // add to deletion state
1640
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1641 1642 1643 1644
             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;
1645 1646
        alive_log_files_.pop_front();
      }
1647
    }
J
jorlow@chromium.org 已提交
1648
  }
1649

1650
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1651 1652 1653 1654 1655
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1656
  RecordFlushIOStats();
J
jorlow@chromium.org 已提交
1657 1658 1659
  return s;
}

1660
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1661
                            const Slice* begin, const Slice* end,
1662 1663
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
1664
  if (target_path_id >= db_options_.db_paths.size()) {
1665 1666 1667
    return Status::InvalidArgument("Invalid target path ID");
  }

1668 1669
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1670 1671

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1672
  if (!s.ok()) {
1673
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1674 1675 1676
    return s;
  }

I
Igor Canadi 已提交
1677
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1678 1679
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1680 1681
    Version* base = cfd->current();
    for (int level = 1; level < cfd->NumberLevels(); level++) {
G
Gabor Cselle 已提交
1682 1683 1684 1685 1686
      if (base->OverlapInLevel(level, begin, end)) {
        max_level_with_files = level;
      }
    }
  }
1687 1688
  for (int level = 0; level <= max_level_with_files; level++) {
    // in case the compaction is unversal or if we're compacting the
1689 1690 1691
    // 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)
1692 1693
    if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
        cfd->ioptions()->compaction_style == kCompactionStyleFIFO ||
1694
        (level == max_level_with_files && level > 0)) {
1695
      s = RunManualCompaction(cfd, level, level, target_path_id, begin, end);
1696
    } else {
1697 1698
      s = RunManualCompaction(cfd, level, level + 1, target_path_id, begin,
                              end);
L
Lei Jin 已提交
1699 1700
    }
    if (!s.ok()) {
1701
      LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1702
      return s;
1703
    }
G
Gabor Cselle 已提交
1704
  }
1705 1706

  if (reduce_level) {
I
Igor Canadi 已提交
1707
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1708
  }
1709
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1710

1711 1712 1713 1714 1715 1716 1717
  {
    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 已提交
1718
  return s;
1719 1720
}

1721 1722 1723 1724 1725 1726 1727
bool DBImpl::SetOptions(ColumnFamilyHandle* column_family,
    const std::unordered_map<std::string, std::string>& options_map) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  MutexLock l(&mutex_);
  return cfh->cfd()->SetOptions(options_map);
}

1728
// return the same level if it cannot be moved
I
Igor Canadi 已提交
1729
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd, int level) {
1730
  mutex_.AssertHeld();
I
Igor Canadi 已提交
1731
  Version* current = cfd->current();
1732
  int minimum_level = level;
1733
  for (int i = level - 1; i > 0; --i) {
1734
    // stop if level i is not empty
1735
    if (current->NumLevelFiles(i) > 0) break;
1736
    // stop if level i is too small (cannot fit the level files)
I
Igor Canadi 已提交
1737
    if (cfd->compaction_picker()->MaxBytesForLevel(i) <
1738 1739 1740
        current->NumLevelBytes(level)) {
      break;
    }
1741 1742 1743 1744 1745 1746

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1750
  SuperVersion* superversion_to_free = nullptr;
1751
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1752 1753

  mutex_.Lock();
1754 1755 1756

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1757
    mutex_.Unlock();
1758
    Log(db_options_.info_log, "ReFitLevel: another thread is refitting");
I
Igor Canadi 已提交
1759
    delete new_superversion;
L
Lei Jin 已提交
1760
    return Status::NotSupported("another thread is refitting");
1761 1762 1763 1764 1765
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1766
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1767
    Log(db_options_.info_log,
1768 1769
        "RefitLevel: waiting for background threads to stop: %d %d",
        bg_compaction_scheduled_, bg_flush_scheduled_);
1770 1771 1772 1773
    bg_cv_.Wait();
  }

  // move to a smaller level
1774 1775
  int to_level = target_level;
  if (target_level < 0) {
I
Igor Canadi 已提交
1776
    to_level = FindMinimumEmptyLevelFitting(cfd, level);
1777
  }
1778 1779 1780

  assert(to_level <= level);

L
Lei Jin 已提交
1781
  Status status;
1782
  if (to_level < level) {
1783 1784
    Log(db_options_.info_log, "[%s] Before refitting:\n%s",
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1785

1786
    VersionEdit edit;
I
Igor Canadi 已提交
1787 1788
    edit.SetColumnFamily(cfd->GetID());
    for (const auto& f : cfd->current()->files_[level]) {
1789
      edit.DeleteFile(level, f->fd.GetNumber());
1790 1791 1792
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1793
    }
1794
    Log(db_options_.info_log, "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1795
        cfd->GetName().c_str(), edit.DebugString().data());
1796

I
Igor Canadi 已提交
1797
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
1798
    // Use latest MutableCFOptions
1799
    superversion_to_free = cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
1800
    new_superversion = nullptr;
1801

1802
    Log(db_options_.info_log, "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1803
        status.ToString().data());
1804 1805

    if (status.ok()) {
1806
      Log(db_options_.info_log, "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1807
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1808 1809 1810 1811 1812
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1813 1814 1815 1816

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1817
  return status;
G
Gabor Cselle 已提交
1818 1819
}

1820 1821 1822
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1823 1824
}

1825 1826 1827
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->max_mem_compaction_level;
1828 1829
}

1830 1831 1832
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->options()->level0_stop_writes_trigger;
1833 1834
}

1835
Status DBImpl::Flush(const FlushOptions& options,
1836 1837 1838
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return FlushMemTable(cfh->cfd(), options);
H
heyongqiang 已提交
1839 1840
}

1841
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1842 1843 1844
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1845
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1846 1847
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1848
  assert(input_level >= 0);
1849

G
Gabor Cselle 已提交
1850 1851
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1852
  ManualCompaction manual;
I
Igor Canadi 已提交
1853
  manual.cfd = cfd;
1854 1855
  manual.input_level = input_level;
  manual.output_level = output_level;
1856
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
1857
  manual.done = false;
1858
  manual.in_progress = false;
1859 1860 1861
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
1862 1863
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1864
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1865 1866 1867 1868
  } else {
    begin_storage = InternalKey(*begin, kMaxSequenceNumber, kValueTypeForSeek);
    manual.begin = &begin_storage;
  }
1869
  if (end == nullptr ||
1870 1871
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1872
    manual.end = nullptr;
G
Gabor Cselle 已提交
1873 1874 1875 1876 1877 1878
  } else {
    end_storage = InternalKey(*end, 0, static_cast<ValueType>(0));
    manual.end = &end_storage;
  }

  MutexLock l(&mutex_);
1879

1880 1881 1882 1883 1884 1885
  // 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
1886
  // RunManualCompaction(), i.e. during that time no other compaction will
1887 1888 1889
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1890
  // RunManualCompaction() from getting to the second while loop below.
1891 1892 1893 1894 1895
  // 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) {
1896
    Log(db_options_.info_log,
I
Igor Canadi 已提交
1897 1898 1899
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1900 1901
    bg_cv_.Wait();
  }
1902

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

1906 1907 1908 1909
  // We don't check bg_error_ here, because if we get the error in compaction,
  // the compaction will set manual.status to bg_error_ and set manual.done to
  // true.
  while (!manual.done) {
1910 1911 1912
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
1913
      bg_cv_.Wait();
1914 1915
    } else {
      manual_compaction_ = &manual;
1916 1917
      bg_compaction_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
G
Gabor Cselle 已提交
1918
    }
H
hans@chromium.org 已提交
1919
  }
1920

1921 1922 1923
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1924
  return manual.status;
J
jorlow@chromium.org 已提交
1925 1926
}

1927 1928
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
                             const FlushOptions& options) {
S
Stanislau Hlebik 已提交
1929 1930 1931 1932
  Status s;
  {
    WriteContext context;
    MutexLock guard_lock(&mutex_);
1933 1934 1935 1936 1937 1938

    if (cfd->imm()->size() == 0 && cfd->mem()->IsEmpty()) {
      // Nothing to flush
      return Status::OK();
    }

I
Igor Canadi 已提交
1939 1940
    WriteThread::Writer w(&mutex_);
    s = write_thread_.EnterWriteThread(&w, 0);
S
Stanislau Hlebik 已提交
1941 1942 1943 1944 1945 1946 1947 1948
    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();

I
Igor Canadi 已提交
1949
    write_thread_.ExitWriteThread(&w, &w, s);
S
Stanislau Hlebik 已提交
1950
  }
S
Stanislau Hlebik 已提交
1951

H
heyongqiang 已提交
1952
  if (s.ok() && options.wait) {
1953
    // Wait until the compaction completes
1954
    s = WaitForFlushMemTable(cfd);
1955 1956
  }
  return s;
J
jorlow@chromium.org 已提交
1957 1958
}

1959
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1960 1961 1962
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1963
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1964 1965
    bg_cv_.Wait();
  }
1966
  if (!bg_error_.ok()) {
1967 1968 1969
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1970 1971
}

1972
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1973
  mutex_.AssertHeld();
1974
  bg_schedule_needed_ = false;
1975 1976
  if (bg_work_gate_closed_) {
    // gate closed for backgrond work
J
jorlow@chromium.org 已提交
1977 1978 1979
  } else if (shutting_down_.Acquire_Load()) {
    // DB is being deleted; no more background compactions
  } else {
1980
    bool is_flush_pending = false;
1981
    // no need to refcount since we're under a mutex
1982
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1983
      if (cfd->imm()->IsFlushPending()) {
1984 1985 1986
        is_flush_pending = true;
      }
    }
1987
    if (is_flush_pending) {
1988
      // memtable flush needed
1989
      if (bg_flush_scheduled_ < db_options_.max_background_flushes) {
1990 1991
        bg_flush_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH);
1992
      } else if (db_options_.max_background_flushes > 0) {
1993
        bg_schedule_needed_ = true;
1994
      }
1995
    }
1996
    bool is_compaction_needed = false;
1997
    // no need to refcount since we're under a mutex
1998
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1999
      if (cfd->current()->NeedsCompaction()) {
2000 2001 2002 2003
        is_compaction_needed = true;
        break;
      }
    }
2004

2005 2006
    // Schedule BGWorkCompaction if there's a compaction pending (or a memtable
    // flush, but the HIGH pool is not enabled)
2007 2008 2009 2010
    // Do it only if max_background_compactions hasn't been reached and
    // bg_manual_only_ == 0
    if (!bg_manual_only_ &&
        (is_compaction_needed ||
2011 2012
         (is_flush_pending && db_options_.max_background_flushes == 0))) {
      if (bg_compaction_scheduled_ < db_options_.max_background_compactions) {
2013 2014 2015 2016 2017
        bg_compaction_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
      } else {
        bg_schedule_needed_ = true;
      }
2018 2019 2020 2021
    }
  }
}

2022
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
2023
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
2024 2025 2026 2027
  IOSTATS_RESET(bytes_written);
}

void DBImpl::RecordCompactionIOStats() {
L
Lei Jin 已提交
2028
  RecordTick(stats_, COMPACT_READ_BYTES, IOSTATS(bytes_read));
2029
  IOSTATS_RESET(bytes_read);
L
Lei Jin 已提交
2030
  RecordTick(stats_, COMPACT_WRITE_BYTES, IOSTATS(bytes_written));
2031 2032 2033
  IOSTATS_RESET(bytes_written);
}

2034
void DBImpl::BGWorkFlush(void* db) {
2035
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
2036 2037 2038 2039
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
2040
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
2041 2042 2043
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
2044
Status DBImpl::BackgroundFlush(bool* madeProgress,
H
Haobo Xu 已提交
2045 2046
                               DeletionState& deletion_state,
                               LogBuffer* log_buffer) {
2047
  mutex_.AssertHeld();
2048 2049 2050 2051 2052

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

2053 2054 2055 2056 2057
  // 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
2058
  for (auto cfd : *versions_->GetColumnFamilySet()) {
2059 2060 2061
    cfd->Ref();
    Status flush_status;
    while (flush_status.ok() && cfd->imm()->IsFlushPending()) {
2062 2063
      LogToBuffer(
          log_buffer,
2064
          "BackgroundCallFlush doing FlushMemTableToOutputFile with column "
I
Igor Canadi 已提交
2065 2066
          "family [%s], flush slots available %d",
          cfd->GetName().c_str(),
2067
          db_options_.max_background_flushes - bg_flush_scheduled_);
2068 2069
      flush_status = FlushMemTableToOutputFile(cfd, madeProgress,
                                               deletion_state, log_buffer);
2070 2071 2072
    }
    if (call_status.ok() && !flush_status.ok()) {
      call_status = flush_status;
2073
    }
2074
    cfd->Unref();
J
jorlow@chromium.org 已提交
2075
  }
2076
  versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
2077
  return call_status;
J
jorlow@chromium.org 已提交
2078 2079
}

2080
void DBImpl::BackgroundCallFlush() {
2081
  bool madeProgress = false;
K
Kai Liu 已提交
2082
  DeletionState deletion_state(true);
2083 2084
  assert(bg_flush_scheduled_);

2085
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2086 2087 2088 2089 2090 2091 2092 2093 2094 2095 2096
  {
    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.
2097 2098
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
H
Haobo Xu 已提交
2099 2100
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
2101
        Log(db_options_.info_log,
2102 2103 2104
            "Waiting after background flush error: %s"
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
H
Haobo Xu 已提交
2105
        log_buffer.FlushBufferToLog();
2106
        LogFlush(db_options_.info_log);
H
Haobo Xu 已提交
2107 2108 2109 2110 2111 2112 2113 2114 2115
        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
2116
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2117
      mutex_.Unlock();
2118 2119 2120 2121 2122
      // 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 已提交
2123
      log_buffer.FlushBufferToLog();
2124 2125 2126
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
2127 2128
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2129

H
Haobo Xu 已提交
2130
    bg_flush_scheduled_--;
2131 2132 2133 2134 2135
    // 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 已提交
2136 2137
      MaybeScheduleFlushOrCompaction();
    }
I
Igor Canadi 已提交
2138
    RecordFlushIOStats();
H
Haobo Xu 已提交
2139
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2140 2141 2142 2143
    // 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.
2144
  }
J
jorlow@chromium.org 已提交
2145 2146
}

2147
void DBImpl::BackgroundCallCompaction() {
2148
  bool madeProgress = false;
K
Kai Liu 已提交
2149
  DeletionState deletion_state(true);
H
Haobo Xu 已提交
2150 2151

  MaybeDumpStats();
2152
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2153 2154 2155 2156 2157 2158 2159 2160 2161 2162 2163
  {
    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.
2164 2165
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2166 2167
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
H
Haobo Xu 已提交
2168
        log_buffer.FlushBufferToLog();
2169
        Log(db_options_.info_log,
2170 2171 2172
            "Waiting after background compaction error: %s, "
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
2173
        LogFlush(db_options_.info_log);
2174 2175 2176 2177
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }
H
Haobo Xu 已提交
2178

2179 2180 2181 2182 2183 2184 2185
    // 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
2186
    if (deletion_state.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2187
      mutex_.Unlock();
2188 2189 2190 2191 2192
      // 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 已提交
2193
      log_buffer.FlushBufferToLog();
2194 2195 2196
      if (deletion_state.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(deletion_state);
      }
2197 2198
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2199

2200
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2201

2202 2203
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2204 2205 2206
    // Previous compaction may have produced too many files in a level,
    // So reschedule another compaction if we made progress in the
    // last compaction.
2207 2208 2209 2210 2211 2212
    //
    // 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_) {
2213 2214
      MaybeScheduleFlushOrCompaction();
    }
2215 2216
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
I
Igor Canadi 已提交
2217
      // * madeProgress -- need to wakeup DelayWrite
2218 2219 2220 2221 2222 2223
      // * 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 已提交
2224 2225 2226 2227
    // 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.
2228
  }
J
jorlow@chromium.org 已提交
2229 2230
}

A
Abhishek Kona 已提交
2231
Status DBImpl::BackgroundCompaction(bool* madeProgress,
2232 2233
                                    DeletionState& deletion_state,
                                    LogBuffer* log_buffer) {
2234
  *madeProgress = false;
J
jorlow@chromium.org 已提交
2235
  mutex_.AssertHeld();
2236

2237 2238
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2239

2240 2241 2242 2243 2244 2245 2246 2247 2248 2249
  if (!bg_error_.ok()) {
    if (is_manual) {
      manual_compaction_->status = bg_error_;
      manual_compaction_->done = true;
      manual_compaction_->in_progress = false;
      manual_compaction_ = nullptr;
    }
    return bg_error_;
  }

2250 2251 2252
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2253 2254 2255 2256
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2257 2258 2259 2260 2261 2262 2263 2264 2265 2266
  }

  // 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",
2267
          db_options_.max_background_compactions - bg_compaction_scheduled_);
2268 2269 2270
      cfd->Ref();
      flush_stat = FlushMemTableToOutputFile(cfd, madeProgress, deletion_state,
                                             log_buffer);
2271
      cfd->Unref();
2272 2273 2274 2275 2276 2277 2278
      if (!flush_stat.ok()) {
        if (is_manual) {
          manual_compaction_->status = flush_stat;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
2279
        return flush_stat;
2280 2281 2282 2283 2284
      }
    }
  }

  unique_ptr<Compaction> c;
2285 2286
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2287
  if (is_manual) {
G
Gabor Cselle 已提交
2288
    ManualCompaction* m = manual_compaction_;
2289
    assert(m->in_progress);
2290 2291 2292
    c.reset(m->cfd->CompactRange(m->input_level, m->output_level,
                                 m->output_path_id, m->begin, m->end,
                                 &manual_end));
2293
    if (!c) {
2294
      m->done = true;
G
Gabor Cselle 已提交
2295
    }
I
Igor Canadi 已提交
2296 2297 2298 2299 2300 2301 2302 2303 2304
    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 已提交
2305
  } else {
2306
    // no need to refcount in iteration since it's always under a mutex
I
Igor Canadi 已提交
2307
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
2308
      if (!cfd->options()->disable_auto_compactions) {
2309
        c.reset(cfd->PickCompaction(log_buffer));
I
Igor Canadi 已提交
2310 2311
        if (c != nullptr) {
          // update statistics
L
Lei Jin 已提交
2312
          MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
I
Igor Canadi 已提交
2313 2314 2315
                      c->inputs(0)->size());
          break;
        }
I
Igor Canadi 已提交
2316 2317
      }
    }
J
jorlow@chromium.org 已提交
2318 2319 2320
  }

  Status status;
2321
  if (!c) {
H
hans@chromium.org 已提交
2322
    // Nothing to do
2323
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2324 2325 2326 2327 2328
  } 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);
2329
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2330 2331
           kCompactionStyleFIFO);
    for (const auto& f : *c->inputs(0)) {
2332
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2333 2334 2335
    }
    status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_,
                                    db_directory_.get());
2336
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2337 2338 2339 2340 2341 2342
    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 已提交
2343
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
2344
    // Move file to next level
2345
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2346
    FileMetaData* f = c->input(0, 0);
2347
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
2348 2349 2350
    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 已提交
2351
    status = versions_->LogAndApply(c->column_family_data(), c->edit(), &mutex_,
2352
                                    db_directory_.get());
2353
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2354
    InstallSuperVersion(c->column_family_data(), deletion_state);
2355

2356
    Version::LevelSummaryStorage tmp;
2357
    LogToBuffer(
2358 2359
        log_buffer,
        "[%s] Moved #%" PRIu64 " to level-%d %" PRIu64 " bytes %s: %s\n",
2360
        c->column_family_data()->GetName().c_str(),
2361 2362
        f->fd.GetNumber(), c->level() + 1,
        f->fd.GetFileSize(),
2363
        status.ToString().c_str(), c->input_version()->LevelSummary(&tmp));
I
Igor Canadi 已提交
2364
    c->ReleaseCompactionFiles(status);
2365
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2366
  } else {
2367
    MaybeScheduleFlushOrCompaction(); // do more compaction work in parallel.
2368
    CompactionState* compact = new CompactionState(c.get());
H
Haobo Xu 已提交
2369
    status = DoCompactionWork(compact, deletion_state, log_buffer);
2370
    CleanupCompaction(compact, status);
I
Igor Canadi 已提交
2371
    c->ReleaseCompactionFiles(status);
2372
    c->ReleaseInputs();
2373
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2374
  }
2375
  c.reset();
J
jorlow@chromium.org 已提交
2376 2377 2378

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2379
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2380 2381
    // Ignore compaction errors found during shutting down
  } else {
2382
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2383
        status.ToString().c_str());
2384
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2385 2386 2387
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2388 2389

  if (is_manual) {
G
Gabor Cselle 已提交
2390
    ManualCompaction* m = manual_compaction_;
2391
    if (!status.ok()) {
L
Lei Jin 已提交
2392
      m->status = status;
2393 2394
      m->done = true;
    }
2395 2396 2397 2398 2399 2400 2401 2402 2403
    // 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.
2404 2405 2406 2407 2408
    //
    // 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) {
2409 2410
      m->done = true;
    }
G
Gabor Cselle 已提交
2411 2412 2413
    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 已提交
2414
      // Universal and FIFO compactions should always compact the whole range
2415 2416
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleUniversal);
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2417
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2418 2419
      m->begin = &m->tmp_storage;
    }
2420
    m->in_progress = false; // not being processed anymore
2421
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2422
  }
2423
  return status;
J
jorlow@chromium.org 已提交
2424 2425
}

2426
void DBImpl::CleanupCompaction(CompactionState* compact, Status status) {
J
jorlow@chromium.org 已提交
2427
  mutex_.AssertHeld();
2428
  if (compact->builder != nullptr) {
J
jorlow@chromium.org 已提交
2429 2430
    // May happen if we get a shutdown call in the middle of compaction
    compact->builder->Abandon();
2431
    compact->builder.reset();
J
jorlow@chromium.org 已提交
2432
  } else {
2433
    assert(compact->outfile == nullptr);
J
jorlow@chromium.org 已提交
2434
  }
D
dgrogan@chromium.org 已提交
2435
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2436 2437
    const CompactionState::Output& out = compact->outputs[i];
    pending_outputs_.erase(out.number);
2438 2439 2440 2441

    // 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 已提交
2442
      TableCache::Evict(table_cache_.get(), out.number);
2443
    }
J
jorlow@chromium.org 已提交
2444 2445 2446 2447
  }
  delete compact;
}

2448
// Allocate the file numbers for the output file. We allocate as
2449
// many output file numbers as there are files in level+1 (at least one)
2450 2451 2452
// Insert them into pending_outputs so that they do not get deleted.
void DBImpl::AllocateCompactionOutputFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2453 2454
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
2455
  int filesNeeded = compact->compaction->num_input_files(1);
2456
  for (int i = 0; i < std::max(filesNeeded, 1); i++) {
2457
    uint64_t file_number = versions_->NewFileNumber();
2458
    pending_outputs_[file_number] = compact->compaction->GetOutputPathId();
2459 2460 2461 2462 2463 2464 2465
    compact->allocated_file_numbers.push_back(file_number);
  }
}

// Frees up unused file number.
void DBImpl::ReleaseCompactionUnusedFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2466
  for (const auto file_number : compact->allocated_file_numbers) {
2467 2468 2469 2470
    pending_outputs_.erase(file_number);
  }
}

J
jorlow@chromium.org 已提交
2471
Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
2472 2473
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
J
jorlow@chromium.org 已提交
2474
  uint64_t file_number;
2475 2476 2477 2478 2479 2480 2481
  // 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 已提交
2482 2483
    mutex_.Lock();
    file_number = versions_->NewFileNumber();
2484
    pending_outputs_[file_number] = compact->compaction->GetOutputPathId();
J
jorlow@chromium.org 已提交
2485 2486
    mutex_.Unlock();
  }
2487 2488
  CompactionState::Output out;
  out.number = file_number;
2489
  out.path_id = compact->compaction->GetOutputPathId();
2490 2491
  out.smallest.Clear();
  out.largest.Clear();
2492
  out.smallest_seqno = out.largest_seqno = 0;
2493
  compact->outputs.push_back(out);
J
jorlow@chromium.org 已提交
2494 2495

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

J
jorlow@chromium.org 已提交
2500
  if (s.ok()) {
L
Lei Jin 已提交
2501
    compact->outfile->SetIOPriority(Env::IO_LOW);
2502
    compact->outfile->SetPreallocationBlockSize(
2503
        compact->compaction->OutputFilePreallocationSize());
2504

2505
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
2506
    compact->builder.reset(NewTableBuilder(
L
Lei Jin 已提交
2507 2508
        *cfd->ioptions(), cfd->internal_comparator(), compact->outfile.get(),
        compact->compaction->OutputCompressionType(),
2509
        cfd->ioptions()->compression_opts));
J
jorlow@chromium.org 已提交
2510
  }
2511
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
2512 2513 2514 2515 2516
  return s;
}

Status DBImpl::FinishCompactionOutputFile(CompactionState* compact,
                                          Iterator* input) {
2517
  assert(compact != nullptr);
2518
  assert(compact->outfile);
2519
  assert(compact->builder != nullptr);
J
jorlow@chromium.org 已提交
2520 2521

  const uint64_t output_number = compact->current_output()->number;
2522
  const uint32_t output_path_id = compact->current_output()->path_id;
J
jorlow@chromium.org 已提交
2523 2524 2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535
  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;
2536
  compact->builder.reset();
J
jorlow@chromium.org 已提交
2537 2538

  // Finish and check for file errors
2539 2540
  if (s.ok() && !db_options_.disableDataSync) {
    if (db_options_.use_fsync) {
L
Lei Jin 已提交
2541
      StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
2542 2543
      s = compact->outfile->Fsync();
    } else {
L
Lei Jin 已提交
2544
      StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
2545 2546
      s = compact->outfile->Sync();
    }
J
jorlow@chromium.org 已提交
2547 2548 2549 2550
  }
  if (s.ok()) {
    s = compact->outfile->Close();
  }
2551
  compact->outfile.reset();
J
jorlow@chromium.org 已提交
2552 2553 2554

  if (s.ok() && current_entries > 0) {
    // Verify that the table is usable
I
Igor Canadi 已提交
2555
    ColumnFamilyData* cfd = compact->compaction->column_family_data();
2556
    FileDescriptor fd(output_number, output_path_id, current_bytes);
I
Igor Canadi 已提交
2557
    Iterator* iter = cfd->table_cache()->NewIterator(
L
Lei Jin 已提交
2558
        ReadOptions(), env_options_, cfd->internal_comparator(), fd);
J
jorlow@chromium.org 已提交
2559 2560 2561
    s = iter->status();
    delete iter;
    if (s.ok()) {
2562
      Log(db_options_.info_log, "[%s] Generated table #%" PRIu64 ": %" PRIu64
2563 2564 2565
                             " keys, %" PRIu64 " bytes",
          cfd->GetName().c_str(), output_number, current_entries,
          current_bytes);
J
jorlow@chromium.org 已提交
2566 2567 2568 2569 2570 2571
    }
  }
  return s;
}


2572 2573
Status DBImpl::InstallCompactionResults(CompactionState* compact,
                                        LogBuffer* log_buffer) {
J
jorlow@chromium.org 已提交
2574
  mutex_.AssertHeld();
2575 2576 2577 2578 2579

  // 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.
2580
  if (!versions_->VerifyCompactionFileConsistency(compact->compaction)) {
2581
    Log(db_options_.info_log, "[%s] Compaction %d@%d + %d@%d files aborted",
I
Igor Canadi 已提交
2582 2583 2584 2585
        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 已提交
2586
    return Status::Corruption("Compaction input files inconsistent");
2587 2588
  }

I
Igor Canadi 已提交
2589 2590
  LogToBuffer(log_buffer, "[%s] Compacted %d@%d + %d@%d files => %lld bytes",
              compact->compaction->column_family_data()->GetName().c_str(),
2591 2592 2593 2594 2595
              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 已提交
2596 2597 2598

  // Add compaction outputs
  compact->compaction->AddInputDeletions(compact->compaction->edit());
D
dgrogan@chromium.org 已提交
2599
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2600
    const CompactionState::Output& out = compact->outputs[i];
2601 2602 2603 2604
    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 已提交
2605
  }
I
Igor Canadi 已提交
2606 2607 2608
  return versions_->LogAndApply(compact->compaction->column_family_data(),
                                compact->compaction->edit(), &mutex_,
                                db_directory_.get());
J
jorlow@chromium.org 已提交
2609 2610
}

2611 2612 2613 2614 2615 2616 2617
// 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(
2618 2619
  SequenceNumber in, std::vector<SequenceNumber>& snapshots,
  SequenceNumber* prev_snapshot) {
2620
  SequenceNumber prev __attribute__((unused)) = 0;
2621 2622 2623
  for (const auto cur : snapshots) {
    assert(prev <= cur);
    if (cur >= in) {
2624
      *prev_snapshot = prev;
2625
      return cur;
2626
    }
2627 2628
    prev = cur; // assignment
    assert(prev);
2629
  }
2630
  Log(db_options_.info_log,
2631 2632
      "Looking for seqid %" PRIu64 " but maxseqid is %" PRIu64 "", in,
      snapshots[snapshots.size() - 1]);
2633 2634 2635 2636
  assert(0);
  return 0;
}

I
Igor Canadi 已提交
2637 2638 2639
uint64_t DBImpl::CallFlushDuringCompaction(ColumnFamilyData* cfd,
                                           DeletionState& deletion_state,
                                           LogBuffer* log_buffer) {
2640
  if (db_options_.max_background_flushes > 0) {
2641 2642 2643
    // flush thread will take care of this
    return 0;
  }
I
Igor Canadi 已提交
2644 2645 2646 2647 2648 2649 2650
  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();
I
Igor Canadi 已提交
2651
      bg_cv_.SignalAll();  // Wakeup DelayWrite() if necessary
I
Igor Canadi 已提交
2652 2653 2654 2655 2656 2657 2658 2659
    }
    mutex_.Unlock();
    log_buffer->FlushBufferToLog();
    return env_->NowMicros() - imm_start;
  }
  return 0;
}

D
Danny Guo 已提交
2660
Status DBImpl::ProcessKeyValueCompaction(
2661
    bool is_snapshot_supported,
D
Danny Guo 已提交
2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672
    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 已提交
2673
  Status status;
D
Danny Guo 已提交
2674
  std::string compaction_filter_value;
J
jorlow@chromium.org 已提交
2675
  ParsedInternalKey ikey;
2676
  IterKey current_user_key;
J
jorlow@chromium.org 已提交
2677
  bool has_current_user_key = false;
2678
  IterKey delete_key;
2679 2680
  SequenceNumber last_sequence_for_key __attribute__((unused)) =
    kMaxSequenceNumber;
2681
  SequenceNumber visible_in_snapshot = kMaxSequenceNumber;
2682
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
I
Igor Canadi 已提交
2683
  MergeHelper merge(
2684
      cfd->user_comparator(), cfd->ioptions()->merge_operator,
2685
      db_options_.info_log.get(), cfd->options()->min_partial_merge_operands,
2686
      false /* internal key corruption is expected */);
2687
  auto compaction_filter = cfd->ioptions()->compaction_filter;
2688 2689
  std::unique_ptr<CompactionFilter> compaction_filter_from_factory = nullptr;
  if (!compaction_filter) {
2690
    auto context = compact->GetFilterContextV1();
2691
    compaction_filter_from_factory =
2692
        cfd->ioptions()->compaction_filter_factory->CreateCompactionFilter(
I
Igor Canadi 已提交
2693
            context);
2694 2695
    compaction_filter = compaction_filter_from_factory.get();
  }
2696

2697 2698 2699 2700
  int64_t key_drop_user = 0;
  int64_t key_drop_newer_entry = 0;
  int64_t key_drop_obsolete = 0;
  int64_t loop_cnt = 0;
2701 2702
  while (input->Valid() && !shutting_down_.Acquire_Load() &&
         !cfd->IsDropped()) {
2703 2704 2705 2706 2707 2708 2709 2710 2711 2712 2713 2714 2715 2716 2717 2718 2719
    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;
    }
2720 2721 2722 2723
    // 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 已提交
2724
    imm_micros += CallFlushDuringCompaction(cfd, deletion_state, log_buffer);
2725

D
Danny Guo 已提交
2726 2727 2728 2729 2730 2731 2732 2733 2734 2735 2736 2737 2738 2739 2740 2741 2742 2743 2744 2745 2746
    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 已提交
2747

2748
    if (compact->compaction->ShouldStopBefore(key) &&
2749
        compact->builder != nullptr) {
D
Danny Guo 已提交
2750
      status = FinishCompactionOutputFile(compact, input);
2751 2752 2753 2754 2755 2756
      if (!status.ok()) {
        break;
      }
    }

    // Handle key/value, add to state, etc.
J
jorlow@chromium.org 已提交
2757
    bool drop = false;
2758
    bool current_entry_is_merging = false;
J
jorlow@chromium.org 已提交
2759 2760
    if (!ParseInternalKey(key, &ikey)) {
      // Do not hide error keys
2761 2762
      // 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.
2763
      current_user_key.Clear();
J
jorlow@chromium.org 已提交
2764 2765
      has_current_user_key = false;
      last_sequence_for_key = kMaxSequenceNumber;
2766
      visible_in_snapshot = kMaxSequenceNumber;
J
jorlow@chromium.org 已提交
2767 2768
    } else {
      if (!has_current_user_key ||
2769
          cfd->user_comparator()->Compare(ikey.user_key,
2770
                                          current_user_key.GetKey()) != 0) {
J
jorlow@chromium.org 已提交
2771
        // First occurrence of this user key
L
Lei Jin 已提交
2772
        current_user_key.SetKey(ikey.user_key);
J
jorlow@chromium.org 已提交
2773 2774
        has_current_user_key = true;
        last_sequence_for_key = kMaxSequenceNumber;
2775
        visible_in_snapshot = kMaxSequenceNumber;
H
Haobo Xu 已提交
2776
        // apply the compaction filter to the first occurrence of the user key
D
Danny Guo 已提交
2777
        if (compaction_filter && !is_compaction_v2 &&
H
Haobo Xu 已提交
2778 2779 2780 2781 2782 2783 2784 2785 2786
            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 已提交
2787 2788 2789
          bool to_delete = compaction_filter->Filter(
              compact->compaction->level(), ikey.user_key, value,
              &compaction_filter_value, &value_changed);
H
Haobo Xu 已提交
2790
          if (to_delete) {
2791 2792 2793
            // make a copy of the original key and convert it to a delete
            delete_key.SetInternalKey(ExtractUserKey(key), ikey.sequence,
                                      kTypeDeletion);
H
Haobo Xu 已提交
2794
            // anchor the key again
2795
            key = delete_key.GetKey();
H
Haobo Xu 已提交
2796 2797 2798 2799
            // needed because ikey is backed by key
            ParseInternalKey(key, &ikey);
            // no value associated with delete
            value.clear();
2800
            ++key_drop_user;
H
Haobo Xu 已提交
2801 2802 2803 2804
          } else if (value_changed) {
            value = compaction_filter_value;
          }
        }
J
jorlow@chromium.org 已提交
2805 2806
      }

2807 2808 2809
      // 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.
2810
      SequenceNumber prev_snapshot = 0; // 0 means no previous snapshot
2811 2812 2813 2814
      SequenceNumber visible = visible_at_tip ? visible_at_tip :
        is_snapshot_supported ?  findEarliestVisibleSnapshot(ikey.sequence,
                                  compact->existing_snapshots, &prev_snapshot)
                              : 0;
2815 2816 2817 2818 2819

      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 已提交
2820
        // Hidden by an newer entry for same user key
2821
        // TODO: why not > ?
2822
        assert(last_sequence_for_key >= ikey.sequence);
J
jorlow@chromium.org 已提交
2823
        drop = true;    // (A)
2824
        ++key_drop_newer_entry;
J
jorlow@chromium.org 已提交
2825
      } else if (ikey.type == kTypeDeletion &&
D
Danny Guo 已提交
2826
          ikey.sequence <= earliest_snapshot &&
2827
          compact->compaction->KeyNotExistsBeyondOutputLevel(ikey.user_key)) {
J
jorlow@chromium.org 已提交
2828 2829 2830 2831 2832 2833 2834 2835
        // 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;
2836
        ++key_drop_obsolete;
2837
      } else if (ikey.type == kTypeMerge) {
2838 2839 2840 2841 2842 2843
        if (!merge.HasOperator()) {
          LogToBuffer(log_buffer, "Options::merge_operator is null.");
          status = Status::InvalidArgument(
              "merge_operator is not properly initialized.");
          break;
        }
2844 2845 2846 2847 2848 2849
        // 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 已提交
2850 2851
        int steps = 0;
        merge.MergeUntil(input, prev_snapshot, bottommost_level,
2852
            db_options_.statistics.get(), &steps);
D
Danny Guo 已提交
2853 2854 2855
        // Skip the Merge ops
        combined_idx = combined_idx - 1 + steps;

2856 2857 2858 2859 2860 2861 2862 2863 2864 2865 2866 2867 2868 2869 2870 2871 2872 2873
        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 已提交
2874 2875 2876
      }

      last_sequence_for_key = ikey.sequence;
2877
      visible_in_snapshot = visible;
J
jorlow@chromium.org 已提交
2878 2879 2880
    }

    if (!drop) {
2881 2882 2883 2884 2885 2886 2887 2888 2889 2890 2891 2892 2893 2894 2895 2896
      // 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);
2897
      }
2898

2899 2900 2901 2902 2903 2904 2905 2906 2907 2908 2909
      // 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.
2910
        if (bottommost_level && ikey.sequence < earliest_snapshot &&
2911 2912 2913 2914 2915 2916 2917 2918 2919 2920 2921
            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
2922

2923 2924 2925 2926 2927 2928 2929
        // Open output file if necessary
        if (compact->builder == nullptr) {
          status = OpenCompactionOutputFile(compact);
          if (!status.ok()) {
            break;
          }
        }
2930 2931

        SequenceNumber seqno = GetInternalKeySeqno(newkey);
2932 2933
        if (compact->builder->NumEntries() == 0) {
          compact->current_output()->smallest.DecodeFrom(newkey);
2934 2935 2936 2937
          compact->current_output()->smallest_seqno = seqno;
        } else {
          compact->current_output()->smallest_seqno =
            std::min(compact->current_output()->smallest_seqno, seqno);
2938 2939 2940
        }
        compact->current_output()->largest.DecodeFrom(newkey);
        compact->builder->Add(newkey, value);
2941 2942
        compact->current_output()->largest_seqno =
          std::max(compact->current_output()->largest_seqno, seqno);
2943 2944 2945 2946

        // Close output file if it is big enough
        if (compact->builder->FileSize() >=
            compact->compaction->MaxOutputFileSize()) {
D
Danny Guo 已提交
2947
          status = FinishCompactionOutputFile(compact, input);
2948 2949 2950
          if (!status.ok()) {
            break;
          }
J
jorlow@chromium.org 已提交
2951 2952
        }

2953 2954 2955 2956 2957 2958 2959 2960 2961 2962 2963 2964 2965 2966 2967 2968 2969 2970 2971 2972
        // 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 已提交
2973 2974 2975 2976 2977
          break;
        }
      }
    }

2978
    // MergeUntil has moved input to the next entry
2979
    if (!current_entry_is_merging) {
2980 2981
      input->Next();
    }
J
jorlow@chromium.org 已提交
2982
  }
2983 2984 2985 2986 2987 2988 2989 2990 2991
  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);
  }
2992 2993
  RecordCompactionIOStats();

D
Danny Guo 已提交
2994 2995 2996 2997 2998 2999 3000 3001 3002
  return status;
}

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

3003
  // Assemble slice vectors for user keys and existing values.
3004
  // We also keep track of our parsed internal key structs because
3005 3006 3007
  // 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 已提交
3008
  std::vector<Slice> user_key_buf;
3009 3010 3011 3012 3013 3014 3015 3016 3017 3018
  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 已提交
3019 3020 3021 3022 3023 3024 3025 3026 3027
  }

  // 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(),
3028
      user_key_buf, existing_value_buf,
D
Danny Guo 已提交
3029 3030 3031 3032 3033 3034
      &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() ==
3035
      compact->key_str_buf_.size());
D
Danny Guo 已提交
3036
  assert(compact->to_delete_buf_.size() ==
3037
      compact->existing_value_str_buf_.size());
D
Danny Guo 已提交
3038 3039 3040 3041 3042 3043 3044 3045 3046
  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],
3047 3048 3049
                        compact->key_str_buf_[i].size(),
                        ikey_buf[i].sequence,
                        kTypeDeletion);
D
Danny Guo 已提交
3050 3051

      // no value associated with delete
3052
      compact->existing_value_str_buf_[i].clear();
L
Lei Jin 已提交
3053
      RecordTick(stats_, COMPACTION_KEY_DROP_USER);
D
Danny Guo 已提交
3054
    } else if (compact->value_changed_buf_[i]) {
3055 3056
      compact->existing_value_str_buf_[i] =
          compact->new_value_buf_[new_value_idx++];
D
Danny Guo 已提交
3057 3058 3059 3060 3061 3062 3063 3064 3065 3066 3067
    }
  }  // for
}

Status DBImpl::DoCompactionWork(CompactionState* compact,
                                DeletionState& deletion_state,
                                LogBuffer* log_buffer) {
  assert(compact);
  compact->CleanupBatchBuffer();
  compact->CleanupMergedBuffer();

F
Feng Zhu 已提交
3068 3069
  // Generate file_levels_ for compaction berfore making Iterator
  compact->compaction->GenerateFileLevels();
D
Danny Guo 已提交
3070
  int64_t imm_micros = 0;  // Micros spent doing imm_ compactions
3071
  ColumnFamilyData* cfd = compact->compaction->column_family_data();
3072 3073
  LogToBuffer(
      log_buffer,
I
Igor Canadi 已提交
3074 3075
      "[%s] Compacting %d@%d + %d@%d files, score %.2f slots available %d",
      cfd->GetName().c_str(), compact->compaction->num_input_files(0),
3076 3077
      compact->compaction->level(), compact->compaction->num_input_files(1),
      compact->compaction->output_level(), compact->compaction->score(),
3078
      db_options_.max_background_compactions - bg_compaction_scheduled_);
D
Danny Guo 已提交
3079 3080
  char scratch[2345];
  compact->compaction->Summary(scratch, sizeof(scratch));
I
Igor Canadi 已提交
3081 3082
  LogToBuffer(log_buffer, "[%s] Compaction start summary: %s\n",
              cfd->GetName().c_str(), scratch);
D
Danny Guo 已提交
3083

3084
  assert(cfd->current()->NumLevelFiles(compact->compaction->level()) > 0);
D
Danny Guo 已提交
3085 3086 3087 3088 3089 3090 3091 3092 3093 3094 3095 3096 3097 3098 3099 3100 3101 3102 3103 3104 3105 3106 3107 3108 3109
  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);

3110
  bool is_snapshot_supported = IsSnapshotSupported();
D
Danny Guo 已提交
3111 3112
  // Release mutex while we're actually doing the compaction work
  mutex_.Unlock();
3113
  log_buffer->FlushBufferToLog();
D
Danny Guo 已提交
3114 3115 3116 3117 3118 3119 3120 3121 3122 3123 3124

  const uint64_t start_micros = env_->NowMicros();
  unique_ptr<Iterator> input(versions_->MakeInputIterator(compact->compaction));
  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 =
3125 3126
      cfd->ioptions()->compaction_filter_factory_v2->
          CreateCompactionFilterV2(context);
D
Danny Guo 已提交
3127 3128 3129
  auto compaction_filter_v2 =
    compaction_filter_from_factory_v2.get();

3130 3131 3132 3133 3134 3135 3136 3137 3138 3139 3140 3141 3142 3143 3144 3145 3146 3147 3148 3149 3150
  if (!compaction_filter_v2) {
    status = ProcessKeyValueCompaction(
      is_snapshot_supported,
      visible_at_tip,
      earliest_snapshot,
      latest_snapshot,
      deletion_state,
      bottommost_level,
      imm_micros,
      input.get(),
      compact,
      false,
      log_buffer);
  } else {
    // 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.
3151
    bool prefix_initialized = false;
3152 3153 3154
    shared_ptr<Iterator> backup_input(
        versions_->MakeInputIterator(compact->compaction));
    backup_input->SeekToFirst();
3155 3156
    while (backup_input->Valid() && !shutting_down_.Acquire_Load() &&
           !cfd->IsDropped()) {
I
Igor Canadi 已提交
3157 3158 3159 3160 3161 3162
      // 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 已提交
3163 3164 3165 3166 3167
      Slice key = backup_input->key();
      Slice value = backup_input->value();

      if (!ParseInternalKey(key, &ikey)) {
        // log error
3168
        Log(db_options_.info_log, "[%s] Failed to parse key: %s",
I
Igor Canadi 已提交
3169
            cfd->GetName().c_str(), key.ToString().c_str());
D
Danny Guo 已提交
3170 3171
        continue;
      } else {
3172
        const SliceTransform* transformer =
3173
            cfd->ioptions()->compaction_filter_factory_v2->GetPrefixExtractor();
3174 3175 3176 3177 3178
        const auto key_prefix = transformer->Transform(ikey.user_key);
        if (!prefix_initialized) {
          compact->cur_prefix_ = key_prefix.ToString();
          prefix_initialized = true;
        }
D
Danny Guo 已提交
3179
        // If the prefix remains the same, keep buffering
3180
        if (key_prefix.compare(Slice(compact->cur_prefix_)) == 0) {
D
Danny Guo 已提交
3181 3182 3183 3184 3185 3186 3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197
          // 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
3198
          if (compact->key_str_buf_.size() > 0) {
D
Danny Guo 已提交
3199 3200
            CallCompactionFilterV2(compact, compaction_filter_v2);
          }
3201
          compact->cur_prefix_ = key_prefix.ToString();
D
Danny Guo 已提交
3202 3203 3204 3205
        }
      }

      // Merge this batch of data (values + ineligible keys)
3206
      compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3207 3208 3209 3210

      // Done buffering for the current prefix. Spit it out to disk
      // Now just iterate through all the kv-pairs
      status = ProcessKeyValueCompaction(
3211
          is_snapshot_supported,
D
Danny Guo 已提交
3212 3213 3214 3215 3216 3217 3218 3219 3220 3221 3222 3223 3224 3225 3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237 3238 3239 3240
          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.
3241
        if (compact->key_str_buf_.size() > 0) {
D
Danny Guo 已提交
3242 3243
          CallCompactionFilterV2(compact, compaction_filter_v2);
        }
3244
        compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3245 3246

        status = ProcessKeyValueCompaction(
3247
            is_snapshot_supported,
D
Danny Guo 已提交
3248 3249 3250 3251 3252 3253 3254 3255 3256 3257 3258 3259 3260 3261 3262 3263
            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
3264
    if (compact->key_str_buf_.size() > 0) {
D
Danny Guo 已提交
3265 3266
      CallCompactionFilterV2(compact, compaction_filter_v2);
    }
3267
    compact->MergeKeyValueSliceBuffer(&cfd->internal_comparator());
D
Danny Guo 已提交
3268
    status = ProcessKeyValueCompaction(
3269
        is_snapshot_supported,
D
Danny Guo 已提交
3270 3271 3272 3273 3274 3275 3276 3277 3278 3279 3280 3281
        visible_at_tip,
        earliest_snapshot,
        latest_snapshot,
        deletion_state,
        bottommost_level,
        imm_micros,
        input.get(),
        compact,
        true,
        log_buffer);
  }  // checking for compaction filter v2

3282
  if (status.ok() && (shutting_down_.Acquire_Load() || cfd->IsDropped())) {
L
Lei Jin 已提交
3283
    status = Status::ShutdownInProgress(
3284
        "Database shutdown or Column family drop during compaction");
J
jorlow@chromium.org 已提交
3285
  }
3286
  if (status.ok() && compact->builder != nullptr) {
3287
    status = FinishCompactionOutputFile(compact, input.get());
J
jorlow@chromium.org 已提交
3288 3289 3290 3291
  }
  if (status.ok()) {
    status = input->status();
  }
3292
  input.reset();
J
jorlow@chromium.org 已提交
3293

3294
  if (!db_options_.disableDataSync) {
3295 3296
    db_directory_->Fsync();
  }
I
Igor Canadi 已提交
3297

L
Lei Jin 已提交
3298
  InternalStats::CompactionStats stats(1);
3299
  stats.micros = env_->NowMicros() - start_micros - imm_micros;
M
Mark Callaghan 已提交
3300 3301
  stats.files_in_leveln = compact->compaction->num_input_files(0);
  stats.files_in_levelnp1 = compact->compaction->num_input_files(1);
L
Lei Jin 已提交
3302
  MeasureTime(stats_, COMPACTION_TIME, stats.micros);
3303 3304

  int num_output_files = compact->outputs.size();
3305
  if (compact->builder != nullptr) {
P
Pascal Borreli 已提交
3306
    // An error occurred so ignore the last output.
3307 3308 3309 3310
    assert(num_output_files > 0);
    --num_output_files;
  }
  stats.files_out_levelnp1 = num_output_files;
M
Mark Callaghan 已提交
3311

3312
  for (int i = 0; i < compact->compaction->num_input_files(0); i++) {
3313
    stats.bytes_readn += compact->compaction->input(0, i)->fd.GetFileSize();
3314
  }
M
Mark Callaghan 已提交
3315

3316
  for (int i = 0; i < compact->compaction->num_input_files(1); i++) {
3317
    stats.bytes_readnp1 += compact->compaction->input(1, i)->fd.GetFileSize();
3318
  }
M
Mark Callaghan 已提交
3319

3320
  for (int i = 0; i < num_output_files; i++) {
3321 3322 3323
    stats.bytes_written += compact->outputs[i].file_size;
  }

3324 3325
  RecordCompactionIOStats();

3326
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
3327
  mutex_.Lock();
3328 3329
  cfd->internal_stats()->AddCompactionStats(
      compact->compaction->output_level(), stats);
J
jorlow@chromium.org 已提交
3330

3331 3332 3333 3334
  // 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 已提交
3335
  if (status.ok()) {
3336
    status = InstallCompactionResults(compact, log_buffer);
3337
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
3338
    InstallSuperVersion(cfd, deletion_state);
J
jorlow@chromium.org 已提交
3339
  }
3340
  Version::LevelSummaryStorage tmp;
3341 3342
  LogToBuffer(
      log_buffer,
3343 3344
      "[%s] compacted to: %s, MB/sec: %.1f rd, %.1f wr, level %d, "
      "files in(%d, %d) out(%d) "
3345 3346
      "MB in(%.1f, %.1f) out(%.1f), read-write-amplify(%.1f) "
      "write-amplify(%.1f) %s\n",
I
Igor Canadi 已提交
3347
      cfd->GetName().c_str(), cfd->current()->LevelSummary(&tmp),
3348 3349 3350
      (stats.bytes_readn + stats.bytes_readnp1) /
          static_cast<double>(stats.micros),
      stats.bytes_written / static_cast<double>(stats.micros),
3351 3352 3353
      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 已提交
3354
      stats.bytes_written / 1048576.0,
3355
      (stats.bytes_written + stats.bytes_readnp1 + stats.bytes_readn) /
3356 3357
          (double)stats.bytes_readn,
      stats.bytes_written / (double)stats.bytes_readn,
3358
      status.ToString().c_str());
M
Mark Callaghan 已提交
3359

J
jorlow@chromium.org 已提交
3360 3361 3362
  return status;
}

3363 3364
namespace {
struct IterState {
3365 3366
  IterState(DBImpl* db, port::Mutex* mu, SuperVersion* super_version)
      : db(db), mu(mu), super_version(super_version) {}
3367 3368

  DBImpl* db;
3369
  port::Mutex* mu;
3370
  SuperVersion* super_version;
3371 3372 3373 3374
};

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

3376
  if (state->super_version->Unref()) {
3377 3378
    DBImpl::DeletionState deletion_state;

3379 3380 3381 3382 3383 3384
    state->mu->Lock();
    state->super_version->Cleanup();
    state->db->FindObsoleteFiles(deletion_state, false, true);
    state->mu->Unlock();

    delete state->super_version;
3385 3386 3387
    if (deletion_state.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(deletion_state);
    }
I
Igor Canadi 已提交
3388
  }
T
Tomislav Novak 已提交
3389

3390 3391
  delete state;
}
H
Hans Wennborg 已提交
3392
}  // namespace
3393

J
jorlow@chromium.org 已提交
3394
Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
3395
                                      ColumnFamilyData* cfd,
3396 3397 3398
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
3399 3400 3401 3402 3403 3404 3405 3406 3407 3408 3409 3410
  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();
3411
  IterState* cleanup = new IterState(this, &mutex_, super_version);
3412
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
3413 3414 3415 3416

  return internal_iter;
}

3417 3418 3419 3420
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

J
jorlow@chromium.org 已提交
3421
Status DBImpl::Get(const ReadOptions& options,
3422
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
3423
                   std::string* value) {
3424
  return GetImpl(options, column_family, key, value);
3425 3426
}

I
Igor Canadi 已提交
3427 3428 3429
// DeletionState gets created and destructed outside of the lock -- we
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
3430
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
3431 3432 3433 3434 3435 3436
//
// 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
3437 3438
void DBImpl::InstallSuperVersion(ColumnFamilyData* cfd,
                                 DeletionState& deletion_state) {
3439
  mutex_.AssertHeld();
I
Igor Canadi 已提交
3440 3441 3442 3443
  // if new_superversion == nullptr, it means somebody already used it
  SuperVersion* new_superversion =
    (deletion_state.new_superversion != nullptr) ?
    deletion_state.new_superversion : new SuperVersion();
3444
  // Use latest MutableCFOptions
3445 3446
  SuperVersion* old_superversion =
      cfd->InstallSuperVersion(new_superversion, &mutex_);
I
Igor Canadi 已提交
3447
  deletion_state.new_superversion = nullptr;
3448
  deletion_state.superversions_to_free.push_back(old_superversion);
I
Igor Canadi 已提交
3449 3450
}

3451
Status DBImpl::GetImpl(const ReadOptions& options,
3452 3453
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
L
Lei Jin 已提交
3454
  StopWatch sw(env_, stats_, DB_GET);
3455
  PERF_TIMER_GUARD(get_snapshot_time);
3456

3457 3458 3459
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3460
  SequenceNumber snapshot;
3461
  if (options.snapshot != nullptr) {
3462 3463 3464
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
3465
  }
3466

3467
  // Acquire SuperVersion
3468
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
I
Igor Canadi 已提交
3469

3470
  // Prepare to store a list of merge operations if merge occurs.
3471
  MergeContext merge_context;
3472

3473
  Status s;
3474
  // First look in the memtable, then in the immutable memtable (if any).
3475
  // s is both in/out. When in, s could either be OK or MergeInProgress.
3476
  // merge_operands will contain the sequence of merges in the latter case.
3477
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
3478
  PERF_TIMER_STOP(get_snapshot_time);
3479

3480
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
3481
    // Done
L
Lei Jin 已提交
3482
    RecordTick(stats_, MEMTABLE_HIT);
3483
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
3484
    // Done
L
Lei Jin 已提交
3485
    RecordTick(stats_, MEMTABLE_HIT);
3486
  } else {
3487
    PERF_TIMER_GUARD(get_from_output_files_time);
I
Igor Canadi 已提交
3488
    sv->current->Get(options, lkey, value, &s, &merge_context, value_found);
L
Lei Jin 已提交
3489
    RecordTick(stats_, MEMTABLE_MISS);
3490
  }
3491

3492 3493
  {
    PERF_TIMER_GUARD(get_post_process_time);
3494

3495
    ReturnAndCleanupSuperVersion(cfd, sv);
3496

3497 3498 3499
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
3500
  return s;
J
jorlow@chromium.org 已提交
3501 3502
}

3503 3504
std::vector<Status> DBImpl::MultiGet(
    const ReadOptions& options,
3505
    const std::vector<ColumnFamilyHandle*>& column_family,
3506
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
3507

L
Lei Jin 已提交
3508
  StopWatch sw(env_, stats_, DB_MULTIGET);
3509
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
3510

3511
  SequenceNumber snapshot;
3512

3513
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
3514
    ColumnFamilyData* cfd;
3515 3516 3517 3518 3519
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
3520 3521 3522 3523 3524 3525
    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});
3526 3527 3528
    }
  }

3529
  mutex_.Lock();
3530 3531 3532 3533 3534
  if (options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
  }
3535
  for (auto mgd_iter : multiget_cf_data) {
3536 3537
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
3538
  }
3539
  mutex_.Unlock();
3540

3541 3542
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
3543

3544
  // Note: this always resizes the values array
3545 3546 3547
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
3548 3549

  // Keep track of bytes that we read for statistics-recording later
3550
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
3551
  PERF_TIMER_STOP(get_snapshot_time);
3552 3553 3554 3555

  // 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.
3556
  // merge_operands will contain the sequence of merges in the latter case.
3557
  for (size_t i = 0; i < num_keys; ++i) {
3558
    merge_context.Clear();
3559
    Status& s = stat_list[i];
3560 3561 3562
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
3563 3564
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
3565 3566 3567
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
3568
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
3569
      // Done
3570
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
3571 3572
      // Done
    } else {
I
Igor Canadi 已提交
3573
      super_version->current->Get(options, lkey, value, &s, &merge_context);
3574 3575 3576
    }

    if (s.ok()) {
3577
      bytes_read += value->size();
3578 3579 3580 3581
    }
  }

  // Post processing (decrement reference counts and record statistics)
3582
  PERF_TIMER_GUARD(get_post_process_time);
3583 3584
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
3585
  // TODO(icanadi) do we need lock here or just around Cleanup()?
3586 3587 3588 3589 3590 3591
  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);
3592 3593
    }
  }
3594 3595 3596 3597 3598 3599 3600
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3601
  }
3602

L
Lei Jin 已提交
3603 3604 3605
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3606
  PERF_TIMER_STOP(get_post_process_time);
3607

3608
  return stat_list;
3609 3610
}

3611
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& options,
3612
                                  const std::string& column_family_name,
3613
                                  ColumnFamilyHandle** handle) {
I
Igor Canadi 已提交
3614 3615 3616
  *handle = nullptr;
  MutexLock l(&mutex_);

I
Igor Canadi 已提交
3617 3618
  if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
      nullptr) {
I
Igor Canadi 已提交
3619 3620
    return Status::InvalidArgument("Column family already exists");
  }
3621
  VersionEdit edit;
3622
  edit.AddColumnFamily(column_family_name);
3623 3624
  uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
  edit.SetColumnFamily(new_id);
I
Igor Canadi 已提交
3625
  edit.SetLogNumber(logfile_number_);
I
Igor Canadi 已提交
3626
  edit.SetComparatorName(options.comparator->Name());
3627

I
Igor Canadi 已提交
3628 3629
  // LogAndApply will both write the creation in MANIFEST and create
  // ColumnFamilyData object
3630 3631
  Status s = versions_->LogAndApply(nullptr, &edit, &mutex_,
                                    db_directory_.get(), false, &options);
3632
  if (s.ok()) {
3633
    single_column_family_mode_ = false;
3634 3635 3636
    auto cfd =
        versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
    assert(cfd != nullptr);
3637
    // Use latest MutableCFOptions
3638
    delete cfd->InstallSuperVersion(new SuperVersion(), &mutex_);
3639
    *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
3640
    Log(db_options_.info_log, "Created column family [%s] (ID %u)",
3641
        column_family_name.c_str(), (unsigned)cfd->GetID());
I
Igor Canadi 已提交
3642 3643
    max_total_in_memory_state_ += cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3644
  } else {
3645
    Log(db_options_.info_log, "Creating column family [%s] FAILED -- %s",
3646 3647
        column_family_name.c_str(), s.ToString().c_str());
  }
3648
  return s;
3649 3650
}

3651 3652 3653 3654
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3655 3656
    return Status::InvalidArgument("Can't drop default column family");
  }
3657

I
Igor Canadi 已提交
3658 3659
  VersionEdit edit;
  edit.DropColumnFamily();
3660 3661
  edit.SetColumnFamily(cfd->GetID());

3662

3663
  Status s;
3664 3665 3666 3667 3668 3669
  {
    MutexLock l(&mutex_);
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
3670
      // we drop column family from a single write thread
I
Igor Canadi 已提交
3671 3672
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
3673
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
3674
      s = versions_->LogAndApply(cfd, &edit, &mutex_);
I
Igor Canadi 已提交
3675
      write_thread_.ExitWriteThread(&w, &w, s);
3676
    }
3677
  }
3678

3679
  if (s.ok()) {
I
Igor Canadi 已提交
3680
    assert(cfd->IsDropped());
I
Igor Canadi 已提交
3681 3682
    max_total_in_memory_state_ -= cfd->options()->write_buffer_size *
                                  cfd->options()->max_write_buffer_number;
3683 3684
    Log(db_options_.info_log, "Dropped column family with id %u\n",
        cfd->GetID());
3685
  } else {
3686 3687
    Log(db_options_.info_log,
        "Dropping column family with id %u FAILED -- %s\n",
3688 3689 3690
        cfd->GetID(), s.ToString().c_str());
  }

3691
  return s;
3692 3693
}

3694
bool DBImpl::KeyMayExist(const ReadOptions& options,
3695 3696
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3697
  if (value_found != nullptr) {
K
Kai Liu 已提交
3698 3699
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3700
  }
3701 3702
  ReadOptions roptions = options;
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3703
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3704

3705
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
3706 3707 3708
  // 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();
3709 3710
}

3711
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
3712 3713 3714
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3715

3716
  if (read_options.tailing) {
I
Igor Canadi 已提交
3717 3718 3719 3720
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
3721 3722 3723 3724 3725
    auto iter = new ForwardIterator(this, read_options, cfd);
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
                         kMaxSequenceNumber,
                         cfd->options()->max_sequential_skip_in_iterations,
                         read_options.iterate_upper_bound);
I
Igor Canadi 已提交
3726
#endif
T
Tomislav Novak 已提交
3727
  } else {
3728 3729 3730 3731
    SequenceNumber latest_snapshot = versions_->LastSequence();
    SuperVersion* sv = nullptr;
    sv = cfd->GetReferencedSuperVersion(&mutex_);

I
Igor Canadi 已提交
3732
    auto snapshot =
3733 3734 3735
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
3736
            : latest_snapshot;
T
Tomislav Novak 已提交
3737

3738 3739 3740 3741 3742 3743 3744 3745 3746 3747 3748 3749 3750 3751 3752 3753 3754 3755 3756 3757 3758 3759 3760 3761 3762 3763 3764 3765 3766 3767 3768 3769 3770 3771 3772 3773 3774 3775 3776 3777 3778 3779 3780
    // 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(
3781 3782 3783
        env_, *cfd->ioptions(), cfd->user_comparator(),
        snapshot, cfd->options()->max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
3784

3785
    Iterator* internal_iter =
3786
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3787 3788 3789 3790
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
3791 3792
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
3793 3794
}

3795
Status DBImpl::NewIterators(
3796
    const ReadOptions& read_options,
I
Igor Canadi 已提交
3797
    const std::vector<ColumnFamilyHandle*>& column_families,
3798
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3799 3800 3801 3802 3803 3804
  iterators->clear();
  iterators->reserve(column_families.size());
  SequenceNumber latest_snapshot = 0;
  std::vector<SuperVersion*> super_versions;
  super_versions.reserve(column_families.size());

3805
  if (!read_options.tailing) {
I
Igor Canadi 已提交
3806 3807 3808 3809 3810 3811 3812 3813 3814
    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();
  }

3815
  if (read_options.tailing) {
I
Igor Canadi 已提交
3816 3817 3818 3819
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3820 3821
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
3822
      auto iter = new ForwardIterator(this, read_options, cfd);
L
Lei Jin 已提交
3823
      iterators->push_back(
3824 3825 3826
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
                        kMaxSequenceNumber,
                        cfd->options()->max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
3827
    }
I
Igor Canadi 已提交
3828
#endif
I
Igor Canadi 已提交
3829 3830 3831 3832 3833 3834
  } 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 =
3835 3836 3837
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
3838 3839
              : latest_snapshot;

3840
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3841 3842
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
          cfd->options()->max_sequential_skip_in_iterations);
3843
      Iterator* internal_iter = NewInternalIterator(
3844
          read_options, cfd, super_versions[i], db_iter->GetArena());
3845 3846
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3847 3848 3849 3850
    }
  }

  return Status::OK();
3851 3852
}

3853 3854 3855 3856 3857 3858 3859 3860 3861
bool DBImpl::IsSnapshotSupported() const {
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (!cfd->mem()->IsSnapshotSupported()) {
      return false;
    }
  }
  return true;
}

J
jorlow@chromium.org 已提交
3862
const Snapshot* DBImpl::GetSnapshot() {
3863
  MutexLock l(&mutex_);
3864 3865
  // returns null if the underlying memtable does not support snapshot.
  if (!IsSnapshotSupported()) return nullptr;
3866
  return snapshots_.New(versions_->LastSequence());
J
jorlow@chromium.org 已提交
3867 3868 3869 3870
}

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

// Convenience methods
3875 3876
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3877
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3878 3879
}

3880 3881 3882
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
3883
  if (!cfh->cfd()->ioptions()->merge_operator) {
3884 3885
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3886
    return DB::Merge(o, column_family, key, val);
3887 3888 3889
  }
}

3890
Status DBImpl::Delete(const WriteOptions& options,
3891
                      ColumnFamilyHandle* column_family, const Slice& key) {
3892
  return DB::Delete(options, column_family, key);
J
jorlow@chromium.org 已提交
3893 3894
}

S
Stanislau Hlebik 已提交
3895 3896 3897 3898
Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
3899
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
I
Igor Canadi 已提交
3900
  WriteThread::Writer w(&mutex_);
S
Stanislau Hlebik 已提交
3901 3902 3903 3904 3905 3906 3907 3908
  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;
I
Igor Canadi 已提交
3909
  bool has_timeout = false;
S
Stanislau Hlebik 已提交
3910
  if (w.timeout_hint_us == 0) {
I
Igor Canadi 已提交
3911
    w.timeout_hint_us = WriteThread::kNoTimeOut;
S
Stanislau Hlebik 已提交
3912 3913
  } else {
    expiration_time = env_->NowMicros() + w.timeout_hint_us;
I
Igor Canadi 已提交
3914
    has_timeout = true;
S
Stanislau Hlebik 已提交
3915 3916 3917 3918 3919 3920 3921 3922 3923
  }

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

  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
3924
  Status status = write_thread_.EnterWriteThread(&w, expiration_time);
S
Stanislau Hlebik 已提交
3925 3926
  assert(status.ok() || status.IsTimedOut());
  if (status.IsTimedOut()) {
3927
    mutex_.Unlock();
L
Lei Jin 已提交
3928
    RecordTick(stats_, WRITE_TIMEDOUT);
3929
    return Status::TimedOut();
3930
  }
S
Stanislau Hlebik 已提交
3931 3932 3933 3934 3935 3936 3937 3938 3939 3940
  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);
3941

3942 3943 3944 3945
  // 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.
3946 3947 3948
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

3949
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
3950
                                    ? 4 * max_total_in_memory_state_
3951
                                    : db_options_.max_total_wal_size;
3952
  if (UNLIKELY(!single_column_family_mode_) &&
3953
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3954
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
3955
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
3956
    alive_log_files_.begin()->getting_flushed = true;
3957
    Log(db_options_.info_log,
3958 3959 3960
        "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 已提交
3961 3962
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
3963
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
3964
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
S
Stanislau Hlebik 已提交
3965
        status = SetNewMemtableAndNewLogFile(cfd, &context);
I
Igor Canadi 已提交
3966 3967 3968
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
3969
        cfd->imm()->FlushRequested();
3970
      }
3971
    }
I
Igor Canadi 已提交
3972 3973 3974 3975 3976 3977 3978 3979 3980 3981 3982 3983 3984 3985 3986 3987 3988 3989 3990
    MaybeScheduleFlushOrCompaction();
  }

  if (UNLIKELY(status.ok() && !bg_error_.ok())) {
    status = bg_error_;
  }

  if (UNLIKELY(status.ok() && !flush_scheduler_.Empty())) {
    status = ScheduleFlushes(&context);
  }

  if (UNLIKELY(status.ok()) &&
      (write_controller_.IsStopped() || write_controller_.GetDelay() > 0)) {
    DelayWrite(expiration_time);
  }

  if (UNLIKELY(status.ok() && has_timeout &&
               env_->NowMicros() > expiration_time)) {
    status = Status::TimedOut();
3991
  }
3992

D
dgrogan@chromium.org 已提交
3993
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
3994
  WriteThread::Writer* last_writer = &w;
S
Stanislau Hlebik 已提交
3995
  if (status.ok()) {
3996
    autovector<WriteBatch*> write_batch_group;
I
Igor Canadi 已提交
3997
    write_thread_.BuildBatchGroup(&last_writer, &write_batch_group);
3998

3999 4000 4001
    // 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
4002
    // into memtables
4003
    {
4004
      mutex_.Unlock();
4005 4006 4007 4008 4009 4010 4011 4012 4013 4014
      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]);
        }
      }

4015 4016 4017 4018
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
4019
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
4020
      // Record statistics
L
Lei Jin 已提交
4021 4022
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
      RecordTick(stats_, BYTES_WRITTEN, WriteBatchInternal::ByteSize(updates));
4023 4024
      if (options.disableWAL) {
        flush_on_destroy_ = true;
4025
      }
L
Lei Jin 已提交
4026
      PERF_TIMER_STOP(write_pre_and_post_process_time);
4027

4028
      uint64_t log_size = 0;
4029
      if (!options.disableWAL) {
4030
        PERF_TIMER_GUARD(write_wal_time);
4031 4032
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
4033 4034
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
4035
        log_empty_ = false;
4036
        log_size = log_entry.size();
L
Lei Jin 已提交
4037
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
H
heyongqiang 已提交
4038
        if (status.ok() && options.sync) {
I
Igor Canadi 已提交
4039
          RecordTick(stats_, WAL_FILE_SYNCED);
4040
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
4041
          if (db_options_.use_fsync) {
4042
            status = log_->file()->Fsync();
4043
          } else {
4044
            status = log_->file()->Sync();
4045
          }
H
heyongqiang 已提交
4046
        }
4047 4048
      }
      if (status.ok()) {
4049
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
4050

4051
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
4052 4053
            updates, column_family_memtables_.get(),
            options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
4054 4055 4056 4057 4058 4059 4060 4061
        // 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 已提交
4062
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
4063
      }
L
Lei Jin 已提交
4064
      PERF_TIMER_START(write_pre_and_post_process_time);
4065 4066 4067
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
4068
      mutex_.Lock();
4069 4070 4071 4072 4073 4074 4075 4076 4077
      // 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);
      }
4078
      if (status.ok()) {
4079
        versions_->SetLastSequence(last_sequence);
4080
      }
J
jorlow@chromium.org 已提交
4081 4082
    }
  }
4083
  if (db_options_.paranoid_checks && !status.ok() &&
4084
      !status.IsTimedOut() && bg_error_.ok()) {
I
Igor Canadi 已提交
4085 4086
    bg_error_ = status; // stop compaction & fail any further writes
  }
4087

I
Igor Canadi 已提交
4088
  write_thread_.ExitWriteThread(&w, last_writer, status);
I
Igor Canadi 已提交
4089
  mutex_.Unlock();
4090

4091
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
4092
    RecordTick(stats_, WRITE_TIMEDOUT);
4093 4094
  }

J
jorlow@chromium.org 已提交
4095 4096 4097
  return status;
}

4098 4099 4100 4101 4102 4103 4104 4105 4106 4107 4108 4109 4110 4111 4112 4113 4114 4115 4116 4117 4118 4119
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
void DBImpl::DelayWrite(uint64_t expiration_time) {
  StopWatch sw(env_, stats_, WRITE_STALL);
  bool has_timeout = (expiration_time > 0);
  auto delay = write_controller_.GetDelay();
  if (write_controller_.IsStopped() == false && delay > 0) {
    mutex_.Unlock();
    env_->SleepForMicroseconds(delay);
    mutex_.Lock();
  }

  while (write_controller_.IsStopped()) {
    if (has_timeout) {
      bg_cv_.TimedWait(expiration_time);
      if (env_->NowMicros() > expiration_time) {
        break;
      }
    } else {
      bg_cv_.Wait();
    }
  }
4120 4121
}

I
Igor Canadi 已提交
4122 4123 4124 4125 4126 4127 4128 4129 4130 4131 4132
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  bool schedule_bg_work = false;
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
    schedule_bg_work = true;
    auto status = SetNewMemtableAndNewLogFile(cfd, context);
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
4133 4134
    }
  }
I
Igor Canadi 已提交
4135 4136 4137 4138
  if (schedule_bg_work) {
    MaybeScheduleFlushOrCompaction();
  }
  return Status::OK();
S
Stanislau Hlebik 已提交
4139 4140 4141 4142 4143 4144 4145 4146 4147 4148 4149 4150 4151 4152 4153 4154 4155 4156
}

// 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;
4157
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
4158 4159 4160 4161
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
4162 4163 4164
      s = env_->NewWritableFile(
          LogFileName(db_options_.wal_dir, new_log_number),
          &lfile, env_->OptimizeForLogWrite(env_options_));
S
Stanislau Hlebik 已提交
4165 4166 4167 4168 4169 4170
      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 已提交
4171
      }
S
Stanislau Hlebik 已提交
4172 4173 4174
    }

    if (s.ok()) {
L
Lei Jin 已提交
4175
      new_mem = new MemTable(cfd->internal_comparator(),
4176 4177
          *cfd->ioptions(), MemTableOptions(mutable_cf_options,
          *cfd->options()));
S
Stanislau Hlebik 已提交
4178 4179 4180 4181 4182 4183 4184 4185 4186 4187 4188 4189 4190 4191 4192 4193 4194 4195 4196 4197 4198 4199 4200 4201 4202 4203 4204 4205
      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_);
4206
      }
4207 4208
    }
  }
S
Stanislau Hlebik 已提交
4209 4210 4211 4212
  cfd->mem()->SetNextLogNumber(logfile_number_);
  cfd->imm()->Add(cfd->mem());
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
4213
  Log(db_options_.info_log,
S
Stanislau Hlebik 已提交
4214 4215 4216
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), logfile_number_);
  context->superversions_to_free_.push_back(
4217
      cfd->InstallSuperVersion(new_superversion, &mutex_, mutable_cf_options));
4218 4219 4220
  return s;
}

I
Igor Canadi 已提交
4221
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
4222 4223 4224 4225 4226
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

4227 4228
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
4229
  auto version = cfd->current();
4230 4231 4232 4233 4234 4235 4236 4237 4238 4239 4240 4241
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
4242
#endif  // ROCKSDB_LITE
4243

I
Igor Canadi 已提交
4244 4245 4246 4247
const std::string& DBImpl::GetName() const {
  return dbname_;
}

4248 4249 4250 4251
Env* DBImpl::GetEnv() const {
  return env_;
}

4252 4253
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
4254
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
4255 4256
}

4257
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
4258
                         const Slice& property, std::string* value) {
4259 4260
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4261 4262 4263
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

4264
  value->clear();
4265 4266 4267 4268 4269 4270 4271 4272 4273 4274 4275 4276 4277 4278 4279
  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 已提交
4280 4281
}

4282 4283
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
4284 4285
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4286 4287 4288 4289 4290 4291 4292 4293 4294 4295 4296 4297
  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) {
4298 4299
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4300 4301 4302

  if (!need_out_of_mutex) {
    MutexLock l(&mutex_);
4303
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
4304 4305 4306 4307 4308 4309 4310 4311 4312 4313 4314 4315 4316 4317
  } 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
I
Igor Canadi 已提交
4318
  return cfd->GetThreadLocalSuperVersion(&mutex_);
4319 4320 4321 4322
}

void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
4323
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
4324 4325 4326 4327 4328 4329 4330 4331 4332 4333 4334 4335 4336

  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);
  }
4337 4338
}

4339
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4340
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
4341 4342
  // TODO(opt): better implementation
  Version* v;
4343 4344
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
4345 4346
  {
    MutexLock l(&mutex_);
4347
    v = cfd->current();
4348
    v->Ref();
J
jorlow@chromium.org 已提交
4349 4350 4351 4352 4353 4354 4355 4356 4357 4358 4359 4360 4361 4362 4363 4364 4365
  }

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

I
Igor Canadi 已提交
4366 4367 4368 4369 4370
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
4371
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
4372 4373 4374 4375 4376 4377 4378 4379 4380 4381 4382 4383 4384 4385 4386 4387
  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;
  }
4388
  iter->reset(new TransactionLogIteratorImpl(db_options_.wal_dir, &db_options_,
L
Lei Jin 已提交
4389
                                             read_options, env_options_,
I
Igor Canadi 已提交
4390 4391 4392 4393
                                             seq, std::move(wal_files), this));
  return (*iter)->status();
}

4394 4395 4396
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4397 4398 4399
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
4400
    Log(db_options_.info_log, "DeleteFile %s failed.\n", name.c_str());
4401 4402 4403
    return Status::InvalidArgument("Invalid file name");
  }

4404 4405 4406 4407
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4408
      Log(db_options_.info_log, "DeleteFile %s failed - not archived log.\n",
4409
          name.c_str());
4410 4411
      return Status::NotSupported("Delete only supported for archived logs");
    }
4412
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
4413
    if (!status.ok()) {
4414
      Log(db_options_.info_log, "DeleteFile %s failed -- %s.\n",
4415
          name.c_str(), status.ToString().c_str());
4416 4417 4418 4419
    }
    return status;
  }

4420
  int level;
I
Igor Canadi 已提交
4421
  FileMetaData* metadata;
4422
  ColumnFamilyData* cfd;
4423
  VersionEdit edit;
K
Kai Liu 已提交
4424
  DeletionState deletion_state(true);
D
Dhruba Borthakur 已提交
4425 4426
  {
    MutexLock l(&mutex_);
4427
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4428
    if (!status.ok()) {
4429
      Log(db_options_.info_log, "DeleteFile %s failed. File not found\n",
4430
                             name.c_str());
D
Dhruba Borthakur 已提交
4431 4432
      return Status::InvalidArgument("File not found");
    }
I
Igor Canadi 已提交
4433
    assert((level > 0) && (level < cfd->NumberLevels()));
4434

D
Dhruba Borthakur 已提交
4435
    // If the file is being compacted no need to delete.
4436
    if (metadata->being_compacted) {
4437
      Log(db_options_.info_log,
4438
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
D
Dhruba Borthakur 已提交
4439
      return Status::OK();
4440 4441
    }

D
Dhruba Borthakur 已提交
4442 4443 4444
    // 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 已提交
4445
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
4446
      if (cfd->current()->NumLevelFiles(i) != 0) {
4447
        Log(db_options_.info_log,
4448
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
D
Dhruba Borthakur 已提交
4449 4450 4451 4452
        return Status::InvalidArgument("File not in last level");
      }
    }
    edit.DeleteFile(level, number);
4453
    status = versions_->LogAndApply(cfd, &edit, &mutex_, db_directory_.get());
I
Igor Canadi 已提交
4454
    if (status.ok()) {
4455
      InstallSuperVersion(cfd, deletion_state);
I
Igor Canadi 已提交
4456
    }
I
Igor Canadi 已提交
4457
    FindObsoleteFiles(deletion_state, false);
D
Dhruba Borthakur 已提交
4458
  } // lock released here
4459
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
4460
  // remove files outside the db-lock
4461 4462 4463
  if (deletion_state.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(deletion_state);
  }
4464 4465 4466 4467 4468 4469
  {
    MutexLock l(&mutex_);
    // schedule flush if file deletion means we freed the space for flushes to
    // continue
    MaybeScheduleFlushOrCompaction();
  }
4470 4471 4472
  return status;
}

4473
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
4474
  MutexLock l(&mutex_);
4475
  versions_->GetLiveFilesMetaData(metadata);
4476
}
I
Igor Canadi 已提交
4477
#endif  // ROCKSDB_LITE
4478

I
Igor Canadi 已提交
4479 4480 4481 4482 4483 4484 4485
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
4488 4489 4490 4491 4492 4493
    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) {
4494
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
4495 4496 4497 4498 4499 4500 4501 4502 4503 4504 4505 4506
                             ". 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);
  }
}

4507 4508 4509 4510 4511 4512 4513 4514 4515 4516 4517 4518 4519 4520 4521 4522 4523 4524 4525 4526 4527 4528 4529 4530 4531 4532 4533
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 已提交
4534 4535
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
4536
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
4537
               const Slice& key, const Slice& value) {
4538 4539 4540 4541
  // 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);
4542
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
4543 4544 4545
  return Write(opt, &batch);
}

4546 4547
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
4548
  WriteBatch batch;
4549
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
4550 4551 4552
  return Write(opt, &batch);
}

4553 4554
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
4555
  WriteBatch batch;
4556
  batch.Merge(column_family, key, value);
4557 4558 4559
  return Write(opt, &batch);
}

4560 4561
// Default implementation -- returns not supported status
Status DB::CreateColumnFamily(const ColumnFamilyOptions& options,
4562
                              const std::string& column_family_name,
4563
                              ColumnFamilyHandle** handle) {
4564
  return Status::NotSupported("");
4565
}
4566
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4567
  return Status::NotSupported("");
4568 4569
}

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

J
Jim Paton 已提交
4572
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4573 4574 4575 4576
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4577
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4578
  std::vector<ColumnFamilyHandle*> handles;
4579
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
4580 4581 4582 4583 4584 4585 4586
  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;
4587 4588
}

4589 4590
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4591
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4592 4593 4594 4595
  Status s = SanitizeDBOptionsByCFOptions(&db_options, column_families);
  if (!s.ok()) {
    return s;
  }
4596
  if (db_options.db_paths.size() > 1) {
4597 4598 4599 4600 4601 4602 4603 4604 4605 4606 4607 4608
    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. ");
    }
4609 4610
  }

4611
  *dbptr = nullptr;
4612
  handles->clear();
J
jorlow@chromium.org 已提交
4613

I
Igor Canadi 已提交
4614 4615 4616 4617
  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);
4618
  }
4619

I
Igor Canadi 已提交
4620
  DBImpl* impl = new DBImpl(db_options, dbname);
4621
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
4622
  if (s.ok()) {
4623
    for (auto db_path : impl->db_options_.db_paths) {
4624
      s = impl->env_->CreateDirIfMissing(db_path.path);
4625 4626 4627 4628 4629 4630
      if (!s.ok()) {
        break;
      }
    }
  }

4631 4632 4633 4634 4635 4636
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4637 4638 4639 4640
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4641
  impl->mutex_.Lock();
4642 4643
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4644
  if (s.ok()) {
4645
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4646
    unique_ptr<WritableFile> lfile;
4647
    EnvOptions soptions(db_options);
4648 4649 4650
    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 已提交
4651
    if (s.ok()) {
4652
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4653
      impl->logfile_number_ = new_log_number;
4654
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
4655

4656 4657
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4658 4659
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
4660 4661 4662 4663 4664 4665 4666 4667 4668 4669 4670 4671 4672 4673 4674 4675 4676 4677 4678
        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 已提交
4679
        }
4680
      }
I
Igor Canadi 已提交
4681 4682
    }
    if (s.ok()) {
4683
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
4684
        // Use latest MutableCFOptions
4685
        delete cfd->InstallSuperVersion(new SuperVersion(), &impl->mutex_);
4686
      }
I
Igor Canadi 已提交
4687 4688
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4689
      impl->DeleteObsoleteFiles();
4690
      impl->MaybeScheduleFlushOrCompaction();
4691
      s = impl->db_directory_->Fsync();
J
jorlow@chromium.org 已提交
4692 4693
    }
  }
4694

I
Igor Canadi 已提交
4695 4696
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
4697 4698
      if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
I
Igor Canadi 已提交
4699 4700 4701 4702
        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 已提交
4703 4704 4705
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
                "open with universal or FIFO compaction style.");
I
Igor Canadi 已提交
4706 4707 4708 4709
            break;
          }
        }
      }
4710
      if (cfd->ioptions()->merge_operator != nullptr &&
4711 4712 4713 4714 4715
          !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 已提交
4716
      if (!s.ok()) {
4717 4718 4719 4720 4721
        break;
      }
    }
  }

4722 4723
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4724
  if (s.ok()) {
4725
    impl->opened_successfully_ = true;
J
jorlow@chromium.org 已提交
4726 4727
    *dbptr = impl;
  } else {
4728 4729 4730
    for (auto h : *handles) {
      delete h;
    }
4731
    handles->clear();
J
jorlow@chromium.org 已提交
4732 4733 4734 4735 4736
    delete impl;
  }
  return s;
}

4737 4738 4739
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4740
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4741 4742
}

4743 4744 4745
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4746
Status DestroyDB(const std::string& dbname, const Options& options) {
4747
  const InternalKeyComparator comparator(options.comparator);
4748
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
4749
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4750
  std::vector<std::string> filenames;
4751 4752
  std::vector<std::string> archiveFiles;

4753
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
4754 4755
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
4756 4757 4758 4759 4760 4761 4762

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

J
jorlow@chromium.org 已提交
4764 4765 4766 4767 4768
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
4769 4770
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4771 4772 4773
  if (result.ok()) {
    uint64_t number;
    FileType type;
4774
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
4775
    for (size_t i = 0; i < filenames.size(); i++) {
4776
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
4777
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4778 4779 4780
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
4781 4782
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
4783 4784 4785
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
4786 4787 4788 4789 4790
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4791

4792 4793
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
4794 4795 4796 4797 4798
      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
4799
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
4800 4801 4802 4803 4804 4805 4806
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

4807
    env->GetChildren(archivedir, &archiveFiles);
4808 4809
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4810 4811
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4812
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4813 4814 4815 4816 4817
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4818
    // ignore case where no archival directory is present.
4819
    env->DeleteDir(archivedir);
4820

J
jorlow@chromium.org 已提交
4821
    env->UnlockFile(lock);  // Ignore error since state is already gone
4822
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4823
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4824
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4825 4826 4827 4828
  }
  return result;
}

4829 4830
//
// A global method that can dump out the build version
K
kailiu 已提交
4831
void DumpLeveldbBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4832 4833
#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
4834
  Log(log, "Git sha %s", rocksdb_build_git_sha);
4835
  Log(log, "Compile time %s %s",
4836
      rocksdb_build_compile_time, rocksdb_build_compile_date);
I
Igor Canadi 已提交
4837
#endif
4838 4839
}

4840
}  // namespace rocksdb