db_impl.cc 163.1 KB
Newer Older
1 2 3 4 5
//  Copyright (c) 2013, Facebook, Inc.  All rights reserved.
//  This source code is licensed under the BSD-style license found in the
//  LICENSE file in the root directory of this source tree. An additional grant
//  of patent rights can be found in the PATENTS file in the same directory.
//
J
jorlow@chromium.org 已提交
6 7 8 9 10 11
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.

#include "db/db_impl.h"

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

I
Igor Canadi 已提交
16
#include <inttypes.h>
17 18
#include <stdint.h>

J
jorlow@chromium.org 已提交
19
#include <algorithm>
20 21
#include <climits>
#include <cstdio>
J
jorlow@chromium.org 已提交
22
#include <set>
23
#include <stdexcept>
24
#include <string>
25
#include <unordered_map>
26
#include <unordered_set>
T
Tomislav Novak 已提交
27
#include <utility>
28
#include <vector>
29

J
jorlow@chromium.org 已提交
30
#include "db/builder.h"
I
Igor Canadi 已提交
31
#include "db/compaction_job.h"
32
#include "db/db_iter.h"
K
kailiu 已提交
33
#include "db/dbformat.h"
34
#include "db/event_helpers.h"
J
jorlow@chromium.org 已提交
35
#include "db/filename.h"
36 37
#include "db/flush_job.h"
#include "db/forward_iterator.h"
I
Igor Canadi 已提交
38
#include "db/job_context.h"
J
jorlow@chromium.org 已提交
39 40
#include "db/log_reader.h"
#include "db/log_writer.h"
V
Venkatesh Radhakrishnan 已提交
41
#include "db/managed_iterator.h"
J
jorlow@chromium.org 已提交
42
#include "db/memtable.h"
K
kailiu 已提交
43
#include "db/memtable_list.h"
44
#include "db/merge_context.h"
45
#include "db/merge_helper.h"
J
jorlow@chromium.org 已提交
46
#include "db/table_cache.h"
K
kailiu 已提交
47
#include "db/table_properties_collector.h"
48
#include "db/transaction_log_impl.h"
J
jorlow@chromium.org 已提交
49 50
#include "db/version_set.h"
#include "db/write_batch_internal.h"
A
agiardullo 已提交
51
#include "db/write_callback.h"
52 53
#include "db/writebuffer.h"
#include "port/likely.h"
54
#include "port/port.h"
I
Igor Canadi 已提交
55
#include "rocksdb/cache.h"
56 57
#include "rocksdb/compaction_filter.h"
#include "rocksdb/db.h"
I
Islam AbdelRahman 已提交
58
#include "rocksdb/delete_scheduler.h"
59 60 61 62
#include "rocksdb/env.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
S
Siying Dong 已提交
63
#include "rocksdb/table.h"
64
#include "rocksdb/version.h"
J
jorlow@chromium.org 已提交
65
#include "table/block.h"
66
#include "table/block_based_table_factory.h"
J
jorlow@chromium.org 已提交
67
#include "table/merger.h"
K
kailiu 已提交
68
#include "table/table_builder.h"
J
jorlow@chromium.org 已提交
69
#include "table/two_level_iterator.h"
70
#include "util/auto_roll_logger.h"
K
kailiu 已提交
71
#include "util/autovector.h"
72
#include "util/build_version.h"
J
jorlow@chromium.org 已提交
73
#include "util/coding.h"
I
Igor Canadi 已提交
74
#include "util/compression.h"
75
#include "util/crc32c.h"
76
#include "util/db_info_dumper.h"
77
#include "util/file_reader_writer.h"
78
#include "util/file_util.h"
79
#include "util/hash_linklist_rep.h"
80 81
#include "util/hash_skiplist_rep.h"
#include "util/iostats_context_imp.h"
H
Haobo Xu 已提交
82
#include "util/log_buffer.h"
83
#include "util/logging.h"
J
jorlow@chromium.org 已提交
84
#include "util/mutexlock.h"
85
#include "util/perf_context_imp.h"
86
#include "util/stop_watch.h"
87
#include "util/string_util.h"
88
#include "util/sync_point.h"
89 90
#include "util/thread_status_updater.h"
#include "util/thread_status_util.h"
V
Venkatesh Radhakrishnan 已提交
91
#include "util/xfunc.h"
J
jorlow@chromium.org 已提交
92

93
namespace rocksdb {
J
jorlow@chromium.org 已提交
94

95
const std::string kDefaultColumnFamilyName("default");
96

I
Igor Canadi 已提交
97
void DumpRocksDBBuildVersion(Logger * log);
98

S
Stanislau Hlebik 已提交
99 100
struct DBImpl::WriteContext {
  autovector<SuperVersion*> superversions_to_free_;
101
  autovector<MemTable*> memtables_to_free_;
S
Stanislau Hlebik 已提交
102 103 104 105 106

  ~WriteContext() {
    for (auto& sv : superversions_to_free_) {
      delete sv;
    }
107 108 109
    for (auto& m : memtables_to_free_) {
      delete m;
    }
S
Stanislau Hlebik 已提交
110 111 112
  }
};

J
jorlow@chromium.org 已提交
113 114 115
Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
                        const Options& src) {
116
  auto db_options = SanitizeOptions(dbname, DBOptions(src));
117
  auto cf_options = SanitizeOptions(db_options, icmp, ColumnFamilyOptions(src));
118 119 120 121 122
  return Options(db_options, cf_options);
}

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

124 125
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
I
Igor Canadi 已提交
126 127 128 129 130
    int max_max_open_files = port::GetMaxOpenFiles();
    if (max_max_open_files == -1) {
      max_max_open_files = 1000000;
    }
    ClipToRange(&result.max_open_files, 20, max_max_open_files);
131
  }
132

133
  if (result.info_log == nullptr) {
K
Kai Liu 已提交
134 135
    Status s = CreateLoggerFromOptions(dbname, result.db_log_dir, src.env,
                                       result, &result.info_log);
J
jorlow@chromium.org 已提交
136 137
    if (!s.ok()) {
      // No place suitable for logging
138
      result.info_log = nullptr;
J
jorlow@chromium.org 已提交
139 140
    }
  }
141 142 143 144
  result.env->IncBackgroundThreadsIfNeeded(src.max_background_compactions,
                                           Env::Priority::LOW);
  result.env->IncBackgroundThreadsIfNeeded(src.max_background_flushes,
                                           Env::Priority::HIGH);
145

I
Igor Canadi 已提交
146
  if (result.rate_limiter.get() != nullptr) {
147 148 149 150 151
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

152 153 154 155
  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
156
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
157
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
158
  }
159

160
  if (result.db_paths.size() == 0) {
161
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
162 163
  }

164 165 166 167
  if (result.compaction_readahead_size > 0) {
    result.new_table_reader_for_compaction_inputs = true;
  }

J
jorlow@chromium.org 已提交
168 169 170
  return result;
}

171 172
namespace {

173 174
Status SanitizeOptionsByTable(
    const DBOptions& db_opts,
175 176 177
    const std::vector<ColumnFamilyDescriptor>& column_families) {
  Status s;
  for (auto cf : column_families) {
178
    s = cf.options.table_factory->SanitizeOptions(db_opts, cf.options);
179 180 181 182 183 184 185
    if (!s.ok()) {
      return s;
    }
  }
  return Status::OK();
}

186
CompressionType GetCompressionFlush(const ImmutableCFOptions& ioptions) {
187 188 189 190 191 192
  // 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;

193
  if (ioptions.compaction_style == kCompactionStyleUniversal) {
194
    can_compress =
195
        (ioptions.compaction_options_universal.compression_size_percent < 0);
196 197
  } else {
    // For leveled compress when min_level_to_compress == 0.
198 199
    can_compress = ioptions.compression_per_level.empty() ||
                   ioptions.compression_per_level[0] != kNoCompression;
200 201 202
  }

  if (can_compress) {
203
    return ioptions.compression;
204 205 206 207
  } else {
    return kNoCompression;
  }
}
I
Igor Canadi 已提交
208

209
void DumpSupportInfo(Logger* logger) {
I
Igor Canadi 已提交
210 211 212 213 214 215 216 217
  Log(InfoLogLevel::INFO_LEVEL, logger, "Compression algorithms supported:");
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tSnappy supported: %d",
      Snappy_Supported());
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tZlib supported: %d",
      Zlib_Supported());
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tBzip supported: %d",
      BZip2_Supported());
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tLZ4 supported: %d", LZ4_Supported());
218 219
  Log(InfoLogLevel::INFO_LEVEL, logger, "Fast CRC32 supported: %d",
      crc32c::IsFastCrc32Supported());
I
Igor Canadi 已提交
220 221
}

222
}  // namespace
223

I
Igor Canadi 已提交
224
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
J
jorlow@chromium.org 已提交
225
    : env_(options.env),
H
heyongqiang 已提交
226
      dbname_(dbname),
227 228
      db_options_(SanitizeOptions(dbname, options)),
      stats_(db_options_.statistics.get()),
229
      db_lock_(nullptr),
I
Igor Canadi 已提交
230
      mutex_(stats_, env_, DB_MUTEX_WAIT_MICROS, options.use_adaptive_mutex),
I
Igor Canadi 已提交
231
      shutting_down_(false),
J
jorlow@chromium.org 已提交
232
      bg_cv_(&mutex_),
233
      logfile_number_(0),
234
      log_dir_synced_(false),
I
Igor Canadi 已提交
235
      log_empty_(true),
236
      default_cf_handle_(nullptr),
237
      log_sync_cv_(&mutex_),
I
Igor Canadi 已提交
238 239
      total_log_size_(0),
      max_total_in_memory_state_(0),
S
sdong 已提交
240
      is_snapshot_supported_(true),
241
      write_buffer_(options.db_write_buffer_size),
S
sdong 已提交
242 243
      write_controller_(options.delayed_write_rate),
      last_batch_group_size_(0),
244 245
      unscheduled_flushes_(0),
      unscheduled_compactions_(0),
246
      bg_compaction_scheduled_(0),
247
      bg_manual_only_(0),
248
      bg_flush_scheduled_(0),
249
      manual_compaction_(nullptr),
250
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
251 252 253
      delete_obsolete_files_next_run_(
          options.env->NowMicros() +
          db_options_.delete_obsolete_files_period_micros),
254
      last_stats_dump_time_microsec_(0),
255
      next_job_id_(1),
256
      flush_on_destroy_(false),
257
      env_options_(db_options_),
I
Igor Canadi 已提交
258 259 260
#ifndef ROCKSDB_LITE
      wal_manager_(db_options_, env_options_),
#endif  // ROCKSDB_LITE
I
Igor Canadi 已提交
261
      event_logger_(db_options_.info_log.get()),
262
      bg_work_gate_closed_(false),
263
      refitting_level_(false),
264
      opened_successfully_(false) {
H
heyongqiang 已提交
265
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
266

J
jorlow@chromium.org 已提交
267
  // Reserve ten files or so for other uses and give the rest to TableCache.
268
  // Give a large number for setting of "infinite" open files.
269 270
  const int table_cache_size = (db_options_.max_open_files == -1) ?
        4194304 : db_options_.max_open_files - 10;
271
  table_cache_ =
272
      NewLRUCache(table_cache_size, db_options_.table_cache_numshardbits);
273

274
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
275 276
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
I
Igor Canadi 已提交
277 278
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
279

I
Igor Canadi 已提交
280
  DumpRocksDBBuildVersion(db_options_.info_log.get());
281 282
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
283
  DumpSupportInfo(db_options_.info_log.get());
284

285
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
286 287
}

288
// Will lock the mutex_,  will wait for completion if wait is true
289
void DBImpl::CancelAllBackgroundWork(bool wait) {
290
  InstrumentedMutexLock l(&mutex_);
291
  shutting_down_.store(true, std::memory_order_release);
292
  bg_cv_.SignalAll();
293 294 295 296
  if (!wait) {
    return;
  }
  // Wait for background work to finish
297
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
298 299
    bg_cv_.Wait();
  }
300 301
}

J
jorlow@chromium.org 已提交
302
DBImpl::~DBImpl() {
303
  mutex_.Lock();
304

305
  if (!shutting_down_.load(std::memory_order_acquire) && flush_on_destroy_) {
306
    for (auto cfd : *versions_->GetColumnFamilySet()) {
307
      if (!cfd->IsDropped() && !cfd->mem()->IsEmpty()) {
308 309 310 311
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
312
        cfd->Unref();
313 314
      }
    }
315
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
316
  }
317 318 319
  mutex_.Unlock();
  // CancelAllBackgroundWork called with false means we just set the shutdown
  // marker. After this we do a variant of the waiting and unschedule work
320 321
  // (to consider: moving all the waiting into CancelAllBackgroundWork(true))
  CancelAllBackgroundWork(false);
322 323 324 325 326 327 328
  int compactions_unscheduled = env_->UnSchedule(this, Env::Priority::LOW);
  int flushes_unscheduled = env_->UnSchedule(this, Env::Priority::HIGH);
  mutex_.Lock();
  bg_compaction_scheduled_ -= compactions_unscheduled;
  bg_flush_scheduled_ -= flushes_unscheduled;

  // Wait for background work to finish
329
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
330 331
    bg_cv_.Wait();
  }
332
  EraseThreadStatusDbInfo();
I
Igor Canadi 已提交
333 334
  flush_scheduler_.Clear();

335 336 337 338 339 340 341 342 343 344 345 346 347
  while (!flush_queue_.empty()) {
    auto cfd = PopFirstFromFlushQueue();
    if (cfd->Unref()) {
      delete cfd;
    }
  }
  while (!compaction_queue_.empty()) {
    auto cfd = PopFirstFromCompactionQueue();
    if (cfd->Unref()) {
      delete cfd;
    }
  }

I
Igor Canadi 已提交
348 349 350 351 352
  if (default_cf_handle_ != nullptr) {
    // we need to delete handle outside of lock because it does its own locking
    mutex_.Unlock();
    delete default_cf_handle_;
    mutex_.Lock();
353 354
  }

I
Igor Canadi 已提交
355 356 357 358 359 360 361 362 363 364
  // Clean up obsolete files due to SuperVersion release.
  // (1) Need to delete to obsolete files before closing because RepairDB()
  // scans all existing files in the file system and builds manifest file.
  // Keeping obsolete files confuses the repair process.
  // (2) Need to check if we Open()/Recover() the DB successfully before
  // deleting because if VersionSet recover fails (may be due to corrupted
  // manifest file), it is not able to identify live files correctly. As a
  // result, all "live" files can get deleted by accident. However, corrupted
  // manifest is recoverable by RepairDB().
  if (opened_successfully_) {
365
    JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
366
    FindObsoleteFiles(&job_context, true);
367 368

    mutex_.Unlock();
I
Igor Canadi 已提交
369
    // manifest number starting from 2
I
Igor Canadi 已提交
370 371 372
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
373
    }
I
Igor Canadi 已提交
374
    job_context.Clean();
375
    mutex_.Lock();
376 377
  }

378 379 380
  for (auto l : logs_to_free_) {
    delete l;
  }
381 382 383
  for (auto& log : logs_) {
    log.ClearWriter();
  }
384
  logs_.clear();
385

386
  // versions need to be destroyed before table_cache since it can hold
387 388
  // references to table_cache.
  versions_.reset();
389
  mutex_.Unlock();
I
Igor Canadi 已提交
390 391 392
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
393

394
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
395 396 397
}

Status DBImpl::NewDB() {
398
  VersionEdit new_db;
399
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
400 401 402
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

403 404
  Status s;

405 406
  Log(InfoLogLevel::INFO_LEVEL,
      db_options_.info_log, "Creating manifest 1 \n");
J
jorlow@chromium.org 已提交
407 408
  const std::string manifest = DescriptorFileName(dbname_, 1);
  {
409 410 411 412 413 414 415 416 417 418
    unique_ptr<WritableFile> file;
    EnvOptions env_options = env_->OptimizeForManifestWrite(env_options_);
    s = env_->NewWritableFile(manifest, &file, env_options);
    if (!s.ok()) {
      return s;
    }
    file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
    unique_ptr<WritableFileWriter> file_writer(
        new WritableFileWriter(std::move(file), env_options));
    log::Writer log(std::move(file_writer));
J
jorlow@chromium.org 已提交
419 420 421
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
S
sdong 已提交
422 423 424
    if (s.ok()) {
      s = SyncManifest(env_, &db_options_, log.file());
    }
J
jorlow@chromium.org 已提交
425 426 427
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
428
    s = SetCurrentFile(env_, dbname_, 1, directories_.GetDbDir());
J
jorlow@chromium.org 已提交
429 430 431 432 433 434 435
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
436
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
437 438
    // No change needed
  } else {
439 440
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
441 442 443 444
    *s = Status::OK();
  }
}

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

453
void DBImpl::PrintStatistics() {
454
  auto dbstats = db_options_.statistics.get();
455
  if (dbstats) {
Y
Yueh-Hsuan Chiang 已提交
456
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
M
Mark Callaghan 已提交
457
        "STATISTICS:\n %s",
458
        dbstats->ToString().c_str());
459 460 461
  }
}

462
void DBImpl::MaybeDumpStats() {
463
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
464 465 466 467

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
468
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
469 470 471 472 473 474
      <= 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;
475

476
#ifndef ROCKSDB_LITE
477 478 479
    bool tmp1 = false;
    bool tmp2 = false;
    DBPropertyType cf_property_type =
480
        GetPropertyType(DB::Properties::kCFStats, &tmp1, &tmp2);
481
    DBPropertyType db_property_type =
482
        GetPropertyType(DB::Properties::kDBStats, &tmp1, &tmp2);
H
Haobo Xu 已提交
483
    std::string stats;
484
    {
485
      InstrumentedMutexLock l(&mutex_);
486
      for (auto cfd : *versions_->GetColumnFamilySet()) {
487
        cfd->internal_stats()->GetStringProperty(cf_property_type,
488 489
                                                 DB::Properties::kCFStats,
                                                 &stats);
490
      }
491
      default_cf_internal_stats_->GetStringProperty(db_property_type,
492 493
                                                    DB::Properties::kDBStats,
                                                    &stats);
494
    }
Y
Yueh-Hsuan Chiang 已提交
495
    Log(InfoLogLevel::WARN_LEVEL,
496
        db_options_.info_log, "------- DUMPING STATS -------");
Y
Yueh-Hsuan Chiang 已提交
497
    Log(InfoLogLevel::WARN_LEVEL,
498
        db_options_.info_log, "%s", stats.c_str());
499
#endif  // !ROCKSDB_LITE
500

501
    PrintStatistics();
502 503 504
  }
}

I
Igor Canadi 已提交
505
// * Returns the list of live files in 'sst_live'
I
Igor Canadi 已提交
506
// If it's doing full scan:
I
Igor Canadi 已提交
507 508
// * Returns the list of all files in the filesystem in
// 'full_scan_candidate_files'.
I
Igor Canadi 已提交
509
// Otherwise, gets obsolete files from VersionSet.
I
Igor Canadi 已提交
510 511
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
512
//  db_options_.delete_obsolete_files_period_micros
I
Igor Canadi 已提交
513
// force = true -- force the full scan
I
Igor Canadi 已提交
514
void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
I
Igor Canadi 已提交
515
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
516 517
  mutex_.AssertHeld();

518
  // if deletion is disabled, do nothing
519
  if (disable_delete_obsolete_files_ > 0) {
520 521 522
    return;
  }

523 524 525 526 527
  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;
528
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
529 530 531
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
I
Igor Canadi 已提交
532
    if (delete_obsolete_files_next_run_ < now_micros) {
533
      doing_the_full_scan = true;
I
Igor Canadi 已提交
534 535
      delete_obsolete_files_next_run_ =
          now_micros + db_options_.delete_obsolete_files_period_micros;
536 537 538
    }
  }

I
Igor Canadi 已提交
539 540
  // don't delete files that might be currently written to from compaction
  // threads
541 542 543
  // Since job_context->min_pending_output is set, until file scan finishes,
  // mutex_ cannot be released. Otherwise, we might see no min_pending_output
  // here but later find newer generated unfinalized files while scannint.
