db_impl.cc 134.7 KB
Newer Older
1 2 3 4 5
//  Copyright (c) 2013, Facebook, Inc.  All rights reserved.
//  This source code is licensed under the BSD-style license found in the
//  LICENSE file in the root directory of this source tree. An additional grant
//  of patent rights can be found in the PATENTS file in the same directory.
//
J
jorlow@chromium.org 已提交
6 7 8 9 10 11 12
// 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"

#include <algorithm>
13 14
#include <climits>
#include <cstdio>
J
jorlow@chromium.org 已提交
15
#include <set>
16
#include <stdexcept>
17 18
#include <stdint.h>
#include <string>
19
#include <unordered_set>
20
#include <vector>
21

J
jorlow@chromium.org 已提交
22 23
#include "db/builder.h"
#include "db/dbformat.h"
24
#include "db/db_iter.h"
J
jorlow@chromium.org 已提交
25 26 27 28
#include "db/filename.h"
#include "db/log_reader.h"
#include "db/log_writer.h"
#include "db/memtable.h"
29
#include "db/memtablelist.h"
30
#include "db/merge_context.h"
31
#include "db/merge_helper.h"
T
Tyler Harter 已提交
32
#include "db/prefix_filter_iterator.h"
J
jorlow@chromium.org 已提交
33
#include "db/table_cache.h"
K
kailiu 已提交
34
#include "db/table_properties_collector.h"
35
#include "db/transaction_log_impl.h"
J
jorlow@chromium.org 已提交
36 37
#include "db/version_set.h"
#include "db/write_batch_internal.h"
38
#include "port/port.h"
39 40 41 42 43 44
#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 已提交
45 46
#include "rocksdb/table.h"
#include "port/port.h"
J
jorlow@chromium.org 已提交
47
#include "table/block.h"
48
#include "table/block_based_table_factory.h"
J
jorlow@chromium.org 已提交
49 50
#include "table/merger.h"
#include "table/two_level_iterator.h"
51 52
#include "util/auto_roll_logger.h"
#include "util/build_version.h"
J
jorlow@chromium.org 已提交
53
#include "util/coding.h"
I
Igor Canadi 已提交
54
#include "util/hash_skiplist_rep.h"
J
jorlow@chromium.org 已提交
55 56
#include "util/logging.h"
#include "util/mutexlock.h"
57
#include "util/perf_context_imp.h"
58
#include "util/stop_watch.h"
59
#include "util/autovector.h"
J
jorlow@chromium.org 已提交
60

61
namespace rocksdb {
J
jorlow@chromium.org 已提交
62

63 64
void dumpLeveldbBuildVersion(Logger * log);

65 66 67 68 69
// Information kept for every waiting writer
struct DBImpl::Writer {
  Status status;
  WriteBatch* batch;
  bool sync;
H
heyongqiang 已提交
70
  bool disableWAL;
71 72 73 74 75 76
  bool done;
  port::CondVar cv;

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

J
jorlow@chromium.org 已提交
77 78 79
struct DBImpl::CompactionState {
  Compaction* const compaction;

80 81 82 83 84
  // 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 已提交
85 86 87 88 89 90

  // Files produced by compaction
  struct Output {
    uint64_t number;
    uint64_t file_size;
    InternalKey smallest, largest;
91
    SequenceNumber smallest_seqno, largest_seqno;
J
jorlow@chromium.org 已提交
92 93
  };
  std::vector<Output> outputs;
94
  std::list<uint64_t> allocated_file_numbers;
J
jorlow@chromium.org 已提交
95 96

  // State kept for output being generated
97 98
  unique_ptr<WritableFile> outfile;
  unique_ptr<TableBuilder> builder;
J
jorlow@chromium.org 已提交
99 100 101 102 103 104 105 106 107

  uint64_t total_bytes;

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

  explicit CompactionState(Compaction* c)
      : compaction(c),
        total_bytes(0) {
  }
108 109 110 111 112 113 114