I
Igor Canadi 已提交
544 545 546 547 548 549 550
  if (!pending_outputs_.empty()) {
    job_context->min_pending_output = *pending_outputs_.begin();
  } else {
    // delete all of them
    job_context->min_pending_output = std::numeric_limits<uint64_t>::max();
  }

551 552
  // Get obsolete files.  This function will also update the list of
  // pending files in VersionSet().
I
Igor Canadi 已提交
553 554
  versions_->GetObsoleteFiles(&job_context->sst_delete_files,
                              job_context->min_pending_output);
I
Igor Canadi 已提交
555

I
Igor Canadi 已提交
556
  // store the current filenum, lognum, etc
557
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
558
  job_context->pending_manifest_file_number =
559
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
560
  job_context->log_number = versions_->MinLogNumber();
561
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
562

I
Igor Canadi 已提交
563
  versions_->AddLiveFiles(&job_context->sst_live);
564
  if (doing_the_full_scan) {
I
Igor Canadi 已提交
565 566
    for (uint32_t path_id = 0; path_id < db_options_.db_paths.size();
         path_id++) {
567 568 569
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
570
      env_->GetChildren(db_options_.db_paths[path_id].path,
571
                        &files);  // Ignore errors
572
      for (std::string file : files) {
I
Igor Canadi 已提交
573
        // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
I
Igor Canadi 已提交
574 575
        job_context->full_scan_candidate_files.emplace_back("/" + file,
                                                            path_id);
576 577
      }
    }
578 579

    //Add log files in wal_dir
580
    if (db_options_.wal_dir != dbname_) {
581
      std::vector<std::string> log_files;
582
      env_->GetChildren(db_options_.wal_dir, &log_files);  // Ignore errors
583
      for (std::string log_file : log_files) {
I
Igor Canadi 已提交
584
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
585 586
      }
    }
587
    // Add info log files in db_log_dir
588
    if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) {
589
      std::vector<std::string> info_log_files;
590 591
      // Ignore errors
      env_->GetChildren(db_options_.db_log_dir, &info_log_files);
592
      for (std::string log_file : info_log_files) {
I
Igor Canadi 已提交
593
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
594 595
      }
    }
596
  }
597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627

  if (!alive_log_files_.empty()) {
    uint64_t min_log_number = versions_->MinLogNumber();
    // find newly obsoleted log files
    while (alive_log_files_.begin()->number < min_log_number) {
      auto& earliest = *alive_log_files_.begin();
      job_context->log_delete_files.push_back(earliest.number);
      total_log_size_ -= earliest.size;
      alive_log_files_.pop_front();
      // Current log should always stay alive since it can't have
      // number < MinLogNumber().
      assert(alive_log_files_.size());
    }
    while (!logs_.empty() && logs_.front().number < min_log_number) {
      auto& log = logs_.front();
      if (log.getting_synced) {
        log_sync_cv_.Wait();
        // logs_ could have changed while we were waiting.
        continue;
      }
      logs_to_free_.push_back(log.ReleaseWriter());
      logs_.pop_front();
    }
    // Current log cannot be obsolete.
    assert(!logs_.empty());
  }

  // We're just cleaning up for DB::Write().
  assert(job_context->logs_to_free.empty());
  job_context->logs_to_free = logs_to_free_;
  logs_to_free_.clear();
628 629
}

630
namespace {
I
Igor Canadi 已提交
631 632
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
633 634 635 636 637
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
638
    return (first.path_id > second.path_id);
639 640 641 642
  }
}
};  // namespace

D
Dhruba Borthakur 已提交
643
// Diffs the files listed in filenames and those that do not
I
Igor Canadi 已提交
644
// belong to live files are posibly removed. Also, removes all the
645
// files in sst_delete_files and log_delete_files.
646
// It is not necessary to hold the mutex when invoking this method.
I
Igor Canadi 已提交
647
void DBImpl::PurgeObsoleteFiles(const JobContext& state) {
648 649
  // we'd better have sth to delete
  assert(state.HaveSomethingToDelete());
650

I
Igor Canadi 已提交
651 652 653 654
  // 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 已提交
655 656
    return;
  }
657

658
  // Now, convert live list to an unordered map, WITHOUT mutex held;
659
  // set is slow.
660
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
I
Igor Canadi 已提交
661
  for (const FileDescriptor& fd : state.sst_live) {
662 663
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
664

I
Igor Canadi 已提交
665
  auto candidate_files = state.full_scan_candidate_files;
I
Igor Canadi 已提交
666 667 668
  candidate_files.reserve(candidate_files.size() +
                          state.sst_delete_files.size() +
                          state.log_delete_files.size());
K
kailiu 已提交
669 670
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
671
  for (auto file : state.sst_delete_files) {
672 673 674
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
675
    delete file;
I
Igor Canadi 已提交
676 677
  }

K
kailiu 已提交
678 679
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
680 681
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
682 683
    }
  }
684

K
kailiu 已提交
685
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
686
  // file twice
687
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
688 689
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
690

691
  std::vector<std::string> old_info_log_files;
692
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
693 694 695
  for (const auto& candidate_file : candidate_files) {
    std::string to_delete = candidate_file.file_name;
    uint32_t path_id = candidate_file.path_id;
K
kailiu 已提交
696 697 698
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
699
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
700 701
      continue;
    }
J
jorlow@chromium.org 已提交
702

K
kailiu 已提交
703 704 705 706 707 708 709 710
    bool keep = true;
    switch (type) {
      case kLogFile:
        keep = ((number >= state.log_number) ||
                (number == state.prev_log_number));
        break;
      case kDescriptorFile:
        // Keep my manifest file, and any newer incarnations'
711
        // (can happen during manifest roll)
K
kailiu 已提交
712 713 714
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
I
Igor Canadi 已提交
715 716 717 718
        // If the second condition is not there, this makes
        // DontDeletePendingOutputs fail
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
               number >= state.min_pending_output;
K
kailiu 已提交
719 720 721
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
722 723 724 725
        // be recorded in pending_outputs_, which is inserted into "live".
        // Also, SetCurrentFile creates a temp file when writing out new
        // manifest, which is equal to state.pending_manifest_file_number. We
        // should not delete that file
726
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
727
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
728 729 730 731
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
732
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
733
        }
K
kailiu 已提交
734 735 736 737 738 739 740 741 742 743 744 745 746
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

747
    std::string fname;
K
kailiu 已提交
748 749
    if (type == kTableFile) {
      // evict from cache
750
      TableCache::Evict(table_cache_.get(), number);
751
      fname = TableFileName(db_options_.db_paths, number, path_id);
752
    } else {
753 754
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
755
    }
756

757
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
758
    if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 ||
759
                              db_options_.WAL_size_limit_MB > 0)) {
I
Igor Canadi 已提交
760
      wal_manager_.ArchiveWALFile(fname, number);
761 762 763
      continue;
    }
#endif  // !ROCKSDB_LITE
I
Islam AbdelRahman 已提交
764
    Status file_deletion_status;
765 766
    if (type == kTableFile && path_id == 0) {
      file_deletion_status = DeleteOrMoveToTrash(&db_options_, fname);
I
Islam AbdelRahman 已提交
767 768 769
    } else {
      file_deletion_status = env_->DeleteFile(fname);
    }
770
    if (file_deletion_status.ok()) {
771
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
772
          "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", state.job_id,
773 774 775 776 777 778 779 780 781
          fname.c_str(), type, number,
          file_deletion_status.ToString().c_str());
    } else {
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "[JOB %d] Failed to delete %s type=%d #%" PRIu64 " -- %s\n",
          state.job_id, fname.c_str(), type, number,
          file_deletion_status.ToString().c_str());
    }
    if (type == kTableFile) {
782 783 784 785
      EventHelpers::LogAndNotifyTableFileDeletion(
          &event_logger_, state.job_id, number, fname,
          file_deletion_status, GetName(),
          db_options_.listeners);
J
jorlow@chromium.org 已提交
786 787
    }
  }
H
heyongqiang 已提交
788

789
  // Delete old info log files.
790
  size_t old_info_log_file_count = old_info_log_files.size();
791
  if (old_info_log_file_count >= db_options_.keep_log_file_num) {
792
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
793
    size_t end = old_info_log_file_count - db_options_.keep_log_file_num;
794
    for (unsigned int i = 0; i <= end; i++) {
795
      std::string& to_delete = old_info_log_files.at(i);
796 797
      std::string full_path_to_delete = (db_options_.db_log_dir.empty() ?
           dbname_ : db_options_.db_log_dir) + "/" + to_delete;
798
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
799
          "[JOB %d] Delete info log file %s\n", state.job_id,
800 801
          full_path_to_delete.c_str());
      Status s = env_->DeleteFile(full_path_to_delete);
802
      if (!s.ok()) {
803 804
        Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
            "[JOB %d] Delete info log file %s FAILED -- %s\n", state.job_id,
805 806
            to_delete.c_str(), s.ToString().c_str());
      }
H
heyongqiang 已提交
807 808
    }
  }
I
Igor Canadi 已提交
809 810 811
#ifndef ROCKSDB_LITE
  wal_manager_.PurgeObsoleteWALFiles();
#endif  // ROCKSDB_LITE
812
  LogFlush(db_options_.info_log);
D
Dhruba Borthakur 已提交
813 814 815 816
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
817
  JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
818
  FindObsoleteFiles(&job_context, true);
819 820

  mutex_.Unlock();
I
Igor Canadi 已提交
821 822
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
823
  }
I
Igor Canadi 已提交
824
  job_context.Clean();
825
  mutex_.Lock();
826 827
}

828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843
Status DBImpl::Directories::CreateAndNewDirectory(
    Env* env, const std::string& dirname,
    std::unique_ptr<Directory>* directory) const {
  // 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(dirname);
  if (!s.ok()) {
    return s;
  }
  return env->NewDirectory(dirname, directory);
}
J
jorlow@chromium.org 已提交
844

845 846 847 848 849 850 851 852 853
Status DBImpl::Directories::SetDirectories(
    Env* env, const std::string& dbname, const std::string& wal_dir,
    const std::vector<DbPath>& data_paths) {
  Status s = CreateAndNewDirectory(env, dbname, &db_dir_);
  if (!s.ok()) {
    return s;
  }
  if (!wal_dir.empty() && dbname != wal_dir) {
    s = CreateAndNewDirectory(env, wal_dir, &wal_dir_);
854 855 856
    if (!s.ok()) {
      return s;
    }
857
  }
858

859 860 861 862 863 864 865 866
  data_dirs_.clear();
  for (auto& p : data_paths) {
    const std::string db_path = p.path;
    if (db_path == dbname) {
      data_dirs_.emplace_back(nullptr);
    } else {
      std::unique_ptr<Directory> path_directory;
      s = CreateAndNewDirectory(env, db_path, &path_directory);
867 868 869
      if (!s.ok()) {
        return s;
      }
870
      data_dirs_.emplace_back(path_directory.release());
871
    }
872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890
  }
  assert(data_dirs_.size() == data_paths.size());
  return Status::OK();
}

Directory* DBImpl::Directories::GetDataDir(size_t path_id) {
  assert(path_id < data_dirs_.size());
  Directory* ret_dir = data_dirs_[path_id].get();
  if (ret_dir == nullptr) {
    // Should use db_dir_
    return db_dir_.get();
  }
  return ret_dir;
}

Status DBImpl::Recover(
    const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
    bool error_if_log_file_exist) {
  mutex_.AssertHeld();
891

892 893 894 895 896
  bool is_new_db = false;
  assert(db_lock_ == nullptr);
  if (!read_only) {
    Status s = directories_.SetDirectories(env_, dbname_, db_options_.wal_dir,
                                           db_options_.db_paths);
897 898 899 900
    if (!s.ok()) {
      return s;
    }

901
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
902 903 904
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
905

A
agiardullo 已提交
906 907
    s = env_->FileExists(CurrentFileName(dbname_));
    if (s.IsNotFound()) {
908
      if (db_options_.create_if_missing) {
909
        s = NewDB();
910
        is_new_db = true;
911 912 913 914 915 916
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
917
      }
A
agiardullo 已提交
918
    } else if (s.ok()) {
919
      if (db_options_.error_if_exists) {
920 921 922
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
A
agiardullo 已提交
923 924 925 926
    } else {
      // Unexpected error reading file
      assert(s.IsIOError());
      return s;
J
jorlow@chromium.org 已提交
927
    }
M
Mayank Agarwal 已提交
928
    // Check for the IDENTITY file and create it if not there
A
agiardullo 已提交
929 930
    s = env_->FileExists(IdentityFileName(dbname_));
    if (s.IsNotFound()) {
M
Mayank Agarwal 已提交
931 932 933 934
      s = SetIdentityFile(env_, dbname_);
      if (!s.ok()) {
        return s;
      }
A
agiardullo 已提交
935 936 937
    } else if (!s.ok()) {
      assert(s.IsIOError());
      return s;
M
Mayank Agarwal 已提交
938
    }
J
jorlow@chromium.org 已提交
939 940
  }

941
  Status s = versions_->Recover(column_families, read_only);
942
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
943 944
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
945
  if (s.ok()) {
A
agiardullo 已提交
946
    SequenceNumber max_sequence(kMaxSequenceNumber);
947 948
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
949
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
950 951
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
952 953 954 955 956

    // 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).
    //
957
    // Note that prev_log_number() is no longer used, but we pay
958
    // attention to it in case we are recovering a database
959
    // produced by an older version of rocksdb.
960
    const uint64_t min_log = versions_->MinLogNumber();
961
    const uint64_t prev_log = versions_->prev_log_number();
962
    std::vector<std::string> filenames;
963
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
964 965
    if (!s.ok()) {
      return s;
966
    }
K
kailiu 已提交
967

968 969
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
970 971
      uint64_t number;
      FileType type;
972 973 974 975 976 977 978 979 980
      if (ParseFileName(filenames[i], &number, &type) && type == kLogFile) {
        if (is_new_db) {
          return Status::Corruption(
              "While creating a new Db, wal_dir contains "
              "existing log file: ",
              filenames[i]);
        } else if ((number >= min_log) || (number == prev_log)) {
          logs.push_back(number);
        }
981
      }
J
jorlow@chromium.org 已提交
982
    }
983

H
heyongqiang 已提交
984 985 986 987 988 989
    if (logs.size() > 0 && error_if_log_file_exist) {
      return Status::Corruption(""
          "The db was opened in readonly mode with error_if_log_file_exist"
          "flag but a log file already exists");
    }

S
Stanislau Hlebik 已提交
990 991 992 993 994 995 996
    if (!logs.empty()) {
      // Recover in the order in which the logs were generated
      std::sort(logs.begin(), logs.end());
      s = RecoverLogFiles(logs, &max_sequence, read_only);
      if (!s.ok()) {
        // Clear memtables if recovery failed
        for (auto cfd : *versions_->GetColumnFamilySet()) {
A
agiardullo 已提交
997 998
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 kMaxSequenceNumber);
S
Stanislau Hlebik 已提交
999 1000
        }
      }
1001
    }
L
Lei Jin 已提交
1002
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
1003 1004
  }

L
Lei Jin 已提交
1005 1006
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
1007
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
1008 1009 1010
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ += mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
I
Igor Canadi 已提交
1011 1012
  }

J
jorlow@chromium.org 已提交
1013 1014 1015
  return s;
}

S
Stanislau Hlebik 已提交
1016 1017 1018
// REQUIRES: log_numbers are sorted in ascending order
Status DBImpl::RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
                               SequenceNumber* max_sequence, bool read_only) {
J
jorlow@chromium.org 已提交
1019 1020
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
1021
    Logger* info_log;
J
jorlow@chromium.org 已提交
1022
    const char* fname;
1023
    Status* status;  // nullptr if db_options_.paranoid_checks==false
I
Igor Sugak 已提交
1024
    virtual void Corruption(size_t bytes, const Status& s) override {
1025 1026
      Log(InfoLogLevel::WARN_LEVEL,
          info_log, "%s%s: dropping %d bytes; %s",
1027
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
1028
          fname, static_cast<int>(bytes), s.ToString().c_str());
K
krad 已提交
1029 1030 1031
      if (this->status != nullptr && this->status->ok()) {
        *this->status = s;
      }
J
jorlow@chromium.org 已提交
1032 1033 1034 1035
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
1036
  Status status;
1037
  std::unordered_map<int, VersionEdit> version_edits;
1038
  // no need to refcount because iteration is under mutex
1039 1040
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1041 1042
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1043
  }
1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055
  int job_id = next_job_id_.fetch_add(1);
  {
    auto stream = event_logger_.Log();
    stream << "job" << job_id << "event"
           << "recovery_started";
    stream << "log_files";
    stream.StartArray();
    for (auto log_number : log_numbers) {
      stream << log_number;
    }
    stream.EndArray();
  }
I
Igor Canadi 已提交
1056

K
krad 已提交
1057
  bool continue_replay_log = true;
S
Stanislau Hlebik 已提交
1058 1059 1060 1061
  for (auto log_number : log_numbers) {
    // The previous incarnation may not have written any MANIFEST
    // records after allocating this log number.  So we manually
    // update the file number allocation counter in VersionSet.
1062
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
1063 1064
    // Open the log file
    std::string fname = LogFileName(db_options_.wal_dir, log_number);
1065 1066 1067 1068
    unique_ptr<SequentialFileReader> file_reader;
    {
      unique_ptr<SequentialFile> file;
      status = env_->NewSequentialFile(fname, &file, env_options_);
S
Stanislau Hlebik 已提交
1069
      if (!status.ok()) {
1070 1071 1072 1073 1074 1075 1076 1077
        MaybeIgnoreError(&status);
        if (!status.ok()) {
          return status;
        } else {
          // Fail with one log file, but that's ok.
          // Try next one.
          continue;
        }
S
Stanislau Hlebik 已提交
1078
      }
1079
      file_reader.reset(new SequentialFileReader(std::move(file)));
J
jorlow@chromium.org 已提交
1080 1081
    }

S
Stanislau Hlebik 已提交
1082 1083 1084 1085 1086
    // Create the log reader.
    LogReporter reporter;
    reporter.env = env_;
    reporter.info_log = db_options_.info_log.get();
    reporter.fname = fname.c_str();
K
krad 已提交
1087 1088 1089 1090 1091 1092 1093
    if (!db_options_.paranoid_checks ||
        db_options_.wal_recovery_mode ==
            WALRecoveryMode::kSkipAnyCorruptedRecords) {
      reporter.status = nullptr;
    } else {
      reporter.status = &status;
    }
S
Stanislau Hlebik 已提交
1094 1095 1096 1097
    // 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).
1098
    log::Reader reader(std::move(file_reader), &reporter, true /*checksum*/,
S
Stanislau Hlebik 已提交
1099
                       0 /*initial_offset*/);
K
krad 已提交
1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Recovering log #%" PRIu64 " mode %d skip-recovery %d", log_number,
        db_options_.wal_recovery_mode, !continue_replay_log);

    // Determine if we should tolerate incomplete records at the tail end of the
    // log
    bool report_eof_inconsistency;
    if (db_options_.wal_recovery_mode ==
        WALRecoveryMode::kAbsoluteConsistency) {
      // in clean shutdown we don't expect any error in the log files
      report_eof_inconsistency = true;
    } else {
      // for other modes ignore only incomplete records in the last log file
      // which is presumably due to write in progress during restart
      report_eof_inconsistency = false;

      // TODO krad: Evaluate if we need to move to a more strict mode where we
      // restrict the inconsistency to only the last log
    }
S
Stanislau Hlebik 已提交
1119 1120 1121 1122 1123

    // Read all the records and add to a memtable
    std::string scratch;
    Slice record;
    WriteBatch batch;
K
krad 已提交
1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136

    if (!continue_replay_log) {
      uint64_t bytes;
      if (env_->GetFileSize(fname, &bytes).ok()) {
        auto info_log = db_options_.info_log.get();
        Log(InfoLogLevel::WARN_LEVEL, info_log, "%s: dropping %d bytes",
            fname.c_str(), static_cast<int>(bytes));
      }
    }

    while (continue_replay_log &&
           reader.ReadRecord(&record, &scratch, report_eof_inconsistency) &&
           status.ok()) {
S
Stanislau Hlebik 已提交
1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153
      if (record.size() < 12) {
        reporter.Corruption(record.size(),
                            Status::Corruption("log record too small"));
        continue;
      }
      WriteBatchInternal::SetContents(&batch, record);

      // If column family was not found, it might mean that the WAL write
      // batch references to the column family that was dropped after the
      // insert. We don't want to fail the whole write batch in that case --
      // we just ignore the update.
      // That's why we set ignore missing column families to true
      status = WriteBatchInternal::InsertInto(
          &batch, column_family_memtables_.get(), true, log_number);

      MaybeIgnoreError(&status);
      if (!status.ok()) {
1154 1155 1156 1157
        // We are treating this as a failure while reading since we read valid
        // blocks that do not form coherent data
        reporter.Corruption(record.size(), status);
        continue;
S
Stanislau Hlebik 已提交
1158
      }
1159

S
Stanislau Hlebik 已提交
1160 1161
      const SequenceNumber last_seq = WriteBatchInternal::Sequence(&batch) +
                                      WriteBatchInternal::Count(&batch) - 1;
A
agiardullo 已提交
1162
      if ((*max_sequence == kMaxSequenceNumber) || (last_seq > *max_sequence)) {
S
Stanislau Hlebik 已提交
1163 1164 1165 1166
        *max_sequence = last_seq;
      }

      if (!read_only) {
I
Igor Canadi 已提交
1167 1168 1169 1170 1171 1172 1173 1174 1175 1176 1177 1178
        // we can do this because this is called before client has access to the
        // DB and there is only a single thread operating on DB
        ColumnFamilyData* cfd;

        while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
          cfd->Unref();
          // If this asserts, it means that InsertInto failed in
          // filtering updates to already-flushed column families
          assert(cfd->GetLogNumber() <= log_number);
          auto iter = version_edits.find(cfd->GetID());
          assert(iter != version_edits.end());
          VersionEdit* edit = &iter->second;
1179
          status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
I
Igor Canadi 已提交
1180 1181 1182 1183
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
1184
          }
A
agiardullo 已提交
1185 1186 1187

          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 *max_sequence);
1188
        }
J
jorlow@chromium.org 已提交
1189 1190 1191
      }
    }

1192
    if (!status.ok()) {
K
krad 已提交
1193
      if (db_options_.wal_recovery_mode ==
1194 1195 1196 1197
             WALRecoveryMode::kSkipAnyCorruptedRecords) {
        // We should ignore all errors unconditionally
        status = Status::OK();
      } else if (db_options_.wal_recovery_mode ==
K
krad 已提交
1198 1199 1200 1201 1202 1203 1204 1205
                 WALRecoveryMode::kPointInTimeRecovery) {
        // We should ignore the error but not continue replaying
        status = Status::OK();
        continue_replay_log = false;

        Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
            "Point in time recovered to log #%" PRIu64 " seq #%" PRIu64,
            log_number, *max_sequence);
1206 1207 1208 1209 1210
      } else {
        assert(db_options_.wal_recovery_mode ==
                  WALRecoveryMode::kTolerateCorruptedTailRecords
               || db_options_.wal_recovery_mode ==
                  WALRecoveryMode::kAbsoluteConsistency);
K
krad 已提交
1211 1212
        return status;
      }
1213 1214
    }

I
Igor Canadi 已提交
1215
    flush_scheduler_.Clear();
A
agiardullo 已提交
1216 1217
    if ((*max_sequence != kMaxSequenceNumber) &&
        (versions_->LastSequence() < *max_sequence)) {
S
Stanislau Hlebik 已提交
1218 1219
      versions_->SetLastSequence(*max_sequence);
    }
1220 1221
  }

1222
  if (!read_only) {
1223 1224
    // no need to refcount since client still doesn't have access
    // to the DB and can not drop column families while we iterate
S
Stanislau Hlebik 已提交
1225
    auto max_log_number = log_numbers.back();
1226
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1227
      auto iter = version_edits.find(cfd->GetID());
1228 1229 1230
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
1231
      if (cfd->GetLogNumber() > max_log_number) {
1232
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
1233
        // from all logs. Memtable has to be empty because
1234
        // we filter the updates based on log_number
1235
        // (in WriteBatch::InsertInto)
1236 1237 1238 1239 1240 1241 1242
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1243
        status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
1244 1245 1246 1247
        if (!status.ok()) {
          // Recovery failed
          break;
        }
A
agiardullo 已提交
1248 1249 1250

        cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                               *max_sequence);
1251
      }
J
jorlow@chromium.org 已提交
1252

1253
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
1254
      // writing log_number in the manifest means that any log file
1255 1256
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
1257 1258 1259
      // Since we already recovered max_log_number, we want all logs
      // with numbers `<= max_log_number` (includes this one) to be ignored
      edit->SetLogNumber(max_log_number + 1);
1260 1261 1262
      // we must mark the next log number as used, even though it's
      // not actually used. that is because VersionSet assumes
      // VersionSet::next_file_number_ always to be strictly greater than any
I
Igor Canadi 已提交
1263
      // log number
1264
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
1265 1266
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
1267
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1268 1269
        // Recovery failed
        break;
1270 1271
      }
    }
1272
  }
I
Igor Canadi 已提交
1273

1274 1275 1276
  event_logger_.Log() << "job" << job_id << "event"
                      << "recovery_finished";

J
jorlow@chromium.org 已提交
1277 1278 1279
  return status;
}

1280 1281
Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
                                           MemTable* mem, VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1282
  mutex_.AssertHeld();
1283
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1284
  FileMetaData meta;
1285
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1286 1287
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1288 1289
  ReadOptions ro;
  ro.total_order_seek = true;
1290
  Arena arena;
1291
  Status s;
1292
  TableProperties table_properties;
1293
  {
1294
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
1295 1296 1297
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1298
        cfd->GetName().c_str(), meta.fd.GetNumber());
1299

1300 1301
    bool paranoid_file_checks =
        cfd->GetLatestMutableCFOptions()->paranoid_file_checks;
1302 1303
    {
      mutex_.Unlock();
1304
      TableFileCreationInfo info;
1305 1306
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
1307
          iter.get(), &meta, cfd->internal_comparator(),
I
Igor Canadi 已提交
1308 1309
          cfd->int_tbl_prop_collector_factories(), snapshots_.GetAll(),
          GetCompressionFlush(*cfd->ioptions()),
1310 1311
          cfd->ioptions()->compression_opts, paranoid_file_checks,
          cfd->internal_stats(), Env::IO_HIGH, &info.table_properties);
1312
      LogFlush(db_options_.info_log);
1313 1314 1315 1316 1317 1318 1319 1320
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] [WriteLevel0TableForRecovery]"
          " Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
          cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
          s.ToString().c_str());

      // output to event logger
      if (s.ok()) {
1321 1322 1323 1324 1325 1326 1327 1328 1329
        info.db_name = dbname_;
        info.cf_name = cfd->GetName();
        info.file_path = TableFileName(db_options_.db_paths,
                                       meta.fd.GetNumber(),
                                       meta.fd.GetPathId());
        info.file_size = meta.fd.GetFileSize();
        info.job_id = job_id;
        EventHelpers::LogAndNotifyTableFileCreation(
            &event_logger_, db_options_.listeners, meta.fd, info);
1330
      }
1331 1332
      mutex_.Lock();
    }
1333
  }
I
Igor Canadi 已提交
1334
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1335 1336 1337 1338

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1339
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1340 1341
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
1342 1343
                  meta.smallest_seqno, meta.largest_seqno,
                  meta.marked_for_compaction);
1344 1345
  }

L
Lei Jin 已提交
1346
  InternalStats::CompactionStats stats(1);
1347
  stats.micros = env_->NowMicros() - start_micros;
1348
  stats.bytes_written = meta.fd.GetFileSize();
1349
  stats.num_output_files = 1;
1350
  cfd->internal_stats()->AddCompactionStats(level, stats);
1351 1352
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1353
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1354 1355 1356
  return s;
}

1357 1358
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
1359
    bool* made_progress, JobContext* job_context, LogBuffer* log_buffer) {
1360
  mutex_.AssertHeld();
1361
  assert(cfd->imm()->NumNotFlushed() != 0);
1362
  assert(cfd->imm()->IsFlushPending());
1363

I
Igor Canadi 已提交
1364 1365
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1366
                     snapshots_.GetAll(), job_context, log_buffer,
1367
                     directories_.GetDbDir(), directories_.GetDataDir(0U),
I
Igor Canadi 已提交
1368 1369
                     GetCompressionFlush(*cfd->ioptions()), stats_,
                     &event_logger_);
1370

1371
  FileMetaData file_meta;
1372 1373 1374 1375 1376 1377 1378

  // Within flush_job.Run, rocksdb may call event listener to notify
  // file creation and deletion.
  //
  // Note that flush_job.Run will unlock and lock the db_mutex,
  // and EventListener callback will be called when the db_mutex
  // is unlocked by the current thread.
1379
  Status s = flush_job.Run(&file_meta);
J
jorlow@chromium.org 已提交
1380 1381

  if (s.ok()) {
I
Igor Canadi 已提交
1382 1383
    InstallSuperVersionAndScheduleWorkWrapper(cfd, job_context,
                                              mutable_cf_options);
1384 1385
    if (made_progress) {
      *made_progress = 1;
1386
    }
S
sdong 已提交
1387
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1388
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1389
                cfd->current()->storage_info()->LevelSummary(&tmp));
J
jorlow@chromium.org 已提交
1390
  }
1391

1392
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1393 1394 1395 1396 1397
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1398
  RecordFlushIOStats();
1399 1400 1401
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1402
    NotifyOnFlushCompleted(cfd, &file_meta, mutable_cf_options,
1403
                           job_context->job_id);
1404 1405
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1406 1407 1408
  return s;
}

1409
void DBImpl::NotifyOnFlushCompleted(
1410
    ColumnFamilyData* cfd, FileMetaData* file_meta,
1411
    const MutableCFOptions& mutable_cf_options, int job_id) {
1412
#ifndef ROCKSDB_LITE
1413
  if (db_options_.listeners.size() == 0U) {
1414 1415
    return;
  }
1416 1417 1418 1419
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
1420
  bool triggered_writes_slowdown =
1421
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1422
       mutable_cf_options.level0_slowdown_writes_trigger);
1423
  bool triggered_writes_stop =
1424
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1425
       mutable_cf_options.level0_stop_writes_trigger);
1426 1427
  // release lock while notifying events
  mutex_.Unlock();
1428
  {
1429 1430 1431 1432 1433
    FlushJobInfo info;
    info.cf_name = cfd->GetName();
    // TODO(yhchiang): make db_paths dynamic in case flush does not
    //                 go to L0 in the future.
    info.file_path = MakeTableFileName(db_options_.db_paths[0].path,
1434
                                       file_meta->fd.GetNumber());
1435
    info.thread_id = env_->GetThreadID();
1436 1437 1438
    info.job_id = job_id;
    info.triggered_writes_slowdown = triggered_writes_slowdown;
    info.triggered_writes_stop = triggered_writes_stop;
1439 1440
    info.smallest_seqno = file_meta->smallest_seqno;
    info.largest_seqno = file_meta->largest_seqno;
1441
    for (auto listener : db_options_.listeners) {
1442
      listener->OnFlushCompleted(this, info);
1443 1444
    }
  }
1445 1446 1447
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
1448
#endif  // ROCKSDB_LITE
1449 1450
}

1451 1452 1453 1454
Status DBImpl::CompactRange(const CompactRangeOptions& options,
                            ColumnFamilyHandle* column_family,
                            const Slice* begin, const Slice* end) {
  if (options.target_path_id >= db_options_.db_paths.size()) {
1455 1456 1457
    return Status::InvalidArgument("Invalid target path ID");
  }

1458 1459
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1460 1461

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1462
  if (!s.ok()) {
1463
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1464 1465 1466
    return s;
  }

I
Igor Canadi 已提交
1467
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1468
  {
1469
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
1470
    Version* base = cfd->current();
1471 1472
    for (int level = 1; level < base->storage_info()->num_non_empty_levels();
         level++) {
S
sdong 已提交
1473
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1474 1475 1476 1477
        max_level_with_files = level;
      }
    }
  }
1478

1479
  int final_output_level = 0;
1480 1481 1482 1483
  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
1484 1485
                            cfd->NumberLevels() - 1, options.target_path_id,
                            begin, end);
1486
    final_output_level = cfd->NumberLevels() - 1;
1487 1488
  } else {
    for (int level = 0; level <= max_level_with_files; level++) {
1489
      int output_level;
A
Andres Notzli 已提交
1490
      // in case the compaction is universal or if we're compacting the
1491 1492 1493 1494
      // bottom-most level, the output level will be the same as input one.
      // level 0 can never be the bottommost level (i.e. if all files are in
      // level 0, we will compact to level 1)
      if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
1495 1496 1497
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
        output_level = level;
      } else if (level == max_level_with_files && level > 0) {
1498 1499 1500 1501 1502 1503 1504
        if (options.bottommost_level_compaction ==
            BottommostLevelCompaction::kSkip) {
          // Skip bottommost level compaction
          continue;
        } else if (options.bottommost_level_compaction ==
                       BottommostLevelCompaction::kIfHaveCompactionFilter &&
                   cfd->ioptions()->compaction_filter == nullptr &&
I
Igor Canadi 已提交
1505
                   cfd->ioptions()->compaction_filter_factory == nullptr) {
A
Andres Notzli 已提交
1506 1507
          // Skip bottommost level compaction since we don't have a compaction
          // filter
1508 1509
          continue;
        }
1510
        output_level = level;
1511
      } else {
1512
        output_level = level + 1;
1513 1514 1515 1516 1517
        if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
            cfd->ioptions()->level_compaction_dynamic_level_bytes &&
            level == 0) {
          output_level = ColumnFamilyData::kCompactToBaseLevel;
        }
1518
      }
1519 1520
      s = RunManualCompaction(cfd, level, output_level, options.target_path_id,
                              begin, end);
1521 1522 1523
      if (!s.ok()) {
        break;
      }
1524 1525 1526 1527 1528
      if (output_level == ColumnFamilyData::kCompactToBaseLevel) {
        final_output_level = cfd->NumberLevels() - 1;
      } else if (output_level > final_output_level) {
        final_output_level = output_level;
      }
1529 1530
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
1531
    }
G
Gabor Cselle 已提交
1532
  }
1533 1534 1535 1536
  if (!s.ok()) {
    LogFlush(db_options_.info_log);
    return s;
  }
1537

1538 1539
  if (options.change_level) {
    s = ReFitLevel(cfd, final_output_level, options.target_level);
1540
  }
1541
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1542

1543
  {
1544
    InstrumentedMutexLock l(&mutex_);
1545 1546 1547 1548 1549
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1550
  return s;
1551 1552
}

1553 1554 1555 1556 1557
Status DBImpl::CompactFiles(
    const CompactionOptions& compact_options,
    ColumnFamilyHandle* column_family,
    const std::vector<std::string>& input_file_names,
    const int output_level, const int output_path_id) {
I
Igor Canadi 已提交
1558 1559 1560 1561
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1562 1563 1564 1565 1566 1567
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);
1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578 1579 1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592

  Status s;
  JobContext job_context(0, true);
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
                       db_options_.info_log.get());

  // Perform CompactFiles
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
  {
    InstrumentedMutexLock l(&mutex_);

    s = CompactFilesImpl(compact_options, cfd, sv->current,
                         input_file_names, output_level,
                         output_path_id, &job_context, &log_buffer);
  }
  ReturnAndCleanupSuperVersion(cfd, sv);

  // Find and delete obsolete files
  {
    InstrumentedMutexLock l(&mutex_);
    // 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 job_context does not
    // catch all created files if compaction failed.
    FindObsoleteFiles(&job_context, !s.ok());
1593
  }  // release the mutex
1594 1595 1596 1597 1598 1599 1600 1601 1602 1603

  // delete unnecessary files if any, this is done outside the mutex
  if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
    // Have to flush the info logs before bg_compaction_scheduled_--
    // because if bg_flush_scheduled_ becomes 0 and the lock is
    // released, the deconstructor of DB can kick in and destroy all the
    // states of DB so info_log might not be available after that point.
    // It also applies to access other states that DB owns.
    log_buffer.FlushBufferToLog();
    if (job_context.HaveSomethingToDelete()) {
1604
      // no mutex is locked here.  No need to Unlock() and Lock() here.
1605 1606 1607 1608 1609
      PurgeObsoleteFiles(job_context);
    }
    job_context.Clean();
  }

1610
  return s;
I
Igor Canadi 已提交
1611
#endif  // ROCKSDB_LITE
1612 1613
}

I
Igor Canadi 已提交
1614
#ifndef ROCKSDB_LITE
1615 1616 1617
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
1618 1619
    const int output_level, int output_path_id, JobContext* job_context,
    LogBuffer* log_buffer) {
1620 1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651
  mutex_.AssertHeld();

  if (shutting_down_.load(std::memory_order_acquire)) {
    return Status::ShutdownInProgress();
  }

  std::unordered_set<uint64_t> input_set;
  for (auto file_name : input_file_names) {
    input_set.insert(TableFileNameToNumber(file_name));
  }

  ColumnFamilyMetaData cf_meta;
  // TODO(yhchiang): can directly use version here if none of the
  // following functions call is pluggable to external developers.
  version->GetColumnFamilyMetaData(&cf_meta);

  if (output_path_id < 0) {
    if (db_options_.db_paths.size() == 1U) {
      output_path_id = 0;
    } else {
      return Status::NotSupported(
          "Automatic output path selection is not "
          "yet supported in CompactFiles()");
    }
  }

  Status s = cfd->compaction_picker()->SanitizeCompactionInputFiles(
      &input_set, cf_meta, output_level);
  if (!s.ok()) {
    return s;
  }

1652
  std::vector<CompactionInputFiles> input_files;
1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671 1672
  s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers(
      &input_files, &input_set, version->storage_info(), compact_options);
  if (!s.ok()) {
    return s;
  }

  for (auto inputs : input_files) {
    if (cfd->compaction_picker()->FilesInCompaction(inputs.files)) {
      return Status::Aborted(
          "Some of the necessary compaction input "
          "files are already being compacted");
    }
  }

  // At this point, CompactFiles will be run.
  bg_compaction_scheduled_++;

  unique_ptr<Compaction> c;
  assert(cfd->compaction_picker());
  c.reset(cfd->compaction_picker()->FormCompaction(
1673 1674
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
1675 1676 1677
  assert(c);
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
1678
  assert(!c->deletion_compaction());
1679

I
Igor Canadi 已提交
1680
  assert(is_snapshot_supported_ || snapshots_.empty());
1681
  CompactionJob compaction_job(
I
Igor Canadi 已提交
1682 1683
      job_context->job_id, c.get(), db_options_, env_options_, versions_.get(),
      &shutting_down_, log_buffer, directories_.GetDbDir(),
S
sdong 已提交
1684
      directories_.GetDataDir(c->output_path_id()), stats_, snapshots_.GetAll(),
1685
      table_cache_, &event_logger_,
1686 1687
      c->mutable_cf_options()->paranoid_file_checks,
      c->mutable_cf_options()->compaction_measure_io_stats, dbname_,
1688 1689 1690 1691 1692 1693 1694 1695 1696 1697 1698 1699 1700
      nullptr);  // Here we pass a nullptr for CompactionJobStats because
                 // CompactFiles does not trigger OnCompactionCompleted(),
                 // which is the only place where CompactionJobStats is
                 // returned.  The idea of not triggering OnCompationCompleted()
                 // is that CompactFiles runs in the caller thread, so the user
                 // should always know when it completes.  As a result, it makes
                 // less sense to notify the users something they should already
                 // know.
                 //
                 // In the future, if we would like to add CompactionJobStats
                 // support for CompactFiles, we should have CompactFiles API
                 // pass a pointer of CompactionJobStats as the out-value
                 // instead of using EventListener.
1701 1702 1703
  compaction_job.Prepare();

  mutex_.Unlock();
1704
  compaction_job.Run();
1705
  mutex_.Lock();
1706

1707
  Status status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
1708
  if (status.ok()) {
I
Igor Canadi 已提交
1709 1710
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
1711 1712 1713 1714 1715 1716 1717 1718 1719
  }
  c->ReleaseCompactionFiles(s);
  c.reset();

  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
1720 1721
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] [JOB %d] Compaction error: %s",
1722
        c->column_family_data()->GetName().c_str(), job_context->job_id,
1723 1724 1725 1726 1727 1728 1729
        status.ToString().c_str());
    if (db_options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }

  bg_compaction_scheduled_--;
1730 1731 1732
  if (bg_compaction_scheduled_ == 0) {
    bg_cv_.SignalAll();
  }
1733 1734 1735

  return status;
}
I
Igor Canadi 已提交
1736
#endif  // ROCKSDB_LITE
1737