  // Create a client visible context of this compaction
  CompactionFilter::Context GetFilterContext() {
    CompactionFilter::Context context;
    context.is_full_compaction = compaction->IsFullCompaction();
    return context;
  }
J
jorlow@chromium.org 已提交
115 116 117
};

// Fix user-supplied options to be reasonable
118
template <class T, class V>
J
jorlow@chromium.org 已提交
119
static void ClipToRange(T* ptr, V minvalue, V maxvalue) {
D
dgrogan@chromium.org 已提交
120 121
  if (static_cast<V>(*ptr) > maxvalue) *ptr = maxvalue;
  if (static_cast<V>(*ptr) < minvalue) *ptr = minvalue;
J
jorlow@chromium.org 已提交
122 123 124
}
Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
S
Sanjay Ghemawat 已提交
125
                        const InternalFilterPolicy* ipolicy,
J
jorlow@chromium.org 已提交
126 127 128
                        const Options& src) {
  Options result = src;
  result.comparator = icmp;
129
  result.filter_policy = (src.filter_policy != nullptr) ? ipolicy : nullptr;
130
  ClipToRange(&result.max_open_files,            20,     1000000);
131 132
  ClipToRange(&result.write_buffer_size,         ((size_t)64)<<10,
                                                 ((size_t)64)<<30);
S
Sanjay Ghemawat 已提交
133
  ClipToRange(&result.block_size,                1<<10,  4<<20);
134

X
Xing Jin 已提交
135 136 137 138 139 140
  // if user sets arena_block_size, we trust user to use this value. Otherwise,
  // calculate a proper value from writer_buffer_size;
  if (result.arena_block_size <= 0) {
    result.arena_block_size = result.write_buffer_size / 10;
  }

141 142
  result.min_write_buffer_number_to_merge = std::min(
    result.min_write_buffer_number_to_merge, result.max_write_buffer_number-1);
143
  if (result.info_log == nullptr) {
K
Kai Liu 已提交
144 145
    Status s = CreateLoggerFromOptions(dbname, result.db_log_dir, src.env,
                                       result, &result.info_log);
J
jorlow@chromium.org 已提交
146 147
    if (!s.ok()) {
      // No place suitable for logging
148
      result.info_log = nullptr;
J
jorlow@chromium.org 已提交
149 150
    }
  }
151
  if (result.block_cache == nullptr && !result.no_block_cache) {
152 153
    result.block_cache = NewLRUCache(8 << 20);
  }
154
  result.compression_per_level = src.compression_per_level;
155 156 157
  if (result.block_size_deviation < 0 || result.block_size_deviation > 100) {
    result.block_size_deviation = 0;
  }
158 159 160
  if (result.max_mem_compaction_level >= result.num_levels) {
    result.max_mem_compaction_level = result.num_levels - 1;
  }
J
Jim Paton 已提交
161 162 163
  if (result.soft_rate_limit > result.hard_rate_limit) {
    result.soft_rate_limit = result.hard_rate_limit;
  }
164 165
  if (result.compaction_filter) {
    Log(result.info_log, "Compaction filter specified, ignore factory");
166
  }
J
Jim Paton 已提交
167
  if (result.prefix_extractor) {
I
Igor Canadi 已提交
168
    // If a prefix extractor has been supplied and a HashSkipListRepFactory is
J
Jim Paton 已提交
169 170
    // being used, make sure that the latter uses the former as its transform
    // function.
I
Igor Canadi 已提交
171
    auto factory = dynamic_cast<HashSkipListRepFactory*>(
J
Jim Paton 已提交
172
      result.memtable_factory.get());
173
    if (factory &&
174
        factory->GetTransform() != result.prefix_extractor) {
J
Jim Paton 已提交
175 176 177 178
      Log(result.info_log, "A prefix hash representation factory was supplied "
          "whose prefix extractor does not match options.prefix_extractor. "
          "Falling back to skip list representation factory");
      result.memtable_factory = std::make_shared<SkipListFactory>();
179 180
    } else if (factory) {
      Log(result.info_log, "Prefix hash memtable rep is in use.");
J
Jim Paton 已提交
181 182
    }
  }
183 184 185 186 187

  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
188

K
kailiu 已提交
189 190 191 192 193 194
  // -- Sanitize the table properties collector
  // All user defined properties collectors will be wrapped by
  // UserKeyTablePropertiesCollector since for them they only have the
  // knowledge of the user keys; internal keys are invisible to them.
  auto& collectors = result.table_properties_collectors;
  for (size_t i = 0; i < result.table_properties_collectors.size(); ++i) {
195 196
    assert(collectors[i]);
    collectors[i] =
K
kailiu 已提交
197
      std::make_shared<UserKeyTablePropertiesCollector>(collectors[i]);
198 199 200 201
  }

  // Add collector to collect internal key statistics
  collectors.push_back(
K
kailiu 已提交
202
      std::make_shared<InternalKeyPropertiesCollector>()
203 204
  );

J
jorlow@chromium.org 已提交
205 206 207
  return result;
}

S
Siying Dong 已提交
208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228
CompressionType GetCompressionType(const Options& options, int level,
                                   const bool enable_compression) {
  if (!enable_compression) {
    // disable compression
    return kNoCompression;
  }
  // If the use has specified a different compression level for each level,
  // then pick the compresison for that level.
  if (!options.compression_per_level.empty()) {
    const int n = options.compression_per_level.size() - 1;
    // It is possible for level_ to be -1; in that case, we use level
    // 0's compression.  This occurs mostly in backwards compatibility
    // situations when the builder doesn't know what level the file
    // belongs to.  Likewise, if level_ is beyond the end of the
    // specified compression levels, use the last value.
    return options.compression_per_level[std::max(0, std::min(level, n))];
  } else {
    return options.compression;
  }
}

229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250
CompressionType GetCompressionFlush(const Options& options) {
  // Compressing memtable flushes might not help unless the sequential load
  // optimization is used for leveled compaction. Otherwise the CPU and
  // latency overhead is not offset by saving much space.

  bool can_compress;

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

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

J
jorlow@chromium.org 已提交
251 252
DBImpl::DBImpl(const Options& options, const std::string& dbname)
    : env_(options.env),
H
heyongqiang 已提交
253
      dbname_(dbname),
J
jorlow@chromium.org 已提交
254
      internal_comparator_(options.comparator),
255 256
      options_(SanitizeOptions(dbname, &internal_comparator_,
                               &internal_filter_policy_, options)),
H
heyongqiang 已提交
257
      internal_filter_policy_(options.filter_policy),
J
jorlow@chromium.org 已提交
258
      owns_info_log_(options_.info_log != options.info_log),
259
      db_lock_(nullptr),
H
Haobo Xu 已提交
260
      mutex_(options.use_adaptive_mutex),
261
      shutting_down_(nullptr),
J
jorlow@chromium.org 已提交
262
      bg_cv_(&mutex_),
I
Igor Canadi 已提交
263
      mem_rep_factory_(options_.memtable_factory.get()),
264
      mem_(new MemTable(internal_comparator_, options_)),
265
      logfile_number_(0),
I
Igor Canadi 已提交
266
      super_version_(nullptr),
267
      tmp_batch_(),
268
      bg_compaction_scheduled_(0),
269
      bg_manual_only_(0),
270
      bg_flush_scheduled_(0),
271
      bg_logstats_scheduled_(false),
272 273
      manual_compaction_(nullptr),
      logger_(nullptr),
274
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
275
      delete_obsolete_files_last_run_(options.env->NowMicros()),
276
      purge_wal_files_last_run_(0),
277
      last_stats_dump_time_microsec_(0),
278
      default_interval_to_delete_obsolete_WAL_(600),
M
Mark Callaghan 已提交
279 280 281
      stall_level0_slowdown_(0),
      stall_memtable_compaction_(0),
      stall_level0_num_files_(0),
J
Jim Paton 已提交
282 283 284
      stall_level0_slowdown_count_(0),
      stall_memtable_compaction_count_(0),
      stall_level0_num_files_count_(0),
285
      started_at_(options.env->NowMicros()),
286
      flush_on_destroy_(false),
287
      stats_(options.num_levels),
288
      delayed_writes_(0),
289 290 291
      storage_options_(options),
      bg_work_gate_closed_(false),
      refitting_level_(false) {
292

293
  mem_->Ref();
294

H
heyongqiang 已提交
295
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
296 297

  stall_leveln_slowdown_.resize(options.num_levels);
J
Jim Paton 已提交
298 299
  stall_leveln_slowdown_count_.resize(options.num_levels);
  for (int i = 0; i < options.num_levels; ++i) {
300
    stall_leveln_slowdown_[i] = 0;
J
Jim Paton 已提交
301 302
    stall_leveln_slowdown_count_[i] = 0;
  }
303

J
jorlow@chromium.org 已提交
304
  // Reserve ten files or so for other uses and give the rest to TableCache.
305
  const int table_cache_size = options_.max_open_files - 10;
306 307
  table_cache_.reset(new TableCache(dbname_, &options_,
                                    storage_options_, table_cache_size));
J
jorlow@chromium.org 已提交
308

309 310
  versions_.reset(new VersionSet(dbname_, &options_, storage_options_,
                                 table_cache_.get(), &internal_comparator_));
311

312 313
  dumpLeveldbBuildVersion(options_.info_log.get());
  options_.Dump(options_.info_log.get());
314

315
  char name[100];
316
  Status st = env_->GetHostName(name, 100L);
317
  if (st.ok()) {
318 319 320 321 322 323
    host_name_ = name;
  } else {
    Log(options_.info_log, "Can't get hostname, use localhost as host name.");
    host_name_ = "localhost";
  }
  last_log_ts = 0;
324

I
Igor Canadi 已提交
325
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
326 327 328
}

DBImpl::~DBImpl() {
329 330 331
  std::vector<MemTable*> to_delete;
  to_delete.reserve(options_.max_write_buffer_number);

J
jorlow@chromium.org 已提交
332
  // Wait for background work to finish
333
  if (flush_on_destroy_ && mem_->GetFirstSequenceNumber() != 0) {
334 335
    FlushMemTable(FlushOptions());
  }
336
  mutex_.Lock();
337
  shutting_down_.Release_Store(this);  // Any non-nullptr value is ok
338 339 340
  while (bg_compaction_scheduled_ ||
         bg_flush_scheduled_ ||
         bg_logstats_scheduled_) {
H
hans@chromium.org 已提交
341
    bg_cv_.Wait();
J
jorlow@chromium.org 已提交
342
  }
I
Igor Canadi 已提交
343 344 345 346 347 348 349
  if (super_version_ != nullptr) {
    bool is_last_reference __attribute__((unused));
    is_last_reference = super_version_->Unref();
    assert(is_last_reference);
    super_version_->Cleanup();
    delete super_version_;
  }
J
jorlow@chromium.org 已提交
350 351
  mutex_.Unlock();

352
  if (db_lock_ != nullptr) {
J
jorlow@chromium.org 已提交
353 354 355
    env_->UnlockFile(db_lock_);
  }

356 357 358 359 360 361 362 363
  if (mem_ != nullptr) {
    delete mem_->Unref();
  }

  imm_.UnrefAll(&to_delete);
  for (MemTable* m: to_delete) {
    delete m;
  }
I
Igor Canadi 已提交
364
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
365 366
}

A
Abhishek Kona 已提交
367
// Do not flush and close database elegantly. Simulate a crash.
368 369 370 371 372 373
void DBImpl::TEST_Destroy_DBImpl() {
  // ensure that no new memtable flushes can occur
  flush_on_destroy_ = false;

  // wait till all background compactions are done.
  mutex_.Lock();
374 375 376
  while (bg_compaction_scheduled_ ||
         bg_flush_scheduled_ ||
         bg_logstats_scheduled_) {
377 378
    bg_cv_.Wait();
  }
I
Igor Canadi 已提交
379 380 381 382 383 384 385
  if (super_version_ != nullptr) {
    bool is_last_reference __attribute__((unused));
    is_last_reference = super_version_->Unref();
    assert(is_last_reference);
    super_version_->Cleanup();
    delete super_version_;
  }
386 387

  // Prevent new compactions from occuring.
388
  bg_work_gate_closed_ = true;
389 390
  const int LargeNumber = 10000000;
  bg_compaction_scheduled_ += LargeNumber;
391

392
  mutex_.Unlock();
I
Igor Canadi 已提交
393
  LogFlush(options_.info_log);
394 395

  // force release the lock file.
396
  if (db_lock_ != nullptr) {
397 398
    env_->UnlockFile(db_lock_);
  }
399 400 401 402

  log_.reset();
  versions_.reset();
  table_cache_.reset();
403 404
}

A
Abhishek Kona 已提交
405 406 407
uint64_t DBImpl::TEST_Current_Manifest_FileNo() {
  return versions_->ManifestFileNumber();
}
408

J
jorlow@chromium.org 已提交
409
Status DBImpl::NewDB() {
410
  VersionEdit new_db;
J
jorlow@chromium.org 已提交
411
  new_db.SetComparatorName(user_comparator()->Name());
412
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
413 414 415 416
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

  const std::string manifest = DescriptorFileName(dbname_, 1);
417
  unique_ptr<WritableFile> file;
418
  Status s = env_->NewWritableFile(manifest, &file, storage_options_);
J
jorlow@chromium.org 已提交
419 420 421
  if (!s.ok()) {
    return s;
  }
422
  file->SetPreallocationBlockSize(options_.manifest_preallocation_size);
J
jorlow@chromium.org 已提交
423
  {
424
    log::Writer log(std::move(file));
J
jorlow@chromium.org 已提交
425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
    s = SetCurrentFile(env_, dbname_, 1);
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

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

447
const Status DBImpl::CreateArchivalDirectory() {
448
  if (options_.WAL_ttl_seconds > 0 || options_.WAL_size_limit_MB > 0) {
449
    std::string archivalPath = ArchivalDirectory(options_.wal_dir);
450 451 452 453 454
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

455
void DBImpl::PrintStatistics() {
456
  auto dbstats = options_.statistics.get();
457 458
  if (dbstats) {
    Log(options_.info_log,
459 460
        "STATISTCS:\n %s",
        dbstats->ToString().c_str());
461 462 463
  }
}

464
void DBImpl::MaybeDumpStats() {
H
Haobo Xu 已提交
465 466 467 468 469 470 471 472 473 474 475 476 477
  if (options_.stats_dump_period_sec == 0) return;

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
      options_.stats_dump_period_sec * 1000000
      <= now_micros) {
    // Multiple threads could race in here simultaneously.
    // However, the last one will update last_stats_dump_time_microsec_
    // atomically. We could see more than one dump during one dump
    // period in rare cases.
    last_stats_dump_time_microsec_ = now_micros;
    std::string stats;
478
    GetProperty("rocksdb.stats", &stats);
H
Haobo Xu 已提交
479
    Log(options_.info_log, "%s", stats.c_str());
480
    PrintStatistics();
481 482 483
  }
}

I
Igor Canadi 已提交
484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526
// DBImpl::SuperVersion methods
DBImpl::SuperVersion::SuperVersion(const int num_memtables) {
  to_delete.resize(num_memtables);
}

DBImpl::SuperVersion::~SuperVersion() {
  for (auto td : to_delete) {
    delete td;
  }
}

DBImpl::SuperVersion* DBImpl::SuperVersion::Ref() {
  refs.fetch_add(1, std::memory_order_relaxed);
  return this;
}

bool DBImpl::SuperVersion::Unref() {
  assert(refs > 0);
  // fetch_sub returns the previous value of ref
  return refs.fetch_sub(1, std::memory_order_relaxed) == 1;
}

void DBImpl::SuperVersion::Cleanup() {
  assert(refs.load(std::memory_order_relaxed) == 0);
  imm.UnrefAll(&to_delete);
  MemTable* m = mem->Unref();
  if (m != nullptr) {
    to_delete.push_back(m);
  }
  current->Unref();
}

void DBImpl::SuperVersion::Init(MemTable* new_mem, const MemTableList& new_imm,
                                Version* new_current) {
  mem = new_mem;
  imm = new_imm;
  current = new_current;
  mem->Ref();
  imm.RefAll();
  current->Ref();
  refs.store(1, std::memory_order_relaxed);
}

527 528
// Returns the list of live files in 'sst_live' and the list
// of all files in the filesystem in 'all_files'.
I
Igor Canadi 已提交
529 530 531 532 533 534 535
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
//  options_.delete_obsolete_files_period_micros
// force = true -- force the full scan
void DBImpl::FindObsoleteFiles(DeletionState& deletion_state,
                               bool force,
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
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 548 549 550 551 552 553 554 555 556 557 558
  bool doing_the_full_scan = false;

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

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

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

567 568 569 570 571 572 573 574
  if (!doing_the_full_scan && !deletion_state.HaveSomethingToDelete()) {
    // avoid filling up sst_live if we're sure that we
    // are not going to do the full scan and that we don't have
    // anything to delete at the moment
    return;
  }

  // don't delete live files
I
Igor Canadi 已提交
575 576 577 578
  deletion_state.sst_live.assign(pending_outputs_.begin(),
                                 pending_outputs_.end());
  versions_->AddLiveFiles(&deletion_state.sst_live);

579 580 581 582 583 584 585 586 587 588 589 590 591
  if (doing_the_full_scan) {
    // set of all files in the directory
    env_->GetChildren(dbname_, &deletion_state.all_files); // Ignore errors

    //Add log files in wal_dir
    if (options_.wal_dir != dbname_) {
      std::vector<std::string> log_files;
      env_->GetChildren(options_.wal_dir, &log_files); // Ignore errors
      deletion_state.all_files.insert(
        deletion_state.all_files.end(),
        log_files.begin(),
        log_files.end()
      );
592
    }
593
  }
594 595
}

D
Dhruba Borthakur 已提交
596
// Diffs the files listed in filenames and those that do not
I
Igor Canadi 已提交
597
// belong to live files are posibly removed. Also, removes all the
598
// files in sst_delete_files and log_delete_files.
599
// It is not necessary to hold the mutex when invoking this method.
D
Dhruba Borthakur 已提交
600
void DBImpl::PurgeObsoleteFiles(DeletionState& state) {
601 602 603 604 605 606 607 608

  // check if there is anything to do
  if (!state.all_files.size() &&
      !state.sst_delete_files.size() &&
      !state.log_delete_files.size()) {
    return;
  }

I
Igor Canadi 已提交
609 610 611 612
  // 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 已提交
613 614 615
    return;
  }

J
jorlow@chromium.org 已提交
616 617
  uint64_t number;
  FileType type;
H
heyongqiang 已提交
618
  std::vector<std::string> old_log_files;
619

620 621
  // Now, convert live list to an unordered set, WITHOUT mutex held;
  // set is slow.
622 623
  std::unordered_set<uint64_t> live_set(state.sst_live.begin(),
                                        state.sst_live.end());
I
Igor Canadi 已提交
624

625 626 627
  state.all_files.reserve(state.all_files.size() +
      state.sst_delete_files.size());
  for (auto file : state.sst_delete_files) {
I
Igor Canadi 已提交
628
    state.all_files.push_back(TableFileName("", file->number).substr(1));
629
    delete file;
I
Igor Canadi 已提交
630 631
  }

632 633 634
  state.all_files.reserve(state.all_files.size() +
      state.log_delete_files.size());
  for (auto filenum : state.log_delete_files) {
I
Igor Canadi 已提交
635
    if (filenum > 0) {
I
Igor Canadi 已提交
636
      state.all_files.push_back(LogFileName("", filenum).substr(1));
I
Igor Canadi 已提交
637 638
    }
  }
639

I
Igor Canadi 已提交
640 641 642 643 644 645
  // dedup state.all_files so we don't try to delete the same
  // file twice
  sort(state.all_files.begin(), state.all_files.end());
  auto unique_end = unique(state.all_files.begin(), state.all_files.end());

  for (size_t i = 0; state.all_files.begin() + i < unique_end; i++) {
646
    if (ParseFileName(state.all_files[i], &number, &type)) {
J
jorlow@chromium.org 已提交
647 648 649
      bool keep = true;
      switch (type) {
        case kLogFile:
I
Igor Canadi 已提交
650 651
          keep = ((number >= state.log_number) ||
                  (number == state.prev_log_number));
J
jorlow@chromium.org 已提交
652 653 654 655
          break;
        case kDescriptorFile:
          // Keep my manifest file, and any newer incarnations'
          // (in case there is a race that allows other incarnations)
I
Igor Canadi 已提交
656
          keep = (number >= state.manifest_file_number);
J
jorlow@chromium.org 已提交
657 658
          break;
        case kTableFile:
659
          keep = (live_set.find(number) != live_set.end());
J
jorlow@chromium.org 已提交
660 661 662 663
          break;
        case kTempFile:
          // Any temp files that are currently being written to must
          // be recorded in pending_outputs_, which is inserted into "live"
664
          keep = (live_set.find(number) != live_set.end());
J
jorlow@chromium.org 已提交
665
          break;
H
heyongqiang 已提交
666 667 668
        case kInfoLogFile:
          keep = true;
          if (number != 0) {
669
            old_log_files.push_back(state.all_files[i]);
H
heyongqiang 已提交
670 671
          }
          break;
J
jorlow@chromium.org 已提交
672 673
        case kCurrentFile:
        case kDBLockFile:
M
Mayank Agarwal 已提交
674
        case kIdentityFile:
K
Kosie van der Merwe 已提交
675
        case kMetaDatabase:
J
jorlow@chromium.org 已提交
676 677 678 679 680 681
          keep = true;
          break;
      }

      if (!keep) {
        if (type == kTableFile) {
I
Igor Canadi 已提交
682 683
          // evict from cache
          table_cache_->Evict(number);
J
jorlow@chromium.org 已提交
684
        }
I
Igor Canadi 已提交
685 686
        std::string fname = ((type == kLogFile) ? options_.wal_dir : dbname_) +
            "/" + state.all_files[i];
K
Kai Liu 已提交
687 688 689 690
        Log(options_.info_log,
            "Delete type=%d #%lu",
            int(type),
            (unsigned long)number);
691

I
Igor Canadi 已提交
692 693 694
        Status st;
        if (type == kLogFile && (options_.WAL_ttl_seconds > 0 ||
              options_.WAL_size_limit_MB > 0)) {
I
Igor Canadi 已提交
695 696
            st = env_->RenameFile(fname,
                ArchivedLogFileName(options_.wal_dir, number));
I
Igor Canadi 已提交
697
            if (!st.ok()) {
K
Kai Liu 已提交
698
              Log(options_.info_log,
I
Igor Canadi 已提交
699 700
                  "RenameFile logfile #%lu FAILED -- %s\n",
                  (unsigned long)number, st.ToString().c_str());
I
Igor Canadi 已提交
701
            }
702
        } else {
I
Igor Canadi 已提交
703
          st = env_->DeleteFile(fname);
704
          if (!st.ok()) {
I
Igor Canadi 已提交
705 706
            Log(options_.info_log, "Delete type=%d #%lu FAILED -- %s\n",
                int(type), (unsigned long)number, st.ToString().c_str());
707
          }
H
heyongqiang 已提交
708
        }
J
jorlow@chromium.org 已提交
709 710 711
      }
    }
  }
H
heyongqiang 已提交
712

713
  // Delete old info log files.
K
Kai Liu 已提交
714 715 716 717 718
  size_t old_log_file_count = old_log_files.size();
  // NOTE: Currently we only support log purge when options_.db_log_dir is
  // located in `dbname` directory.
  if (old_log_file_count >= options_.keep_log_file_num &&
      options_.db_log_dir.empty()) {
H
heyongqiang 已提交
719
    std::sort(old_log_files.begin(), old_log_files.end());
K
Kai Liu 已提交
720
    size_t end = old_log_file_count - options_.keep_log_file_num;
721
    for (unsigned int i = 0; i <= end; i++) {
H
heyongqiang 已提交
722
      std::string& to_delete = old_log_files.at(i);
D
Dhruba Borthakur 已提交
723 724
      // Log(options_.info_log, "Delete type=%d %s\n",
      //     int(kInfoLogFile), to_delete.c_str());
H
heyongqiang 已提交
725 726 727
      env_->DeleteFile(dbname_ + "/" + to_delete);
    }
  }
728
  PurgeObsoleteWALFiles();
I
Igor Canadi 已提交
729
  LogFlush(options_.info_log);
D
Dhruba Borthakur 已提交
730 731 732 733 734
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
  DeletionState deletion_state;
I
Igor Canadi 已提交
735
  FindObsoleteFiles(deletion_state, true);
D
Dhruba Borthakur 已提交
736
  PurgeObsoleteFiles(deletion_state);
737 738
}

739 740 741 742 743 744 745 746
// 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
747
void DBImpl::PurgeObsoleteWALFiles() {
748 749 750 751 752 753
  bool const ttl_enabled = options_.WAL_ttl_seconds > 0;
  bool const size_limit_enabled =  options_.WAL_size_limit_MB > 0;
  if (!ttl_enabled && !size_limit_enabled) {
    return;
  }

754 755
  int64_t current_time;
  Status s = env_->GetCurrentTime(&current_time);
756 757 758 759 760 761 762 763
  if (!s.ok()) {
    Log(options_.info_log, "Can't get current time: %s", s.ToString().c_str());
    assert(false);
    return;
  }
  uint64_t const now_seconds = static_cast<uint64_t>(current_time);
  uint64_t const time_to_check = (ttl_enabled && !size_limit_enabled) ?
    options_.WAL_ttl_seconds / 2 : default_interval_to_delete_obsolete_WAL_;
764

765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804
  if (purge_wal_files_last_run_ + time_to_check > now_seconds) {
    return;
  }

  purge_wal_files_last_run_ = now_seconds;

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

  size_t log_files_num = 0;
  uint64_t log_file_size = 0;

  for (auto& f : files) {
    uint64_t number;
    FileType type;
    if (ParseFileName(f, &number, &type) && type == kLogFile) {
      std::string const file_path = archival_dir + "/" + f;
      if (ttl_enabled) {
        uint64_t file_m_time;
        Status const s = env_->GetFileModificationTime(file_path,
          &file_m_time);
        if (!s.ok()) {
          Log(options_.info_log, "Can't get file mod time: %s: %s",
              file_path.c_str(), s.ToString().c_str());
          continue;
        }
        if (now_seconds - file_m_time > options_.WAL_ttl_seconds) {
          Status const s = env_->DeleteFile(file_path);
          if (!s.ok()) {
            Log(options_.info_log, "Can't delete file: %s: %s",
                file_path.c_str(), s.ToString().c_str());
            continue;
          }
          continue;
805
        }
806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858
      }

      if (size_limit_enabled) {
        uint64_t file_size;
        Status const s = env_->GetFileSize(file_path, &file_size);
        if (!s.ok()) {
          Log(options_.info_log, "Can't get file size: %s: %s",
              file_path.c_str(), s.ToString().c_str());
          return;
        } else {
          if (file_size > 0) {
            log_file_size = std::max(log_file_size, file_size);
            ++log_files_num;
          } else {
            Status s = env_->DeleteFile(file_path);
            if (!s.ok()) {
              Log(options_.info_log, "Can't delete file: %s: %s",
                  file_path.c_str(), s.ToString().c_str());
              continue;
            }
          }
        }
      }
    }
  }

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

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

  size_t files_del_num = log_files_num - files_keep_num;
  VectorLogPtr archived_logs;
  AppendSortedWalsOfType(archival_dir, archived_logs, kArchivedLogFile);

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

  for (size_t i = 0; i < files_del_num; ++i) {
    std::string const file_path = archived_logs[i]->PathName();
    Status const s = DeleteFile(file_path);
    if (!s.ok()) {
      Log(options_.info_log, "Can't delete file: %s: %s",
          file_path.c_str(), s.ToString().c_str());
      continue;
859 860
    }
  }
D
Dhruba Borthakur 已提交
861 862
}

863 864 865
// If externalTable is set, then apply recovered transactions
// to that table. This is used for readonly mode.
Status DBImpl::Recover(VersionEdit* edit, MemTable* external_table,
866
                       bool error_if_log_file_exist) {
J
jorlow@chromium.org 已提交
867 868
  mutex_.AssertHeld();

869
  assert(db_lock_ == nullptr);
870
  if (!external_table) {
871 872 873 874 875 876 877 878 879 880 881 882 883
    // 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;
    }

    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
884 885 886
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
887

888 889
    if (!env_->FileExists(CurrentFileName(dbname_))) {
      if (options_.create_if_missing) {
890
        // TODO: add merge_operator name check
891 892 893 894 895 896 897
        s = NewDB();
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
898 899
      }
    } else {
900 901 902 903
      if (options_.error_if_exists) {
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
904
    }
M
Mayank Agarwal 已提交
905 906 907 908 909 910 911
    // 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 已提交
912 913
  }

914
  Status s = versions_->Recover();
J
jorlow@chromium.org 已提交
915 916
  if (s.ok()) {
    SequenceNumber max_sequence(0);
917 918 919 920 921 922 923

    // 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
924
    // produced by an older version of rocksdb.
925 926 927
    const uint64_t min_log = versions_->LogNumber();
    const uint64_t prev_log = versions_->PrevLogNumber();
    std::vector<std::string> filenames;
928
    s = env_->GetChildren(options_.wal_dir, &filenames);
929 930
    if (!s.ok()) {
      return s;
931
    }
932 933 934 935 936 937 938 939 940
    uint64_t number;
    FileType type;
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
      if (ParseFileName(filenames[i], &number, &type)
          && type == kLogFile
          && ((number >= min_log) || (number == prev_log))) {
        logs.push_back(number);
      }
J
jorlow@chromium.org 已提交
941
    }
942

H
heyongqiang 已提交
943 944 945 946 947 948
    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");
    }

949 950 951
    // Recover in the order in which the logs were generated
    std::sort(logs.begin(), logs.end());
    for (size_t i = 0; i < logs.size(); i++) {
952
      s = RecoverLogFile(logs[i], edit, &max_sequence, external_table);
953 954 955 956
      // 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(logs[i]);
957 958
    }

J
jorlow@chromium.org 已提交
959
    if (s.ok()) {
960 961 962
      if (versions_->LastSequence() < max_sequence) {
        versions_->SetLastSequence(max_sequence);
      }
963
      SetTickerCount(options_.statistics.get(), SEQUENCE_NUMBER,
964
                     versions_->LastSequence());
J
jorlow@chromium.org 已提交
965 966 967 968 969 970 971 972
    }
  }

  return s;
}

Status DBImpl::RecoverLogFile(uint64_t log_number,
                              VersionEdit* edit,
973 974
                              SequenceNumber* max_sequence,
                              MemTable* external_table) {
J
jorlow@chromium.org 已提交
975 976
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
977
    Logger* info_log;
J
jorlow@chromium.org 已提交
978
    const char* fname;
979 980
    Status* status;  // nullptr if options_.paranoid_checks==false or
                     //            options_.skip_log_error_on_recovery==true
J
jorlow@chromium.org 已提交
981
    virtual void Corruption(size_t bytes, const Status& s) {
982
      Log(info_log, "%s%s: dropping %d bytes; %s",
983
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
984
          fname, static_cast<int>(bytes), s.ToString().c_str());
985
      if (this->status != nullptr && this->status->ok()) *this->status = s;
J
jorlow@chromium.org 已提交
986 987 988 989 990 991
    }
  };

  mutex_.AssertHeld();

  // Open the log file
992
  std::string fname = LogFileName(options_.wal_dir, log_number);
993
  unique_ptr<SequentialFile> file;
994
  Status status = env_->NewSequentialFile(fname, &file, storage_options_);
J
jorlow@chromium.org 已提交
995 996 997 998 999 1000 1001 1002
  if (!status.ok()) {
    MaybeIgnoreError(&status);
    return status;
  }

  // Create the log reader.
  LogReporter reporter;
  reporter.env = env_;
1003
  reporter.info_log = options_.info_log.get();
J
jorlow@chromium.org 已提交
1004
  reporter.fname = fname.c_str();
1005 1006
  reporter.status = (options_.paranoid_checks &&
                     !options_.skip_log_error_on_recovery ? &status : nullptr);
J
jorlow@chromium.org 已提交
1007 1008 1009 1010
  // 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).
1011
  log::Reader reader(std::move(file), &reporter, true/*checksum*/,
1012
                     0/*initial_offset*/);
K
Kai Liu 已提交
1013 1014
  Log(options_.info_log, "Recovering log #%lu",
      (unsigned long) log_number);
J
jorlow@chromium.org 已提交
1015 1016 1017 1018 1019

  // Read all the records and add to a memtable
  std::string scratch;
  Slice record;
  WriteBatch batch;
1020
  MemTable* mem = nullptr;
1021 1022 1023
  if (external_table) {
    mem = external_table;
  }
1024
  while (reader.ReadRecord(&record, &scratch) && status.ok()) {
J
jorlow@chromium.org 已提交
1025 1026 1027 1028 1029 1030 1031
    if (record.size() < 12) {
      reporter.Corruption(
          record.size(), Status::Corruption("log record too small"));
      continue;
    }
    WriteBatchInternal::SetContents(&batch, record);

1032
    if (mem == nullptr) {
1033
      mem = new MemTable(internal_comparator_, options_);
1034
      mem->Ref();
J
jorlow@chromium.org 已提交
1035
    }
1036
    status = WriteBatchInternal::InsertInto(&batch, mem, &options_);
J
jorlow@chromium.org 已提交
1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047
    MaybeIgnoreError(&status);
    if (!status.ok()) {
      break;
    }
    const SequenceNumber last_seq =
        WriteBatchInternal::Sequence(&batch) +
        WriteBatchInternal::Count(&batch) - 1;
    if (last_seq > *max_sequence) {
      *max_sequence = last_seq;
    }

1048 1049
    if (!external_table &&
        mem->ApproximateMemoryUsage() > options_.write_buffer_size) {
1050
      status = WriteLevel0TableForRecovery(mem, edit);
J
jorlow@chromium.org 已提交
1051 1052 1053 1054 1055
      if (!status.ok()) {
        // Reflect errors immediately so that conditions like full
        // file-systems cause the DB::Open() to fail.
        break;
      }
1056
      delete mem->Unref();
1057
      mem = nullptr;
J
jorlow@chromium.org 已提交
1058 1059 1060
    }
  }

1061
  if (status.ok() && mem != nullptr && !external_table) {
1062
    status = WriteLevel0TableForRecovery(mem, edit);
J
jorlow@chromium.org 已提交
1063 1064 1065 1066
    // Reflect errors immediately so that conditions like full
    // file-systems cause the DB::Open() to fail.
  }

1067 1068 1069
  if (mem != nullptr && !external_table) {
    delete mem->Unref();
  }
J
jorlow@chromium.org 已提交
1070 1071 1072
  return status;
}

1073
Status DBImpl::WriteLevel0TableForRecovery(MemTable* mem, VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1074
  mutex_.AssertHeld();
1075
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1076 1077 1078 1079
  FileMetaData meta;
  meta.number = versions_->NewFileNumber();
  pending_outputs_.insert(meta.number);
  Iterator* iter = mem->NewIterator();
1080 1081 1082
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
    mem->GetFirstSequenceNumber();
K
Kai Liu 已提交
1083 1084
  Log(options_.info_log, "Level-0 table #%lu: started",
      (unsigned long) meta.number);
1085 1086 1087 1088

  Status s;
  {
    mutex_.Unlock();
1089 1090
    s = BuildTable(dbname_, env_, options_, storage_options_,
                   table_cache_.get(), iter, &meta,
1091
                   user_comparator(), newest_snapshot,
1092 1093
                   earliest_seqno_in_memtable,
                   GetCompressionFlush(options_));
I
Igor Canadi 已提交
1094
    LogFlush(options_.info_log);
1095 1096 1097
    mutex_.Lock();
  }

K
Kai Liu 已提交
1098 1099 1100
  Log(options_.info_log, "Level-0 table #%lu: %lu bytes %s",
      (unsigned long) meta.number,
      (unsigned long) meta.file_size,
J
jorlow@chromium.org 已提交
1101 1102
      s.ToString().c_str());
  delete iter;
1103

1104
  pending_outputs_.erase(meta.number);
1105 1106 1107 1108 1109 1110

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

1115 1116 1117
  CompactionStats stats;
  stats.micros = env_->NowMicros() - start_micros;
  stats.bytes_written = meta.file_size;
M
Mark Callaghan 已提交
1118
  stats.files_out_levelnp1 = 1;
1119
  stats_[level].Add(stats);
1120
  RecordTick(options_.statistics.get(), COMPACT_WRITE_BYTES, meta.file_size);
J
jorlow@chromium.org 已提交
1121 1122 1123
  return s;
}

1124

1125
Status DBImpl::WriteLevel0Table(std::vector<MemTable*> &mems, VersionEdit* edit,
1126
                                uint64_t* filenumber) {
J
jorlow@chromium.org 已提交
1127
  mutex_.AssertHeld();
1128 1129 1130 1131 1132
  const uint64_t start_micros = env_->NowMicros();
  FileMetaData meta;
  meta.number = versions_->NewFileNumber();
  *filenumber = meta.number;
  pending_outputs_.insert(meta.number);
1133

1134 1135
  const SequenceNumber newest_snapshot = snapshots_.GetNewest();
  const SequenceNumber earliest_seqno_in_memtable =
1136
    mems[0]->GetFirstSequenceNumber();
1137
  Version* base = versions_->current();
1138
  base->Ref();          // it is likely that we do not need this reference
1139 1140 1141
  Status s;
  {
    mutex_.Unlock();
1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153
    std::vector<Iterator*> list;
    for (MemTable* m : mems) {
      Log(options_.info_log,
          "Flushing memtable with log file: %lu\n",
          (unsigned long)m->GetLogNumber());
      list.push_back(m->NewIterator());
    }
    Iterator* iter = NewMergingIterator(&internal_comparator_, &list[0],
                                        list.size());
    Log(options_.info_log,
        "Level-0 flush table #%lu: started",
        (unsigned long)meta.number);
1154

1155 1156
    s = BuildTable(dbname_, env_, options_, storage_options_,
                   table_cache_.get(), iter, &meta,
1157
                   user_comparator(), newest_snapshot,
1158
                   earliest_seqno_in_memtable, GetCompressionFlush(options_));
I
Igor Canadi 已提交
1159
    LogFlush(options_.info_log);
1160 1161 1162 1163 1164
    delete iter;
    Log(options_.info_log, "Level-0 flush table #%lu: %lu bytes %s",
        (unsigned long) meta.number,
        (unsigned long) meta.file_size,
        s.ToString().c_str());
1165 1166
    mutex_.Lock();
  }
1167 1168
  base->Unref();

1169 1170 1171 1172 1173 1174 1175 1176 1177

  // re-acquire the most current version
  base = versions_->current();

  // 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 已提交
1178
  // committed so that other threads can recognize this file as a
1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191
  // valid one.
  // pending_outputs_.erase(meta.number);

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
  if (s.ok() && meta.file_size > 0) {
    const Slice min_user_key = meta.smallest.user_key();
    const Slice max_user_key = meta.largest.user_key();
    // if we have more than 1 background thread, then we cannot
    // insert files directly into higher levels because some other
    // threads could be concurrently producing compacted files for
    // that key range.
1192
    if (base != nullptr && options_.max_background_compactions <= 1 &&
1193
        options_.compaction_style == kCompactionStyleLevel) {
1194 1195 1196
      level = base->PickLevelForMemTableOutput(min_user_key, max_user_key);
    }
    edit->AddFile(level, meta.number, meta.file_size,
1197 1198
                  meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1199 1200 1201 1202 1203 1204
  }

  CompactionStats stats;
  stats.micros = env_->NowMicros() - start_micros;
  stats.bytes_written = meta.file_size;
  stats_[level].Add(stats);
1205
  RecordTick(options_.statistics.get(), COMPACT_WRITE_BYTES, meta.file_size);
1206 1207 1208
  return s;
}

I
Igor Canadi 已提交
1209 1210
Status DBImpl::FlushMemTableToOutputFile(bool* madeProgress,
                                         DeletionState& deletion_state) {
1211 1212 1213
  mutex_.AssertHeld();
  assert(imm_.size() != 0);

1214
  if (!imm_.IsFlushPending(options_.min_write_buffer_number_to_merge)) {
1215 1216
    Log(options_.info_log, "FlushMemTableToOutputFile already in progress");
    Status s = Status::IOError("FlushMemTableToOutputFile already in progress");
1217 1218 1219 1220 1221
    return s;
  }

  // Save the contents of the earliest memtable as a new Table
  uint64_t file_number;
1222 1223 1224
  std::vector<MemTable*> mems;
  imm_.PickMemtablesToFlush(&mems);
  if (mems.empty()) {
1225 1226 1227 1228 1229 1230
    Log(options_.info_log, "Nothing in memstore to flush");
    Status s = Status::IOError("Nothing in memstore to flush");
    return s;
  }

  // record the logfile_number_ before we release the mutex
1231 1232 1233
  // 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.
1234
  MemTable* m = mems[0];
1235 1236
  VersionEdit* edit = m->GetEdits();
  edit->SetPrevLogNumber(0);
1237 1238 1239 1240 1241 1242 1243 1244 1245 1246
  // SetLogNumber(log_num) indicates logs with number smaller than log_num
  // will no longer be picked up for recovery.
  edit->SetLogNumber(
      mems.back()->GetNextLogNumber()
  );

  std::vector<uint64_t> logs_to_delete;
  for (auto mem : mems) {
    logs_to_delete.push_back(mem->GetLogNumber());
  }
1247

1248
  // This will release and re-acquire the mutex.
1249
  Status s = WriteLevel0Table(mems, edit, &file_number);
1250

1251
  if (s.ok() && shutting_down_.Acquire_Load()) {
1252 1253 1254
    s = Status::IOError(
      "Database shutdown started during memtable compaction"
    );
1255
  }
J
jorlow@chromium.org 已提交
1256

1257
  // Replace immutable memtable with the generated Table
1258
  s = imm_.InstallMemtableFlushResults(
1259
    mems, versions_.get(), s, &mutex_, options_.info_log.get(),
1260
    file_number, pending_outputs_, &deletion_state.memtables_to_free);
J
jorlow@chromium.org 已提交
1261 1262

  if (s.ok()) {
I
Igor Canadi 已提交
1263
    InstallSuperVersion(deletion_state);
1264 1265 1266
    if (madeProgress) {
      *madeProgress = 1;
    }
1267

1268
    MaybeScheduleLogDBDeployStats();
I
Igor Canadi 已提交
1269

1270
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1271
      // add to deletion state
1272 1273 1274 1275
      deletion_state.log_delete_files.insert(
          deletion_state.log_delete_files.end(),
          logs_to_delete.begin(),
          logs_to_delete.end());
1276
    }
J
jorlow@chromium.org 已提交
1277 1278 1279 1280
  }
  return s;
}

1281 1282 1283 1284 1285
void DBImpl::CompactRange(const Slice* begin,
                          const Slice* end,
                          bool reduce_level,
                          int target_level) {
  FlushMemTable(FlushOptions());
G
Gabor Cselle 已提交
1286 1287 1288 1289
  int max_level_with_files = 1;
  {
    MutexLock l(&mutex_);
    Version* base = versions_->current();
1290
    for (int level = 1; level < NumberLevels(); level++) {
G
Gabor Cselle 已提交
1291 1292 1293 1294 1295
      if (base->OverlapInLevel(level, begin, end)) {
        max_level_with_files = level;
      }
    }
  }
1296 1297 1298 1299 1300 1301 1302 1303 1304
  for (int level = 0; level <= max_level_with_files; level++) {
    // in case the compaction is unversal or if we're compacting the
    // bottom-most level, the output level will be the same as input one
    if (options_.compaction_style == kCompactionStyleUniversal ||
        level == max_level_with_files) {
      RunManualCompaction(level, level, begin, end);
    } else {
      RunManualCompaction(level, level + 1, begin, end);
    }
G
Gabor Cselle 已提交
1305
  }
1306 1307

  if (reduce_level) {
1308
    ReFitLevel(max_level_with_files, target_level);
1309
  }
I
Igor Canadi 已提交
1310
  LogFlush(options_.info_log);
1311 1312 1313 1314 1315
}

// return the same level if it cannot be moved
int DBImpl::FindMinimumEmptyLevelFitting(int level) {
  mutex_.AssertHeld();
1316
  Version* current = versions_->current();
1317
  int minimum_level = level;
1318
  for (int i = level - 1; i > 0; --i) {
1319
    // stop if level i is not empty
1320
    if (current->NumLevelFiles(i) > 0) break;
1321
    // stop if level i is too small (cannot fit the level files)
1322
    if (versions_->MaxBytesForLevel(i) < current->NumLevelBytes(level)) break;
1323 1324 1325 1326 1327 1328

    minimum_level = i;
  }
  return minimum_level;
}

1329
void DBImpl::ReFitLevel(int level, int target_level) {
1330 1331
  assert(level < NumberLevels());

I
Igor Canadi 已提交
1332 1333 1334 1335 1336
  SuperVersion* superversion_to_free = nullptr;
  SuperVersion* new_superversion =
      new SuperVersion(options_.max_write_buffer_number);

  mutex_.Lock();
1337 1338 1339

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1340
    mutex_.Unlock();
1341
    Log(options_.info_log, "ReFitLevel: another thread is refitting");
I
Igor Canadi 已提交
1342
    delete new_superversion;
1343 1344 1345 1346 1347 1348
    return;
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1349
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1350
    Log(options_.info_log,
1351 1352
        "RefitLevel: waiting for background threads to stop: %d %d",
        bg_compaction_scheduled_, bg_flush_scheduled_);
1353 1354 1355 1356
    bg_cv_.Wait();
  }