O
Ori Bernstein 已提交
1738
void DBImpl::NotifyOnCompactionCompleted(
1739 1740
    ColumnFamilyData* cfd, Compaction *c, const Status &st,
    const CompactionJobStats& compaction_job_stats,
1741
    const int job_id) {
O
Ori Bernstein 已提交
1742
#ifndef ROCKSDB_LITE
1743
  if (db_options_.listeners.size() == 0U) {
O
Ori Bernstein 已提交
1744 1745 1746 1747 1748 1749 1750 1751
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  // release lock while notifying events
  mutex_.Unlock();
1752 1753 1754 1755
  {
    CompactionJobInfo info;
    info.cf_name = cfd->GetName();
    info.status = st;
1756
    info.thread_id = env_->GetThreadID();
1757 1758
    info.job_id = job_id;
    info.base_input_level = c->start_level();
1759
    info.output_level = c->output_level();
1760
    info.stats = compaction_job_stats;
1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778
    for (size_t i = 0; i < c->num_input_levels(); ++i) {
      for (const auto fmd : *c->inputs(i)) {
        info.input_files.push_back(
            TableFileName(db_options_.db_paths,
                          fmd->fd.GetNumber(),
                          fmd->fd.GetPathId()));
      }
    }
    for (const auto newf : c->edit()->GetNewFiles()) {
      info.output_files.push_back(
          TableFileName(db_options_.db_paths,
                        newf.second.fd.GetNumber(),
                        newf.second.fd.GetPathId()));
    }
    for (auto listener : db_options_.listeners) {
      listener->OnCompactionCompleted(this, info);
    }
  }
O
Ori Bernstein 已提交
1779 1780 1781 1782 1783 1784
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
#endif  // ROCKSDB_LITE
}

1785
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1786
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1787 1788 1789
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1790 1791
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1792 1793
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1794
        cfd->GetName().c_str());
1795
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1796 1797 1798
  }

  MutableCFOptions new_options;
1799
  Status s;
L
Lei Jin 已提交
1800
  {
1801
    InstrumentedMutexLock l(&mutex_);
1802 1803
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1804 1805 1806 1807
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

1808 1809
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1810 1811
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1812 1813
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1814
  }
1815
  if (s.ok()) {
1816 1817
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1818 1819 1820
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
  } else {
1821 1822
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1823
  }
S
sdong 已提交
1824
  LogFlush(db_options_.info_log);
1825
  return s;
I
Igor Canadi 已提交
1826
#endif  // ROCKSDB_LITE
1827 1828
}

1829
// return the same level if it cannot be moved
1830 1831
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
1832
  mutex_.AssertHeld();
S
sdong 已提交
1833
  const auto* vstorage = cfd->current()->storage_info();
1834
  int minimum_level = level;
1835
  for (int i = level - 1; i > 0; --i) {
1836
    // stop if level i is not empty
S
sdong 已提交
1837
    if (vstorage->NumLevelFiles(i) > 0) break;
1838
    // stop if level i is too small (cannot fit the level files)
1839
    if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) {
1840 1841
      break;
    }
1842 1843 1844 1845 1846 1847

    minimum_level = i;
  }
  return minimum_level;
}

I
Igor Canadi 已提交
1848 1849
Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
  assert(level < cfd->NumberLevels());
1850 1851 1852
  if (target_level >= cfd->NumberLevels()) {
    return Status::InvalidArgument("Target level exceeds number of levels");
  }
1853

I
Igor Canadi 已提交
1854
  SuperVersion* superversion_to_free = nullptr;
1855
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1856

I
Islam AbdelRahman 已提交
1857
  InstrumentedMutexLock guard_lock(&mutex_);
1858 1859 1860

  // only allow one thread refitting
  if (refitting_level_) {
1861 1862
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
I
Igor Canadi 已提交
1863
    delete new_superversion;
L
Lei Jin 已提交
1864
    return Status::NotSupported("another thread is refitting");
1865 1866 1867 1868 1869
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1870
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1871 1872
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop: %d %d",
1873
        bg_compaction_scheduled_, bg_flush_scheduled_);
1874 1875 1876
    bg_cv_.Wait();
  }

1877 1878
  const MutableCFOptions mutable_cf_options =
    *cfd->GetLatestMutableCFOptions();
1879
  // move to a smaller level
1880 1881
  int to_level = target_level;
  if (target_level < 0) {
1882
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1883
  }
1884

L
Lei Jin 已提交
1885
  Status status;
1886 1887 1888
  auto* vstorage = cfd->current()->storage_info();
  if (to_level > level) {
    if (level == 0) {
I
Islam AbdelRahman 已提交
1889
      delete new_superversion;
1890 1891 1892 1893 1894 1895
      return Status::NotSupported(
          "Cannot change from level 0 to other levels.");
    }
    // Check levels are empty for a trivial move
    for (int l = level + 1; l <= to_level; l++) {
      if (vstorage->NumLevelFiles(l) > 0) {
I
Islam AbdelRahman 已提交
1896
        delete new_superversion;
1897 1898 1899 1900 1901 1902
        return Status::NotSupported(
            "Levels between source and target are not empty for a move.");
      }
    }
  }
  if (to_level != level) {
1903 1904
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Before refitting:\n%s",
1905
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1906

1907
    VersionEdit edit;
I
Igor Canadi 已提交
1908
    edit.SetColumnFamily(cfd->GetID());
1909
    for (const auto& f : vstorage->LevelFiles(level)) {
1910
      edit.DeleteFile(level, f->fd.GetNumber());
1911 1912
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
1913 1914
                   f->smallest_seqno, f->largest_seqno,
                   f->marked_for_compaction);
1915
    }
1916 1917
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1918
        cfd->GetName().c_str(), edit.DebugString().data());
1919

1920 1921
    status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                    directories_.GetDbDir());
I
Igor Canadi 已提交
1922
    superversion_to_free = InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
1923
        cfd, new_superversion, mutable_cf_options);
I
Igor Canadi 已提交
1924
    new_superversion = nullptr;
1925

1926 1927
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1928
        status.ToString().data());
1929 1930

    if (status.ok()) {
1931 1932
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1933
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1934 1935 1936 1937 1938
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1939 1940 1941

  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1942
  return status;
G
Gabor Cselle 已提交
1943 1944
}

1945 1946 1947
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1948 1949
}

1950
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
1951
  return 0;
1952 1953
}

1954 1955
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1956
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
1957 1958
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
1959 1960
}

L
Lei Jin 已提交
1961
Status DBImpl::Flush(const FlushOptions& flush_options,
1962 1963
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1964
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
1965 1966
}

1967 1968 1969 1970 1971 1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983 1984 1985 1986 1987 1988 1989 1990 1991 1992 1993 1994 1995 1996 1997 1998
Status DBImpl::SyncWAL() {
  autovector<log::Writer*, 1> logs_to_sync;
  bool need_log_dir_sync;
  uint64_t current_log_number;

  {
    InstrumentedMutexLock l(&mutex_);
    assert(!logs_.empty());

    // This SyncWAL() call only cares about logs up to this number.
    current_log_number = logfile_number_;

    while (logs_.front().number <= current_log_number &&
           logs_.front().getting_synced) {
      log_sync_cv_.Wait();
    }
    // First check that logs are safe to sync in background.
    for (auto it = logs_.begin();
         it != logs_.end() && it->number <= current_log_number; ++it) {
      if (!it->writer->file()->writable_file()->IsSyncThreadSafe()) {
        return Status::NotSupported(
          "SyncWAL() is not supported for this implementation of WAL file",
          db_options_.allow_mmap_writes
            ? "try setting Options::allow_mmap_writes to false"
            : Slice());
      }
    }
    for (auto it = logs_.begin();
         it != logs_.end() && it->number <= current_log_number; ++it) {
      auto& log = *it;
      assert(!log.getting_synced);
      log.getting_synced = true;
1999
      logs_to_sync.push_back(log.writer);
2000 2001 2002 2003 2004
    }

    need_log_dir_sync = !log_dir_synced_;
  }

2005
  RecordTick(stats_, WAL_FILE_SYNCED);
2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031 2032 2033 2034 2035 2036
  Status status;
  for (log::Writer* log : logs_to_sync) {
    status = log->file()->SyncWithoutFlush(db_options_.use_fsync);
    if (!status.ok()) {
      break;
    }
  }
  if (status.ok() && need_log_dir_sync) {
    status = directories_.GetWalDir()->Fsync();
  }

  {
    InstrumentedMutexLock l(&mutex_);
    MarkLogsSynced(current_log_number, need_log_dir_sync, status);
  }

  return status;
}

void DBImpl::MarkLogsSynced(
    uint64_t up_to, bool synced_dir, const Status& status) {
  mutex_.AssertHeld();
  if (synced_dir &&
      logfile_number_ == up_to &&
      status.ok()) {
    log_dir_synced_ = true;
  }
  for (auto it = logs_.begin(); it != logs_.end() && it->number <= up_to;) {
    auto& log = *it;
    assert(log.getting_synced);
    if (status.ok() && logs_.size() > 1) {
2037
      logs_to_free_.push_back(log.ReleaseWriter());
A
Andres Noetzli 已提交
2038
      it = logs_.erase(it);
2039 2040 2041 2042 2043
    } else {
      log.getting_synced = false;
      ++it;
    }
  }
A
Andres Noetzli 已提交
2044
  assert(logs_.empty() || (logs_.size() == 1 && !logs_[0].getting_synced));
2045 2046 2047
  log_sync_cv_.SignalAll();
}

2048
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
2049 2050 2051
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
2052
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
2053
                                   int output_level, uint32_t output_path_id,
2054 2055
                                   const Slice* begin, const Slice* end,
                                   bool disallow_trivial_move) {
2056 2057
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);
2058

G
Gabor Cselle 已提交
2059 2060
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
2061
  ManualCompaction manual;
I
Igor Canadi 已提交
2062
  manual.cfd = cfd;
2063 2064
  manual.input_level = input_level;
  manual.output_level = output_level;
2065
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
2066
  manual.done = false;
2067
  manual.in_progress = false;
2068
  manual.disallow_trivial_move = disallow_trivial_move;
2069 2070 2071
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
2072 2073
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2074
    manual.begin = nullptr;
G
Gabor Cselle 已提交
2075
  } else {
2076
    begin_storage.SetMaxPossibleForUserKey(*begin);
G
Gabor Cselle 已提交
2077 2078
    manual.begin = &begin_storage;
  }
2079
  if (end == nullptr ||
2080 2081
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2082
    manual.end = nullptr;
G
Gabor Cselle 已提交
2083
  } else {
2084
    end_storage.SetMinPossibleForUserKey(*end);
G
Gabor Cselle 已提交
2085 2086 2087
    manual.end = &end_storage;
  }

2088
  InstrumentedMutexLock l(&mutex_);
2089

2090 2091 2092 2093 2094 2095
  // 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
2096
  // RunManualCompaction(), i.e. during that time no other compaction will
2097 2098 2099
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
2100
  // RunManualCompaction() from getting to the second while loop below.
2101 2102 2103 2104 2105
  // 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) {
2106
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
2107 2108 2109
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
2110 2111
    bg_cv_.Wait();
  }
2112

2113 2114
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
2115
      cfd->GetName().c_str());
2116

2117 2118 2119 2120
  // We don't check bg_error_ here, because if we get the error in compaction,
  // the compaction will set manual.status to bg_error_ and set manual.done to
  // true.
  while (!manual.done) {
2121 2122 2123
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
2124
      bg_cv_.Wait();
2125 2126
    } else {
      manual_compaction_ = &manual;
2127
      bg_compaction_scheduled_++;
2128
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
G
Gabor Cselle 已提交
2129
    }
H
hans@chromium.org 已提交
2130
  }
2131

2132 2133 2134
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
2135
  return manual.status;
J
jorlow@chromium.org 已提交
2136 2137
}

2138
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
2139
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
2140 2141 2142
  Status s;
  {
    WriteContext context;
2143
    InstrumentedMutexLock guard_lock(&mutex_);
2144

2145
    if (cfd->imm()->NumNotFlushed() == 0 && cfd->mem()->IsEmpty()) {
2146 2147 2148 2149
      // Nothing to flush
      return Status::OK();
    }

2150 2151
    WriteThread::Writer w;
    write_thread_.EnterUnbatched(&w, &mutex_);
S
Stanislau Hlebik 已提交
2152

I
Igor Canadi 已提交
2153
    // SwitchMemtable() will release and reacquire mutex
S
Stanislau Hlebik 已提交
2154
    // during execution
I
Igor Canadi 已提交
2155
    s = SwitchMemtable(cfd, &context);
2156
    write_thread_.ExitUnbatched(&w);
2157

S
Stanislau Hlebik 已提交
2158 2159
    cfd->imm()->FlushRequested();

2160 2161 2162
    // schedule flush
    SchedulePendingFlush(cfd);
    MaybeScheduleFlushOrCompaction();
S
Stanislau Hlebik 已提交
2163
  }
S
Stanislau Hlebik 已提交
2164

L
Lei Jin 已提交
2165
  if (s.ok() && flush_options.wait) {
2166
    // Wait until the compaction completes
2167
    s = WaitForFlushMemTable(cfd);
2168 2169
  }
  return s;
J
jorlow@chromium.org 已提交
2170 2171
}

2172
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
2173 2174
  Status s;
  // Wait until the compaction completes
2175
  InstrumentedMutexLock l(&mutex_);
2176
  while (cfd->imm()->NumNotFlushed() > 0 && bg_error_.ok()) {
2177 2178 2179
    if (shutting_down_.load(std::memory_order_acquire)) {
      return Status::ShutdownInProgress();
    }
2180 2181
    bg_cv_.Wait();
  }
2182
  if (!bg_error_.ok()) {
2183 2184 2185
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
2186 2187
}

2188
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
2189
  mutex_.AssertHeld();
2190 2191 2192 2193
  if (!opened_successfully_) {
    // Compaction may introduce data race to DB open
    return;
  }
2194
  if (bg_work_gate_closed_) {
2195 2196
    // gate closed for background work
    return;
I
Igor Canadi 已提交
2197
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
2198
    // DB is being deleted; no more background compactions
2199 2200
    return;
  }
2201

2202 2203 2204 2205
  while (unscheduled_flushes_ > 0 &&
         bg_flush_scheduled_ < db_options_.max_background_flushes) {
    unscheduled_flushes_--;
    bg_flush_scheduled_++;
2206
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
2207 2208
  }

2209 2210 2211 2212 2213 2214 2215 2216 2217 2218 2219 2220
  // special case -- if max_background_flushes == 0, then schedule flush on a
  // compaction thread
  if (db_options_.max_background_flushes == 0) {
    while (unscheduled_flushes_ > 0 &&
           bg_flush_scheduled_ + bg_compaction_scheduled_ <
               db_options_.max_background_compactions) {
      unscheduled_flushes_--;
      bg_flush_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::LOW, this);
    }
  }

2221 2222 2223 2224 2225 2226
  if (bg_manual_only_) {
    // only manual compactions are allowed to run. don't schedule automatic
    // compactions
    return;
  }

2227 2228 2229 2230
  while (bg_compaction_scheduled_ < db_options_.max_background_compactions &&
         unscheduled_compactions_ > 0) {
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
2231
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
2232 2233 2234 2235 2236 2237 2238 2239 2240 2241 2242 2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255 2256 2257 2258 2259 2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270 2271 2272 2273 2274 2275 2276 2277
  }
}

void DBImpl::AddToCompactionQueue(ColumnFamilyData* cfd) {
  assert(!cfd->pending_compaction());
  cfd->Ref();
  compaction_queue_.push_back(cfd);
  cfd->set_pending_compaction(true);
}

ColumnFamilyData* DBImpl::PopFirstFromCompactionQueue() {
  assert(!compaction_queue_.empty());
  auto cfd = *compaction_queue_.begin();
  compaction_queue_.pop_front();
  assert(cfd->pending_compaction());
  cfd->set_pending_compaction(false);
  return cfd;
}

void DBImpl::AddToFlushQueue(ColumnFamilyData* cfd) {
  assert(!cfd->pending_flush());
  cfd->Ref();
  flush_queue_.push_back(cfd);
  cfd->set_pending_flush(true);
}

ColumnFamilyData* DBImpl::PopFirstFromFlushQueue() {
  assert(!flush_queue_.empty());
  auto cfd = *flush_queue_.begin();
  flush_queue_.pop_front();
  assert(cfd->pending_flush());
  cfd->set_pending_flush(false);
  return cfd;
}

void DBImpl::SchedulePendingFlush(ColumnFamilyData* cfd) {
  if (!cfd->pending_flush() && cfd->imm()->IsFlushPending()) {
    AddToFlushQueue(cfd);
    ++unscheduled_flushes_;
  }
}

void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) {
  if (!cfd->pending_compaction() && cfd->NeedsCompaction()) {
    AddToCompactionQueue(cfd);
    ++unscheduled_compactions_;
2278 2279 2280
  }
}

2281
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
2282
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
2283 2284 2285
  IOSTATS_RESET(bytes_written);
}

2286
void DBImpl::BGWorkFlush(void* db) {
2287
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
2288
  TEST_SYNC_POINT("DBImpl::BGWorkFlush");
2289
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
2290
  TEST_SYNC_POINT("DBImpl::BGWorkFlush:done");
2291 2292 2293
}

void DBImpl::BGWorkCompaction(void* db) {
2294
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
2295
  TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
2296 2297 2298
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

2299
Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
H
Haobo Xu 已提交
2300
                               LogBuffer* log_buffer) {
2301
  mutex_.AssertHeld();
2302

2303 2304 2305 2306 2307 2308 2309
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

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

2312 2313 2314
  ColumnFamilyData* cfd = nullptr;
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
I
Igor Canadi 已提交
2315
    auto first_cfd = PopFirstFromFlushQueue();
2316

I
Igor Canadi 已提交
2317
    if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
2318
      // can't flush this CF, try next one
I
Igor Canadi 已提交
2319 2320
      if (first_cfd->Unref()) {
        delete first_cfd;
2321 2322
      }
      continue;
2323
    }
2324 2325

    // found a flush!
I
Igor Canadi 已提交
2326
    cfd = first_cfd;
2327 2328 2329 2330 2331 2332 2333 2334 2335 2336 2337 2338 2339
    break;
  }

  if (cfd != nullptr) {
    const MutableCFOptions mutable_cf_options =
        *cfd->GetLatestMutableCFOptions();
    LogToBuffer(
        log_buffer,
        "Calling FlushMemTableToOutputFile with column "
        "family [%s], flush slots available %d, compaction slots available %d",
        cfd->GetName().c_str(),
        db_options_.max_background_flushes - bg_flush_scheduled_,
        db_options_.max_background_compactions - bg_compaction_scheduled_);
2340
    status = FlushMemTableToOutputFile(cfd, mutable_cf_options, made_progress,
2341 2342 2343
                                       job_context, log_buffer);
    if (cfd->Unref()) {
      delete cfd;
2344
    }
J
jorlow@chromium.org 已提交
2345
  }