  // move to a smaller level
1357 1358 1359 1360
  int to_level = target_level;
  if (target_level < 0) {
    to_level = FindMinimumEmptyLevelFitting(level);
  }
1361 1362 1363 1364 1365 1366 1367

  assert(to_level <= level);

  if (to_level < level) {
    Log(options_.info_log, "Before refitting:\n%s",
        versions_->current()->DebugString().data());

1368
    VersionEdit edit;
1369 1370
    for (const auto& f : versions_->current()->files_[level]) {
      edit.DeleteFile(level, f->number);
1371 1372
      edit.AddFile(to_level, f->number, f->file_size, f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1373 1374 1375 1376 1377
    }
    Log(options_.info_log, "Apply version edit:\n%s",
        edit.DebugString().data());

    auto status = versions_->LogAndApply(&edit, &mutex_);
I
Igor Canadi 已提交
1378 1379
    superversion_to_free = InstallSuperVersion(new_superversion);
    new_superversion = nullptr;
1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390

    Log(options_.info_log, "LogAndApply: %s\n", status.ToString().data());

    if (status.ok()) {
      Log(options_.info_log, "After refitting:\n%s",
          versions_->current()->DebugString().data());
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1391 1392 1393 1394

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
G
Gabor Cselle 已提交
1395 1396
}

1397
int DBImpl::NumberLevels() {
1398
  return options_.num_levels;
1399 1400 1401
}

int DBImpl::MaxMemCompactionLevel() {
1402
  return options_.max_mem_compaction_level;
1403 1404 1405
}

int DBImpl::Level0StopWriteTrigger() {
1406
  return options_.level0_stop_writes_trigger;
1407 1408
}

H
heyongqiang 已提交
1409 1410 1411 1412 1413
Status DBImpl::Flush(const FlushOptions& options) {
  Status status = FlushMemTable(options);
  return status;
}

1414
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1415 1416 1417
  return versions_->LastSequence();
}

1418
Status DBImpl::GetUpdatesSince(SequenceNumber seq,
1419
                               unique_ptr<TransactionLogIterator>* iter) {
1420

1421
  RecordTick(options_.statistics.get(), GET_UPDATES_SINCE_CALLS);
1422
  if (seq > versions_->LastSequence()) {
1423 1424 1425
    return Status::IOError("Requested sequence not yet written in the db");
  }
  //  Get all sorted Wal Files.
1426 1427
  //  Do binary search and open files and find the seq number.

1428 1429
  std::unique_ptr<VectorLogPtr> wal_files(new VectorLogPtr);
  Status s = GetSortedWalFiles(*wal_files);
1430 1431 1432 1433
  if (!s.ok()) {
    return s;
  }

1434
  s = RetainProbableWalFiles(*wal_files, seq);
1435 1436
  if (!s.ok()) {
    return s;
1437
  }
1438
  iter->reset(
1439
    new TransactionLogIteratorImpl(options_.wal_dir,
1440
                                   &options_,
1441
                                   storage_options_,
1442
                                   seq,
1443
                                   std::move(wal_files),
1444
                                   this));
1445
  return (*iter)->status();
1446 1447
}

1448 1449
Status DBImpl::RetainProbableWalFiles(VectorLogPtr& all_logs,
                                      const SequenceNumber target) {
1450
  long start = 0; // signed to avoid overflow when target is < first file.
1451
  long end = static_cast<long>(all_logs.size()) - 1;
1452
  // Binary Search. avoid opening all files.
1453 1454
  while (end >= start) {
    long mid = start + (end - start) / 2;  // Avoid overflow.
1455 1456
    SequenceNumber current_seq_num = all_logs.at(mid)->StartSequence();
    if (current_seq_num == target) {
1457
      end = mid;
1458
      break;
1459
    } else if (current_seq_num < target) {
1460
      start = mid + 1;
1461
    } else {
1462
      end = mid - 1;
1463 1464
    }
  }
1465 1466 1467
  size_t start_index = std::max(0l, end); // end could be -ve.
  // The last wal file is always included
  all_logs.erase(all_logs.begin(), all_logs.begin() + start_index);
1468 1469 1470
  return Status::OK();
}

1471 1472 1473
bool DBImpl::CheckWalFileExistsAndEmpty(const WalFileType type,
                                        const uint64_t number) {
  const std::string fname = (type == kAliveLogFile) ?
1474 1475
    LogFileName(options_.wal_dir, number) :
    ArchivedLogFileName(options_.wal_dir, number);
1476 1477
  uint64_t file_size;
  Status s = env_->GetFileSize(fname, &file_size);
1478
  return (s.ok() && (file_size == 0));
1479 1480
}

1481 1482
Status DBImpl::ReadFirstRecord(const WalFileType type, const uint64_t number,
                               WriteBatch* const result) {
1483

1484
  if (type == kAliveLogFile) {
1485
    std::string fname = LogFileName(options_.wal_dir, number);
1486 1487 1488
    Status status = ReadFirstLine(fname, result);
    if (!status.ok()) {
      //  check if the file got moved to archive.
1489 1490
      std::string archived_file =
        ArchivedLogFileName(options_.wal_dir, number);
1491
      Status s = ReadFirstLine(archived_file, result);
1492
      if (!s.ok()) {
1493
        return Status::IOError("Log File has been deleted: " + archived_file);
1494 1495 1496
      }
    }
    return Status::OK();
1497
  } else if (type == kArchivedLogFile) {
1498
    std::string fname = ArchivedLogFileName(options_.wal_dir, number);
1499 1500 1501
    Status status = ReadFirstLine(fname, result);
    return status;
  }
1502
  return Status::NotSupported("File Type Not Known: " + std::to_string(type));
1503 1504 1505 1506 1507 1508 1509 1510
}

Status DBImpl::ReadFirstLine(const std::string& fname,
                             WriteBatch* const batch) {
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
    Logger* info_log;
    const char* fname;
1511
    Status* status;  // nullptr if options_.paranoid_checks==false
1512 1513
    virtual void Corruption(size_t bytes, const Status& s) {
      Log(info_log, "%s%s: dropping %d bytes; %s",
1514
          (this->status == nullptr ? "(ignoring error) " : ""),
1515
          fname, static_cast<int>(bytes), s.ToString().c_str());
1516
      if (this->status != nullptr && this->status->ok()) *this->status = s;
1517 1518 1519
    }
  };

1520
  unique_ptr<SequentialFile> file;
1521
  Status status = env_->NewSequentialFile(fname, &file, storage_options_);
1522 1523 1524 1525 1526 1527 1528 1529

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


  LogReporter reporter;
  reporter.env = env_;
1530
  reporter.info_log = options_.info_log.get();
1531
  reporter.fname = fname.c_str();
1532
  reporter.status = (options_.paranoid_checks ? &status : nullptr);
1533
  log::Reader reader(std::move(file), &reporter, true/*checksum*/,
1534 1535 1536
                     0/*initial_offset*/);
  std::string scratch;
  Slice record;
1537

1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550
  if (reader.ReadRecord(&record, &scratch) && status.ok()) {
    if (record.size() < 12) {
      reporter.Corruption(
          record.size(), Status::Corruption("log record too small"));
      return Status::IOError("Corruption noted");
      //  TODO read record's till the first no corrupt entry?
    }
    WriteBatchInternal::SetContents(batch, record);
    return Status::OK();
  }
  return Status::IOError("Error reading from file " + fname);
}

1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563
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;
  }
};

Status DBImpl::AppendSortedWalsOfType(const std::string& path,
    VectorLogPtr& log_files, WalFileType log_type) {
  std::vector<std::string> all_files;
  const Status status = env_->GetChildren(path, &all_files);
1564 1565 1566
  if (!status.ok()) {
    return status;
  }
1567
  log_files.reserve(log_files.size() + all_files.size());
1568 1569 1570 1571 1572 1573
  VectorLogPtr::iterator pos_start;
  if (!log_files.empty()) {
    pos_start = log_files.end() - 1;
  } else {
    pos_start = log_files.begin();
  }
1574
  for (const auto& f : all_files) {
1575 1576
    uint64_t number;
    FileType type;
1577
    if (ParseFileName(f, &number, &type) && type == kLogFile){
1578 1579 1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593 1594 1595

      WriteBatch batch;
      Status s = ReadFirstRecord(log_type, number, &batch);
      if (!s.ok()) {
        if (CheckWalFileExistsAndEmpty(log_type, number)) {
          continue;
        }
        return s;
      }

      uint64_t size_bytes;
      s = env_->GetFileSize(LogFileName(path, number), &size_bytes);
      if (!s.ok()) {
        return s;
      }

      log_files.push_back(std::move(unique_ptr<LogFile>(new LogFileImpl(
        number, log_type, WriteBatchInternal::Sequence(&batch), size_bytes))));
1596 1597
    }
  }
1598
  CompareLogByPointer compare_log_files;
1599
  std::sort(pos_start, log_files.end(), compare_log_files);
1600 1601 1602
  return status;
}

1603 1604 1605 1606 1607
void DBImpl::RunManualCompaction(int input_level,
                                 int output_level,
                                 const Slice* begin,
                                 const Slice* end) {
  assert(input_level >= 0);
1608

G
Gabor Cselle 已提交
1609 1610
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1611
  ManualCompaction manual;
1612 1613
  manual.input_level = input_level;
  manual.output_level = output_level;
G
Gabor Cselle 已提交
1614
  manual.done = false;
1615
  manual.in_progress = false;
1616 1617 1618 1619
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
      options_.compaction_style == kCompactionStyleUniversal) {
1620
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1621 1622 1623 1624
  } else {
    begin_storage = InternalKey(*begin, kMaxSequenceNumber, kValueTypeForSeek);
    manual.begin = &begin_storage;
  }
1625 1626
  if (end == nullptr ||
      options_.compaction_style == kCompactionStyleUniversal) {
1627
    manual.end = nullptr;
G
Gabor Cselle 已提交
1628 1629 1630 1631 1632 1633
  } else {
    end_storage = InternalKey(*end, 0, static_cast<ValueType>(0));
    manual.end = &end_storage;
  }

  MutexLock l(&mutex_);
1634

1635 1636 1637 1638 1639 1640
  // 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
1641
  // RunManualCompaction(), i.e. during that time no other compaction will
1642 1643 1644
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1645
  // RunManualCompaction() from getting to the second while loop below.
1646 1647 1648 1649 1650 1651 1652 1653
  // However, only one of them will actually schedule compaction, while
  // others will wait on a condition variable until it completes.

  ++bg_manual_only_;
  while (bg_compaction_scheduled_ > 0) {
    Log(options_.info_log,
        "Manual compaction waiting for all other scheduled background "
        "compactions to finish");
1654 1655
    bg_cv_.Wait();
  }
1656

1657 1658
  Log(options_.info_log, "Manual compaction starting");

1659 1660 1661 1662
  while (!manual.done && !shutting_down_.Acquire_Load() && bg_error_.ok()) {
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
1663
      bg_cv_.Wait();
1664 1665 1666
    } else {
      manual_compaction_ = &manual;
      MaybeScheduleFlushOrCompaction();
G
Gabor Cselle 已提交
1667
    }
H
hans@chromium.org 已提交
1668
  }
1669

1670 1671 1672
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
J
jorlow@chromium.org 已提交
1673 1674
}

1675 1676 1677 1678 1679 1680 1681 1682 1683
void DBImpl::TEST_CompactRange(int level,
                               const Slice* begin,
                               const Slice* end) {
  int output_level = (options_.compaction_style == kCompactionStyleUniversal)
                         ? level
                         : level + 1;
  RunManualCompaction(level, output_level, begin, end);
}

H
heyongqiang 已提交
1684
Status DBImpl::FlushMemTable(const FlushOptions& options) {
1685 1686
  // nullptr batch means just wait for earlier writes to be done
  Status s = Write(WriteOptions(), nullptr);
H
heyongqiang 已提交
1687
  if (s.ok() && options.wait) {
1688
    // Wait until the compaction completes
1689
    s = WaitForFlushMemTable();
1690 1691
  }
  return s;
J
jorlow@chromium.org 已提交
1692 1693
}

1694
Status DBImpl::WaitForFlushMemTable() {
1695 1696 1697
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1698
  while (imm_.size() > 0 && bg_error_.ok()) {
1699 1700
    bg_cv_.Wait();
  }
1701
  if (imm_.size() != 0) {
1702 1703 1704
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1705 1706
}

1707
Status DBImpl::TEST_FlushMemTable() {
H
heyongqiang 已提交
1708 1709 1710
  return FlushMemTable(FlushOptions());
}

1711 1712
Status DBImpl::TEST_WaitForFlushMemTable() {
  return WaitForFlushMemTable();
1713 1714 1715
}

Status DBImpl::TEST_WaitForCompact() {
1716
  // Wait until the compaction completes
1717 1718 1719 1720 1721

  // TODO: a bug here. This function actually does not necessarily
  // wait for compact. It actually waits for scheduled compaction
  // OR flush to finish.

1722
  MutexLock l(&mutex_);
1723 1724
  while ((bg_compaction_scheduled_ || bg_flush_scheduled_) &&
         bg_error_.ok()) {
1725 1726 1727
    bg_cv_.Wait();
  }
  return bg_error_;
1728 1729
}

1730
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1731
  mutex_.AssertHeld();
1732 1733
  if (bg_work_gate_closed_) {
    // gate closed for backgrond work
J
jorlow@chromium.org 已提交
1734 1735 1736
  } else if (shutting_down_.Acquire_Load()) {
    // DB is being deleted; no more background compactions
  } else {
1737 1738 1739 1740 1741 1742 1743 1744 1745
    bool is_flush_pending =
      imm_.IsFlushPending(options_.min_write_buffer_number_to_merge);
    if (is_flush_pending &&
        (bg_flush_scheduled_ < options_.max_background_flushes)) {
      // memtable flush needed
      bg_flush_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH);
    }

1746 1747 1748 1749
    // Schedule BGWorkCompaction if there's a compaction pending (or a memtable
    // flush, but the HIGH pool is not enabled). Do it only if
    // max_background_compactions hasn't been reached and, in case
    // bg_manual_only_ > 0, if it's a manual compaction.
1750 1751 1752
    if ((manual_compaction_ ||
         versions_->NeedsCompaction() ||
         (is_flush_pending && (options_.max_background_flushes <= 0))) &&
1753 1754 1755
        bg_compaction_scheduled_ < options_.max_background_compactions &&
        (!bg_manual_only_ || manual_compaction_)) {

1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769
      bg_compaction_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
    }
  }
}

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

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

I
Igor Canadi 已提交
1770 1771
Status DBImpl::BackgroundFlush(bool* madeProgress,
                               DeletionState& deletion_state) {
1772 1773 1774 1775
  Status stat;
  while (stat.ok() &&
         imm_.IsFlushPending(options_.min_write_buffer_number_to_merge)) {
    Log(options_.info_log,
1776
        "BackgroundCallFlush doing FlushMemTableToOutputFile, flush slots available %d",
1777
        options_.max_background_flushes - bg_flush_scheduled_);
I
Igor Canadi 已提交
1778
    stat = FlushMemTableToOutputFile(madeProgress, deletion_state);
J
jorlow@chromium.org 已提交
1779
  }
1780
  return stat;
J
jorlow@chromium.org 已提交
1781 1782
}

1783
void DBImpl::BackgroundCallFlush() {
1784
  bool madeProgress = false;
I
Igor Canadi 已提交
1785
  DeletionState deletion_state(options_.max_write_buffer_number, true);
1786 1787 1788
  assert(bg_flush_scheduled_);
  MutexLock l(&mutex_);

I
Igor Canadi 已提交
1789
  Status s;
1790
  if (!shutting_down_.Acquire_Load()) {
I
Igor Canadi 已提交
1791
    s = BackgroundFlush(&madeProgress, deletion_state);
1792 1793 1794 1795 1796 1797 1798 1799 1800
    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.
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      Log(options_.info_log, "Waiting after background flush error: %s",
          s.ToString().c_str());
      mutex_.Unlock();
I
Igor Canadi 已提交
1801
      LogFlush(options_.info_log);
1802 1803 1804 1805 1806
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
    }
  }

I
Igor Canadi 已提交
1807 1808 1809
  // 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());
I
Igor Canadi 已提交
1810
  // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
1811 1812 1813 1814 1815 1816
  if (deletion_state.HaveSomethingToDelete()) {
    mutex_.Unlock();
    PurgeObsoleteFiles(deletion_state);
    mutex_.Lock();
  }

1817
  bg_flush_scheduled_--;
1818 1819 1820
  if (madeProgress) {
    MaybeScheduleFlushOrCompaction();
  }
1821
  bg_cv_.SignalAll();
J
jorlow@chromium.org 已提交
1822 1823
}

1824

1825 1826 1827 1828
void DBImpl::TEST_PurgeObsoleteteWAL() {
  PurgeObsoleteWALFiles();
}

1829 1830 1831 1832 1833
uint64_t DBImpl::TEST_GetLevel0TotalSize() {
  MutexLock l(&mutex_);
  return versions_->current()->NumLevelBytes(0);
}

1834
void DBImpl::BackgroundCallCompaction() {
1835
  bool madeProgress = false;
I
Igor Canadi 已提交
1836
  DeletionState deletion_state(options_.max_write_buffer_number, true);
H
Haobo Xu 已提交
1837 1838 1839

  MaybeDumpStats();

J
jorlow@chromium.org 已提交
1840
  MutexLock l(&mutex_);
1841
  // Log(options_.info_log, "XXX BG Thread %llx process new work item", pthread_self());
J
jorlow@chromium.org 已提交
1842
  assert(bg_compaction_scheduled_);
I
Igor Canadi 已提交
1843
  Status s;
H
hans@chromium.org 已提交
1844
  if (!shutting_down_.Acquire_Load()) {
I
Igor Canadi 已提交
1845
    s = BackgroundCompaction(&madeProgress, deletion_state);
1846 1847 1848 1849 1850 1851 1852 1853 1854
    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.
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      Log(options_.info_log, "Waiting after background compaction error: %s",
          s.ToString().c_str());
      mutex_.Unlock();
I
Igor Canadi 已提交
1855
      LogFlush(options_.info_log);
1856 1857 1858
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
    }
J
jorlow@chromium.org 已提交
1859
  }
1860

I
Igor Canadi 已提交
1861 1862 1863 1864 1865
  // 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());
1866

I
Igor Canadi 已提交
1867
  // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
1868
  if (deletion_state.HaveSomethingToDelete()) {
D
Dhruba Borthakur 已提交
1869 1870
    mutex_.Unlock();
    PurgeObsoleteFiles(deletion_state);
1871
    mutex_.Lock();
D
Dhruba Borthakur 已提交
1872 1873
  }

1874
  bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
1875

1876 1877
  MaybeScheduleLogDBDeployStats();

J
jorlow@chromium.org 已提交
1878
  // Previous compaction may have produced too many files in a level,
A
Abhishek Kona 已提交
1879
  // So reschedule another compaction if we made progress in the
1880 1881
  // last compaction.
  if (madeProgress) {
1882
    MaybeScheduleFlushOrCompaction();
1883
  }
H
hans@chromium.org 已提交
1884
  bg_cv_.SignalAll();
1885

J
jorlow@chromium.org 已提交
1886 1887
}

A
Abhishek Kona 已提交
1888
Status DBImpl::BackgroundCompaction(bool* madeProgress,
I
Igor Canadi 已提交
1889
                                    DeletionState& deletion_state) {
1890
  *madeProgress = false;
J
jorlow@chromium.org 已提交
1891
  mutex_.AssertHeld();
1892

1893
  // TODO: remove memtable flush from formal compaction
1894
  while (imm_.IsFlushPending(options_.min_write_buffer_number_to_merge)) {
A
Abhishek Kona 已提交
1895
    Log(options_.info_log,
1896 1897
        "BackgroundCompaction doing FlushMemTableToOutputFile, compaction slots "
        "available %d",
1898
        options_.max_background_compactions - bg_compaction_scheduled_);
I
Igor Canadi 已提交
1899
    Status stat = FlushMemTableToOutputFile(madeProgress, deletion_state);
1900 1901 1902
    if (!stat.ok()) {
      return stat;
    }
1903 1904
  }

1905
  unique_ptr<Compaction> c;
1906
  bool is_manual = (manual_compaction_ != nullptr) &&
1907
                   (manual_compaction_->in_progress == false);
1908 1909
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
1910
  if (is_manual) {
G
Gabor Cselle 已提交
1911
    ManualCompaction* m = manual_compaction_;
1912 1913
    assert(!m->in_progress);
    m->in_progress = true; // another thread cannot pick up the same work
1914 1915 1916
    c.reset(versions_->CompactRange(
        m->input_level, m->output_level, m->begin, m->end, &manual_end));
    if (!c) {
1917
      m->done = true;
G
Gabor Cselle 已提交
1918 1919
    }
    Log(options_.info_log,
1920 1921 1922 1923
        "Manual compaction from level-%d to level-%d from %s .. %s; will stop "
        "at %s\n",
        m->input_level,
        m->output_level,
G
Gabor Cselle 已提交
1924 1925
        (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
        (m->end ? m->end->DebugString().c_str() : "(end)"),
1926 1927 1928
        ((m->done || manual_end == nullptr)
             ? "(end)"
             : manual_end->DebugString().c_str()));
1929
  } else if (!options_.disable_auto_compactions) {
1930
    c.reset(versions_->PickCompaction());
J
jorlow@chromium.org 已提交
1931 1932 1933
  }

  Status status;
1934
  if (!c) {
H
hans@chromium.org 已提交
1935
    // Nothing to do
1936
    Log(options_.info_log, "Compaction nothing to do");
H
hans@chromium.org 已提交
1937
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
1938
    // Move file to next level
1939
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
1940 1941 1942
    FileMetaData* f = c->input(0, 0);
    c->edit()->DeleteFile(c->level(), f->number);
    c->edit()->AddFile(c->level() + 1, f->number, f->file_size,
1943 1944
                       f->smallest, f->largest,
                       f->smallest_seqno, f->largest_seqno);
1945
    status = versions_->LogAndApply(c->edit(), &mutex_);
I
Igor Canadi 已提交
1946
    InstallSuperVersion(deletion_state);
1947
    Version::LevelSummaryStorage tmp;
1948
    Log(options_.info_log, "Moved #%lld to level-%d %lld bytes %s: %s\n",
1949
        static_cast<unsigned long long>(f->number), c->level() + 1,
J
jorlow@chromium.org 已提交
1950
        static_cast<unsigned long long>(f->file_size),
1951
        status.ToString().c_str(), versions_->current()->LevelSummary(&tmp));
1952
    versions_->ReleaseCompactionFiles(c.get(), status);
1953
    *madeProgress = true;
J
jorlow@chromium.org 已提交
1954
  } else {
1955
    MaybeScheduleFlushOrCompaction(); // do more compaction work in parallel.
1956
    CompactionState* compact = new CompactionState(c.get());
I
Igor Canadi 已提交
1957
    status = DoCompactionWork(compact, deletion_state);
1958
    CleanupCompaction(compact, status);
1959
    versions_->ReleaseCompactionFiles(c.get(), status);
1960
    c->ReleaseInputs();
1961
    *madeProgress = true;
J
jorlow@chromium.org 已提交
1962
  }
1963
  c.reset();
J
jorlow@chromium.org 已提交
1964 1965 1966 1967 1968 1969

  if (status.ok()) {
    // Done
  } else if (shutting_down_.Acquire_Load()) {
    // Ignore compaction errors found during shutting down
  } else {
1970
    Log(options_.info_log,
J
jorlow@chromium.org 已提交
1971 1972 1973 1974 1975
        "Compaction error: %s", status.ToString().c_str());
    if (options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
1976 1977

  if (is_manual) {
G
Gabor Cselle 已提交
1978
    ManualCompaction* m = manual_compaction_;
1979 1980 1981
    if (!status.ok()) {
      m->done = true;
    }
1982 1983 1984 1985 1986 1987 1988 1989 1990
    // 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.
1991 1992 1993 1994 1995
    //
    // 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) {
1996 1997
      m->done = true;
    }
G
Gabor Cselle 已提交
1998 1999 2000
    if (!m->done) {
      // We only compacted part of the requested range.  Update *m
      // to the range that is left to be compacted.
2001 2002 2003
      // Universal compaction should always compact the whole range
      assert(options_.compaction_style != kCompactionStyleUniversal);
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2004 2005
      m->begin = &m->tmp_storage;
    }
2006
    m->in_progress = false; // not being processed anymore
2007
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2008
  }
2009
  return status;
J
jorlow@chromium.org 已提交
2010 2011
}

2012
void DBImpl::CleanupCompaction(CompactionState* compact, Status status) {
J
jorlow@chromium.org 已提交
2013
  mutex_.AssertHeld();
2014
  if (compact->builder != nullptr) {
J
jorlow@chromium.org 已提交
2015 2016
    // May happen if we get a shutdown call in the middle of compaction
    compact->builder->Abandon();
2017
    compact->builder.reset();
J
jorlow@chromium.org 已提交
2018
  } else {
2019
    assert(compact->outfile == nullptr);
J
jorlow@chromium.org 已提交
2020
  }
D
dgrogan@chromium.org 已提交
2021
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2022 2023
    const CompactionState::Output& out = compact->outputs[i];
    pending_outputs_.erase(out.number);
2024 2025 2026 2027 2028 2029

    // If this file was inserted into the table cache then remove
    // them here because this compaction was not committed.
    if (!status.ok()) {
      table_cache_->Evict(out.number);
    }
J
jorlow@chromium.org 已提交
2030 2031 2032 2033
  }
  delete compact;
}

2034
// Allocate the file numbers for the output file. We allocate as
2035
// many output file numbers as there are files in level+1 (at least one)
2036 2037 2038
// Insert them into pending_outputs so that they do not get deleted.
void DBImpl::AllocateCompactionOutputFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2039 2040
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
2041
  int filesNeeded = compact->compaction->num_input_files(1);
2042
  for (int i = 0; i < std::max(filesNeeded, 1); i++) {
2043 2044 2045 2046 2047 2048 2049 2050 2051
    uint64_t file_number = versions_->NewFileNumber();
    pending_outputs_.insert(file_number);
    compact->allocated_file_numbers.push_back(file_number);
  }
}

// Frees up unused file number.
void DBImpl::ReleaseCompactionUnusedFileNumbers(CompactionState* compact) {
  mutex_.AssertHeld();
2052
  for (const auto file_number : compact->allocated_file_numbers) {
2053 2054 2055 2056 2057
    pending_outputs_.erase(file_number);
    // Log(options_.info_log, "XXX releasing unused file num %d", file_number);
  }
}

J
jorlow@chromium.org 已提交
2058
Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
2059 2060
  assert(compact != nullptr);
  assert(compact->builder == nullptr);
J
jorlow@chromium.org 已提交
2061
  uint64_t file_number;
2062 2063 2064 2065 2066 2067 2068
  // 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 已提交
2069 2070 2071 2072 2073
    mutex_.Lock();
    file_number = versions_->NewFileNumber();
    pending_outputs_.insert(file_number);
    mutex_.Unlock();
  }