2346
  return status;
J
jorlow@chromium.org 已提交
2347 2348
}

2349
void DBImpl::BackgroundCallFlush() {
2350
  bool made_progress = false;
2351
  JobContext job_context(next_job_id_.fetch_add(1), true);
2352 2353
  assert(bg_flush_scheduled_);

2354
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2355
  {
2356
    InstrumentedMutexLock l(&mutex_);
H
Haobo Xu 已提交
2357

I
Igor Canadi 已提交
2358 2359 2360
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2361
    Status s = BackgroundFlush(&made_progress, &job_context, &log_buffer);
2362 2363 2364 2365 2366 2367 2368 2369 2370 2371 2372 2373 2374 2375 2376 2377 2378
    if (!s.ok() && !s.IsShutdownInProgress()) {
      // Wait a little bit before retrying background flush in
      // case this is an environmental problem and we do not want to
      // chew up resources for failed flushes for the duration of
      // the problem.
      uint64_t error_cnt =
        default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "Waiting after background flush error: %s"
          "Accumulated background error counts: %" PRIu64,
          s.ToString().c_str(), error_cnt);
      log_buffer.FlushBufferToLog();
      LogFlush(db_options_.info_log);
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
H
Haobo Xu 已提交
2379 2380
    }

I
Igor Canadi 已提交
2381 2382
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2383 2384 2385
    // If flush failed, we want to delete all temporary files that we might have
    // created. Thus, we force full scan in FindObsoleteFiles()
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
H
Haobo Xu 已提交
2386
    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2387
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2388
      mutex_.Unlock();
2389 2390 2391 2392 2393
      // Have to flush the info logs before bg_flush_scheduled_--
      // because if bg_flush_scheduled_ becomes 0 and the lock is
      // released, the deconstructor of DB can kick in and destroy all the
      // states of DB so info_log might not be available after that point.
      // It also applies to access other states that DB owns.
H
Haobo Xu 已提交
2394
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2395 2396
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2397
      }
I
Igor Canadi 已提交
2398
      job_context.Clean();
2399 2400
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2401

H
Haobo Xu 已提交
2402
    bg_flush_scheduled_--;
2403 2404
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2405
    RecordFlushIOStats();
H
Haobo Xu 已提交
2406
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2407 2408 2409 2410
    // IMPORTANT: there should be no code after calling SignalAll. This call may
    // signal the DB destructor that it's OK to proceed with destruction. In
    // that case, all DB variables will be dealloacated and referencing them
    // will cause trouble.
2411
  }
J
jorlow@chromium.org 已提交
2412 2413
}

2414
void DBImpl::BackgroundCallCompaction() {
2415
  bool made_progress = false;
2416
  JobContext job_context(next_job_id_.fetch_add(1), true);
H
Haobo Xu 已提交
2417 2418

  MaybeDumpStats();
2419
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2420
  {
2421
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
2422 2423 2424 2425

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2426
    assert(bg_compaction_scheduled_);
2427
    Status s = BackgroundCompaction(&made_progress, &job_context, &log_buffer);
2428 2429 2430 2431 2432 2433
    if (!s.ok() && !s.IsShutdownInProgress()) {
      // 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.
      uint64_t error_cnt =
2434
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2435 2436 2437 2438 2439 2440 2441 2442 2443 2444
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
      log_buffer.FlushBufferToLog();
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "Waiting after background compaction error: %s, "
          "Accumulated background error counts: %" PRIu64,
          s.ToString().c_str(), error_cnt);
      LogFlush(db_options_.info_log);
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
2445
    }
H
Haobo Xu 已提交
2446

I
Igor Canadi 已提交
2447 2448
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2449 2450 2451 2452
    // If compaction failed, we want to delete all temporary files that we might
    // have created (they might not be all recorded in job_context in case of a
    // failure). Thus, we force full scan in FindObsoleteFiles()
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
2453 2454

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2455
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2456
      mutex_.Unlock();
2457 2458 2459 2460 2461
      // Have to flush the info logs before bg_compaction_scheduled_--
      // because if bg_flush_scheduled_ becomes 0 and the lock is
      // released, the deconstructor of DB can kick in and destroy all the
      // states of DB so info_log might not be available after that point.
      // It also applies to access other states that DB owns.
H
Haobo Xu 已提交
2462
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2463 2464
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2465
      }
I
Igor Canadi 已提交
2466
      job_context.Clean();
2467 2468
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2469

2470
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2471

2472 2473
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2474 2475
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2476
    if (made_progress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
2477
      // signal if
2478
      // * made_progress -- need to wakeup DelayWrite
2479 2480 2481 2482 2483 2484
      // * bg_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl
      // * bg_manual_only_ > 0 -- need to wakeup RunManualCompaction
      // If none of this is true, there is no need to signal since nobody is
      // waiting for it
      bg_cv_.SignalAll();
    }
I
Igor Canadi 已提交
2485 2486 2487 2488
    // IMPORTANT: there should be no code after calling SignalAll. This call may
    // signal the DB destructor that it's OK to proceed with destruction. In
    // that case, all DB variables will be dealloacated and referencing them
    // will cause trouble.
2489
  }
J
jorlow@chromium.org 已提交
2490 2491
}

2492 2493
Status DBImpl::BackgroundCompaction(bool* made_progress,
                                    JobContext* job_context,
2494
                                    LogBuffer* log_buffer) {
2495
  *made_progress = false;
J
jorlow@chromium.org 已提交
2496
  mutex_.AssertHeld();
2497

2498 2499
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2500 2501
  bool trivial_move_disallowed = is_manual &&
                                 manual_compaction_->disallow_trivial_move;
2502

2503
  CompactionJobStats compaction_job_stats;
2504 2505 2506 2507 2508 2509
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
2510
    if (is_manual) {
2511
      manual_compaction_->status = status;
2512 2513 2514 2515
      manual_compaction_->done = true;
      manual_compaction_->in_progress = false;
      manual_compaction_ = nullptr;
    }
2516
    return status;
2517 2518
  }

2519 2520 2521
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2522 2523 2524 2525
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2526 2527 2528
  }

  unique_ptr<Compaction> c;
2529 2530
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2531
  if (is_manual) {
G
Gabor Cselle 已提交
2532
    ManualCompaction* m = manual_compaction_;
2533
    assert(m->in_progress);
2534 2535 2536
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
2537
    if (!c) {
2538
      m->done = true;
2539 2540 2541 2542 2543 2544 2545 2546 2547 2548 2549 2550 2551 2552 2553 2554
      LogToBuffer(log_buffer,
                  "[%s] Manual compaction from level-%d from %s .. "
                  "%s; nothing to do\n",
                  m->cfd->GetName().c_str(), m->input_level,
                  (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                  (m->end ? m->end->DebugString().c_str() : "(end)"));
    } else {
      LogToBuffer(log_buffer,
                  "[%s] Manual compaction from level-%d to level-%d from %s .. "
                  "%s; will stop at %s\n",
                  m->cfd->GetName().c_str(), m->input_level, c->output_level(),
                  (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                  (m->end ? m->end->DebugString().c_str() : "(end)"),
                  ((m->done || manual_end == nullptr)
                       ? "(end)"
                       : manual_end->DebugString().c_str()));
G
Gabor Cselle 已提交
2555
    }
2556 2557 2558 2559 2560 2561 2562 2563 2564 2565 2566 2567 2568 2569 2570 2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587
  } else if (!compaction_queue_.empty()) {
    // cfd is referenced here
    auto cfd = PopFirstFromCompactionQueue();
    // We unreference here because the following code will take a Ref() on
    // this cfd if it is going to use it (Compaction class holds a
    // reference).
    // This will all happen under a mutex so we don't have to be afraid of
    // somebody else deleting it.
    if (cfd->Unref()) {
      delete cfd;
      // This was the last reference of the column family, so no need to
      // compact.
      return Status::OK();
    }

    // Pick up latest mutable CF Options and use it throughout the
    // compaction job
    // Compaction makes a copy of the latest MutableCFOptions. It should be used
    // throughout the compaction procedure to make sure consistency. It will
    // eventually be installed into SuperVersion
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    if (!mutable_cf_options->disable_auto_compactions && !cfd->IsDropped()) {
      // NOTE: try to avoid unnecessary copy of MutableCFOptions if
      // compaction is not necessary. Need to make sure mutex is held
      // until we make a copy in the following code
      c.reset(cfd->PickCompaction(*mutable_cf_options, log_buffer));
      if (c != nullptr) {
        // update statistics
        MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
                    c->inputs(0)->size());
        // There are three things that can change compaction score:
        // 1) When flush or compaction finish. This case is covered by
I
Igor Canadi 已提交
2588
        // InstallSuperVersionAndScheduleWork
2589
        // 2) When MutableCFOptions changes. This case is also covered by
I
Igor Canadi 已提交
2590 2591
        // InstallSuperVersionAndScheduleWork, because this is when the new
        // options take effect.
2592 2593 2594 2595 2596 2597 2598 2599 2600 2601 2602
        // 3) When we Pick a new compaction, we "remove" those files being
        // compacted from the calculation, which then influences compaction
        // score. Here we check if we need the new compaction even without the
        // files that are currently being compacted. If we need another
        // compaction, we might be able to execute it in parallel, so we add it
        // to the queue and schedule a new thread.
        if (cfd->NeedsCompaction()) {
          // Yes, we need more compactions!
          AddToCompactionQueue(cfd);
          ++unscheduled_compactions_;
          MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2603
        }
I
Igor Canadi 已提交
2604 2605
      }
    }
J
jorlow@chromium.org 已提交
2606 2607
  }

2608
  if (!c) {
H
hans@chromium.org 已提交
2609
    // Nothing to do
2610
    LogToBuffer(log_buffer, "Compaction nothing to do");
2611
  } else if (c->deletion_compaction()) {
I
Igor Canadi 已提交
2612 2613 2614 2615
    // TODO(icanadi) Do we want to honor snapshots here? i.e. not delete old
    // file if there is alive snapshot pointing to it
    assert(c->num_input_files(1) == 0);
    assert(c->level() == 0);
2616
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2617
           kCompactionStyleFIFO);
2618 2619 2620

    compaction_job_stats.num_input_files = c->num_input_files(0);

I
Igor Canadi 已提交
2621
    for (const auto& f : *c->inputs(0)) {
2622
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2623
    }
2624 2625 2626
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
2627 2628
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2629 2630 2631
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
2632
    *made_progress = true;
2633
  } else if (!trivial_move_disallowed && c->IsTrivialMove()) {
2634
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2635 2636 2637 2638 2639
    // Instrument for event update
    // TODO(yhchiang): add op details for showing trivial-move.
    ThreadStatusUtil::SetColumnFamily(c->column_family_data());
    ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);

2640 2641
    compaction_job_stats.num_input_files = c->num_input_files(0);

2642 2643 2644
    // Move files to next level
    int32_t moved_files = 0;
    int64_t moved_bytes = 0;
2645
    for (unsigned int l = 0; l < c->num_input_levels(); l++) {
2646
      if (c->level(l) == c->output_level()) {
2647 2648 2649 2650
        continue;
      }
      for (size_t i = 0; i < c->num_input_files(l); i++) {
        FileMetaData* f = c->input(l, i);
2651
        c->edit()->DeleteFile(c->level(l), f->fd.GetNumber());
2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663
        c->edit()->AddFile(c->output_level(), f->fd.GetNumber(),
                           f->fd.GetPathId(), f->fd.GetFileSize(), f->smallest,
                           f->largest, f->smallest_seqno, f->largest_seqno,
                           f->marked_for_compaction);

        LogToBuffer(log_buffer,
                    "[%s] Moving #%" PRIu64 " to level-%d %" PRIu64 " bytes\n",
                    c->column_family_data()->GetName().c_str(),
                    f->fd.GetNumber(), c->output_level(), f->fd.GetFileSize());
        ++moved_files;
        moved_bytes += f->fd.GetFileSize();
      }
2664
    }
2665

2666
    status = versions_->LogAndApply(c->column_family_data(),
2667 2668
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2669
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2670 2671
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
2672

S
sdong 已提交
2673
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Islam AbdelRahman 已提交
2674
    c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(),
2675
                                                             moved_bytes);
2676 2677 2678 2679
    {
      event_logger_.LogToBuffer(log_buffer)
          << "job" << job_context->job_id << "event"
          << "trivial_move"
I
Islam AbdelRahman 已提交
2680
          << "destination_level" << c->output_level() << "files" << moved_files
2681
          << "total_files_size" << moved_bytes;
2682
    }
2683 2684
    LogToBuffer(
        log_buffer,
2685
        "[%s] Moved #%d files to level-%d %" PRIu64 " bytes %s: %s\n",
I
Islam AbdelRahman 已提交
2686 2687
        c->column_family_data()->GetName().c_str(), moved_files,
        c->output_level(), moved_bytes, status.ToString().c_str(),
2688
        c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
2689
    *made_progress = true;
2690 2691 2692

    // Clear Instrument
    ThreadStatusUtil::ResetThreadStatus();
J
jorlow@chromium.org 已提交
2693
  } else {
2694 2695 2696
    int output_level  __attribute__((unused)) = c->output_level();
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial",
                             &output_level);
I
Igor Canadi 已提交
2697
    assert(is_snapshot_supported_ || snapshots_.empty());
2698
    CompactionJob compaction_job(
I
Igor Canadi 已提交
2699 2700
        job_context->job_id, c.get(), db_options_, env_options_,
        versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(),
2701
        directories_.GetDataDir(c->output_path_id()), stats_,
2702
        snapshots_.GetAll(), table_cache_, &event_logger_,
2703 2704
        c->mutable_cf_options()->paranoid_file_checks,
        c->mutable_cf_options()->compaction_measure_io_stats, dbname_,
S
sdong 已提交
2705
        &compaction_job_stats);
I
Igor Canadi 已提交
2706
    compaction_job.Prepare();
2707

I
Igor Canadi 已提交
2708
    mutex_.Unlock();
2709
    compaction_job.Run();
2710
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun");
I
Igor Canadi 已提交
2711
    mutex_.Lock();
2712

2713
    status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
I
Igor Canadi 已提交
2714
    if (status.ok()) {
I
Igor Canadi 已提交
2715 2716
      InstallSuperVersionAndScheduleWorkWrapper(
          c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2717
    }
2718
    *made_progress = true;
O
Ori Bernstein 已提交
2719 2720
  }
  if (c != nullptr) {
2721 2722 2723
    NotifyOnCompactionCompleted(
        c->column_family_data(), c.get(), status,
        compaction_job_stats, job_context->job_id);
I
Igor Canadi 已提交
2724
    c->ReleaseCompactionFiles(status);
2725
    *made_progress = true;
J
jorlow@chromium.org 已提交
2726
  }
2727
  // this will unref its input_version and column_family_data
2728
  c.reset();
J
jorlow@chromium.org 已提交
2729 2730 2731

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2732
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2733 2734
    // Ignore compaction errors found during shutting down
  } else {
2735
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2736
        status.ToString().c_str());
2737
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2738 2739 2740
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2741 2742

  if (is_manual) {
G
Gabor Cselle 已提交
2743
    ManualCompaction* m = manual_compaction_;
2744
    if (!status.ok()) {
L
Lei Jin 已提交
2745
      m->status = status;
2746 2747
      m->done = true;
    }
2748 2749 2750 2751 2752 2753 2754 2755 2756
    // 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.
2757 2758 2759 2760 2761
    //
    // 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) {
2762 2763
      m->done = true;
    }
G
Gabor Cselle 已提交
2764 2765 2766
    if (!m->done) {
      // We only compacted part of the requested range.  Update *m
      // to the range that is left to be compacted.
I
Igor Canadi 已提交
2767
      // Universal and FIFO compactions should always compact the whole range
S
sdong 已提交
2768 2769 2770
      assert(m->cfd->ioptions()->compaction_style !=
                 kCompactionStyleUniversal ||
             m->cfd->ioptions()->num_levels > 1);
2771
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2772
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2773 2774
      m->begin = &m->tmp_storage;
    }
2775
    m->in_progress = false; // not being processed anymore
2776
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2777
  }
2778
  return status;
J
jorlow@chromium.org 已提交
2779 2780
}

2781 2782
namespace {
struct IterState {
2783
  IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version)
I
Igor Canadi 已提交
2784
      : db(_db), mu(_mu), super_version(_super_version) {}
2785 2786

  DBImpl* db;
2787
  InstrumentedMutex* mu;
2788
  SuperVersion* super_version;
2789 2790 2791 2792
};

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

2794
  if (state->super_version->Unref()) {
2795 2796 2797
    // Job id == 0 means that this is not our background process, but rather
    // user thread
    JobContext job_context(0);
2798

2799 2800
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2801
    state->db->FindObsoleteFiles(&job_context, false, true);
2802 2803 2804
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2805 2806
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2807
    }
I
Igor Canadi 已提交
2808
    job_context.Clean();
I
Igor Canadi 已提交
2809
  }
T
Tomislav Novak 已提交
2810

2811 2812
  delete state;
}
H
Hans Wennborg 已提交
2813
}  // namespace
2814

L
Lei Jin 已提交
2815
Iterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
2816
                                      ColumnFamilyData* cfd,
2817 2818 2819
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
2820 2821 2822 2823 2824
  assert(arena != nullptr);
  // Need to create internal iterator from the arena.
  MergeIteratorBuilder merge_iter_builder(&cfd->internal_comparator(), arena);
  // Collect iterator for mutable mem
  merge_iter_builder.AddIterator(
L
Lei Jin 已提交
2825
      super_version->mem->NewIterator(read_options, arena));
2826
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
2827
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
2828
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
2829
  super_version->current->AddIterators(read_options, env_options_,
2830 2831
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
2832
  IterState* cleanup = new IterState(this, &mutex_, super_version);
2833
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2834 2835 2836 2837

  return internal_iter;
}

2838 2839 2840 2841
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2842
Status DBImpl::Get(const ReadOptions& read_options,
2843
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2844
                   std::string* value) {
L
Lei Jin 已提交
2845
  return GetImpl(read_options, column_family, key, value);
2846 2847
}

I
Igor Canadi 已提交
2848 2849
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
2850 2851
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
2852
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
2853
//
I
Igor Canadi 已提交
2854 2855 2856
// However, if InstallSuperVersionAndScheduleWork() gets called twice with the
// same job_context, we can't reuse the SuperVersion() that got
// malloced because
I
Igor Canadi 已提交
2857 2858 2859
// 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
I
Igor Canadi 已提交
2860
void DBImpl::InstallSuperVersionAndScheduleWorkWrapper(
I
Igor Canadi 已提交
2861
    ColumnFamilyData* cfd, JobContext* job_context,
2862
    const MutableCFOptions& mutable_cf_options) {
2863
  mutex_.AssertHeld();
I
Igor Canadi 已提交
2864
  SuperVersion* old_superversion = InstallSuperVersionAndScheduleWork(
I
Igor Canadi 已提交
2865 2866 2867
      cfd, job_context->new_superversion, mutable_cf_options);
  job_context->new_superversion = nullptr;
  job_context->superversions_to_free.push_back(old_superversion);
I
Igor Canadi 已提交
2868 2869
}

I
Igor Canadi 已提交
2870
SuperVersion* DBImpl::InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
2871
    ColumnFamilyData* cfd, SuperVersion* new_sv,