2074 2075 2076 2077
  CompactionState::Output out;
  out.number = file_number;
  out.smallest.Clear();
  out.largest.Clear();
2078
  out.smallest_seqno = out.largest_seqno = 0;
2079
  compact->outputs.push_back(out);
J
jorlow@chromium.org 已提交
2080 2081 2082

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

J
jorlow@chromium.org 已提交
2085
  if (s.ok()) {
2086 2087 2088
    // Over-estimate slightly so we don't end up just barely crossing
    // the threshold.
    compact->outfile->SetPreallocationBlockSize(
2089
      1.1 * versions_->MaxFileSizeForLevel(compact->compaction->output_level()));
2090

S
Siying Dong 已提交
2091 2092 2093 2094
    CompressionType compression_type = GetCompressionType(
        options_, compact->compaction->output_level(),
        compact->compaction->enable_compression());

S
Siying Dong 已提交
2095
    compact->builder.reset(
S
Siying Dong 已提交
2096
        GetTableBuilder(options_, compact->outfile.get(), compression_type));
J
jorlow@chromium.org 已提交
2097
  }
I
Igor Canadi 已提交
2098
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
2099 2100 2101 2102 2103
  return s;
}

Status DBImpl::FinishCompactionOutputFile(CompactionState* compact,
                                          Iterator* input) {
2104
  assert(compact != nullptr);
2105
  assert(compact->outfile);
2106
  assert(compact->builder != nullptr);
J
jorlow@chromium.org 已提交
2107 2108 2109 2110 2111 2112 2113 2114 2115 2116 2117 2118 2119 2120 2121

  const uint64_t output_number = compact->current_output()->number;
  assert(output_number != 0);

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

  // Finish and check for file errors
2125
  if (s.ok() && !options_.disableDataSync) {
2126
    if (options_.use_fsync) {
2127
      StopWatch sw(env_, options_.statistics.get(),
2128
                   COMPACTION_OUTFILE_SYNC_MICROS, false);
2129 2130
      s = compact->outfile->Fsync();
    } else {
2131
      StopWatch sw(env_, options_.statistics.get(),
2132
                   COMPACTION_OUTFILE_SYNC_MICROS, false);
2133 2134
      s = compact->outfile->Sync();
    }
J
jorlow@chromium.org 已提交
2135 2136 2137 2138
  }
  if (s.ok()) {
    s = compact->outfile->Close();
  }
2139
  compact->outfile.reset();
J
jorlow@chromium.org 已提交
2140 2141 2142

  if (s.ok() && current_entries > 0) {
    // Verify that the table is usable
J
jorlow@chromium.org 已提交
2143
    Iterator* iter = table_cache_->NewIterator(ReadOptions(),
2144
                                               storage_options_,
J
jorlow@chromium.org 已提交
2145 2146
                                               output_number,
                                               current_bytes);
J
jorlow@chromium.org 已提交
2147 2148 2149
    s = iter->status();
    delete iter;
    if (s.ok()) {
2150
      Log(options_.info_log,
K
Kai Liu 已提交
2151 2152 2153 2154
          "Generated table #%lu: %lu keys, %lu bytes",
          (unsigned long) output_number,
          (unsigned long) current_entries,
          (unsigned long) current_bytes);
J
jorlow@chromium.org 已提交
2155 2156 2157 2158 2159 2160 2161 2162
    }
  }
  return s;
}


Status DBImpl::InstallCompactionResults(CompactionState* compact) {
  mutex_.AssertHeld();
2163 2164 2165 2166 2167

  // 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.
2168
  if (!versions_->VerifyCompactionFileConsistency(compact->compaction)) {
2169 2170 2171 2172 2173 2174 2175 2176
    Log(options_.info_log,  "Compaction %d@%d + %d@%d files aborted",
      compact->compaction->num_input_files(0),
      compact->compaction->level(),
      compact->compaction->num_input_files(1),
      compact->compaction->level() + 1);
    return Status::IOError("Compaction input files inconsistent");
  }

2177
  Log(options_.info_log,  "Compacted %d@%d + %d@%d files => %lld bytes",
J
jorlow@chromium.org 已提交
2178 2179 2180 2181 2182 2183 2184 2185
      compact->compaction->num_input_files(0),
      compact->compaction->level(),
      compact->compaction->num_input_files(1),
      compact->compaction->level() + 1,
      static_cast<long long>(compact->total_bytes));

  // Add compaction outputs
  compact->compaction->AddInputDeletions(compact->compaction->edit());
D
dgrogan@chromium.org 已提交
2186
  for (size_t i = 0; i < compact->outputs.size(); i++) {
J
jorlow@chromium.org 已提交
2187 2188
    const CompactionState::Output& out = compact->outputs[i];
    compact->compaction->edit()->AddFile(
2189 2190
        compact->compaction->output_level(), out.number, out.file_size,
        out.smallest, out.largest, out.smallest_seqno, out.largest_seqno);
J
jorlow@chromium.org 已提交
2191
  }
2192
  return versions_->LogAndApply(compact->compaction->edit(), &mutex_);
J
jorlow@chromium.org 已提交
2193 2194
}

2195 2196 2197 2198 2199 2200 2201 2202
//
// 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(
2203 2204
  SequenceNumber in, std::vector<SequenceNumber>& snapshots,
  SequenceNumber* prev_snapshot) {
2205
  SequenceNumber prev __attribute__((unused)) = 0;
2206 2207 2208
  for (const auto cur : snapshots) {
    assert(prev <= cur);
    if (cur >= in) {
2209
      *prev_snapshot = prev;
2210
      return cur;
2211
    }
2212 2213
    prev = cur; // assignment
    assert(prev);
2214 2215
  }
  Log(options_.info_log,
K
Kai Liu 已提交
2216 2217 2218
      "Looking for seqid %lu but maxseqid is %lu",
      (unsigned long)in,
      (unsigned long)snapshots[snapshots.size()-1]);
2219 2220 2221 2222
  assert(0);
  return 0;
}

I
Igor Canadi 已提交
2223 2224
Status DBImpl::DoCompactionWork(CompactionState* compact,
                                DeletionState& deletion_state) {
2225
  assert(compact);
2226
  int64_t imm_micros = 0;  // Micros spent doing imm_ compactions
A
Abhishek Kona 已提交
2227
  Log(options_.info_log,
2228
      "Compacting %d@%d + %d@%d files, score %.2f slots available %d",
J
jorlow@chromium.org 已提交
2229 2230 2231
      compact->compaction->num_input_files(0),
      compact->compaction->level(),
      compact->compaction->num_input_files(1),
2232
      compact->compaction->output_level(),
2233
      compact->compaction->score(),
2234
      options_.max_background_compactions - bg_compaction_scheduled_);
2235 2236
  char scratch[256];
  compact->compaction->Summary(scratch, sizeof(scratch));
H
heyongqiang 已提交
2237
  Log(options_.info_log, "Compaction start summary: %s\n", scratch);
J
jorlow@chromium.org 已提交
2238

2239
  assert(versions_->current()->NumLevelFiles(compact->compaction->level()) > 0);
2240
  assert(compact->builder == nullptr);
2241
  assert(!compact->outfile);
2242 2243 2244

  SequenceNumber visible_at_tip = 0;
  SequenceNumber earliest_snapshot;
H
Haobo Xu 已提交
2245
  SequenceNumber latest_snapshot = 0;
2246 2247 2248 2249 2250
  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;
J
jorlow@chromium.org 已提交
2251
  } else {
H
Haobo Xu 已提交
2252
    latest_snapshot = compact->existing_snapshots.back();
2253 2254 2255 2256
    // 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];
J
jorlow@chromium.org 已提交
2257 2258
  }

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

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

J
jorlow@chromium.org 已提交
2265 2266 2267
  // Release mutex while we're actually doing the compaction work
  mutex_.Unlock();

2268
  const uint64_t start_micros = env_->NowMicros();
2269
  unique_ptr<Iterator> input(versions_->MakeInputIterator(compact->compaction));
J
jorlow@chromium.org 已提交
2270 2271 2272 2273 2274
  input->SeekToFirst();
  Status status;
  ParsedInternalKey ikey;
  std::string current_user_key;
  bool has_current_user_key = false;
2275 2276
  SequenceNumber last_sequence_for_key __attribute__((unused)) =
    kMaxSequenceNumber;
2277
  SequenceNumber visible_in_snapshot = kMaxSequenceNumber;
H
Haobo Xu 已提交
2278
  std::string compaction_filter_value;
H
Haobo Xu 已提交
2279
  std::vector<char> delete_key; // for compaction filter
2280
  MergeHelper merge(user_comparator(), options_.merge_operator.get(),
2281 2282
                    options_.info_log.get(),
                    false /* internal key corruption is expected */);
2283 2284 2285
  auto compaction_filter = options_.compaction_filter;
  std::unique_ptr<CompactionFilter> compaction_filter_from_factory = nullptr;
  if (!compaction_filter) {
2286 2287 2288
    auto context = compact->GetFilterContext();
    compaction_filter_from_factory =
      options_.compaction_filter_factory->CreateCompactionFilter(context);
2289 2290
    compaction_filter = compaction_filter_from_factory.get();
  }
2291

J
jorlow@chromium.org 已提交
2292
  for (; input->Valid() && !shutting_down_.Acquire_Load(); ) {
2293
    // Prioritize immutable compaction work
2294
    // TODO: remove memtable flush from normal compaction work
2295
    if (imm_.imm_flush_needed.NoBarrier_Load() != nullptr) {
2296
      const uint64_t imm_start = env_->NowMicros();
I
Igor Canadi 已提交
2297
      LogFlush(options_.info_log);
2298
      mutex_.Lock();
2299
      if (imm_.IsFlushPending(options_.min_write_buffer_number_to_merge)) {
I
Igor Canadi 已提交
2300
        FlushMemTableToOutputFile(nullptr, deletion_state);
H
hans@chromium.org 已提交
2301
        bg_cv_.SignalAll();  // Wakeup MakeRoomForWrite() if necessary
2302 2303 2304 2305 2306
      }
      mutex_.Unlock();
      imm_micros += (env_->NowMicros() - imm_start);
    }

J
jorlow@chromium.org 已提交
2307
    Slice key = input->key();
2308
    Slice value = input->value();
H
Haobo Xu 已提交
2309

2310
    if (compact->compaction->ShouldStopBefore(key) &&
2311
        compact->builder != nullptr) {
2312
      status = FinishCompactionOutputFile(compact, input.get());
2313 2314 2315 2316 2317 2318
      if (!status.ok()) {
        break;
      }
    }

    // Handle key/value, add to state, etc.
J
jorlow@chromium.org 已提交
2319
    bool drop = false;
2320
    bool current_entry_is_merging = false;
J
jorlow@chromium.org 已提交
2321 2322
    if (!ParseInternalKey(key, &ikey)) {
      // Do not hide error keys
2323 2324
      // 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.
J
jorlow@chromium.org 已提交
2325 2326 2327
      current_user_key.clear();
      has_current_user_key = false;
      last_sequence_for_key = kMaxSequenceNumber;
2328
      visible_in_snapshot = kMaxSequenceNumber;
J
jorlow@chromium.org 已提交
2329 2330 2331 2332 2333 2334 2335 2336
    } else {
      if (!has_current_user_key ||
          user_comparator()->Compare(ikey.user_key,
                                     Slice(current_user_key)) != 0) {
        // First occurrence of this user key
        current_user_key.assign(ikey.user_key.data(), ikey.user_key.size());
        has_current_user_key = true;
        last_sequence_for_key = kMaxSequenceNumber;
2337
        visible_in_snapshot = kMaxSequenceNumber;
H
Haobo Xu 已提交
2338 2339

        // apply the compaction filter to the first occurrence of the user key
2340
        if (compaction_filter &&
H
Haobo Xu 已提交
2341 2342 2343 2344 2345 2346 2347 2348 2349 2350
            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();
          bool to_delete =
2351
            compaction_filter->Filter(compact->compaction->level(),
S
Siying Dong 已提交
2352 2353 2354
                                               ikey.user_key, value,
                                               &compaction_filter_value,
                                               &value_changed);
H
Haobo Xu 已提交
2355 2356 2357 2358 2359 2360 2361 2362 2363 2364 2365 2366
          if (to_delete) {
            // make a copy of the original key
            delete_key.assign(key.data(), key.data() + key.size());
            // convert it to a delete
            UpdateInternalKey(&delete_key[0], delete_key.size(),
                              ikey.sequence, kTypeDeletion);
            // anchor the key again
            key = Slice(&delete_key[0], delete_key.size());
            // needed because ikey is backed by key
            ParseInternalKey(key, &ikey);
            // no value associated with delete
            value.clear();
2367
            RecordTick(options_.statistics.get(), COMPACTION_KEY_DROP_USER);
H
Haobo Xu 已提交
2368 2369 2370 2371 2372
          } else if (value_changed) {
            value = compaction_filter_value;
          }
        }

J
jorlow@chromium.org 已提交
2373 2374
      }

2375 2376 2377
      // 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.
2378 2379 2380 2381 2382 2383
      SequenceNumber prev_snapshot = 0; // 0 means no previous snapshot
      SequenceNumber visible = visible_at_tip ?
        visible_at_tip :
        findEarliestVisibleSnapshot(ikey.sequence,
                                    compact->existing_snapshots,
                                    &prev_snapshot);
2384 2385 2386 2387 2388

      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 已提交
2389
        // Hidden by an newer entry for same user key
2390
        // TODO: why not > ?
2391
        assert(last_sequence_for_key >= ikey.sequence);
J
jorlow@chromium.org 已提交
2392
        drop = true;    // (A)
2393
        RecordTick(options_.statistics.get(), COMPACTION_KEY_DROP_NEWER_ENTRY);
J
jorlow@chromium.org 已提交
2394
      } else if (ikey.type == kTypeDeletion &&
2395
                 ikey.sequence <= earliest_snapshot &&
J
jorlow@chromium.org 已提交
2396 2397 2398 2399 2400 2401 2402 2403 2404
                 compact->compaction->IsBaseLevelForKey(ikey.user_key)) {
        // 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;
2405
        RecordTick(options_.statistics.get(), COMPACTION_KEY_DROP_OBSOLETE);
2406 2407 2408 2409 2410 2411 2412
      } else if (ikey.type == kTypeMerge) {
        // We know the merge type entry is not hidden, otherwise we would
        // have hit (A)
        // We encapsulate the merge related state machine in a different
        // object to minimize change to the existing flow. Turn out this
        // logic could also be nicely re-used for memtable flush purge
        // optimization in BuildTable.
M
Mayank Agarwal 已提交
2413
        merge.MergeUntil(input.get(), prev_snapshot, bottommost_level,
2414
                         options_.statistics.get());
2415 2416 2417 2418 2419 2420 2421 2422 2423 2424 2425 2426 2427 2428 2429 2430 2431 2432
        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 已提交
2433 2434 2435
      }

      last_sequence_for_key = ikey.sequence;
2436
      visible_in_snapshot = visible;
J
jorlow@chromium.org 已提交
2437 2438
    }
#if 0
2439
    Log(options_.info_log,
J
jorlow@chromium.org 已提交
2440
        "  Compact: %s, seq %d, type: %d %d, drop: %d, is_base: %d, "
2441
        "%d smallest_snapshot: %d level: %d bottommost %d",
J
jorlow@chromium.org 已提交
2442
        ikey.user_key.ToString().c_str(),
D
dgrogan@chromium.org 已提交
2443
        (int)ikey.sequence, ikey.type, kTypeValue, drop,
J
jorlow@chromium.org 已提交
2444
        compact->compaction->IsBaseLevelForKey(ikey.user_key),
2445 2446
        (int)last_sequence_for_key, (int)earliest_snapshot,
        compact->compaction->level(), bottommost_level);
J
jorlow@chromium.org 已提交
2447 2448 2449
#endif

    if (!drop) {
2450 2451 2452 2453 2454 2455 2456 2457 2458 2459 2460 2461 2462 2463 2464 2465
      // 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);
2466
      }
2467

2468 2469 2470 2471 2472 2473 2474 2475 2476 2477 2478
      // 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.
2479 2480
        if (options_.compaction_style == kCompactionStyleLevel &&
            bottommost_level && ikey.sequence < earliest_snapshot &&
2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491
            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
2492

2493 2494 2495 2496 2497 2498 2499
        // Open output file if necessary
        if (compact->builder == nullptr) {
          status = OpenCompactionOutputFile(compact);
          if (!status.ok()) {
            break;
          }
        }
2500 2501

        SequenceNumber seqno = GetInternalKeySeqno(newkey);
2502 2503
        if (compact->builder->NumEntries() == 0) {
          compact->current_output()->smallest.DecodeFrom(newkey);
2504 2505 2506 2507
          compact->current_output()->smallest_seqno = seqno;
        } else {
          compact->current_output()->smallest_seqno =
            std::min(compact->current_output()->smallest_seqno, seqno);
2508 2509 2510
        }
        compact->current_output()->largest.DecodeFrom(newkey);
        compact->builder->Add(newkey, value);
2511 2512
        compact->current_output()->largest_seqno =
          std::max(compact->current_output()->largest_seqno, seqno);
2513 2514 2515 2516 2517 2518 2519 2520

        // Close output file if it is big enough
        if (compact->builder->FileSize() >=
            compact->compaction->MaxOutputFileSize()) {
          status = FinishCompactionOutputFile(compact, input.get());
          if (!status.ok()) {
            break;
          }
J
jorlow@chromium.org 已提交
2521 2522
        }

2523 2524 2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538 2539 2540 2541 2542
        // 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 已提交
2543 2544 2545 2546 2547
          break;
        }
      }
    }

2548
    // MergeUntil has moved input to the next entry
2549
    if (!current_entry_is_merging) {
2550 2551
      input->Next();
    }
J
jorlow@chromium.org 已提交
2552 2553 2554
  }

  if (status.ok() && shutting_down_.Acquire_Load()) {
2555
    status = Status::IOError("Database shutdown started during compaction");
J
jorlow@chromium.org 已提交
2556
  }
2557
  if (status.ok() && compact->builder != nullptr) {
2558
    status = FinishCompactionOutputFile(compact, input.get());
J
jorlow@chromium.org 已提交
2559 2560 2561 2562
  }
  if (status.ok()) {
    status = input->status();
  }
2563
  input.reset();
J
jorlow@chromium.org 已提交
2564

2565 2566
  CompactionStats stats;
  stats.micros = env_->NowMicros() - start_micros - imm_micros;
2567 2568
  if (options_.statistics.get()) {
    options_.statistics.get()->measureTime(COMPACTION_TIME, stats.micros);
A
Abhishek Kona 已提交
2569
  }
M
Mark Callaghan 已提交
2570 2571
  stats.files_in_leveln = compact->compaction->num_input_files(0);
  stats.files_in_levelnp1 = compact->compaction->num_input_files(1);
2572 2573

  int num_output_files = compact->outputs.size();
2574
  if (compact->builder != nullptr) {
P
Pascal Borreli 已提交
2575
    // An error occurred so ignore the last output.
2576 2577 2578 2579
    assert(num_output_files > 0);
    --num_output_files;
  }
  stats.files_out_levelnp1 = num_output_files;