I
Igor Canadi 已提交
2872
    const MutableCFOptions& mutable_cf_options) {
L
Lei Jin 已提交
2873
  mutex_.AssertHeld();
2874 2875 2876 2877 2878 2879 2880 2881 2882

  // Update max_total_in_memory_state_
  size_t old_memtable_size = 0;
  auto* old_sv = cfd->GetSuperVersion();
  if (old_sv) {
    old_memtable_size = old_sv->mutable_cf_options.write_buffer_size *
                        old_sv->mutable_cf_options.max_write_buffer_number;
  }

L
Lei Jin 已提交
2883 2884 2885
  auto* old = cfd->InstallSuperVersion(
      new_sv ? new_sv : new SuperVersion(), &mutex_, mutable_cf_options);

2886
  // Whenever we install new SuperVersion, we might need to issue new flushes or
I
Igor Canadi 已提交
2887 2888 2889 2890
  // compactions.
  SchedulePendingFlush(cfd);
  SchedulePendingCompaction(cfd);
  MaybeScheduleFlushOrCompaction();
L
Lei Jin 已提交
2891 2892 2893 2894 2895 2896 2897 2898 2899 2900

  // Update max_total_in_memory_state_
  max_total_in_memory_state_ =
      max_total_in_memory_state_ - old_memtable_size +
      mutable_cf_options.write_buffer_size *
      mutable_cf_options.max_write_buffer_number;
  return old;
}

Status DBImpl::GetImpl(const ReadOptions& read_options,
2901 2902
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
L
Lei Jin 已提交
2903
  StopWatch sw(env_, stats_, DB_GET);
2904
  PERF_TIMER_GUARD(get_snapshot_time);
2905

2906 2907 2908
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2909
  SequenceNumber snapshot;
L
Lei Jin 已提交
2910 2911 2912
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2913 2914
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2915
  }
2916
  // Acquire SuperVersion
2917
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
2918
  // Prepare to store a list of merge operations if merge occurs.
2919
  MergeContext merge_context;
2920

2921
  Status s;
2922
  // First look in the memtable, then in the immutable memtable (if any).
2923
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2924
  // merge_operands will contain the sequence of merges in the latter case.
2925
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2926
  PERF_TIMER_STOP(get_snapshot_time);
2927

2928
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2929
    // Done
L
Lei Jin 已提交
2930
    RecordTick(stats_, MEMTABLE_HIT);
2931
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2932
    // Done
L
Lei Jin 已提交
2933
    RecordTick(stats_, MEMTABLE_HIT);
2934
  } else {
2935
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2936 2937
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2938
    RecordTick(stats_, MEMTABLE_MISS);
2939
  }
2940

2941 2942
  {
    PERF_TIMER_GUARD(get_post_process_time);
2943

2944
    ReturnAndCleanupSuperVersion(cfd, sv);
2945

2946 2947 2948
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2949
  return s;
J
jorlow@chromium.org 已提交
2950 2951
}

2952
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
2953
    const ReadOptions& read_options,
2954
    const std::vector<ColumnFamilyHandle*>& column_family,
2955
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
2956

L
Lei Jin 已提交
2957
  StopWatch sw(env_, stats_, DB_MULTIGET);
2958
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
2959

2960
  SequenceNumber snapshot;
2961

2962
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
2963
    ColumnFamilyData* cfd;
2964 2965 2966 2967 2968
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
2969 2970 2971 2972 2973 2974
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(cf);
    auto cfd = cfh->cfd();
    if (multiget_cf_data.find(cfd->GetID()) == multiget_cf_data.end()) {
      auto mgcfd = new MultiGetColumnFamilyData();
      mgcfd->cfd = cfd;
      multiget_cf_data.insert({cfd->GetID(), mgcfd});
2975 2976 2977
    }
  }

2978
  mutex_.Lock();
L
Lei Jin 已提交
2979 2980 2981
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2982 2983 2984
  } else {
    snapshot = versions_->LastSequence();
  }
2985
  for (auto mgd_iter : multiget_cf_data) {
2986 2987
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
2988
  }
2989
  mutex_.Unlock();
2990

2991 2992
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2993

2994
  // Note: this always resizes the values array
2995 2996 2997
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
2998 2999

  // Keep track of bytes that we read for statistics-recording later
3000
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
3001
  PERF_TIMER_STOP(get_snapshot_time);
3002 3003 3004 3005

  // 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.
3006
  // merge_operands will contain the sequence of merges in the latter case.
3007
  for (size_t i = 0; i < num_keys; ++i) {
3008
    merge_context.Clear();
3009
    Status& s = stat_list[i];
3010 3011 3012
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
3013 3014
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
3015 3016 3017
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
3018
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
3019
      // Done
3020
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
3021 3022
      // Done
    } else {
3023
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
3024 3025
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
3026 3027 3028
    }

    if (s.ok()) {
3029
      bytes_read += value->size();
3030 3031 3032 3033
    }
  }

  // Post processing (decrement reference counts and record statistics)
3034
  PERF_TIMER_GUARD(get_post_process_time);
3035 3036
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
3037
  // TODO(icanadi) do we need lock here or just around Cleanup()?
3038 3039 3040 3041 3042 3043
  mutex_.Lock();
  for (auto mgd_iter : multiget_cf_data) {
    auto mgd = mgd_iter.second;
    if (mgd->super_version->Unref()) {
      mgd->super_version->Cleanup();
      superversions_to_delete.push_back(mgd->super_version);
3044 3045
    }
  }
3046 3047 3048 3049 3050 3051 3052
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3053
  }
3054

L
Lei Jin 已提交
3055 3056 3057
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3058
  PERF_TIMER_STOP(get_post_process_time);
3059

3060
  return stat_list;
3061 3062
}

L
Lei Jin 已提交
3063
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3064
                                  const std::string& column_family_name,
3065
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
3066
  Status s;
I
Igor Canadi 已提交
3067
  *handle = nullptr;
3068 3069 3070 3071 3072 3073

  s = CheckCompressionSupported(cf_options);
  if (!s.ok()) {
    return s;
  }

Y
Yueh-Hsuan Chiang 已提交
3074
  {
3075
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
3076

Y
Yueh-Hsuan Chiang 已提交
3077 3078 3079 3080 3081 3082 3083 3084 3085 3086 3087 3088 3089 3090
    if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
        nullptr) {
      return Status::InvalidArgument("Column family already exists");
    }
    VersionEdit edit;
    edit.AddColumnFamily(column_family_name);
    uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
    edit.SetColumnFamily(new_id);
    edit.SetLogNumber(logfile_number_);
    edit.SetComparatorName(cf_options.comparator->Name());

    // LogAndApply will both write the creation in MANIFEST and create
    // ColumnFamilyData object
    Options opt(db_options_, cf_options);
I
Igor Canadi 已提交
3091
    {  // write thread
3092 3093
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
I
Igor Canadi 已提交
3094 3095 3096 3097
      // LogAndApply will both write the creation in MANIFEST and create
      // ColumnFamilyData object
      s = versions_->LogAndApply(
          nullptr, MutableCFOptions(opt, ImmutableCFOptions(opt)), &edit,
3098
          &mutex_, directories_.GetDbDir(), false, &cf_options);
3099
      write_thread_.ExitUnbatched(&w);
I
Igor Canadi 已提交
3100
    }
Y
Yueh-Hsuan Chiang 已提交
3101 3102 3103 3104 3105
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
I
Igor Canadi 已提交
3106
      delete InstallSuperVersionAndScheduleWork(
Y
Yueh-Hsuan Chiang 已提交
3107
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
S
sdong 已提交
3108 3109 3110 3111 3112

      if (!cfd->mem()->IsSnapshotSupported()) {
        is_snapshot_supported_ = false;
      }

Y
Yueh-Hsuan Chiang 已提交
3113 3114 3115 3116 3117 3118 3119 3120 3121
      *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
          "Created column family [%s] (ID %u)",
          column_family_name.c_str(), (unsigned)cfd->GetID());
    } else {
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "Creating column family [%s] FAILED -- %s",
          column_family_name.c_str(), s.ToString().c_str());
    }
3122
  }  // InstrumentedMutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
3123 3124

  // this is outside the mutex
3125
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
3126 3127
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
3128
  }
3129
  return s;
3130 3131
}

3132 3133 3134 3135
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3136 3137
    return Status::InvalidArgument("Can't drop default column family");
  }
3138

S
sdong 已提交
3139 3140
  bool cf_support_snapshot = cfd->mem()->IsSnapshotSupported();

I
Igor Canadi 已提交
3141 3142
  VersionEdit edit;
  edit.DropColumnFamily();
3143 3144
  edit.SetColumnFamily(cfd->GetID());

3145
  Status s;
3146
  {
3147
    InstrumentedMutexLock l(&mutex_);
3148 3149 3150 3151
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
3152
      // we drop column family from a single write thread
3153 3154
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
3155 3156
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
3157
      write_thread_.ExitUnbatched(&w);
3158
    }
S
sdong 已提交
3159 3160 3161 3162 3163 3164

    if (!cf_support_snapshot) {
      // Dropped Column Family doesn't support snapshot. Need to recalculate
      // is_snapshot_supported_.
      bool new_is_snapshot_supported = true;
      for (auto c : *versions_->GetColumnFamilySet()) {
3165
        if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) {
S
sdong 已提交
3166 3167 3168 3169 3170 3171
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
3172
  }
3173

3174
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
3175 3176 3177 3178
    // Note that here we erase the associated cf_info of the to-be-dropped
    // cfd before its ref-count goes to zero to avoid having to erase cf_info
    // later inside db_mutex.
    EraseThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
3179
    assert(cfd->IsDropped());
L
Lei Jin 已提交
3180 3181 3182
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
3183 3184
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
3185
        cfd->GetID());
3186
  } else {
3187
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
3188
        "Dropping column family with id %u FAILED -- %s\n",
3189 3190 3191
        cfd->GetID(), s.ToString().c_str());
  }

3192
  return s;
3193 3194
}

L
Lei Jin 已提交
3195
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
3196 3197
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3198
  if (value_found != nullptr) {
K
Kai Liu 已提交
3199 3200
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3201
  }
L
Lei Jin 已提交
3202
  ReadOptions roptions = read_options;
3203
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3204
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3205

3206
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
3207 3208 3209
  // 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();
3210 3211
}

3212
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
3213 3214 3215
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3216

V
Venkatesh Radhakrishnan 已提交
3217 3218 3219 3220 3221 3222 3223 3224 3225 3226 3227 3228 3229 3230 3231 3232 3233 3234
  XFUNC_TEST("", "managed_new", managed_new1, xf_manage_new,
             reinterpret_cast<DBImpl*>(this),
             const_cast<ReadOptions*>(&read_options), is_snapshot_supported_);
  if (read_options.managed) {
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return NewErrorIterator(Status::InvalidArgument(
        "Managed Iterators not supported in RocksDBLite."));
#else
    if ((read_options.tailing) || (read_options.snapshot != nullptr) ||
        (is_snapshot_supported_)) {
      return new ManagedIterator(this, read_options, cfd);
    }
    // Managed iter not supported
    return NewErrorIterator(Status::InvalidArgument(
        "Managed Iterators not supported without snapshots."));
#endif
  } else if (read_options.tailing) {
I
Igor Canadi 已提交
3235 3236 3237 3238
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
3239 3240
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
3241
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3242 3243 3244
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
3245
#endif
T
Tomislav Novak 已提交
3246
  } else {
3247
    SequenceNumber latest_snapshot = versions_->LastSequence();
3248
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
3249

I
Igor Canadi 已提交
3250
    auto snapshot =
3251 3252 3253
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
3254
            : latest_snapshot;
T
Tomislav Novak 已提交
3255

3256 3257 3258 3259 3260 3261 3262 3263 3264 3265 3266 3267 3268 3269 3270 3271 3272 3273 3274 3275 3276 3277 3278 3279 3280 3281 3282 3283 3284 3285 3286 3287 3288 3289 3290 3291
    // Try to generate a DB iterator tree in continuous memory area to be
    // cache friendly. Here is an example of result:
    // +-------------------------------+
    // |                               |
    // | ArenaWrappedDBIter            |
    // |  +                            |
    // |  +---> Inner Iterator   ------------+
    // |  |                            |     |
    // |  |    +-- -- -- -- -- -- -- --+     |
    // |  +--- | Arena                 |     |
    // |       |                       |     |
    // |          Allocated Memory:    |     |
    // |       |   +-------------------+     |
    // |       |   | DBIter            | <---+
    // |           |  +                |
    // |       |   |  +-> iter_  ------------+
    // |       |   |                   |     |
    // |       |   +-------------------+     |
    // |       |   | MergingIterator   | <---+
    // |           |  +                |
    // |       |   |  +->child iter1  ------------+
    // |       |   |  |                |          |
    // |           |  +->child iter2  ----------+ |
    // |       |   |  |                |        | |
    // |       |   |  +->child iter3  --------+ | |
    // |           |                   |      | | |
    // |       |   +-------------------+      | | |
    // |       |   | Iterator1         | <--------+
    // |       |   +-------------------+      | |
    // |       |   | Iterator2         | <------+
    // |       |   +-------------------+      |
    // |       |   | Iterator3         | <----+
    // |       |   +-------------------+
    // |       |                       |
    // +-------+-----------------------+
    //
3292 3293
    // ArenaWrappedDBIter inlines an arena area where all the iterators in
    // the iterator tree are allocated in the order of being accessed when
3294 3295 3296 3297 3298
    // querying.
    // Laying out the iterators in the order of being accessed makes it more
    // likely that any iterator pointer is close to the iterator it points to so
    // that they are likely to be in the same cache line and/or page.
    ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3299
        env_, *cfd->ioptions(), cfd->user_comparator(),
3300
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
3301
        read_options.iterate_upper_bound);
3302

3303
    Iterator* internal_iter =
3304
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3305 3306 3307 3308
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
3309 3310
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
3311 3312
}

3313
Status DBImpl::NewIterators(
3314
    const ReadOptions& read_options,
I
Igor Canadi 已提交
3315
    const std::vector<ColumnFamilyHandle*>& column_families,
3316
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3317 3318
  iterators->clear();
  iterators->reserve(column_families.size());
V
Venkatesh Radhakrishnan 已提交
3319 3320 3321 3322 3323 3324 3325 3326 3327 3328 3329 3330 3331 3332 3333 3334 3335 3336 3337 3338
  XFUNC_TEST("", "managed_new", managed_new1, xf_manage_new,
             reinterpret_cast<DBImpl*>(this),
             const_cast<ReadOptions*>(&read_options), is_snapshot_supported_);
  if (read_options.managed) {
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Managed interator not supported in RocksDB lite");
#else
    if ((!read_options.tailing) && (read_options.snapshot == nullptr) &&
        (!is_snapshot_supported_)) {
      return Status::InvalidArgument(
          "Managed interator not supported without snapshots");
    }
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
      auto iter = new ManagedIterator(this, read_options, cfd);
      iterators->push_back(iter);
    }
#endif
  } else if (read_options.tailing) {
I
Igor Canadi 已提交
3339 3340 3341 3342
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3343 3344
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
3345 3346
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
3347
      iterators->push_back(
3348
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3349 3350
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
3351
    }
I
Igor Canadi 已提交
3352
#endif
I
Igor Canadi 已提交
3353
  } else {
3354 3355
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
3356
    for (size_t i = 0; i < column_families.size(); ++i) {
3357 3358 3359
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
3360 3361

      auto snapshot =
3362 3363 3364
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
3365 3366
              : latest_snapshot;

3367
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3368
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
3369
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
3370
      Iterator* internal_iter = NewInternalIterator(
3371
          read_options, cfd, sv, db_iter->GetArena());
3372 3373
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3374 3375 3376 3377
    }
  }

  return Status::OK();
3378 3379
}

J
jorlow@chromium.org 已提交
3380
const Snapshot* DBImpl::GetSnapshot() {
3381 3382
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error
3383
  SnapshotImpl* s = new SnapshotImpl;
3384

3385
  InstrumentedMutexLock l(&mutex_);
3386
  // returns null if the underlying memtable does not support snapshot.
3387 3388 3389 3390 3391
  if (!is_snapshot_supported_) {
    delete s;
    return nullptr;
  }
  return snapshots_.New(s, versions_->LastSequence(), unix_time);
J
jorlow@chromium.org 已提交
3392 3393 3394
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
3395 3396 3397 3398 3399 3400
  const SnapshotImpl* casted_s = reinterpret_cast<const SnapshotImpl*>(s);
  {
    InstrumentedMutexLock l(&mutex_);
    snapshots_.Delete(casted_s);
  }
  delete casted_s;
J
jorlow@chromium.org 已提交
3401 3402 3403
}

// Convenience methods
3404 3405
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3406
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3407 3408
}

3409 3410 3411
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
3412
  if (!cfh->cfd()->ioptions()->merge_operator) {
3413 3414
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3415
    return DB::Merge(o, column_family, key, val);
3416 3417 3418
  }
}

L
Lei Jin 已提交
3419
Status DBImpl::Delete(const WriteOptions& write_options,
3420
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
3421
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
3422 3423
}

L
Lei Jin 已提交
3424
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
A
agiardullo 已提交
3425 3426 3427
  return WriteImpl(write_options, my_batch, nullptr);
}

A
agiardullo 已提交
3428
#ifndef ROCKSDB_LITE
A
agiardullo 已提交
3429 3430 3431 3432 3433
Status DBImpl::WriteWithCallback(const WriteOptions& write_options,
                                 WriteBatch* my_batch,
                                 WriteCallback* callback) {
  return WriteImpl(write_options, my_batch, callback);
}
A
agiardullo 已提交
3434
#endif  // ROCKSDB_LITE
A
agiardullo 已提交
3435 3436 3437

Status DBImpl::WriteImpl(const WriteOptions& write_options,
                         WriteBatch* my_batch, WriteCallback* callback) {
S
Stanislau Hlebik 已提交
3438 3439 3440
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
3441 3442 3443
  if (write_options.timeout_hint_us != 0) {
    return Status::InvalidArgument("timeout_hint_us is deprecated");
  }
A
agiardullo 已提交
3444 3445

  Status status;
A
agiardullo 已提交
3446 3447
  bool callback_failed = false;

A
agiardullo 已提交
3448 3449 3450 3451 3452 3453 3454 3455 3456 3457
  bool xfunc_attempted_write = false;
  XFUNC_TEST("transaction", "transaction_xftest_write_impl",
             xf_transaction_write1, xf_transaction_write, write_options,
             db_options_, my_batch, callback, this, &status,
             &xfunc_attempted_write);
  if (xfunc_attempted_write) {
    // Test already did the write
    return status;
  }

3458
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
3459
  WriteThread::Writer w;
S
Stanislau Hlebik 已提交
3460
  w.batch = my_batch;
L
Lei Jin 已提交
3461 3462
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
3463 3464
  w.in_batch_group = false;
  w.done = false;
A
agiardullo 已提交
3465
  w.has_callback = (callback != nullptr) ? true : false;
S
Stanislau Hlebik 已提交
3466

L
Lei Jin 已提交
3467
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
3468 3469 3470
    RecordTick(stats_, WRITE_WITH_WAL);
  }

3471 3472
  StopWatch write_sw(env_, db_options_.statistics.get(), DB_WRITE);

3473 3474 3475 3476 3477 3478 3479 3480
  write_thread_.JoinBatchGroup(&w);
  if (w.done) {
    // write was done by someone else, no need to grab mutex
    RecordTick(stats_, WRITE_DONE_BY_OTHER);
    return w.status;
  }
  // else we are the leader of the write batch group

S
Stanislau Hlebik 已提交
3481 3482
  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
3483 3484 3485 3486 3487

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

S
Stanislau Hlebik 已提交
3488 3489
  RecordTick(stats_, WRITE_DONE_BY_SELF);
  default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_SELF, 1);
3490

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

3498
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
3499
                                    ? 4 * max_total_in_memory_state_
3500
                                    : db_options_.max_total_wal_size;
3501
  if (UNLIKELY(!single_column_family_mode_) &&
3502
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3503
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
3504
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
3505
    alive_log_files_.begin()->getting_flushed = true;
3506
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3507 3508 3509
        "Flushing all column families with data in WAL number %" PRIu64
        ". Total log size is %" PRIu64 " while max_total_wal_size is %" PRIu64,
        flush_column_family_if_log_file, total_log_size_, max_total_wal_size);
I
Igor Canadi 已提交
3510 3511
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
3512
    for (auto cfd : *versions_->GetColumnFamilySet()) {
3513 3514 3515
      if (cfd->IsDropped()) {
        continue;
      }
I
Igor Canadi 已提交
3516
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
I
Igor Canadi 已提交
3517
        status = SwitchMemtable(cfd, &context);
I
Igor Canadi 已提交
3518 3519 3520
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
3521
        cfd->imm()->FlushRequested();
3522
        SchedulePendingFlush(cfd);
3523
      }
3524
    }
I
Igor Canadi 已提交
3525
    MaybeScheduleFlushOrCompaction();
3526 3527 3528 3529 3530 3531 3532 3533
  } else if (UNLIKELY(write_buffer_.ShouldFlush())) {
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Flushing all column families. Write buffer is using %" PRIu64
        " bytes out of a total of %" PRIu64 ".",
        write_buffer_.memory_usage(), write_buffer_.buffer_size());
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
    for (auto cfd : *versions_->GetColumnFamilySet()) {
3534 3535 3536
      if (cfd->IsDropped()) {
        continue;
      }
3537
      if (!cfd->mem()->IsEmpty()) {
I
Igor Canadi 已提交
3538
        status = SwitchMemtable(cfd, &context);
3539 3540 3541 3542
        if (!status.ok()) {
          break;
        }
        cfd->imm()->FlushRequested();
3543
        SchedulePendingFlush(cfd);
3544 3545 3546
      }
    }
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
3547 3548 3549 3550 3551 3552 3553 3554 3555 3556
  }

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

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

S
sdong 已提交
3557 3558
  if (UNLIKELY(status.ok()) &&
      (write_controller_.IsStopped() || write_controller_.NeedsDelay())) {
3559 3560
    PERF_TIMER_STOP(write_pre_and_post_process_time);
    PERF_TIMER_GUARD(write_delay_time);
S
sdong 已提交
3561 3562 3563 3564
    // We don't know size of curent batch so that we always use the size
    // for previous one. It might create a fairness issue that expiration
    // might happen for smaller writes but larger writes can go through.
    // Can optimize it if it is an issue.
3565
    status = DelayWrite(last_batch_group_size_);
3566
    PERF_TIMER_START(write_pre_and_post_process_time);
I
Igor Canadi 已提交
3567 3568
  }

D
dgrogan@chromium.org 已提交
3569
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
3570
  WriteThread::Writer* last_writer = &w;
A
agiardullo 已提交
3571
  autovector<WriteBatch*> write_batch_group;
3572 3573
  bool need_log_sync = !write_options.disableWAL && write_options.sync;
  bool need_log_dir_sync = need_log_sync && !log_dir_synced_;
A
agiardullo 已提交
3574

S
Stanislau Hlebik 已提交
3575
  if (status.ok()) {
3576 3577
    last_batch_group_size_ = write_thread_.EnterAsBatchGroupLeader(
        &w, &last_writer, &write_batch_group);
3578

3579
    if (need_log_sync) {
3580 3581 3582 3583 3584 3585 3586 3587 3588
      while (logs_.front().getting_synced) {
        log_sync_cv_.Wait();
      }
      for (auto& log : logs_) {
        assert(!log.getting_synced);
        log.getting_synced = true;
      }
    }

3589 3590 3591
    // 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
3592
    // into memtables
A
agiardullo 已提交
3593 3594 3595 3596 3597 3598 3599

    mutex_.Unlock();

    if (callback != nullptr) {
      // If this write has a validation callback, check to see if this write
      // is able to be written.  Must be called on the write thread.
      status = callback->Callback(this);
A
agiardullo 已提交
3600
      callback_failed = true;
A
agiardullo 已提交
3601 3602 3603 3604 3605 3606 3607 3608
    }
  } else {
    mutex_.Unlock();
  }

  // At this point the mutex is unlocked

  if (status.ok()) {
3609 3610 3611 3612 3613 3614 3615 3616 3617 3618
      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]);
        }
      }

3619 3620 3621 3622
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
3623
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
3624
      // Record statistics
L
Lei Jin 已提交
3625
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
3626
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
3627
      if (write_options.disableWAL) {
3628
        flush_on_destroy_ = true;
3629
      }
L
Lei Jin 已提交
3630
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3631

3632
      uint64_t log_size = 0;
L
Lei Jin 已提交
3633
      if (!write_options.disableWAL) {
3634
        PERF_TIMER_GUARD(write_wal_time);
3635
        Slice log_entry = WriteBatchInternal::Contents(updates);
3636
        status = logs_.back().writer->AddRecord(log_entry);
I
Igor Canadi 已提交
3637 3638
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
3639
        log_empty_ = false;
3640
        log_size = log_entry.size();
L
Lei Jin 已提交
3641
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
3642
        if (status.ok() && need_log_sync) {
I
Igor Canadi 已提交
3643
          RecordTick(stats_, WAL_FILE_SYNCED);
3644
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
3645 3646 3647 3648 3649 3650 3651 3652
          // It's safe to access logs_ with unlocked mutex_ here because:
          //  - we've set getting_synced=true for all logs,
          //    so other threads won't pop from logs_ while we're here,
          //  - only writer thread can push to logs_, and we're in
          //    writer thread, so no one will push to logs_,
          //  - as long as other threads don't modify it, it's safe to read
          //    from std::deque from multiple threads concurrently.
          for (auto& log : logs_) {
3653
            status = log.writer->file()->Sync(db_options_.use_fsync);
3654 3655 3656 3657
            if (!status.ok()) {
              break;
            }
          }
3658
          if (status.ok() && need_log_dir_sync) {
3659 3660 3661 3662 3663
            // We only sync WAL directory the first time WAL syncing is
            // requested, so that in case users never turn on WAL sync,
            // we can avoid the disk I/O in the write code path.
            status = directories_.GetWalDir()->Fsync();
          }
H
heyongqiang 已提交
3664
        }
3665 3666
      }
      if (status.ok()) {
3667
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
3668

3669
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3670
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
3671
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
3672 3673 3674 3675 3676 3677 3678 3679
        // A non-OK status here indicates iteration failure (either in-memory
        // writebatch corruption (very bad), or the client specified invalid
        // column family).  This will later on trigger bg_error_.
        //
        // Note that existing logic was not sound. Any partial failure writing
        // into the memtable would result in a state that some write ops might
        // have succeeded in memtable but Status reports error for all writes.

L
Lei Jin 已提交
3680
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
3681
      }
L
Lei Jin 已提交
3682
      PERF_TIMER_START(write_pre_and_post_process_time);
3683 3684 3685
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
3686
      mutex_.Lock();
A
agiardullo 已提交
3687

3688 3689 3690
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
3691 3692
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
3693
      if (!write_options.disableWAL) {
3694 3695 3696 3697 3698
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
3699
      if (status.ok()) {
3700
        versions_->SetLastSequence(last_sequence);
3701
      }
A
agiardullo 已提交
3702 3703 3704
  } else {
    // Operation failed.  Make sure sure mutex is held for cleanup code below.
    mutex_.Lock();
3705
  }
A
agiardullo 已提交
3706

A
agiardullo 已提交
3707
  if (db_options_.paranoid_checks && !status.ok() && !callback_failed &&
3708
      !status.IsBusy() && bg_error_.ok()) {
I
Igor Canadi 已提交
3709 3710
    bg_error_ = status; // stop compaction & fail any further writes
  }
3711

A
agiardullo 已提交
3712
  mutex_.AssertHeld();
3713

3714 3715
  if (need_log_sync) {
    MarkLogsSynced(logfile_number_, need_log_dir_sync, status);
3716 3717
  }

3718 3719 3720 3721 3722 3723 3724 3725 3726
  uint64_t writes_for_other = write_batch_group.size() - 1;
  if (writes_for_other > 0) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_OTHER,
                                           writes_for_other);
    if (!write_options.disableWAL) {
      default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_WITH_WAL,
                                             writes_for_other);
    }
  }
3727

I
Igor Canadi 已提交
3728
  mutex_.Unlock();
3729

3730 3731
  write_thread_.ExitAsBatchGroupLeader(&w, last_writer, status);

J
jorlow@chromium.org 已提交
3732 3733 3734
  return status;
}

3735 3736
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
3737
Status DBImpl::DelayWrite(uint64_t num_bytes) {
S
sdong 已提交
3738 3739 3740 3741
  uint64_t time_delayed = 0;
  bool delayed = false;
  {
    StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed);
S
sdong 已提交
3742 3743
    auto delay = write_controller_.GetDelay(env_, num_bytes);
    if (delay > 0) {
S
sdong 已提交
3744 3745
      mutex_.Unlock();
      delayed = true;
S
sdong 已提交
3746
      TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep");
3747 3748
      // hopefully we don't have to sleep more than 2 billion microseconds
      env_->SleepForMicroseconds(static_cast<int>(delay));
S
sdong 已提交
3749 3750
      mutex_.Lock();
    }
3751

3752
    while (bg_error_.ok() && write_controller_.IsStopped()) {
S
sdong 已提交
3753
      delayed = true;
3754 3755
      TEST_SYNC_POINT("DBImpl::DelayWrite:Wait");
      bg_cv_.Wait();
3756 3757
    }
  }
S
sdong 已提交
3758 3759 3760 3761 3762
  if (delayed) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_STALL_MICROS,
                                           time_delayed);
    RecordTick(stats_, STALL_MICROS, time_delayed);
  }
I
Igor Canadi 已提交
3763 3764

  return bg_error_;
3765 3766
}

I
Igor Canadi 已提交
3767 3768 3769
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
I
Igor Canadi 已提交
3770
    auto status = SwitchMemtable(cfd, context);
I
Igor Canadi 已提交
3771 3772 3773 3774 3775
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
3776 3777
    }
  }
I
Igor Canadi 已提交
3778
  return Status::OK();
S
Stanislau Hlebik 已提交
3779 3780 3781 3782
}

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
3783
Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
S
Stanislau Hlebik 已提交
3784 3785 3786 3787 3788 3789 3790
  mutex_.AssertHeld();
  unique_ptr<WritableFile> lfile;
  log::Writer* new_log = nullptr;
  MemTable* new_mem = nullptr;

  // Attempt to switch to a new memtable and trigger flush of old.
  // Do this without holding the dbmutex lock.
3791
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
3792 3793 3794 3795
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
3796
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
3797 3798 3799 3800
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
3801 3802
      EnvOptions opt_env_opt =
          env_->OptimizeForLogWrite(env_options_, db_options_);
3803
      s = env_->NewWritableFile(
3804
          LogFileName(db_options_.wal_dir, new_log_number), &lfile,
3805
          opt_env_opt);
S
Stanislau Hlebik 已提交
3806 3807 3808
      if (s.ok()) {
        // Our final size should be less than write_buffer_size
        // (compression, etc) but err on the side of caution.
L
Lei Jin 已提交
3809 3810
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
3811 3812 3813
        unique_ptr<WritableFileWriter> file_writer(
            new WritableFileWriter(std::move(lfile), opt_env_opt));
        new_log = new log::Writer(std::move(file_writer));
I
Igor Canadi 已提交
3814
      }
S
Stanislau Hlebik 已提交
3815 3816 3817
    }

    if (s.ok()) {
A
agiardullo 已提交
3818 3819
      SequenceNumber seq = versions_->LastSequence();
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options, seq);
S
Stanislau Hlebik 已提交
3820 3821 3822
      new_superversion = new SuperVersion();
    }
  }
3823
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
3824 3825
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
3826 3827 3828 3829 3830 3831 3832 3833 3834 3835 3836 3837
  mutex_.Lock();
  if (!s.ok()) {
    // how do we fail if we're not creating new log?
    assert(creating_new_log);
    assert(!new_mem);
    assert(!new_log);
    return s;
  }
  if (creating_new_log) {
    logfile_number_ = new_log_number;
    assert(new_log != nullptr);
    log_empty_ = true;
3838
    log_dir_synced_ = false;
3839
    logs_.emplace_back(logfile_number_, new_log);
S
Stanislau Hlebik 已提交
3840
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
3841
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
3842 3843 3844 3845
      // all this is just optimization to delete logs that
      // are no longer needed -- if CF is empty, that means it
      // doesn't need that particular log to stay alive, so we just
      // advance the log number. no need to persist this in the manifest
I
Igor Canadi 已提交
3846
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
3847
          loop_cfd->imm()->NumNotFlushed() == 0) {
I
Igor Canadi 已提交
3848
        loop_cfd->SetLogNumber(logfile_number_);
3849
      }
3850 3851
    }
  }
S
Stanislau Hlebik 已提交
3852
  cfd->mem()->SetNextLogNumber(logfile_number_);
3853
  cfd->imm()->Add(cfd->mem(), &context->memtables_to_free_);
S
Stanislau Hlebik 已提交
3854 3855
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
I
Igor Canadi 已提交
3856 3857
  context->superversions_to_free_.push_back(InstallSuperVersionAndScheduleWork(
      cfd, new_superversion, mutable_cf_options));
3858 3859 3860
  return s;
}

I
Igor Canadi 已提交
3861
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
3862 3863 3864 3865 3866
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3867 3868
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
3869
  auto version = cfd->current();
3870 3871 3872 3873 3874 3875 3876 3877 3878 3879 3880 3881
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
3882
#endif  // ROCKSDB_LITE
3883

I
Igor Canadi 已提交
3884 3885 3886 3887
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3888 3889 3890 3891
Env* DBImpl::GetEnv() const {
  return env_;
}

3892 3893
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
3894
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
3895 3896
}

3897 3898
const DBOptions& DBImpl::GetDBOptions() const { return db_options_; }

3899
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
3900
                         const Slice& property, std::string* value) {
3901 3902
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3903 3904 3905
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

3906
  value->clear();
3907 3908 3909 3910 3911
  if (is_int_property) {
    uint64_t int_value;
    bool ret_value = GetIntPropertyInternal(column_family, property_type,
                                            need_out_of_mutex, &int_value);
    if (ret_value) {
3912
      *value = ToString(int_value);
3913 3914 3915 3916 3917
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
3918
    InstrumentedMutexLock l(&mutex_);
3919 3920 3921
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
3922 3923
}

3924 3925
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
3926 3927
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3928 3929 3930 3931 3932 3933 3934 3935 3936 3937 3938 3939
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);
  if (!is_int_property) {
    return false;
  }
  return GetIntPropertyInternal(column_family, property_type, need_out_of_mutex,
                                value);
}

bool DBImpl::GetIntPropertyInternal(ColumnFamilyHandle* column_family,
                                    DBPropertyType property_type,
                                    bool need_out_of_mutex, uint64_t* value) {
3940 3941
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
3942 3943

  if (!need_out_of_mutex) {
3944
    InstrumentedMutexLock l(&mutex_);
3945
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
3946 3947 3948 3949 3950 3951 3952 3953 3954 3955 3956 3957 3958 3959
  } else {
    SuperVersion* sv = GetAndRefSuperVersion(cfd);

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

    ReturnAndCleanupSuperVersion(cfd, sv);

    return ret;
  }
}

SuperVersion* DBImpl::GetAndRefSuperVersion(ColumnFamilyData* cfd) {
  // TODO(ljin): consider using GetReferencedSuperVersion() directly
I
Igor Canadi 已提交
3960
  return cfd->GetThreadLocalSuperVersion(&mutex_);
3961 3962
}

A
agiardullo 已提交
3963 3964 3965 3966 3967 3968 3969 3970 3971 3972 3973 3974
// REQUIRED: this function should only be called on the write thread or if the
// mutex is held.
SuperVersion* DBImpl::GetAndRefSuperVersion(uint32_t column_family_id) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);
  if (!cfd) {
    return nullptr;
  }

  return GetAndRefSuperVersion(cfd);
}

A
agiardullo 已提交
3975 3976 3977 3978 3979 3980 3981 3982 3983 3984 3985 3986 3987 3988 3989 3990
// REQUIRED:  mutex is NOT held
SuperVersion* DBImpl::GetAndRefSuperVersionUnlocked(uint32_t column_family_id) {
  ColumnFamilyData* cfd;
  {
    InstrumentedMutexLock l(&mutex_);
    auto column_family_set = versions_->GetColumnFamilySet();
    cfd = column_family_set->GetColumnFamily(column_family_id);
  }

  if (!cfd) {
    return nullptr;
  }

  return GetAndRefSuperVersion(cfd);
}

3991 3992
void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
3993
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3994 3995 3996 3997 3998

  if (unref_sv) {
    // Release SuperVersion
    if (sv->Unref()) {
      {
3999
        InstrumentedMutexLock l(&mutex_);
4000 4001 4002 4003 4004 4005 4006
        sv->Cleanup();
      }
      delete sv;
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
    }
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
  }
4007 4008
}

A
agiardullo 已提交
4009 4010 4011 4012 4013 4014 4015 4016 4017 4018 4019 4020
// REQUIRED: this function should only be called on the write thread.
void DBImpl::ReturnAndCleanupSuperVersion(uint32_t column_family_id,
                                          SuperVersion* sv) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);

  // If SuperVersion is held, and we successfully fetched a cfd using
  // GetAndRefSuperVersion(), it must still exist.
  assert(cfd != nullptr);
  ReturnAndCleanupSuperVersion(cfd, sv);
}

A
agiardullo 已提交
4021 4022 4023 4024 4025 4026 4027 4028 4029 4030 4031 4032 4033 4034 4035 4036
// REQUIRED: Mutex should NOT be held.
void DBImpl::ReturnAndCleanupSuperVersionUnlocked(uint32_t column_family_id,
                                                  SuperVersion* sv) {
  ColumnFamilyData* cfd;
  {
    InstrumentedMutexLock l(&mutex_);
    auto column_family_set = versions_->GetColumnFamilySet();
    cfd = column_family_set->GetColumnFamily(column_family_id);
  }

  // If SuperVersion is held, and we successfully fetched a cfd using
  // GetAndRefSuperVersion(), it must still exist.
  assert(cfd != nullptr);
  ReturnAndCleanupSuperVersion(cfd, sv);
}

A
agiardullo 已提交
4037 4038 4039 4040 4041 4042 4043 4044 4045 4046 4047 4048
// REQUIRED: this function should only be called on the write thread or if the
// mutex is held.
ColumnFamilyHandle* DBImpl::GetColumnFamilyHandle(uint32_t column_family_id) {
  ColumnFamilyMemTables* cf_memtables = column_family_memtables_.get();

  if (!cf_memtables->Seek(column_family_id)) {
    return nullptr;
  }

  return cf_memtables->GetColumnFamilyHandle();
}

A
agiardullo 已提交
4049 4050 4051 4052 4053 4054 4055 4056 4057 4058 4059 4060 4061 4062
// REQUIRED: mutex is NOT held.
ColumnFamilyHandle* DBImpl::GetColumnFamilyHandleUnlocked(
    uint32_t column_family_id) {
  ColumnFamilyMemTables* cf_memtables = column_family_memtables_.get();

  InstrumentedMutexLock l(&mutex_);

  if (!cf_memtables->Seek(column_family_id)) {
    return nullptr;
  }

  return cf_memtables->GetColumnFamilyHandle();
}

4063
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4064 4065
                                 const Range* range, int n, uint64_t* sizes,
                                 bool include_memtable) {
J
jorlow@chromium.org 已提交
4066
  Version* v;
4067 4068
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4069 4070
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
  v = sv->current;
J
jorlow@chromium.org 已提交
4071 4072 4073

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
4074 4075 4076
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
    sizes[i] = versions_->ApproximateSize(v, k1.Encode(), k2.Encode());
4077 4078 4079 4080
    if (include_memtable) {
      sizes[i] += sv->mem->ApproximateSize(k1.Encode(), k2.Encode());
      sizes[i] += sv->imm->ApproximateSize(k1.Encode(), k2.Encode());
    }
J
jorlow@chromium.org 已提交
4081 4082
  }

4083
  ReturnAndCleanupSuperVersion(cfd, sv);
J
jorlow@chromium.org 已提交
4084 4085
}

I
Igor Canadi 已提交
4086 4087 4088 4089 4090 4091 4092 4093 4094 4095 4096 4097 4098 4099 4100 4101
std::list<uint64_t>::iterator
DBImpl::CaptureCurrentFileNumberInPendingOutputs() {
  // We need to remember the iterator of our insert, because after the
  // background job is done, we need to remove that element from
  // pending_outputs_.
  pending_outputs_.push_back(versions_->current_next_file_number());
  auto pending_outputs_inserted_elem = pending_outputs_.end();
  --pending_outputs_inserted_elem;
  return pending_outputs_inserted_elem;
}