M
Mark Callaghan 已提交
2580

2581
  for (int i = 0; i < compact->compaction->num_input_files(0); i++) {
M
Mark Callaghan 已提交
2582
    stats.bytes_readn += compact->compaction->input(0, i)->file_size;
2583 2584 2585
    RecordTick(options_.statistics.get(), COMPACT_READ_BYTES,
               compact->compaction->input(0, i)->file_size);
  }
M
Mark Callaghan 已提交
2586

2587
  for (int i = 0; i < compact->compaction->num_input_files(1); i++) {
M
Mark Callaghan 已提交
2588
    stats.bytes_readnp1 += compact->compaction->input(1, i)->file_size;
2589 2590 2591
    RecordTick(options_.statistics.get(), COMPACT_READ_BYTES,
               compact->compaction->input(1, i)->file_size);
  }
M
Mark Callaghan 已提交
2592

2593
  for (int i = 0; i < num_output_files; i++) {
2594
    stats.bytes_written += compact->outputs[i].file_size;
2595 2596
    RecordTick(options_.statistics.get(), COMPACT_WRITE_BYTES,
               compact->outputs[i].file_size);
2597 2598
  }

I
Igor Canadi 已提交
2599
  LogFlush(options_.info_log);
J
jorlow@chromium.org 已提交
2600
  mutex_.Lock();
2601
  stats_[compact->compaction->output_level()].Add(stats);
J
jorlow@chromium.org 已提交
2602

2603 2604 2605 2606
  // 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 已提交
2607 2608
  if (status.ok()) {
    status = InstallCompactionResults(compact);
I
Igor Canadi 已提交
2609
    InstallSuperVersion(deletion_state);
J
jorlow@chromium.org 已提交
2610
  }
2611
  Version::LevelSummaryStorage tmp;
2612
  Log(options_.info_log,
M
Mark Callaghan 已提交
2613
      "compacted to: %s, %.1f MB/sec, level %d, files in(%d, %d) out(%d) "
2614 2615
      "MB in(%.1f, %.1f) out(%.1f), read-write-amplify(%.1f) "
      "write-amplify(%.1f) %s\n",
2616
      versions_->current()->LevelSummary(&tmp),
M
Mark Callaghan 已提交
2617
      (stats.bytes_readn + stats.bytes_readnp1 + stats.bytes_written) /
2618 2619 2620 2621
          (double)stats.micros,
      compact->compaction->output_level(), stats.files_in_leveln,
      stats.files_in_levelnp1, stats.files_out_levelnp1,
      stats.bytes_readn / 1048576.0, stats.bytes_readnp1 / 1048576.0,
M
Mark Callaghan 已提交
2622
      stats.bytes_written / 1048576.0,
2623
      (stats.bytes_written + stats.bytes_readnp1 + stats.bytes_readn) /
2624 2625
          (double)stats.bytes_readn,
      stats.bytes_written / (double)stats.bytes_readn,
2626
      status.ToString().c_str());
M
Mark Callaghan 已提交
2627

J
jorlow@chromium.org 已提交
2628 2629 2630
  return status;
}

2631 2632 2633 2634
namespace {
struct IterState {
  port::Mutex* mu;
  Version* version;
2635
  std::vector<MemTable*> mem; // includes both mem_ and imm_
I
Igor Canadi 已提交
2636
  DBImpl *db;
2637 2638 2639 2640
};

static void CleanupIteratorState(void* arg1, void* arg2) {
  IterState* state = reinterpret_cast<IterState*>(arg1);
2641 2642
  DBImpl::DeletionState deletion_state(state->db->GetOptions().
                                       max_write_buffer_number);
2643
  state->mu->Lock();
2644
  for (unsigned int i = 0; i < state->mem.size(); i++) {
2645 2646
    MemTable* m = state->mem[i]->Unref();
    if (m != nullptr) {
2647
      deletion_state.memtables_to_free.push_back(m);
2648
    }
2649
  }
2650
  state->version->Unref();
I
Igor Canadi 已提交
2651 2652
  // fast path FindObsoleteFiles
  state->db->FindObsoleteFiles(deletion_state, false, true);
2653
  state->mu->Unlock();
I
Igor Canadi 已提交
2654
  state->db->PurgeObsoleteFiles(deletion_state);
2655 2656
  delete state;
}
H
Hans Wennborg 已提交
2657
}  // namespace
2658

J
jorlow@chromium.org 已提交
2659 2660
Iterator* DBImpl::NewInternalIterator(const ReadOptions& options,
                                      SequenceNumber* latest_snapshot) {
2661
  IterState* cleanup = new IterState;
2662 2663 2664
  MemTable* mutable_mem;
  std::vector<MemTable*> immutables;
  Version* version;
J
jorlow@chromium.org 已提交
2665

2666
  // Collect together all needed child iterators for mem
2667 2668
  mutex_.Lock();
  *latest_snapshot = versions_->LastSequence();
2669
  mem_->Ref();
2670
  mutable_mem = mem_;
2671 2672 2673
  // Collect together all needed child iterators for imm_
  imm_.GetMemTables(&immutables);
  for (unsigned int i = 0; i < immutables.size(); i++) {
2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684
    immutables[i]->Ref();
  }
  // Collect iterators for files in L0 - Ln
  versions_->current()->Ref();
  version = versions_->current();
  mutex_.Unlock();

  std::vector<Iterator*> list;
  list.push_back(mutable_mem->NewIterator(options));
  cleanup->mem.push_back(mutable_mem);
  for (MemTable* m : immutables) {
2685
    list.push_back(m->NewIterator(options));
2686
    cleanup->mem.push_back(m);
2687
  }
2688
  version->AddIterators(options, storage_options_, &list);
J
jorlow@chromium.org 已提交
2689
  Iterator* internal_iter =
2690
      NewMergingIterator(&internal_comparator_, &list[0], list.size());
2691
  cleanup->version = version;
2692
  cleanup->mu = &mutex_;
I
Igor Canadi 已提交
2693
  cleanup->db = this;
2694
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2695 2696 2697 2698 2699 2700 2701 2702 2703

  return internal_iter;
}

Iterator* DBImpl::TEST_NewInternalIterator() {
  SequenceNumber ignored;
  return NewInternalIterator(ReadOptions(), &ignored);
}

J
jorlow@chromium.org 已提交
2704
int64_t DBImpl::TEST_MaxNextLevelOverlappingBytes() {
2705
  MutexLock l(&mutex_);
2706
  return versions_->current()->MaxNextLevelOverlappingBytes();
2707 2708
}

J
jorlow@chromium.org 已提交
2709 2710 2711
Status DBImpl::Get(const ReadOptions& options,
                   const Slice& key,
                   std::string* value) {
2712 2713 2714
  return GetImpl(options, key, value);
}

I
Igor Canadi 已提交
2715 2716 2717 2718 2719 2720 2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731 2732 2733 2734 2735 2736 2737 2738 2739 2740 2741 2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752
// DeletionState gets created and destructed outside of the lock -- we
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
// * delete one SuperVersion() outside of the lock -- superversion_to_free
//
// 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
void DBImpl::InstallSuperVersion(DeletionState& deletion_state) {
  // if new_superversion == nullptr, it means somebody already used it
  SuperVersion* new_superversion =
    (deletion_state.new_superversion != nullptr) ?
    deletion_state.new_superversion : new SuperVersion();
  SuperVersion* old_superversion = InstallSuperVersion(new_superversion);
  deletion_state.new_superversion = nullptr;
  if (deletion_state.superversion_to_free != nullptr) {
    // somebody already put it there
    delete old_superversion;
  } else {
    deletion_state.superversion_to_free = old_superversion;
  }
}

DBImpl::SuperVersion* DBImpl::InstallSuperVersion(
    SuperVersion* new_superversion) {
  mutex_.AssertHeld();
  new_superversion->Init(mem_, imm_, versions_->current());
  SuperVersion* old_superversion = super_version_;
  super_version_ = new_superversion;
  if (old_superversion != nullptr && old_superversion->Unref()) {
    old_superversion->Cleanup();
    return old_superversion; // will let caller delete outside of mutex
  }
  return nullptr;
}

2753 2754 2755
Status DBImpl::GetImpl(const ReadOptions& options,
                       const Slice& key,
                       std::string* value,
2756
                       bool* value_found) {
2757
  Status s;
2758

2759
  StopWatch sw(env_, options_.statistics.get(), DB_GET, false);
2760
  SequenceNumber snapshot;
2761
  if (options.snapshot != nullptr) {
2762 2763 2764
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2765
  }
2766

I
Igor Canadi 已提交
2767 2768 2769
  // This can be replaced by using atomics and spinlock instead of big mutex
  mutex_.Lock();
  SuperVersion* get_version = super_version_->Ref();
2770
  mutex_.Unlock();
I
Igor Canadi 已提交
2771

2772
  bool have_stat_update = false;
2773
  Version::GetStats stats;
2774

2775
  // Prepare to store a list of merge operations if merge occurs.
2776
  MergeContext merge_context;
2777

2778
  // First look in the memtable, then in the immutable memtable (if any).
2779
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2780
  // merge_operands will contain the sequence of merges in the latter case.
2781
  LookupKey lkey(key, snapshot);
I
Igor Canadi 已提交
2782
  if (get_version->mem->Get(lkey, value, &s, merge_context, options_)) {
2783
    // Done
2784
    RecordTick(options_.statistics.get(), MEMTABLE_HIT);
I
Igor Canadi 已提交
2785
  } else if (get_version->imm.Get(lkey, value, &s, merge_context, options_)) {
2786
    // Done
2787
    RecordTick(options_.statistics.get(), MEMTABLE_HIT);
2788
  } else {
I
Igor Canadi 已提交
2789 2790
    get_version->current->Get(options, lkey, value, &s, &merge_context, &stats,
                              options_, value_found);
2791
    have_stat_update = true;
2792
    RecordTick(options_.statistics.get(), MEMTABLE_MISS);
2793
  }
2794

I
Igor Canadi 已提交
2795 2796 2797 2798 2799 2800 2801 2802 2803 2804 2805 2806 2807 2808 2809 2810 2811 2812 2813 2814 2815
  bool delete_get_version = false;
  if (!options_.disable_seek_compaction && have_stat_update) {
    mutex_.Lock();
    if (get_version->current->UpdateStats(stats)) {
      MaybeScheduleFlushOrCompaction();
    }
    if (get_version->Unref()) {
      get_version->Cleanup();
      delete_get_version = true;
    }
    mutex_.Unlock();
  } else {
    if (get_version->Unref()) {
      mutex_.Lock();
      get_version->Cleanup();
      mutex_.Unlock();
      delete_get_version = true;
    }
  }
  if (delete_get_version) {
    delete get_version;
2816
  }
2817

2818
  // Note, tickers are atomic now - no lock protection needed any more.
2819 2820
  RecordTick(options_.statistics.get(), NUMBER_KEYS_READ);
  RecordTick(options_.statistics.get(), BYTES_READ, value->size());
2821
  return s;
J
jorlow@chromium.org 已提交
2822 2823
}

2824 2825 2826 2827
std::vector<Status> DBImpl::MultiGet(const ReadOptions& options,
                                     const std::vector<Slice>& keys,
                                     std::vector<std::string>* values) {

2828
  StopWatch sw(env_, options_.statistics.get(), DB_MULTIGET, false);
2829
  SequenceNumber snapshot;
2830 2831
  std::vector<MemTable*> to_delete;

2832
  mutex_.Lock();
2833 2834 2835 2836 2837 2838 2839 2840 2841 2842 2843 2844 2845 2846 2847 2848 2849 2850 2851
  if (options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_;
  } else {
    snapshot = versions_->LastSequence();
  }

  MemTable* mem = mem_;
  MemTableList imm = imm_;
  Version* current = versions_->current();
  mem->Ref();
  imm.RefAll();
  current->Ref();

  // Unlock while reading from files and memtables

  mutex_.Unlock();
  bool have_stat_update = false;
  Version::GetStats stats;

2852 2853
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2854

2855 2856 2857 2858 2859 2860 2861 2862 2863 2864 2865
  // Note: this always resizes the values array
  int numKeys = keys.size();
  std::vector<Status> statList(numKeys);
  values->resize(numKeys);

  // Keep track of bytes that we read for statistics-recording later
  uint64_t bytesRead = 0;

  // 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.
2866 2867
  // merge_operands will contain the sequence of merges in the latter case.
  for (int i=0; i<numKeys; ++i) {
2868
    merge_context.Clear();
2869 2870 2871 2872
    Status& s = statList[i];
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
2873
    if (mem->Get(lkey, value, &s, merge_context, options_)) {
2874
      // Done
2875
    } else if (imm.Get(lkey, value, &s, merge_context, options_)) {
2876 2877
      // Done
    } else {
2878
      current->Get(options, lkey, value, &s, &merge_context, &stats, options_);
2879 2880 2881 2882 2883 2884 2885 2886 2887 2888 2889 2890
      have_stat_update = true;
    }

    if (s.ok()) {
      bytesRead += value->size();
    }
  }

  // Post processing (decrement reference counts and record statistics)
  mutex_.Lock();
  if (!options_.disable_seek_compaction &&
      have_stat_update && current->UpdateStats(stats)) {
2891
    MaybeScheduleFlushOrCompaction();
2892
  }
2893 2894
  MemTable* m = mem->Unref();
  imm.UnrefAll(&to_delete);
2895
  current->Unref();
2896 2897
  mutex_.Unlock();

2898 2899 2900 2901
  // free up all obsolete memtables outside the mutex
  delete m;
  for (MemTable* v: to_delete) delete v;

2902 2903 2904
  RecordTick(options_.statistics.get(), NUMBER_MULTIGET_CALLS);
  RecordTick(options_.statistics.get(), NUMBER_MULTIGET_KEYS_READ, numKeys);
  RecordTick(options_.statistics.get(), NUMBER_MULTIGET_BYTES_READ, bytesRead);
2905 2906 2907 2908

  return statList;
}

2909 2910 2911 2912 2913
bool DBImpl::KeyMayExist(const ReadOptions& options,
                         const Slice& key,
                         std::string* value,
                         bool* value_found) {
  if (value_found != nullptr) {
K
Kai Liu 已提交
2914 2915
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
2916
  }
2917 2918
  ReadOptions roptions = options;
  roptions.read_tier = kBlockCacheTier; // read from block cache only
K
Kai Liu 已提交
2919 2920 2921 2922 2923 2924
  auto s = GetImpl(roptions, key, value, value_found);

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

J
jorlow@chromium.org 已提交
2927 2928
Iterator* DBImpl::NewIterator(const ReadOptions& options) {
  SequenceNumber latest_snapshot;
T
Tyler Harter 已提交
2929 2930 2931 2932 2933 2934 2935 2936 2937 2938 2939 2940 2941
  Iterator* iter = NewInternalIterator(options, &latest_snapshot);
  iter = NewDBIterator(
             &dbname_, env_, options_, user_comparator(), iter,
             (options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(options.snapshot)->number_
              : latest_snapshot));
  if (options.prefix) {
    // use extra wrapper to exclude any keys from the results which
    // don't begin with the prefix
    iter = new PrefixFilterIterator(iter, *options.prefix,
                                    options_.prefix_extractor);
  }
  return iter;
J
jorlow@chromium.org 已提交
2942 2943 2944 2945
}

const Snapshot* DBImpl::GetSnapshot() {
  MutexLock l(&mutex_);
2946
  return snapshots_.New(versions_->LastSequence());
J
jorlow@chromium.org 已提交
2947 2948 2949 2950
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
  MutexLock l(&mutex_);
2951
  snapshots_.Delete(reinterpret_cast<const SnapshotImpl*>(s));
J
jorlow@chromium.org 已提交
2952 2953 2954 2955 2956 2957 2958
}

// Convenience methods
Status DBImpl::Put(const WriteOptions& o, const Slice& key, const Slice& val) {
  return DB::Put(o, key, val);
}

2959 2960 2961 2962 2963 2964 2965 2966 2967
Status DBImpl::Merge(const WriteOptions& o, const Slice& key,
                     const Slice& val) {
  if (!options_.merge_operator) {
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
    return DB::Merge(o, key, val);
  }
}

J
jorlow@chromium.org 已提交
2968 2969 2970 2971
Status DBImpl::Delete(const WriteOptions& options, const Slice& key) {
  return DB::Delete(options, key);
}

2972 2973 2974 2975
Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
  Writer w(&mutex_);
  w.batch = my_batch;
  w.sync = options.sync;
H
heyongqiang 已提交
2976
  w.disableWAL = options.disableWAL;
2977
  w.done = false;
2978

2979
  StopWatch sw(env_, options_.statistics.get(), DB_WRITE, false);
I
Igor Canadi 已提交
2980
  mutex_.Lock();
2981 2982 2983 2984
  writers_.push_back(&w);
  while (!w.done && &w != writers_.front()) {
    w.cv.Wait();
  }
2985 2986 2987 2988 2989

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

2990
  if (w.done) {
I
Igor Canadi 已提交
2991
    mutex_.Unlock();
2992
    RecordTick(options_.statistics.get(), WRITE_DONE_BY_OTHER, 1);
2993
    return w.status;
2994 2995
  } else {
    RecordTick(options_.statistics.get(), WRITE_DONE_BY_SELF, 1);
2996 2997 2998
  }

  // May temporarily unlock and wait.
I
Igor Canadi 已提交
2999 3000
  SuperVersion* superversion_to_free = nullptr;
  Status status = MakeRoomForWrite(my_batch == nullptr, &superversion_to_free);
D
dgrogan@chromium.org 已提交
3001
  uint64_t last_sequence = versions_->LastSequence();
3002
  Writer* last_writer = &w;
3003
  if (status.ok() && my_batch != nullptr) {  // nullptr batch is for compactions
3004 3005
    autovector<WriteBatch*> write_batch_group;
    BuildBatchGroup(&last_writer, &write_batch_group);
3006

3007 3008 3009 3010
    // 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
    // into mem_.
3011
    {
3012
      mutex_.Unlock();
3013 3014 3015 3016 3017 3018 3019 3020 3021 3022
      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]);
        }
      }

3023 3024 3025 3026 3027
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
      // Record statistics
3028 3029 3030
      RecordTick(options_.statistics.get(),
                 NUMBER_KEYS_WRITTEN, my_batch_count);
      RecordTick(options_.statistics.get(),
3031 3032
                 BYTES_WRITTEN,
                 WriteBatchInternal::ByteSize(updates));
3033 3034
      if (options.disableWAL) {
        flush_on_destroy_ = true;
3035 3036 3037
      }

      if (!options.disableWAL) {
3038 3039
        StopWatchNano timer(env_);
        StartPerfTimer(&timer);
3040 3041 3042 3043
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
        RecordTick(options_.statistics.get(), WAL_FILE_SYNCED, 1);
        RecordTick(options_.statistics.get(), WAL_FILE_BYTES, log_entry.size());
3044
        BumpPerfTime(&perf_context.wal_write_time, &timer);
H
heyongqiang 已提交
3045
        if (status.ok() && options.sync) {
3046
          if (options_.use_fsync) {
3047
            StopWatch(env_, options_.statistics.get(), WAL_FILE_SYNC_MICROS);
3048
            status = log_->file()->Fsync();
3049
          } else {
3050
            StopWatch(env_, options_.statistics.get(), WAL_FILE_SYNC_MICROS);
3051
            status = log_->file()->Sync();
3052
          }
H
heyongqiang 已提交
3053
        }
3054 3055
      }
      if (status.ok()) {
3056 3057
        status = WriteBatchInternal::InsertInto(updates, mem_, &options_, this,
                                                options_.filter_deletes);
3058 3059 3060 3061 3062 3063 3064
        if (!status.ok()) {
          // Panic for in-memory corruptions
          // 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.
          throw std::runtime_error("In memory WriteBatch corruption!");
        }
3065 3066
        SetTickerCount(options_.statistics.get(),
                       SEQUENCE_NUMBER, last_sequence);
3067
      }
3068
      if (updates == &tmp_batch_) tmp_batch_.Clear();
3069 3070
      mutex_.Lock();
      if (status.ok()) {
3071
        versions_->SetLastSequence(last_sequence);
3072
      }
J
jorlow@chromium.org 已提交
3073 3074
    }
  }
I
Igor Canadi 已提交
3075 3076 3077
  if (options_.paranoid_checks && !status.ok() && bg_error_.ok()) {
    bg_error_ = status; // stop compaction & fail any further writes
  }
3078

3079 3080 3081 3082 3083 3084 3085
  while (true) {
    Writer* ready = writers_.front();
    writers_.pop_front();
    if (ready != &w) {
      ready->status = status;
      ready->done = true;
      ready->cv.Signal();
3086
    }
3087 3088
    if (ready == last_writer) break;
  }
3089

3090 3091 3092
  // Notify new head of write queue
  if (!writers_.empty()) {
    writers_.front()->cv.Signal();
3093
  }
I
Igor Canadi 已提交
3094 3095
  mutex_.Unlock();
  delete superversion_to_free;
J
jorlow@chromium.org 已提交
3096 3097 3098
  return status;
}

3099
// REQUIRES: Writer list must be non-empty
3100
// REQUIRES: First writer must have a non-nullptr batch
3101 3102
void DBImpl::BuildBatchGroup(Writer** last_writer,
                             autovector<WriteBatch*>* write_batch_group) {
3103 3104
  assert(!writers_.empty());
  Writer* first = writers_.front();
3105
  assert(first->batch != nullptr);
3106 3107

  size_t size = WriteBatchInternal::ByteSize(first->batch);
3108
  write_batch_group->push_back(first->batch);
3109 3110 3111 3112 3113 3114 3115 3116 3117 3118 3119 3120 3121 3122 3123 3124 3125 3126 3127

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

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

H
heyongqiang 已提交
3128 3129 3130 3131 3132 3133
    if (!w->disableWAL && first->disableWAL) {
      // Do not include a write that needs WAL into a batch that has
      // WAL disabled.
      break;
    }

3134
    if (w->batch != nullptr) {
3135 3136 3137 3138 3139 3140
      size += WriteBatchInternal::ByteSize(w->batch);
      if (size > max_size) {
        // Do not make batch too big
        break;
      }

3141
      write_batch_group->push_back(w->batch);
3142 3143 3144 3145 3146
    }
    *last_writer = w;
  }
}

3147 3148 3149
// This function computes the amount of time in microseconds by which a write
// should be delayed based on the number of level-0 files according to the
// following formula:
J
Jim Paton 已提交
3150 3151 3152 3153
// if n < bottom, return 0;
// if n >= top, return 1000;
// otherwise, let r = (n - bottom) /
//                    (top - bottom)
3154 3155 3156 3157
//  and return r^2 * 1000.
// The goal of this formula is to gradually increase the rate at which writes
// are slowed. We also tried linear delay (r * 1000), but it seemed to do
// slightly worse. There is no other particular reason for choosing quadratic.
J
Jim Paton 已提交
3158
uint64_t DBImpl::SlowdownAmount(int n, int top, int bottom) {
3159
  uint64_t delay;
J
Jim Paton 已提交
3160
  if (n >= top) {
3161 3162
    delay = 1000;
  }
J
Jim Paton 已提交
3163
  else if (n < bottom) {
3164 3165 3166 3167
    delay = 0;
  }
  else {
    // If we are here, we know that:
J
Jim Paton 已提交
3168
    //   level0_start_slowdown <= n < level0_slowdown
3169 3170
    // since the previous two conditions are false.
    float how_much =
J
Jim Paton 已提交
3171 3172
      (float) (n - bottom) /
              (top - bottom);
3173 3174 3175 3176 3177 3178
    delay = how_much * how_much * 1000;
  }
  assert(delay <= 1000);
  return delay;
}

3179
// REQUIRES: mutex_ is held
3180
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
3181 3182
Status DBImpl::MakeRoomForWrite(bool force,
                                SuperVersion** superversion_to_free) {
3183
  mutex_.AssertHeld();
3184
  assert(!writers_.empty());
3185
  bool allow_delay = !force;
J
Jim Paton 已提交
3186 3187
  bool allow_hard_rate_limit_delay = !force;
  bool allow_soft_rate_limit_delay = !force;
3188
  uint64_t rate_limit_delay_millis = 0;
3189
  Status s;
3190
  double score;
I
Igor Canadi 已提交
3191
  *superversion_to_free = nullptr;
3192

3193 3194 3195 3196 3197
  while (true) {
    if (!bg_error_.ok()) {
      // Yield previous error
      s = bg_error_;
      break;
3198
    } else if (allow_delay && versions_->NeedSlowdownForNumLevel0Files()) {
3199 3200 3201
      // We are getting close to hitting a hard limit on the number of
      // L0 files.  Rather than delaying a single write by several
      // seconds when we hit the hard limit, start delaying each
3202
      // individual write by 0-1ms to reduce latency variance.  Also,
3203 3204 3205
      // this delay hands over some CPU to the compaction thread in
      // case it is sharing the same core as the writer.
      mutex_.Unlock();
3206
      uint64_t delayed;
J
Jim Paton 已提交
3207
      {
3208
        StopWatch sw(env_, options_.statistics.get(), STALL_L0_SLOWDOWN_COUNT);
J
Jim Paton 已提交
3209
        env_->SleepForMicroseconds(
3210
          SlowdownAmount(versions_->current()->NumLevelFiles(0),
J
Jim Paton 已提交
3211 3212 3213
                         options_.level0_slowdown_writes_trigger,
                         options_.level0_stop_writes_trigger)
        );
3214
        delayed = sw.ElapsedMicros();
J
Jim Paton 已提交
3215
      }
3216
      RecordTick(options_.statistics.get(), STALL_L0_SLOWDOWN_MICROS, delayed);
3217
      stall_level0_slowdown_ += delayed;
J
Jim Paton 已提交
3218
      stall_level0_slowdown_count_++;
3219 3220
      allow_delay = false;  // Do not delay a single write more than once
      mutex_.Lock();
3221
      delayed_writes_++;
3222 3223 3224
    } else if (!force &&
               (mem_->ApproximateMemoryUsage() <= options_.write_buffer_size)) {
      // There is room in current memtable
3225 3226 3227
      if (allow_delay) {
        DelayLoggingAndReset();
      }
3228
      break;
3229
    } else if (imm_.size() == options_.max_write_buffer_number - 1) {
3230
      // We have filled up the current memtable, but the previous
3231 3232
      // ones are still being compacted, so we wait.
      DelayLoggingAndReset();
3233
      Log(options_.info_log, "wait for memtable compaction...\n");
3234
      uint64_t stall;
J
Jim Paton 已提交
3235
      {
3236
        StopWatch sw(env_, options_.statistics.get(),
J
Jim Paton 已提交
3237 3238
          STALL_MEMTABLE_COMPACTION_COUNT);
        bg_cv_.Wait();
3239
        stall = sw.ElapsedMicros();
J
Jim Paton 已提交
3240
      }
3241 3242
      RecordTick(options_.statistics.get(),
                 STALL_MEMTABLE_COMPACTION_MICROS, stall);
3243
      stall_memtable_compaction_ += stall;
J
Jim Paton 已提交
3244
      stall_memtable_compaction_count_++;
3245
    } else if (versions_->current()->NumLevelFiles(0) >=
3246
               options_.level0_stop_writes_trigger) {
3247
      // There are too many level-0 files.
3248 3249
      DelayLoggingAndReset();
      Log(options_.info_log, "wait for fewer level0 files...\n");
3250
      uint64_t stall;
J
Jim Paton 已提交
3251
      {
3252 3253
        StopWatch sw(env_, options_.statistics.get(),
                     STALL_L0_NUM_FILES_COUNT);
J
Jim Paton 已提交
3254
        bg_cv_.Wait();
3255
        stall = sw.ElapsedMicros();
J
Jim Paton 已提交
3256
      }
3257
      RecordTick(options_.statistics.get(), STALL_L0_NUM_FILES_MICROS, stall);
3258
      stall_level0_num_files_ += stall;
J
Jim Paton 已提交
3259
      stall_level0_num_files_count_++;
3260
    } else if (
J
Jim Paton 已提交
3261 3262 3263
        allow_hard_rate_limit_delay &&
        options_.hard_rate_limit > 1.0 &&
        (score = versions_->MaxCompactionScore()) > options_.hard_rate_limit) {
3264
      // Delay a write when the compaction score for any level is too large.
3265
      int max_level = versions_->MaxCompactionScoreLevel();
3266
      mutex_.Unlock();
3267
      uint64_t delayed;
J
Jim Paton 已提交
3268
      {
3269 3270
        StopWatch sw(env_, options_.statistics.get(),
                     HARD_RATE_LIMIT_DELAY_COUNT);
J
Jim Paton 已提交
3271
        env_->SleepForMicroseconds(1000);
3272
        delayed = sw.ElapsedMicros();
J
Jim Paton 已提交
3273
      }
3274
      stall_leveln_slowdown_[max_level] += delayed;
J
Jim Paton 已提交
3275
      stall_leveln_slowdown_count_[max_level]++;
3276
      // Make sure the following value doesn't round to zero.
3277 3278
      uint64_t rate_limit = std::max((delayed / 1000), (uint64_t) 1);
      rate_limit_delay_millis += rate_limit;
3279 3280
      RecordTick(options_.statistics.get(),
                 RATE_LIMIT_DELAY_MILLIS, rate_limit);
J
Jim Paton 已提交
3281 3282 3283 3284
      if (options_.rate_limit_delay_max_milliseconds > 0 &&
          rate_limit_delay_millis >=
          (unsigned)options_.rate_limit_delay_max_milliseconds) {
        allow_hard_rate_limit_delay = false;
3285
      }
3286
      mutex_.Lock();
J
Jim Paton 已提交
3287 3288 3289 3290 3291 3292 3293
    } else if (
        allow_soft_rate_limit_delay &&
        options_.soft_rate_limit > 0.0 &&
        (score = versions_->MaxCompactionScore()) > options_.soft_rate_limit) {
      // Delay a write when the compaction score for any level is too large.
      // TODO: add statistics
      mutex_.Unlock();
J
Jim Paton 已提交
3294
      {
3295 3296
        StopWatch sw(env_, options_.statistics.get(),
                     SOFT_RATE_LIMIT_DELAY_COUNT);
J
Jim Paton 已提交
3297 3298 3299 3300 3301 3302 3303
        env_->SleepForMicroseconds(SlowdownAmount(
          score,
          options_.soft_rate_limit,
          options_.hard_rate_limit)
        );
        rate_limit_delay_millis += sw.ElapsedMicros();
      }
J
Jim Paton 已提交
3304 3305
      allow_soft_rate_limit_delay = false;
      mutex_.Lock();
3306

3307
    } else {
3308 3309 3310 3311 3312
      unique_ptr<WritableFile> lfile;
      MemTable* memtmp = nullptr;

      // Attempt to switch to a new memtable and trigger compaction of old.
      // Do this without holding the dbmutex lock.
3313 3314
      assert(versions_->PrevLogNumber() == 0);
      uint64_t new_log_number = versions_->NewFileNumber();
I
Igor Canadi 已提交
3315
      SuperVersion* new_superversion = nullptr;
3316 3317 3318 3319 3320
      mutex_.Unlock();
      {
        EnvOptions soptions(storage_options_);
        soptions.use_mmap_writes = false;
        DelayLoggingAndReset();
3321 3322
        s = env_->NewWritableFile(LogFileName(options_.wal_dir, new_log_number),
                                  &lfile, soptions);
3323 3324 3325 3326
        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 * options_.write_buffer_size);
3327
          memtmp = new MemTable(internal_comparator_, options_);
I
Igor Canadi 已提交
3328
          new_superversion = new SuperVersion(options_.max_write_buffer_number);
3329 3330 3331
        }
      }
      mutex_.Lock();
3332
      if (!s.ok()) {
H
heyongqiang 已提交
3333
        // Avoid chewing through file number space in a tight loop.
3334
        versions_->ReuseFileNumber(new_log_number);
3335
        assert (!memtmp);
3336 3337
        break;
      }
3338
      logfile_number_ = new_log_number;
3339
      log_.reset(new log::Writer(std::move(lfile)));
3340
      mem_->SetNextLogNumber(logfile_number_);
3341
      imm_.Add(mem_);
3342 3343 3344
      if (force) {
        imm_.FlushRequested();
      }
3345
      mem_ = memtmp;
3346
      mem_->Ref();
3347
      Log(options_.info_log,
K
Kai Liu 已提交
3348 3349
          "New memtable created with log file: #%lu\n",
          (unsigned long)logfile_number_);
3350
      mem_->SetLogNumber(logfile_number_);
3351
      force = false;   // Do not force another compaction if have room
3352
      MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
3353
      *superversion_to_free = InstallSuperVersion(new_superversion);
3354 3355 3356 3357 3358
    }
  }
  return s;
}

I
Igor Canadi 已提交
3359 3360 3361 3362
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3363 3364 3365 3366
Env* DBImpl::GetEnv() const {
  return env_;
}

I
Igor Canadi 已提交
3367 3368 3369 3370
const Options& DBImpl::GetOptions() const {
  return options_;
}