void DBImpl::ReleaseFileNumberFromPendingOutputs(
    std::list<uint64_t>::iterator v) {
  pending_outputs_.erase(v);
}

I
Igor Canadi 已提交
4102 4103 4104 4105 4106
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
4107
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
4108 4109 4110
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
4111
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
4112 4113
}

4114 4115 4116
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4117 4118 4119
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
4120 4121
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
4122 4123 4124
    return Status::InvalidArgument("Invalid file name");
  }

4125 4126 4127 4128
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4129 4130
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
4131
          name.c_str());
4132 4133
      return Status::NotSupported("Delete only supported for archived logs");
    }
4134
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
4135
    if (!status.ok()) {
4136 4137
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
4138
          name.c_str(), status.ToString().c_str());
4139 4140 4141 4142
    }
    return status;
  }

4143
  int level;
I
Igor Canadi 已提交
4144
  FileMetaData* metadata;
4145
  ColumnFamilyData* cfd;
4146
  VersionEdit edit;
4147
  JobContext job_context(next_job_id_.fetch_add(1), true);
D
Dhruba Borthakur 已提交
4148
  {
4149
    InstrumentedMutexLock l(&mutex_);
4150
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4151
    if (!status.ok()) {
4152 4153
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
4154
      job_context.Clean();
D
Dhruba Borthakur 已提交
4155 4156
      return Status::InvalidArgument("File not found");
    }
4157
    assert(level < cfd->NumberLevels());
4158

D
Dhruba Borthakur 已提交
4159
    // If the file is being compacted no need to delete.
4160
    if (metadata->being_compacted) {
4161
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
4162
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
4163
      job_context.Clean();
D
Dhruba Borthakur 已提交
4164
      return Status::OK();
4165 4166
    }

D
Dhruba Borthakur 已提交
4167 4168 4169
    // 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.
S
sdong 已提交
4170
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
4171
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
4172
      if (vstoreage->NumLevelFiles(i) != 0) {
4173
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
4174
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
4175
        job_context.Clean();
D
Dhruba Borthakur 已提交
4176 4177 4178
        return Status::InvalidArgument("File not in last level");
      }
    }
4179
    // if level == 0, it has to be the oldest file
S
sdong 已提交
4180 4181
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
4182 4183
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
4184
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
4185
      job_context.Clean();
4186 4187 4188
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
4189
    edit.DeleteFile(level, number);
4190
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
4191
                                    &edit, &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
4192
    if (status.ok()) {
I
Igor Canadi 已提交
4193 4194
      InstallSuperVersionAndScheduleWorkWrapper(
          cfd, &job_context, *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
4195
    }
I
Igor Canadi 已提交
4196 4197
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
4198

4199
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
4200
  // remove files outside the db-lock
I
Igor Canadi 已提交
4201
  if (job_context.HaveSomethingToDelete()) {
4202
    // Call PurgeObsoleteFiles() without holding mutex.
I
Igor Canadi 已提交
4203
    PurgeObsoleteFiles(job_context);
4204
  }
I
Igor Canadi 已提交
4205
  job_context.Clean();
4206 4207 4208
  return status;
}

4209
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
4210
  InstrumentedMutexLock l(&mutex_);
4211
  versions_->GetLiveFilesMetaData(metadata);
4212
}
4213 4214 4215 4216 4217 4218 4219 4220 4221 4222 4223

void DBImpl::GetColumnFamilyMetaData(
    ColumnFamilyHandle* column_family,
    ColumnFamilyMetaData* cf_meta) {
  assert(column_family);
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  auto* sv = GetAndRefSuperVersion(cfd);
  sv->current->GetColumnFamilyMetaData(cf_meta);
  ReturnAndCleanupSuperVersion(cfd, sv);
}

I
Igor Canadi 已提交
4224
#endif  // ROCKSDB_LITE
4225

I
Igor Canadi 已提交
4226 4227 4228 4229 4230 4231 4232
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

  std::string corruption_messages;
  for (const auto& md : metadata) {
4233 4234
    // md.name has a leading "/".
    std::string file_path = md.db_path + md.name;
4235

I
Igor Canadi 已提交
4236 4237 4238 4239 4240 4241
    uint64_t fsize = 0;
    Status s = env_->GetFileSize(file_path, &fsize);
    if (!s.ok()) {
      corruption_messages +=
          "Can't access " + md.name + ": " + s.ToString() + "\n";
    } else if (fsize != md.size) {
4242
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
4243
                             ". Size recorded in manifest " +
4244 4245
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
4246 4247 4248 4249 4250 4251 4252 4253 4254
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

4255
Status DBImpl::GetDbIdentity(std::string& identity) const {
4256 4257
  std::string idfilename = IdentityFileName(dbname_);
  const EnvOptions soptions;
4258 4259 4260 4261 4262 4263 4264 4265 4266
  unique_ptr<SequentialFileReader> id_file_reader;
  Status s;
  {
    unique_ptr<SequentialFile> idfile;
    s = env_->NewSequentialFile(idfilename, &idfile, soptions);
    if (!s.ok()) {
      return s;
    }
    id_file_reader.reset(new SequentialFileReader(std::move(idfile)));
4267
  }
4268

4269 4270 4271 4272 4273
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
D
Dmitri Smirnov 已提交
4274
  char* buffer = reinterpret_cast<char*>(alloca(file_size));
4275
  Slice id;
4276
  s = id_file_reader->Read(static_cast<size_t>(file_size), &id, buffer);
4277 4278 4279 4280 4281 4282 4283 4284 4285 4286 4287
  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 已提交
4288 4289
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
4290
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
4291
               const Slice& key, const Slice& value) {
4292 4293 4294 4295
  // 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);
4296
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
4297 4298 4299
  return Write(opt, &batch);
}

4300 4301
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
4302
  WriteBatch batch;
4303
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
4304 4305 4306
  return Write(opt, &batch);
}

4307 4308
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
4309
  WriteBatch batch;
4310
  batch.Merge(column_family, key, value);
4311 4312 4313
  return Write(opt, &batch);
}

4314
// Default implementation -- returns not supported status
L
Lei Jin 已提交
4315
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
4316
                              const std::string& column_family_name,
4317
                              ColumnFamilyHandle** handle) {
4318
  return Status::NotSupported("");
4319
}
4320
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4321
  return Status::NotSupported("");
4322 4323
}

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

J
Jim Paton 已提交
4326
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4327 4328 4329 4330
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4331
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4332
  std::vector<ColumnFamilyHandle*> handles;
4333
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
4334 4335 4336 4337 4338 4339 4340
  if (s.ok()) {
    assert(handles.size() == 1);
    // i can delete the handle since DBImpl is always holding a reference to
    // default column family
    delete handles[0];
  }
  return s;
4341 4342
}

4343 4344
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4345
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4346
  Status s = SanitizeOptionsByTable(db_options, column_families);
4347 4348 4349
  if (!s.ok()) {
    return s;
  }
4350

4351 4352 4353 4354 4355 4356
  for (auto& cfd : column_families) {
    s = CheckCompressionSupported(cfd.options);
    if (!s.ok()) {
      return s;
    }
    if (db_options.db_paths.size() > 1) {
4357 4358
      if ((cfd.options.compaction_style != kCompactionStyleUniversal) &&
          (cfd.options.compaction_style != kCompactionStyleLevel)) {
4359 4360
        return Status::NotSupported(
            "More than one DB paths are only supported in "
4361
            "universal and level compaction styles. ");
4362 4363
      }
    }
4364
  }
4365

4366 4367 4368
  if (db_options.db_paths.size() > 4) {
    return Status::NotSupported(
        "More than four DB paths are not supported yet. ");
4369 4370
  }

4371
  *dbptr = nullptr;
4372
  handles->clear();
J
jorlow@chromium.org 已提交
4373

I
Igor Canadi 已提交
4374 4375 4376 4377
  size_t max_write_buffer_size = 0;
  for (auto cf : column_families) {
    max_write_buffer_size =
        std::max(max_write_buffer_size, cf.options.write_buffer_size);
4378
  }
4379

I
Igor Canadi 已提交
4380
  DBImpl* impl = new DBImpl(db_options, dbname);
4381
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
4382
  if (s.ok()) {
4383
    for (auto db_path : impl->db_options_.db_paths) {
4384
      s = impl->env_->CreateDirIfMissing(db_path.path);
4385 4386 4387 4388 4389 4390
      if (!s.ok()) {
        break;
      }
    }
  }

4391 4392 4393 4394 4395 4396
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4397 4398 4399 4400
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4401
  impl->mutex_.Lock();
4402 4403
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4404
  if (s.ok()) {
4405
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4406
    unique_ptr<WritableFile> lfile;
4407
    EnvOptions soptions(db_options);
4408 4409
    EnvOptions opt_env_options =
        impl->db_options_.env->OptimizeForLogWrite(soptions, impl->db_options_);
4410 4411
    s = impl->db_options_.env->NewWritableFile(
        LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
4412
        opt_env_options);
J
jorlow@chromium.org 已提交
4413
    if (s.ok()) {
4414
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4415
      impl->logfile_number_ = new_log_number;
4416 4417
      unique_ptr<WritableFileWriter> file_writer(
          new WritableFileWriter(std::move(lfile), opt_env_options));
4418 4419
      impl->logs_.emplace_back(new_log_number,
                               new log::Writer(std::move(file_writer)));
I
Igor Canadi 已提交
4420

4421 4422
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4423 4424
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
4425 4426 4427
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
4428
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
4429 4430 4431 4432 4433 4434 4435 4436 4437 4438 4439 4440 4441 4442 4443 4444
        } else {
          if (db_options.create_missing_column_families) {
            // missing column family, create it
            ColumnFamilyHandle* handle;
            impl->mutex_.Unlock();
            s = impl->CreateColumnFamily(cf.options, cf.name, &handle);
            impl->mutex_.Lock();
            if (s.ok()) {
              handles->push_back(handle);
            } else {
              break;
            }
          } else {
            s = Status::InvalidArgument("Column family not found: ", cf.name);
            break;
          }
I
Igor Canadi 已提交
4445
        }
4446
      }
I
Igor Canadi 已提交
4447 4448
    }
    if (s.ok()) {
4449
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
4450
        delete impl->InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
4451
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
4452
      }
I
Igor Canadi 已提交
4453 4454
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4455
      impl->DeleteObsoleteFiles();
4456
      s = impl->directories_.GetDbDir()->Fsync();
J
jorlow@chromium.org 已提交
4457 4458
    }
  }
4459

I
Igor Canadi 已提交
4460 4461
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
S
sdong 已提交
4462
      if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
4463
        auto* vstorage = cfd->current()->storage_info();
4464
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
4465
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
4466
          if (num_files > 0) {
I
Igor Canadi 已提交
4467 4468
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
S
sdong 已提交
4469
                "open with FIFO compaction style.");
I
Igor Canadi 已提交
4470 4471 4472 4473
            break;
          }
        }
      }
S
sdong 已提交
4474 4475 4476
      if (!cfd->mem()->IsSnapshotSupported()) {
        impl->is_snapshot_supported_ = false;
      }
4477
      if (cfd->ioptions()->merge_operator != nullptr &&
4478 4479 4480 4481 4482
          !cfd->mem()->IsMergeOperatorSupported()) {
        s = Status::InvalidArgument(
            "The memtable of column family %s does not support merge operator "
            "its options.merge_operator is non-null", cfd->GetName().c_str());
      }
I
Igor Canadi 已提交
4483
      if (!s.ok()) {
4484 4485 4486 4487
        break;
      }
    }
  }
4488 4489 4490 4491 4492
  TEST_SYNC_POINT("DBImpl::Open:Opened");
  if (s.ok()) {
    impl->opened_successfully_ = true;
    impl->MaybeScheduleFlushOrCompaction();
  }
4493 4494
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4495
  if (s.ok()) {
S
sdong 已提交
4496 4497
    Log(InfoLogLevel::INFO_LEVEL, impl->db_options_.info_log, "DB pointer %p",
        impl);
J
jorlow@chromium.org 已提交
4498 4499
    *dbptr = impl;
  } else {
Y
Yueh-Hsuan Chiang 已提交
4500
    for (auto* h : *handles) {
4501 4502
      delete h;
    }
4503
    handles->clear();
J
jorlow@chromium.org 已提交
4504 4505 4506 4507 4508
    delete impl;
  }
  return s;
}

4509 4510 4511
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4512
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4513 4514
}

4515 4516 4517
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4518
Status DestroyDB(const std::string& dbname, const Options& options) {
4519
  const InternalKeyComparator comparator(options.comparator);
4520
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
4521
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4522
  std::vector<std::string> filenames;
4523

J
jorlow@chromium.org 已提交
4524 4525
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
4526

J
jorlow@chromium.org 已提交
4527
  FileLock* lock;
4528 4529
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4530 4531 4532
  if (result.ok()) {
    uint64_t number;
    FileType type;
4533
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
4534
    for (size_t i = 0; i < filenames.size(); i++) {
4535
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
4536
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4537
        Status del;
4538
        std::string path_to_delete = dbname + "/" + filenames[i];
K
Kosie van der Merwe 已提交
4539
        if (type == kMetaDatabase) {
4540 4541 4542
          del = DestroyDB(path_to_delete, options);
        } else if (type == kTableFile) {
          del = DeleteOrMoveToTrash(&options, path_to_delete);
K
Kosie van der Merwe 已提交
4543
        } else {
4544
          del = env->DeleteFile(path_to_delete);
K
Kosie van der Merwe 已提交
4545
        }
J
jorlow@chromium.org 已提交
4546 4547 4548 4549 4550
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4551

4552 4553
    for (size_t path_id = 0; path_id < options.db_paths.size(); path_id++) {
      const auto& db_path = options.db_paths[path_id];
4554
      env->GetChildren(db_path.path, &filenames);
4555 4556 4557
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
4558 4559 4560 4561 4562 4563 4564
          Status del;
          std::string table_path = db_path.path + "/" + filenames[i];
          if (path_id == 0) {
            del = DeleteOrMoveToTrash(&options, table_path);
          } else {
            del = env->DeleteFile(table_path);
          }
4565 4566 4567 4568 4569 4570 4571
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

I
Igor Canadi 已提交
4572 4573 4574 4575 4576 4577 4578 4579 4580 4581 4582 4583 4584 4585 4586 4587 4588 4589
    std::vector<std::string> walDirFiles;
    std::string archivedir = ArchivalDirectory(dbname);
    if (dbname != soptions.wal_dir) {
      env->GetChildren(soptions.wal_dir, &walDirFiles);
      archivedir = ArchivalDirectory(soptions.wal_dir);
    }

    // Delete log files in the WAL dir
    for (const auto& file : walDirFiles) {
      if (ParseFileName(file, &number, &type) && type == kLogFile) {
        Status del = env->DeleteFile(soptions.wal_dir + "/" + file);
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }

    std::vector<std::string> archiveFiles;
4590
    env->GetChildren(archivedir, &archiveFiles);
4591 4592
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4593 4594
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4595
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4596 4597 4598 4599 4600
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4601
    // ignore case where no archival directory is present.
4602
    env->DeleteDir(archivedir);
4603

J
jorlow@chromium.org 已提交
4604
    env->UnlockFile(lock);  // Ignore error since state is already gone
4605
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4606
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4607
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4608 4609 4610 4611
  }
  return result;
}

Y
Yueh-Hsuan Chiang 已提交
4612
#if ROCKSDB_USING_THREAD_STATUS
4613

Y
Yueh-Hsuan Chiang 已提交
4614 4615
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4616
  if (db_options_.enable_thread_tracking) {
4617
    ThreadStatusUtil::NewColumnFamilyInfo(this, cfd);
4618
  }
Y
Yueh-Hsuan Chiang 已提交
4619 4620 4621 4622
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4623
  if (db_options_.enable_thread_tracking) {
4624
    ThreadStatusUtil::EraseColumnFamilyInfo(cfd);
4625
  }
Y
Yueh-Hsuan Chiang 已提交
4626 4627 4628
}

void DBImpl::EraseThreadStatusDbInfo() const {
4629
  if (db_options_.enable_thread_tracking) {
4630
    ThreadStatusUtil::EraseDatabaseInfo(this);
4631
  }
Y
Yueh-Hsuan Chiang 已提交
4632 4633 4634 4635 4636 4637 4638 4639 4640 4641 4642 4643 4644 4645 4646
}

#else
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
}

void DBImpl::EraseThreadStatusDbInfo() const {
}
#endif  // ROCKSDB_USING_THREAD_STATUS

4647 4648
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
4649
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4650
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
4651 4652
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
4653
  Warn(log,
4654
      "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
I
Igor Canadi 已提交
4655
      ROCKSDB_PATCH);
4656 4657
  Warn(log, "Git sha %s", rocksdb_build_git_sha);
  Warn(log, "Compile date %s", rocksdb_build_compile_date);
I
Igor Canadi 已提交
4658
#endif
4659 4660
}

A
agiardullo 已提交
4661 4662 4663 4664 4665 4666 4667 4668 4669 4670 4671 4672 4673 4674 4675 4676 4677 4678 4679 4680 4681 4682 4683 4684 4685 4686 4687 4688 4689 4690 4691 4692 4693 4694 4695 4696 4697 4698 4699 4700 4701 4702 4703 4704 4705 4706 4707 4708 4709 4710 4711 4712 4713 4714 4715 4716 4717 4718 4719 4720 4721 4722 4723 4724 4725 4726 4727 4728 4729 4730 4731 4732 4733 4734 4735 4736 4737 4738 4739
#ifndef ROCKSDB_LITE
SequenceNumber DBImpl::GetEarliestMemTableSequenceNumber(SuperVersion* sv,
                                                         bool include_history) {
  // Find the earliest sequence number that we know we can rely on reading
  // from the memtable without needing to check sst files.
  SequenceNumber earliest_seq =
      sv->imm->GetEarliestSequenceNumber(include_history);
  if (earliest_seq == kMaxSequenceNumber) {
    earliest_seq = sv->mem->GetEarliestSequenceNumber();
  }
  assert(sv->mem->GetEarliestSequenceNumber() >= earliest_seq);

  return earliest_seq;
}
#endif  // ROCKSDB_LITE

#ifndef ROCKSDB_LITE
Status DBImpl::GetLatestSequenceForKeyFromMemtable(SuperVersion* sv,
                                                   const Slice& key,
                                                   SequenceNumber* seq) {
  Status s;
  std::string value;
  MergeContext merge_context;

  SequenceNumber current_seq = versions_->LastSequence();
  LookupKey lkey(key, current_seq);

  *seq = kMaxSequenceNumber;

  // Check if there is a record for this key in the latest memtable
  sv->mem->Get(lkey, &value, &s, &merge_context, seq);

  if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
    // unexpected error reading memtable.
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "Unexpected status returned from MemTable::Get: %s\n",
        s.ToString().c_str());

    return s;
  }

  if (*seq != kMaxSequenceNumber) {
    // Found a sequence number, no need to check immutable memtables
    return Status::OK();
  }

  // Check if there is a record for this key in the immutable memtables
  sv->imm->Get(lkey, &value, &s, &merge_context, seq);

  if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
    // unexpected error reading memtable.
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "Unexpected status returned from MemTableList::Get: %s\n",
        s.ToString().c_str());

    return s;
  }

  if (*seq != kMaxSequenceNumber) {
    // Found a sequence number, no need to check memtable history
    return Status::OK();
  }

  // Check if there is a record for this key in the immutable memtables
  sv->imm->GetFromHistory(lkey, &value, &s, &merge_context, seq);

  if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
    // unexpected error reading memtable.
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "Unexpected status returned from MemTableList::GetFromHistory: %s\n",
        s.ToString().c_str());

    return s;
  }

  return Status::OK();
}
#endif  // ROCKSDB_LITE

4740
}  // namespace rocksdb