3371 3372 3373
bool DBImpl::GetProperty(const Slice& property, std::string* value) {
  value->clear();

J
jorlow@chromium.org 已提交
3374
  MutexLock l(&mutex_);
3375
  Version* current = versions_->current();
J
jorlow@chromium.org 已提交
3376
  Slice in = property;
3377
  Slice prefix("rocksdb.");
J
jorlow@chromium.org 已提交
3378 3379 3380 3381 3382 3383 3384
  if (!in.starts_with(prefix)) return false;
  in.remove_prefix(prefix.size());

  if (in.starts_with("num-files-at-level")) {
    in.remove_prefix(strlen("num-files-at-level"));
    uint64_t level;
    bool ok = ConsumeDecimalNumber(&in, &level) && in.empty();
3385
    if (!ok || (int)level >= NumberLevels()) {
J
jorlow@chromium.org 已提交
3386 3387
      return false;
    } else {
3388
      char buf[100];
D
dgrogan@chromium.org 已提交
3389
      snprintf(buf, sizeof(buf), "%d",
3390
               current->NumLevelFiles(static_cast<int>(level)));
3391
      *value = buf;
J
jorlow@chromium.org 已提交
3392 3393
      return true;
    }
3394 3395 3396 3397 3398 3399 3400 3401 3402 3403 3404
  } else if (in == "levelstats") {
    char buf[1000];
    snprintf(buf, sizeof(buf),
             "Level Files Size(MB)\n"
             "--------------------\n");
    value->append(buf);

    for (int level = 0; level < NumberLevels(); level++) {
      snprintf(buf, sizeof(buf),
               "%3d %8d %8.0f\n",
               level,
3405
               current->NumLevelFiles(level),
3406
               current->NumLevelBytes(level) / 1048576.0);
3407 3408 3409 3410
      value->append(buf);
    }
    return true;

3411
  } else if (in == "stats") {
M
Mark Callaghan 已提交
3412
    char buf[1000];
3413 3414 3415 3416 3417 3418 3419

    uint64_t wal_bytes = 0;
    uint64_t wal_synced = 0;
    uint64_t user_bytes_written = 0;
    uint64_t write_other = 0;
    uint64_t write_self = 0;
    uint64_t write_with_wal = 0;
3420 3421
    uint64_t total_bytes_written = 0;
    uint64_t total_bytes_read = 0;
M
Mark Callaghan 已提交
3422
    uint64_t micros_up = env_->NowMicros() - started_at_;
3423 3424
    // Add "+1" to make sure seconds_up is > 0 and avoid NaN later
    double seconds_up = (micros_up + 1) / 1000000.0;
3425
    uint64_t total_slowdown = 0;
J
Jim Paton 已提交
3426
    uint64_t total_slowdown_count = 0;
3427 3428 3429 3430
    uint64_t interval_bytes_written = 0;
    uint64_t interval_bytes_read = 0;
    uint64_t interval_bytes_new = 0;
    double   interval_seconds_up = 0;
M
Mark Callaghan 已提交
3431

3432 3433 3434 3435 3436 3437 3438 3439 3440 3441
    Statistics* s = options_.statistics.get();
    if (s) {
      wal_bytes = s->getTickerCount(WAL_FILE_BYTES);
      wal_synced = s->getTickerCount(WAL_FILE_SYNCED);
      user_bytes_written = s->getTickerCount(BYTES_WRITTEN);
      write_other = s->getTickerCount(WRITE_DONE_BY_OTHER);
      write_self = s->getTickerCount(WRITE_DONE_BY_SELF);
      write_with_wal = s->getTickerCount(WRITE_WITH_WAL);
    }

M
Mark Callaghan 已提交
3442
    // Pardon the long line but I think it is easier to read this way.
3443 3444
    snprintf(buf, sizeof(buf),
             "                               Compactions\n"
3445
             "Level  Files Size(MB) Score Time(sec)  Read(MB) Write(MB)    Rn(MB)  Rnp1(MB)  Wnew(MB) RW-Amplify Read(MB/s) Write(MB/s)      Rn     Rnp1     Wnp1     NewW    Count  Ln-stall Stall-cnt\n"
J
Jim Paton 已提交
3446
             "--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------\n"
3447 3448
             );
    value->append(buf);
3449
    for (int level = 0; level < current->NumberLevels(); level++) {
3450
      int files = current->NumLevelFiles(level);
3451
      if (stats_[level].micros > 0 || files > 0) {
M
Mark Callaghan 已提交
3452 3453 3454 3455 3456 3457
        int64_t bytes_read = stats_[level].bytes_readn +
                             stats_[level].bytes_readnp1;
        int64_t bytes_new = stats_[level].bytes_written -
                            stats_[level].bytes_readnp1;
        double amplify = (stats_[level].bytes_readn == 0)
            ? 0.0
3458 3459 3460
            : (stats_[level].bytes_written +
               stats_[level].bytes_readnp1 +
               stats_[level].bytes_readn) /
M
Mark Callaghan 已提交
3461 3462
                (double) stats_[level].bytes_readn;

3463 3464 3465
        total_bytes_read += bytes_read;
        total_bytes_written += stats_[level].bytes_written;

3466 3467
        snprintf(
            buf, sizeof(buf),
3468
            "%3d %8d %8.0f %5.1f %9.0f %9.0f %9.0f %9.0f %9.0f %9.0f %10.1f %9.1f %11.1f %8d %8d %8d %8d %8d %9.1f %9lu\n",
3469 3470
            level,
            files,
3471 3472
            current->NumLevelBytes(level) / 1048576.0,
            current->NumLevelBytes(level) /
3473
                versions_->MaxBytesForLevel(level),
3474
            stats_[level].micros / 1e6,
M
Mark Callaghan 已提交
3475 3476 3477 3478 3479 3480
            bytes_read / 1048576.0,
            stats_[level].bytes_written / 1048576.0,
            stats_[level].bytes_readn / 1048576.0,
            stats_[level].bytes_readnp1 / 1048576.0,
            bytes_new / 1048576.0,
            amplify,
3481 3482
            // +1 to avoid division by 0
            (bytes_read / 1048576.0) / ((stats_[level].micros+1) / 1000000.0),
3483
            (stats_[level].bytes_written / 1048576.0) /
3484
                ((stats_[level].micros+1) / 1000000.0),
M
Mark Callaghan 已提交
3485 3486 3487 3488
            stats_[level].files_in_leveln,
            stats_[level].files_in_levelnp1,
            stats_[level].files_out_levelnp1,
            stats_[level].files_out_levelnp1 - stats_[level].files_in_levelnp1,
3489
            stats_[level].count,
J
Jim Paton 已提交
3490 3491
            stall_leveln_slowdown_[level] / 1000000.0,
            (unsigned long) stall_leveln_slowdown_count_[level]);
3492
        total_slowdown += stall_leveln_slowdown_[level];
J
Jim Paton 已提交
3493
        total_slowdown_count += stall_leveln_slowdown_count_[level];
3494 3495 3496
        value->append(buf);
      }
    }
M
Mark Callaghan 已提交
3497

3498 3499 3500 3501
    interval_bytes_new = user_bytes_written - last_stats_.ingest_bytes_;
    interval_bytes_read = total_bytes_read - last_stats_.compaction_bytes_read_;
    interval_bytes_written =
        total_bytes_written - last_stats_.compaction_bytes_written_;
3502 3503 3504 3505 3506 3507
    interval_seconds_up = seconds_up - last_stats_.seconds_up_;

    snprintf(buf, sizeof(buf), "Uptime(secs): %.1f total, %.1f interval\n",
             seconds_up, interval_seconds_up);
    value->append(buf);

3508 3509 3510 3511 3512 3513 3514 3515 3516 3517 3518 3519 3520 3521 3522 3523 3524 3525
    snprintf(buf, sizeof(buf),
             "Writes cumulative: %llu total, %llu batches, "
             "%.1f per batch, %.2f ingest GB\n",
             (unsigned long long) (write_other + write_self),
             (unsigned long long) write_self,
             (write_other + write_self) / (double) (write_self + 1),
             user_bytes_written / (1048576.0 * 1024));
    value->append(buf);

    snprintf(buf, sizeof(buf),
             "WAL cumulative: %llu WAL writes, %llu WAL syncs, "
             "%.2f writes per sync, %.2f GB written\n",
             (unsigned long long) write_with_wal,
             (unsigned long long ) wal_synced,
             write_with_wal / (double) (wal_synced + 1),
             wal_bytes / (1048576.0 * 1024));
    value->append(buf);

M
Mark Callaghan 已提交
3526
    snprintf(buf, sizeof(buf),
3527 3528
             "Compaction IO cumulative (GB): "
             "%.2f new, %.2f read, %.2f write, %.2f read+write\n",
3529
             user_bytes_written / (1048576.0 * 1024),
3530 3531 3532 3533 3534 3535 3536 3537
             total_bytes_read / (1048576.0 * 1024),
             total_bytes_written / (1048576.0 * 1024),
             (total_bytes_read + total_bytes_written) / (1048576.0 * 1024));
    value->append(buf);

    snprintf(buf, sizeof(buf),
             "Compaction IO cumulative (MB/sec): "
             "%.1f new, %.1f read, %.1f write, %.1f read+write\n",
3538
             user_bytes_written / 1048576.0 / seconds_up,
3539 3540 3541 3542 3543 3544 3545 3546
             total_bytes_read / 1048576.0 / seconds_up,
             total_bytes_written / 1048576.0 / seconds_up,
             (total_bytes_read + total_bytes_written) / 1048576.0 / seconds_up);
    value->append(buf);

    // +1 to avoid divide by 0 and NaN
    snprintf(buf, sizeof(buf),
             "Amplification cumulative: %.1f write, %.1f compaction\n",
3547 3548 3549 3550 3551 3552 3553 3554 3555 3556 3557 3558 3559 3560 3561 3562 3563 3564 3565 3566 3567 3568 3569 3570 3571 3572 3573 3574 3575 3576 3577 3578
             (double) (total_bytes_written + wal_bytes)
                 / (user_bytes_written + 1),
             (double) (total_bytes_written + total_bytes_read + wal_bytes)
                 / (user_bytes_written + 1));
    value->append(buf);

    uint64_t interval_write_other = write_other - last_stats_.write_other_;
    uint64_t interval_write_self = write_self - last_stats_.write_self_;

    snprintf(buf, sizeof(buf),
             "Writes interval: %llu total, %llu batches, "
             "%.1f per batch, %.1f ingest MB\n",
             (unsigned long long) (interval_write_other + interval_write_self),
             (unsigned long long) interval_write_self,
             (double) (interval_write_other + interval_write_self)
                 / (interval_write_self + 1),
             (user_bytes_written - last_stats_.ingest_bytes_) /  1048576.0);
    value->append(buf);

    uint64_t interval_write_with_wal =
        write_with_wal - last_stats_.write_with_wal_;

    uint64_t interval_wal_synced = wal_synced - last_stats_.wal_synced_;
    uint64_t interval_wal_bytes = wal_bytes - last_stats_.wal_bytes_;

    snprintf(buf, sizeof(buf),
             "WAL interval: %llu WAL writes, %llu WAL syncs, "
             "%.2f writes per sync, %.2f MB written\n",
             (unsigned long long) interval_write_with_wal,
             (unsigned long long ) interval_wal_synced,
             interval_write_with_wal / (double) (interval_wal_synced + 1),
             interval_wal_bytes / (1048576.0 * 1024));
M
Mark Callaghan 已提交
3579 3580
    value->append(buf);

3581 3582 3583 3584 3585 3586 3587 3588 3589 3590 3591 3592 3593 3594 3595 3596 3597 3598 3599 3600 3601 3602
    snprintf(buf, sizeof(buf),
             "Compaction IO interval (MB): "
             "%.2f new, %.2f read, %.2f write, %.2f read+write\n",
             interval_bytes_new / 1048576.0,
             interval_bytes_read/ 1048576.0,
             interval_bytes_written / 1048576.0,
             (interval_bytes_read + interval_bytes_written) / 1048576.0);
    value->append(buf);

    snprintf(buf, sizeof(buf),
             "Compaction IO interval (MB/sec): "
             "%.1f new, %.1f read, %.1f write, %.1f read+write\n",
             interval_bytes_new / 1048576.0 / interval_seconds_up,
             interval_bytes_read / 1048576.0 / interval_seconds_up,
             interval_bytes_written / 1048576.0 / interval_seconds_up,
             (interval_bytes_read + interval_bytes_written)
                 / 1048576.0 / interval_seconds_up);
    value->append(buf);

    // +1 to avoid divide by 0 and NaN
    snprintf(buf, sizeof(buf),
             "Amplification interval: %.1f write, %.1f compaction\n",
3603 3604 3605 3606
             (double) (interval_bytes_written + wal_bytes)
                 / (interval_bytes_new + 1),
             (double) (interval_bytes_written + interval_bytes_read + wal_bytes)
                 / (interval_bytes_new + 1));
M
Mark Callaghan 已提交
3607 3608 3609 3610
    value->append(buf);

    snprintf(buf, sizeof(buf),
            "Stalls(secs): %.3f level0_slowdown, %.3f level0_numfiles, "
3611
            "%.3f memtable_compaction, %.3f leveln_slowdown\n",
M
Mark Callaghan 已提交
3612 3613
            stall_level0_slowdown_ / 1000000.0,
            stall_level0_num_files_ / 1000000.0,
3614
            stall_memtable_compaction_ / 1000000.0,
3615
            total_slowdown / 1000000.0);
M
Mark Callaghan 已提交
3616 3617
    value->append(buf);

J
Jim Paton 已提交
3618 3619 3620 3621 3622 3623 3624 3625 3626
    snprintf(buf, sizeof(buf),
            "Stalls(count): %lu level0_slowdown, %lu level0_numfiles, "
            "%lu memtable_compaction, %lu leveln_slowdown\n",
            (unsigned long) stall_level0_slowdown_count_,
            (unsigned long) stall_level0_num_files_count_,
            (unsigned long) stall_memtable_compaction_count_,
            (unsigned long) total_slowdown_count);
    value->append(buf);

3627 3628 3629
    last_stats_.compaction_bytes_read_ = total_bytes_read;
    last_stats_.compaction_bytes_written_ = total_bytes_written;
    last_stats_.ingest_bytes_ = user_bytes_written;
3630
    last_stats_.seconds_up_ = seconds_up;
3631 3632 3633 3634 3635
    last_stats_.wal_bytes_ = wal_bytes;
    last_stats_.wal_synced_ = wal_synced;
    last_stats_.write_with_wal_ = write_with_wal;
    last_stats_.write_other_ = write_other;
    last_stats_.write_self_ = write_self;
3636

3637
    return true;
G
Gabor Cselle 已提交
3638 3639 3640
  } else if (in == "sstables") {
    *value = versions_->current()->DebugString();
    return true;
3641 3642 3643
  } else if (in == "num-immutable-mem-table") {
    *value = std::to_string(imm_.size());
    return true;
J
jorlow@chromium.org 已提交
3644
  }
3645

J
jorlow@chromium.org 已提交
3646 3647 3648 3649 3650 3651 3652 3653 3654 3655 3656 3657 3658 3659 3660 3661 3662 3663 3664 3665 3666 3667 3668 3669 3670 3671 3672 3673 3674
  return false;
}

void DBImpl::GetApproximateSizes(
    const Range* range, int n,
    uint64_t* sizes) {
  // TODO(opt): better implementation
  Version* v;
  {
    MutexLock l(&mutex_);
    versions_->current()->Ref();
    v = versions_->current();
  }

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

3675 3676 3677 3678 3679 3680 3681
inline void DBImpl::DelayLoggingAndReset() {
  if (delayed_writes_ > 0) {
    Log(options_.info_log, "delayed %d write...\n", delayed_writes_ );
    delayed_writes_ = 0;
  }
}

3682 3683 3684
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
3685 3686 3687 3688
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
    Log(options_.info_log, "DeleteFile %s failed.\n", name.c_str());
3689 3690 3691
    return Status::InvalidArgument("Invalid file name");
  }

3692 3693 3694 3695 3696 3697 3698 3699 3700 3701 3702 3703 3704 3705
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
      Log(options_.info_log, "DeleteFile %s failed.\n", name.c_str());
      return Status::NotSupported("Delete only supported for archived logs");
    }
    status = env_->DeleteFile(options_.wal_dir + "/" + name.c_str());
    if (!status.ok()) {
      Log(options_.info_log, "DeleteFile %s failed.\n", name.c_str());
    }
    return status;
  }

3706 3707 3708
  int level;
  FileMetaData metadata;
  int maxlevel = NumberLevels();
3709
  VersionEdit edit;
I
Igor Canadi 已提交
3710
  DeletionState deletion_state(0, true);
D
Dhruba Borthakur 已提交
3711 3712 3713 3714
  {
    MutexLock l(&mutex_);
    status = versions_->GetMetadataForFile(number, &level, &metadata);
    if (!status.ok()) {
3715 3716
      Log(options_.info_log, "DeleteFile %s failed. File not found\n",
                             name.c_str());
D
Dhruba Borthakur 已提交
3717 3718 3719
      return Status::InvalidArgument("File not found");
    }
    assert((level > 0) && (level < maxlevel));
3720

D
Dhruba Borthakur 已提交
3721 3722
    // If the file is being compacted no need to delete.
    if (metadata.being_compacted) {
3723
      Log(options_.info_log,
3724
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
D
Dhruba Borthakur 已提交
3725
      return Status::OK();
3726 3727
    }

D
Dhruba Borthakur 已提交
3728 3729 3730 3731
    // 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.
    for (int i = level + 1; i < maxlevel; i++) {
3732
      if (versions_->current()->NumLevelFiles(i) != 0) {
D
Dhruba Borthakur 已提交
3733
        Log(options_.info_log,
3734
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
D
Dhruba Borthakur 已提交
3735 3736 3737 3738 3739
        return Status::InvalidArgument("File not in last level");
      }
    }
    edit.DeleteFile(level, number);
    status = versions_->LogAndApply(&edit, &mutex_);
I
Igor Canadi 已提交
3740 3741 3742
    if (status.ok()) {
      InstallSuperVersion(deletion_state);
    }
I
Igor Canadi 已提交
3743
    FindObsoleteFiles(deletion_state, false);
D
Dhruba Borthakur 已提交
3744
  } // lock released here
I
Igor Canadi 已提交
3745
  LogFlush(options_.info_log);
I
Igor Canadi 已提交
3746 3747
  // remove files outside the db-lock
  PurgeObsoleteFiles(deletion_state);
3748 3749 3750
  return status;
}

3751
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData> *metadata) {
3752 3753 3754 3755
  MutexLock l(&mutex_);
  return versions_->GetLiveFilesMetaData(metadata);
}

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 3781 3782
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 已提交
3783 3784 3785
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
Status DB::Put(const WriteOptions& opt, const Slice& key, const Slice& value) {
3786 3787 3788 3789
  // 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);
J
jorlow@chromium.org 已提交
3790 3791 3792 3793 3794 3795 3796 3797 3798 3799
  batch.Put(key, value);
  return Write(opt, &batch);
}

Status DB::Delete(const WriteOptions& opt, const Slice& key) {
  WriteBatch batch;
  batch.Delete(key);
  return Write(opt, &batch);
}

3800 3801 3802 3803 3804 3805 3806
Status DB::Merge(const WriteOptions& opt, const Slice& key,
                 const Slice& value) {
  WriteBatch batch;
  batch.Merge(key, value);
  return Write(opt, &batch);
}

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

J
Jim Paton 已提交
3809
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
3810
  *dbptr = nullptr;
H
Haobo Xu 已提交
3811
  EnvOptions soptions;
J
jorlow@chromium.org 已提交
3812

3813
  if (options.block_cache != nullptr && options.no_block_cache) {
3814
    return Status::InvalidArgument(
3815
        "no_block_cache is true while block_cache is not nullptr");
3816
  }
3817

J
jorlow@chromium.org 已提交
3818
  DBImpl* impl = new DBImpl(options, dbname);
3819 3820 3821 3822 3823 3824 3825
  Status s = impl->env_->CreateDirIfMissing(impl->options_.wal_dir);
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
3826 3827 3828 3829
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
3830
  impl->mutex_.Lock();
3831
  VersionEdit edit;
3832
  s = impl->Recover(&edit); // Handles create_if_missing, error_if_exists
J
jorlow@chromium.org 已提交
3833
  if (s.ok()) {
3834
    uint64_t new_log_number = impl->versions_->NewFileNumber();
3835
    unique_ptr<WritableFile> lfile;
H
Haobo Xu 已提交
3836
    soptions.use_mmap_writes = false;
3837
    s = impl->options_.env->NewWritableFile(
3838 3839 3840 3841
      LogFileName(impl->options_.wal_dir, new_log_number),
      &lfile,
      soptions
    );
J
jorlow@chromium.org 已提交
3842
    if (s.ok()) {
3843
      lfile->SetPreallocationBlockSize(1.1 * impl->options_.write_buffer_size);
3844
      edit.SetLogNumber(new_log_number);
3845
      impl->logfile_number_ = new_log_number;
3846
      impl->log_.reset(new log::Writer(std::move(lfile)));
3847
      s = impl->versions_->LogAndApply(&edit, &impl->mutex_);
J
jorlow@chromium.org 已提交
3848 3849
    }
    if (s.ok()) {
I
Igor Canadi 已提交
3850
      delete impl->InstallSuperVersion(new DBImpl::SuperVersion());
3851
      impl->mem_->SetLogNumber(impl->logfile_number_);
J
jorlow@chromium.org 已提交
3852
      impl->DeleteObsoleteFiles();
3853
      impl->MaybeScheduleFlushOrCompaction();
3854
      impl->MaybeScheduleLogDBDeployStats();
J
jorlow@chromium.org 已提交
3855 3856
    }
  }
3857

3858 3859
  if (s.ok() && impl->options_.compaction_style == kCompactionStyleUniversal) {
    Version* current = impl->versions_->current();
3860
    for (int i = 1; i < impl->NumberLevels(); i++) {
3861
      int num_files = current->NumLevelFiles(i);
3862 3863 3864 3865 3866 3867 3868 3869
      if (num_files > 0) {
        s = Status::InvalidArgument("Not all files are at level 0. Cannot "
          "open with universal compaction style.");
        break;
      }
    }
  }

3870 3871
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
3872 3873 3874 3875 3876 3877 3878 3879
  if (s.ok()) {
    *dbptr = impl;
  } else {
    delete impl;
  }
  return s;
}

3880 3881 3882
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
3883
Status DestroyDB(const std::string& dbname, const Options& options) {
3884 3885 3886 3887 3888
  const InternalKeyComparator comparator(options.comparator);
  const InternalFilterPolicy filter_policy(options.filter_policy);
  const Options& soptions(SanitizeOptions(
    dbname, &comparator, &filter_policy, options));
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
3889
  std::vector<std::string> filenames;
3890 3891
  std::vector<std::string> archiveFiles;

3892
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
3893 3894
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
3895 3896 3897 3898 3899 3900 3901

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

J
jorlow@chromium.org 已提交
3903 3904 3905 3906 3907
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
3908 3909
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
3910 3911 3912
  if (result.ok()) {
    uint64_t number;
    FileType type;
D
dgrogan@chromium.org 已提交
3913
    for (size_t i = 0; i < filenames.size(); i++) {
3914
      if (ParseFileName(filenames[i], &number, &type) &&
3915
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
3916 3917 3918
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
3919 3920
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
3921 3922 3923
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
3924 3925 3926 3927 3928
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3929

3930
    env->GetChildren(archivedir, &archiveFiles);
3931 3932
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
3933 3934
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
3935
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
3936 3937 3938 3939 3940
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3941
    // ignore case where no archival directory is present.
3942
    env->DeleteDir(archivedir);
3943

J
jorlow@chromium.org 已提交
3944
    env->UnlockFile(lock);  // Ignore error since state is already gone
3945
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
3946
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
3947
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
3948 3949 3950 3951
  }
  return result;
}

3952 3953
//
// A global method that can dump out the build version
3954
void dumpLeveldbBuildVersion(Logger * log) {
3955
  Log(log, "Git sha %s", rocksdb_build_git_sha);
3956
  Log(log, "Compile time %s %s",
3957
      rocksdb_build_compile_time, rocksdb_build_compile_date);
3958 3959
}

3960
}  // namespace rocksdb