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

#include "db/db_impl.h"

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

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

J
jorlow@chromium.org 已提交
29
#include "db/builder.h"
I
Igor Canadi 已提交
30
#include "db/flush_job.h"
I
Igor Canadi 已提交
31
#include "db/compaction_job.h"
32
#include "db/db_iter.h"
K
kailiu 已提交
33
#include "db/dbformat.h"
J
jorlow@chromium.org 已提交
34
#include "db/filename.h"
I
Igor Canadi 已提交
35
#include "db/job_context.h"
J
jorlow@chromium.org 已提交
36 37 38
#include "db/log_reader.h"
#include "db/log_writer.h"
#include "db/memtable.h"
K
kailiu 已提交
39
#include "db/memtable_list.h"
40
#include "db/merge_context.h"
41
#include "db/merge_helper.h"
J
jorlow@chromium.org 已提交
42
#include "db/table_cache.h"
K
kailiu 已提交
43
#include "db/table_properties_collector.h"
L
Lei Jin 已提交
44
#include "db/forward_iterator.h"
45
#include "db/transaction_log_impl.h"
J
jorlow@chromium.org 已提交
46
#include "db/version_set.h"
47
#include "db/writebuffer.h"
J
jorlow@chromium.org 已提交
48
#include "db/write_batch_internal.h"
49
#include "port/port.h"
I
Igor Canadi 已提交
50
#include "rocksdb/cache.h"
51
#include "port/likely.h"
52 53 54 55
#include "rocksdb/compaction_filter.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/merge_operator.h"
I
Igor Canadi 已提交
56
#include "rocksdb/version.h"
57 58
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
S
Siying Dong 已提交
59
#include "rocksdb/table.h"
J
jorlow@chromium.org 已提交
60
#include "table/block.h"
61
#include "table/block_based_table_factory.h"
J
jorlow@chromium.org 已提交
62
#include "table/merger.h"
K
kailiu 已提交
63
#include "table/table_builder.h"
J
jorlow@chromium.org 已提交
64
#include "table/two_level_iterator.h"
65
#include "util/auto_roll_logger.h"
K
kailiu 已提交
66
#include "util/autovector.h"
67
#include "util/build_version.h"
J
jorlow@chromium.org 已提交
68
#include "util/coding.h"
69
#include "util/db_info_dumper.h"
I
Igor Canadi 已提交
70
#include "util/hash_skiplist_rep.h"
71
#include "util/hash_linklist_rep.h"
J
jorlow@chromium.org 已提交
72
#include "util/logging.h"
H
Haobo Xu 已提交
73
#include "util/log_buffer.h"
J
jorlow@chromium.org 已提交
74
#include "util/mutexlock.h"
75
#include "util/perf_context_imp.h"
76
#include "util/iostats_context_imp.h"
77
#include "util/stop_watch.h"
78
#include "util/sync_point.h"
79
#include "util/string_util.h"
Y
Yueh-Hsuan Chiang 已提交
80
#include "util/thread_status_impl.h"
J
jorlow@chromium.org 已提交
81

82
namespace rocksdb {
J
jorlow@chromium.org 已提交
83

84
const std::string kDefaultColumnFamilyName("default");
85

I
Igor Canadi 已提交
86
void DumpRocksDBBuildVersion(Logger * log);
87

S
Stanislau Hlebik 已提交
88 89 90 91 92 93 94 95 96 97 98 99 100 101
struct DBImpl::WriteContext {
  autovector<SuperVersion*> superversions_to_free_;
  autovector<log::Writer*> logs_to_free_;

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

J
jorlow@chromium.org 已提交
102 103 104
Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
                        const Options& src) {
105
  auto db_options = SanitizeOptions(dbname, DBOptions(src));
106
  auto cf_options = SanitizeOptions(icmp, ColumnFamilyOptions(src));
107 108 109 110 111
  return Options(db_options, cf_options);
}

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

113 114
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
115
    ClipToRange(&result.max_open_files, 20, 1000000);
116
  }
117

118
  if (result.info_log == nullptr) {
K
Kai Liu 已提交
119 120
    Status s = CreateLoggerFromOptions(dbname, result.db_log_dir, src.env,
                                       result, &result.info_log);
J
jorlow@chromium.org 已提交
121 122
    if (!s.ok()) {
      // No place suitable for logging
123
      result.info_log = nullptr;
J
jorlow@chromium.org 已提交
124 125
    }
  }
126 127 128 129
  result.env->IncBackgroundThreadsIfNeeded(src.max_background_compactions,
                                           Env::Priority::LOW);
  result.env->IncBackgroundThreadsIfNeeded(src.max_background_flushes,
                                           Env::Priority::HIGH);
130

131 132 133 134 135 136
  if (!result.rate_limiter) {
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

137 138 139 140
  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
141
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
142
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
143
  }
144

145
  if (result.db_paths.size() == 0) {
146
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
147 148
  }

J
jorlow@chromium.org 已提交
149 150 151
  return result;
}

152 153
namespace {

154 155
Status SanitizeOptionsByTable(
    const DBOptions& db_opts,
156 157 158
    const std::vector<ColumnFamilyDescriptor>& column_families) {
  Status s;
  for (auto cf : column_families) {
159
    s = cf.options.table_factory->SanitizeOptions(db_opts, cf.options);
160 161 162 163 164 165 166
    if (!s.ok()) {
      return s;
    }
  }
  return Status::OK();
}

167
CompressionType GetCompressionFlush(const ImmutableCFOptions& ioptions) {
168 169 170 171 172 173
  // 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;

174
  if (ioptions.compaction_style == kCompactionStyleUniversal) {
175
    can_compress =
176
        (ioptions.compaction_options_universal.compression_size_percent < 0);
177 178
  } else {
    // For leveled compress when min_level_to_compress == 0.
179 180
    can_compress = ioptions.compression_per_level.empty() ||
                   ioptions.compression_per_level[0] != kNoCompression;
181 182 183
  }

  if (can_compress) {
184
    return ioptions.compression;
185 186 187 188
  } else {
    return kNoCompression;
  }
}
189
}  // namespace
190

I
Igor Canadi 已提交
191
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
J
jorlow@chromium.org 已提交
192
    : env_(options.env),
H
heyongqiang 已提交
193
      dbname_(dbname),
194 195
      db_options_(SanitizeOptions(dbname, options)),
      stats_(db_options_.statistics.get()),
196
      db_lock_(nullptr),
H
Haobo Xu 已提交
197
      mutex_(options.use_adaptive_mutex),
I
Igor Canadi 已提交
198
      shutting_down_(false),
J
jorlow@chromium.org 已提交
199
      bg_cv_(&mutex_),
200
      logfile_number_(0),
I
Igor Canadi 已提交
201
      log_empty_(true),
202
      default_cf_handle_(nullptr),
I
Igor Canadi 已提交
203 204
      total_log_size_(0),
      max_total_in_memory_state_(0),
205
      write_buffer_(options.db_write_buffer_size),
206
      tmp_batch_(),
207
      bg_schedule_needed_(false),
208
      bg_compaction_scheduled_(0),
209
      bg_manual_only_(0),
210
      bg_flush_scheduled_(0),
211
      manual_compaction_(nullptr),
212
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
213
      delete_obsolete_files_last_run_(options.env->NowMicros()),
214
      last_stats_dump_time_microsec_(0),
215
      flush_on_destroy_(false),
L
Lei Jin 已提交
216
      env_options_(options),
I
Igor Canadi 已提交
217 218 219
#ifndef ROCKSDB_LITE
      wal_manager_(db_options_, env_options_),
#endif  // ROCKSDB_LITE
220
      bg_work_gate_closed_(false),
221
      refitting_level_(false),
222 223
      opened_successfully_(false),
      notifying_events_(0) {
H
heyongqiang 已提交
224
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
225

J
jorlow@chromium.org 已提交
226
  // Reserve ten files or so for other uses and give the rest to TableCache.
227
  // Give a large number for setting of "infinite" open files.
228 229
  const int table_cache_size = (db_options_.max_open_files == -1) ?
        4194304 : db_options_.max_open_files - 10;
230 231
  // Reserve ten files or so for other uses and give the rest to TableCache.
  table_cache_ =
232 233
      NewLRUCache(table_cache_size, db_options_.table_cache_numshardbits,
                  db_options_.table_cache_remove_scan_count_limit);
234

235
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
236 237
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
I
Igor Canadi 已提交
238 239
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
240

I
Igor Canadi 已提交
241
  DumpRocksDBBuildVersion(db_options_.info_log.get());
242 243
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
244

245
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
246 247 248
}

DBImpl::~DBImpl() {
Y
Yueh-Hsuan Chiang 已提交
249
  EraseThreadStatusDbInfo();
250
  mutex_.Lock();
251

252 253
  if (flush_on_destroy_) {
    for (auto cfd : *versions_->GetColumnFamilySet()) {
254
      if (!cfd->mem()->IsEmpty()) {
255 256 257 258
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
259
        cfd->Unref();
260 261
      }
    }
262
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
263
  }
I
Igor Canadi 已提交
264 265

  // Wait for background work to finish
I
Igor Canadi 已提交
266
  shutting_down_.store(true, std::memory_order_release);
267
  while (bg_compaction_scheduled_ || bg_flush_scheduled_ || notifying_events_) {
H
hans@chromium.org 已提交
268
    bg_cv_.Wait();
J
jorlow@chromium.org 已提交
269
  }
270
  listeners_.clear();
I
Igor Canadi 已提交
271 272
  flush_scheduler_.Clear();

I
Igor Canadi 已提交
273 274 275 276 277
  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();
278 279
  }

I
Igor Canadi 已提交
280 281 282 283 284 285 286 287 288 289
  // 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_) {
I
Igor Canadi 已提交
290 291
    JobContext job_context;
    FindObsoleteFiles(&job_context, true);
I
Igor Canadi 已提交
292
    // manifest number starting from 2
I
Igor Canadi 已提交
293 294 295
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
296
    }
I
Igor Canadi 已提交
297
    job_context.Clean();
298 299
  }

300
  // versions need to be destroyed before table_cache since it can hold
301 302
  // references to table_cache.
  versions_.reset();
303
  mutex_.Unlock();
I
Igor Canadi 已提交
304 305 306
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
307

308
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
309 310 311
}

Status DBImpl::NewDB() {
312
  VersionEdit new_db;
313
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
314 315 316
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

317 318
  Log(InfoLogLevel::INFO_LEVEL,
      db_options_.info_log, "Creating manifest 1 \n");
J
jorlow@chromium.org 已提交
319
  const std::string manifest = DescriptorFileName(dbname_, 1);
320
  unique_ptr<WritableFile> file;
I
Igor Canadi 已提交
321
  Status s = env_->NewWritableFile(
L
Lei Jin 已提交
322
      manifest, &file, env_->OptimizeForManifestWrite(env_options_));
J
jorlow@chromium.org 已提交
323 324 325
  if (!s.ok()) {
    return s;
  }
326
  file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
J
jorlow@chromium.org 已提交
327
  {
328
    log::Writer log(std::move(file));
J
jorlow@chromium.org 已提交
329 330 331 332 333 334
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
335
    s = SetCurrentFile(env_, dbname_, 1, db_directory_.get());
J
jorlow@chromium.org 已提交
336 337 338 339 340 341 342
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
343
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
344 345
    // No change needed
  } else {
346 347
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
348 349 350 351
    *s = Status::OK();
  }
}

352
const Status DBImpl::CreateArchivalDirectory() {
353 354
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
355 356 357 358 359
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

360
void DBImpl::PrintStatistics() {
361
  auto dbstats = db_options_.statistics.get();
362
  if (dbstats) {
363
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
364 365
        "STATISTCS:\n %s",
        dbstats->ToString().c_str());
366 367 368
  }
}

369
void DBImpl::MaybeDumpStats() {
370
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
371 372 373 374

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
375
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
376 377 378 379 380 381
      <= 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;
382

383 384 385 386 387 388
    bool tmp1 = false;
    bool tmp2 = false;
    DBPropertyType cf_property_type =
        GetPropertyType("rocksdb.cfstats", &tmp1, &tmp2);
    DBPropertyType db_property_type =
        GetPropertyType("rocksdb.dbstats", &tmp1, &tmp2);
H
Haobo Xu 已提交
389
    std::string stats;
390 391 392
    {
      MutexLock l(&mutex_);
      for (auto cfd : *versions_->GetColumnFamilySet()) {
393 394
        cfd->internal_stats()->GetStringProperty(cf_property_type,
                                                 "rocksdb.cfstats", &stats);
395
      }
396 397
      default_cf_internal_stats_->GetStringProperty(db_property_type,
                                                    "rocksdb.dbstats", &stats);
398
    }
399 400 401 402
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "------- DUMPING STATS -------");
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "%s", stats.c_str());
403

404
    PrintStatistics();
405 406 407
  }
}

408
// Returns the list of live files in 'sst_live' and the list
K
kailiu 已提交
409
// of all files in the filesystem in 'candidate_files'.
I
Igor Canadi 已提交
410 411
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
412
//  db_options_.delete_obsolete_files_period_micros
I
Igor Canadi 已提交
413
// force = true -- force the full scan
I
Igor Canadi 已提交
414
void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
I
Igor Canadi 已提交
415
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
416 417
  mutex_.AssertHeld();

418
  // if deletion is disabled, do nothing
419
  if (disable_delete_obsolete_files_ > 0) {
420 421 422
    return;
  }

423 424 425 426 427
  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;
428
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
429 430 431 432
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
    if (delete_obsolete_files_last_run_ +
433
        db_options_.delete_obsolete_files_period_micros < now_micros) {
434 435 436 437 438
      doing_the_full_scan = true;
      delete_obsolete_files_last_run_ = now_micros;
    }
  }

I
Igor Canadi 已提交
439
  // get obsolete files
I
Igor Canadi 已提交
440
  versions_->GetObsoleteFiles(&job_context->sst_delete_files);
I
Igor Canadi 已提交
441

I
Igor Canadi 已提交
442
  // store the current filenum, lognum, etc
443
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
444
  job_context->pending_manifest_file_number =
445
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
446
  job_context->log_number = versions_->MinLogNumber();
447
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
448

I
Igor Canadi 已提交
449
  if (!doing_the_full_scan && !job_context->HaveSomethingToDelete()) {
450 451 452 453 454 455 456
    // avoid filling up sst_live if we're sure that we
    // are not going to do the full scan and that we don't have
    // anything to delete at the moment
    return;
  }

  // don't delete live files
I
Igor Canadi 已提交
457 458 459 460 461
  if (pending_outputs_.size()) {
    job_context->min_pending_output = *pending_outputs_.begin();
  } else {
    // delete all of them
    job_context->min_pending_output = std::numeric_limits<uint64_t>::max();
462
  }
I
Igor Canadi 已提交
463
  versions_->AddLiveFiles(&job_context->sst_live);
I
Igor Canadi 已提交
464

465
  if (doing_the_full_scan) {
466 467
    for (uint32_t path_id = 0;
         path_id < db_options_.db_paths.size(); path_id++) {
468 469 470
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
471
      env_->GetChildren(db_options_.db_paths[path_id].path,
472
                        &files);  // Ignore errors
473
      for (std::string file : files) {
I
Igor Canadi 已提交
474
        // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
I
Igor Canadi 已提交
475
        job_context->candidate_files.emplace_back("/" + file, path_id);
476 477
      }
    }
478 479

    //Add log files in wal_dir
480
    if (db_options_.wal_dir != dbname_) {
481
      std::vector<std::string> log_files;
482
      env_->GetChildren(db_options_.wal_dir, &log_files);  // Ignore errors
483
      for (std::string log_file : log_files) {
I
Igor Canadi 已提交
484
        job_context->candidate_files.emplace_back(log_file, 0);
485 486
      }
    }
487
    // Add info log files in db_log_dir
488
    if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) {
489
      std::vector<std::string> info_log_files;
490 491
      // Ignore errors
      env_->GetChildren(db_options_.db_log_dir, &info_log_files);
492
      for (std::string log_file : info_log_files) {
I
Igor Canadi 已提交
493
        job_context->candidate_files.emplace_back(log_file, 0);
494 495
      }
    }
496
  }
497 498
}

499
namespace {
I
Igor Canadi 已提交
500 501
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
502 503 504 505 506
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
507
    return (first.path_id > second.path_id);
508 509 510 511
  }
}
};  // namespace

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

I
Igor Canadi 已提交
520 521 522 523
  // 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 已提交
524 525
    return;
  }
526

527
  // Now, convert live list to an unordered map, WITHOUT mutex held;
528
  // set is slow.
529
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
I
Igor Canadi 已提交
530
  for (const FileDescriptor& fd : state.sst_live) {
531 532
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
533

I
Igor Canadi 已提交
534 535 536 537
  auto candidate_files = state.candidate_files;
  candidate_files.reserve(candidate_files.size() +
                          state.sst_delete_files.size() +
                          state.log_delete_files.size());
K
kailiu 已提交
538 539
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
540
  for (auto file : state.sst_delete_files) {
541 542 543
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
544
    delete file;
I
Igor Canadi 已提交
545 546
  }

K
kailiu 已提交
547 548
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
549 550
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
551 552
    }
  }
553

K
kailiu 已提交
554
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
555
  // file twice
556
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
557 558
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
559

560
  std::vector<std::string> old_info_log_files;
561
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
562 563 564
  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 已提交
565 566 567
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
568
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
569 570
      continue;
    }
J
jorlow@chromium.org 已提交
571

K
kailiu 已提交
572 573 574 575 576 577 578 579
    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'
580
        // (can happen during manifest roll)
K
kailiu 已提交
581 582 583
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
I
Igor Canadi 已提交
584 585 586 587
        // 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 已提交
588 589 590
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
591 592 593 594
        // 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
595
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
596
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
597 598 599 600
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
601
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
602
        }
K
kailiu 已提交
603 604 605 606 607 608 609 610 611 612 613 614 615
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

616
    std::string fname;
K
kailiu 已提交
617 618
    if (type == kTableFile) {
      // evict from cache
619
      TableCache::Evict(table_cache_.get(), number);
620
      fname = TableFileName(db_options_.db_paths, number, path_id);
621
    } else {
622 623
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
624
    }
625

I
Igor Canadi 已提交
626 627
#ifdef ROCKSDB_LITE
    Status s = env_->DeleteFile(fname);
628 629
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "Delete %s type=%d #%" PRIu64 " -- %s\n",
I
Igor Canadi 已提交
630 631 632 633 634
        fname.c_str(), type, number, s.ToString().c_str());
#else   // not ROCKSDB_LITE
    if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 ||
                             db_options_.WAL_size_limit_MB > 0)) {
      wal_manager_.ArchiveWALFile(fname, number);
K
kailiu 已提交
635 636
    } else {
      Status s = env_->DeleteFile(fname);
637 638
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "Delete %s type=%d #%" PRIu64 " -- %s\n",
639
          fname.c_str(), type, number, s.ToString().c_str());
J
jorlow@chromium.org 已提交
640
    }
I
Igor Canadi 已提交
641
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
642
  }
H
heyongqiang 已提交
643

644
  // Delete old info log files.
645
  size_t old_info_log_file_count = old_info_log_files.size();
646
  if (old_info_log_file_count >= db_options_.keep_log_file_num) {
647
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
648
    size_t end = old_info_log_file_count - db_options_.keep_log_file_num;
649
    for (unsigned int i = 0; i <= end; i++) {
650
      std::string& to_delete = old_info_log_files.at(i);
651 652
      std::string full_path_to_delete = (db_options_.db_log_dir.empty() ?
           dbname_ : db_options_.db_log_dir) + "/" + to_delete;
653 654
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
          "Delete info log file %s\n",
655 656
          full_path_to_delete.c_str());
      Status s = env_->DeleteFile(full_path_to_delete);
657
      if (!s.ok()) {
658 659
        Log(InfoLogLevel::ERROR_LEVEL,
            db_options_.info_log, "Delete info log file %s FAILED -- %s\n",
660 661
            to_delete.c_str(), s.ToString().c_str());
      }
H
heyongqiang 已提交
662 663
    }
  }
I
Igor Canadi 已提交
664 665 666
#ifndef ROCKSDB_LITE
  wal_manager_.PurgeObsoleteWALFiles();
#endif  // ROCKSDB_LITE
667
  LogFlush(db_options_.info_log);
D
Dhruba Borthakur 已提交
668 669 670 671
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
I
Igor Canadi 已提交
672 673 674 675
  JobContext job_context;
  FindObsoleteFiles(&job_context, true);
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
676
  }
I
Igor Canadi 已提交
677
  job_context.Clean();
678 679
}

680
Status DBImpl::Recover(
681 682
    const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
    bool error_if_log_file_exist) {
J
jorlow@chromium.org 已提交
683 684
  mutex_.AssertHeld();

685
  bool is_new_db = false;
686
  assert(db_lock_ == nullptr);
I
Igor Canadi 已提交
687
  if (!read_only) {
688 689 690 691 692 693 694 695 696 697 698 699
    // We call CreateDirIfMissing() as the directory may already exist (if we
    // are reopening a DB), when this happens we don't want creating the
    // directory to cause an error. However, we need to check if creating the
    // directory fails or else we may get an obscure message about the lock
    // file not existing. One real-world example of this occurring is if
    // env->CreateDirIfMissing() doesn't create intermediate directories, e.g.
    // when dbname_ is "dir/db" but when "dir" doesn't exist.
    Status s = env_->CreateDirIfMissing(dbname_);
    if (!s.ok()) {
      return s;
    }

700
    for (auto& db_path : db_options_.db_paths) {
701
      s = env_->CreateDirIfMissing(db_path.path);
702 703 704 705 706
      if (!s.ok()) {
        return s;
      }
    }

707 708 709 710 711
    s = env_->NewDirectory(dbname_, &db_directory_);
    if (!s.ok()) {
      return s;
    }

712
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
713 714 715
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
716

717
    if (!env_->FileExists(CurrentFileName(dbname_))) {
718
      if (db_options_.create_if_missing) {
719
        s = NewDB();
720
        is_new_db = true;
721 722 723 724 725 726
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
727 728
      }
    } else {
729
      if (db_options_.error_if_exists) {
730 731 732
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
733
    }
M
Mayank Agarwal 已提交
734 735 736 737 738 739 740
    // Check for the IDENTITY file and create it if not there
    if (!env_->FileExists(IdentityFileName(dbname_))) {
      s = SetIdentityFile(env_, dbname_);
      if (!s.ok()) {
        return s;
      }
    }
J
jorlow@chromium.org 已提交
741 742
  }

743
  Status s = versions_->Recover(column_families, read_only);
744
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
745 746
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
747 748
  if (s.ok()) {
    SequenceNumber max_sequence(0);
749 750
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
751
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
752 753
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
754 755 756 757 758

    // 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).
    //
759
    // Note that prev_log_number() is no longer used, but we pay
760
    // attention to it in case we are recovering a database
761
    // produced by an older version of rocksdb.
762
    const uint64_t min_log = versions_->MinLogNumber();
763
    const uint64_t prev_log = versions_->prev_log_number();
764
    std::vector<std::string> filenames;
765
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
766 767
    if (!s.ok()) {
      return s;
768
    }
K
kailiu 已提交
769

770 771
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
772 773
      uint64_t number;
      FileType type;
774 775 776 777 778 779 780 781 782
      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);
        }
783
      }
J
jorlow@chromium.org 已提交
784
    }
785

H
heyongqiang 已提交
786 787 788 789 790 791
    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 已提交
792 793 794 795 796 797 798
    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()) {
L
Lei Jin 已提交
799
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
S
Stanislau Hlebik 已提交
800 801
        }
      }
802
    }
L
Lei Jin 已提交
803
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
804 805
  }

L
Lei Jin 已提交
806 807
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
808
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
809 810 811
    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 已提交
812 813
  }

J
jorlow@chromium.org 已提交
814 815 816
  return s;
}

S
Stanislau Hlebik 已提交
817 818 819
// 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 已提交
820 821
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
822
    Logger* info_log;
J
jorlow@chromium.org 已提交
823
    const char* fname;
824 825
    Status* status;  // nullptr if db_options_.paranoid_checks==false or
                     //            db_options_.skip_log_error_on_recovery==true
J
jorlow@chromium.org 已提交
826
    virtual void Corruption(size_t bytes, const Status& s) {
827 828
      Log(InfoLogLevel::WARN_LEVEL,
          info_log, "%s%s: dropping %d bytes; %s",
829
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
830
          fname, static_cast<int>(bytes), s.ToString().c_str());
831
      if (this->status != nullptr && this->status->ok()) *this->status = s;
J
jorlow@chromium.org 已提交
832 833 834 835
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
836
  Status status;
837
  std::unordered_map<int, VersionEdit> version_edits;
838
  // no need to refcount because iteration is under mutex
839 840
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
841 842
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
843
  }
I
Igor Canadi 已提交
844

S
Stanislau Hlebik 已提交
845 846 847 848
  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.
849
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
850 851 852 853
    // Open the log file
    std::string fname = LogFileName(db_options_.wal_dir, log_number);
    unique_ptr<SequentialFile> file;
    status = env_->NewSequentialFile(fname, &file, env_options_);
J
jorlow@chromium.org 已提交
854
    if (!status.ok()) {
S
Stanislau Hlebik 已提交
855 856 857 858 859 860 861 862
      MaybeIgnoreError(&status);
      if (!status.ok()) {
        return status;
      } else {
        // Fail with one log file, but that's ok.
        // Try next one.
        continue;
      }
J
jorlow@chromium.org 已提交
863 864
    }

S
Stanislau Hlebik 已提交
865 866 867 868 869 870 871 872 873 874 875 876 877 878 879
    // Create the log reader.
    LogReporter reporter;
    reporter.env = env_;
    reporter.info_log = db_options_.info_log.get();
    reporter.fname = fname.c_str();
    reporter.status =
        (db_options_.paranoid_checks && !db_options_.skip_log_error_on_recovery
             ? &status
             : nullptr);
    // We intentially make log::Reader do checksumming even if
    // paranoid_checks==false so that corruptions cause entire commits
    // to be skipped instead of propagating bad information (like overly
    // large sequence numbers).
    log::Reader reader(std::move(file), &reporter, true /*checksum*/,
                       0 /*initial_offset*/);
880 881
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "Recovering log #%" PRIu64 "", log_number);
S
Stanislau Hlebik 已提交
882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913

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

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

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

      if (!read_only) {
I
Igor Canadi 已提交
914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930
        // we can do this because this is called before client has access to the
        // DB and there is only a single thread operating on DB
        ColumnFamilyData* cfd;

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

I
Igor Canadi 已提交
937
    flush_scheduler_.Clear();
S
Stanislau Hlebik 已提交
938 939 940
    if (versions_->LastSequence() < *max_sequence) {
      versions_->SetLastSequence(*max_sequence);
    }
941 942
  }

943
  if (!read_only) {
944 945
    // 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 已提交
946
    auto max_log_number = log_numbers.back();
947
    for (auto cfd : *versions_->GetColumnFamilySet()) {
948
      auto iter = version_edits.find(cfd->GetID());
949 950 951
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
952
      if (cfd->GetLogNumber() > max_log_number) {
953
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
954
        // from all logs. Memtable has to be empty because
955
        // we filter the updates based on log_number
956
        // (in WriteBatch::InsertInto)
957 958 959 960 961 962 963
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
964
        status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
965 966 967 968
        if (!status.ok()) {
          // Recovery failed
          break;
        }
L
Lei Jin 已提交
969
        cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
970
      }
J
jorlow@chromium.org 已提交
971

972
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
973
      // writing log_number in the manifest means that any log file
974 975
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
976 977 978
      // 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);
979 980 981
      // 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 已提交
982
      // log number
983
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
984 985
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
986
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
987 988
        // Recovery failed
        break;
989 990
      }
    }
991
  }
I
Igor Canadi 已提交
992

J
jorlow@chromium.org 已提交
993 994 995
  return status;
}

996 997
Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
                                           VersionEdit* edit) {
J
jorlow@chromium.org 已提交
998
  mutex_.AssertHeld();
999
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1000
  FileMetaData meta;
1001
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1002 1003
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1004 1005
  ReadOptions ro;
  ro.total_order_seek = true;
1006
  Arena arena;
1007 1008
  Status s;
  {
1009 1010 1011 1012
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
    const SequenceNumber newest_snapshot = snapshots_.GetNewest();
    const SequenceNumber earliest_seqno_in_memtable =
        mem->GetFirstSequenceNumber();
1013 1014 1015
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1016
        cfd->GetName().c_str(), meta.fd.GetNumber());
1017

1018 1019 1020 1021 1022
    {
      mutex_.Unlock();
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
          iter.get(), &meta, cfd->internal_comparator(), newest_snapshot,
1023 1024
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
          cfd->ioptions()->compression_opts, Env::IO_HIGH);
1025
      LogFlush(db_options_.info_log);
1026 1027
      mutex_.Lock();
    }
1028

1029 1030 1031
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
1032 1033 1034
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
        s.ToString().c_str());
  }
I
Igor Canadi 已提交
1035
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1036 1037 1038 1039

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1040
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1041 1042 1043
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1044 1045
  }

L
Lei Jin 已提交
1046
  InternalStats::CompactionStats stats(1);
1047
  stats.micros = env_->NowMicros() - start_micros;
1048
  stats.bytes_written = meta.fd.GetFileSize();
M
Mark Callaghan 已提交
1049
  stats.files_out_levelnp1 = 1;
1050
  cfd->internal_stats()->AddCompactionStats(level, stats);
1051 1052
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1053
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1054 1055 1056
  return s;
}

1057 1058
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
I
Igor Canadi 已提交
1059
    bool* madeProgress, JobContext* job_context, LogBuffer* log_buffer) {
1060
  mutex_.AssertHeld();
1061
  assert(cfd->imm()->size() != 0);
1062
  assert(cfd->imm()->IsFlushPending());
1063

I
Igor Canadi 已提交
1064 1065
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1066 1067 1068
                     snapshots_.GetNewest(), job_context, log_buffer,
                     db_directory_.get(), GetCompressionFlush(*cfd->ioptions()),
                     stats_);
1069

1070 1071
  uint64_t file_number;
  Status s = flush_job.Run(&file_number);
J
jorlow@chromium.org 已提交
1072 1073

  if (s.ok()) {
I
Igor Canadi 已提交
1074
    InstallSuperVersionBackground(cfd, job_context, mutable_cf_options);
1075 1076 1077
    if (madeProgress) {
      *madeProgress = 1;
    }
S
sdong 已提交
1078
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1079
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1080
                cfd->current()->storage_info()->LevelSummary(&tmp));
1081

1082
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1083
      // add to deletion state
1084
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1085 1086
             alive_log_files_.begin()->number < versions_->MinLogNumber()) {
        const auto& earliest = *alive_log_files_.begin();
I
Igor Canadi 已提交
1087
        job_context->log_delete_files.push_back(earliest.number);
I
Igor Canadi 已提交
1088
        total_log_size_ -= earliest.size;
1089 1090
        alive_log_files_.pop_front();
      }
1091
    }
J
jorlow@chromium.org 已提交
1092
  }
1093

1094
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1095 1096 1097 1098 1099
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1100
  RecordFlushIOStats();
1101 1102 1103
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1104
    NotifyOnFlushCompleted(cfd, file_number, mutable_cf_options);
1105 1106
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1107 1108 1109
  return s;
}

1110
void DBImpl::NotifyOnFlushCompleted(
1111 1112
    ColumnFamilyData* cfd, uint64_t file_number,
    const MutableCFOptions& mutable_cf_options) {
1113 1114 1115 1116 1117 1118
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  bool triggered_flush_slowdown =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1119
       mutable_cf_options.level0_slowdown_writes_trigger);
1120 1121
  bool triggered_flush_stop =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1122
       mutable_cf_options.level0_stop_writes_trigger);
1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137
  notifying_events_++;
  // release lock while notifying events
  mutex_.Unlock();
  // TODO(yhchiang): make db_paths dynamic.
  cfd->NotifyOnFlushCompleted(
        this, MakeTableFileName(db_options_.db_paths[0].path, file_number),
        triggered_flush_slowdown,
        triggered_flush_stop);
  mutex_.Lock();
  notifying_events_--;
  assert(notifying_events_ >= 0);
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
}

1138
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1139
                            const Slice* begin, const Slice* end,
1140 1141
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
1142
  if (target_path_id >= db_options_.db_paths.size()) {
1143 1144 1145
    return Status::InvalidArgument("Invalid target path ID");
  }

1146 1147
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1148 1149

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1150
  if (!s.ok()) {
1151
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1152 1153 1154
    return s;
  }

I
Igor Canadi 已提交
1155
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1156 1157
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1158 1159
    Version* base = cfd->current();
    for (int level = 1; level < cfd->NumberLevels(); level++) {
S
sdong 已提交
1160
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1161 1162 1163 1164
        max_level_with_files = level;
      }
    }
  }
1165 1166
  for (int level = 0; level <= max_level_with_files; level++) {
    // in case the compaction is unversal or if we're compacting the
1167 1168 1169
    // 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)
1170 1171
    if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
        cfd->ioptions()->compaction_style == kCompactionStyleFIFO ||
1172
        (level == max_level_with_files && level > 0)) {
1173
      s = RunManualCompaction(cfd, level, level, target_path_id, begin, end);
1174
    } else {
1175 1176
      s = RunManualCompaction(cfd, level, level + 1, target_path_id, begin,
                              end);
L
Lei Jin 已提交
1177 1178
    }
    if (!s.ok()) {
1179
      LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1180
      return s;
1181
    }
G
Gabor Cselle 已提交
1182
  }
1183 1184

  if (reduce_level) {
I
Igor Canadi 已提交
1185
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1186
  }
1187
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1188

1189 1190 1191 1192 1193 1194 1195
  {
    MutexLock l(&mutex_);
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1196
  return s;
1197 1198
}

1199 1200 1201 1202 1203
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 已提交
1204 1205 1206 1207
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226
  MutexLock l(&mutex_);
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);
  // TODO(yhchiang): use superversion
  cfd->Ref();
  auto version = cfd->current();
  version->Ref();
  auto s = CompactFilesImpl(compact_options, cfd, version,
                            input_file_names, output_level, output_path_id);
  // TODO(yhchiang): unref could move into CompactFilesImpl().  Otherwise,
  // FindObsoleteFiles might never able to find any file to delete.
  version->Unref();
  // TODO(yhchiang): cfd should be deleted after its last reference.
  cfd->Unref();
  return s;
I
Igor Canadi 已提交
1227
#endif  // ROCKSDB_LITE
1228 1229
}

I
Igor Canadi 已提交
1230
#ifndef ROCKSDB_LITE
1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
    const int output_level, int output_path_id) {
  mutex_.AssertHeld();

  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());

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

  autovector<CompactionInputFiles> input_files;
  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(
        compact_options, input_files,
        output_level, version->storage_info(),
        *cfd->GetLatestMutableCFOptions()));
  assert(c);
  c->SetInputVersion(version);
  c->SetOutputPathId(static_cast<uint32_t>(output_path_id));
  // deletion compaction currently not allowed in CompactFiles.
  assert(!c->IsDeletionCompaction());

  JobContext job_context(true);
  auto yield_callback = [&]() {
    return CallFlushDuringCompaction(c->column_family_data(),
                                     *c->mutable_cf_options(), &job_context,
                                     &log_buffer);
  };
1305 1306 1307 1308 1309
  CompactionJob compaction_job(c.get(), db_options_, *c->mutable_cf_options(),
                               env_options_, versions_.get(), &shutting_down_,
                               &log_buffer, db_directory_.get(), stats_,
                               &snapshots_, IsSnapshotSupported(), table_cache_,
                               std::move(yield_callback));
1310 1311 1312 1313 1314
  compaction_job.Prepare();

  mutex_.Unlock();
  Status status = compaction_job.Run();
  mutex_.Lock();
1315
  compaction_job.Install(&status, &mutex_);
1316
  if (status.ok()) {
1317 1318
    InstallSuperVersionBackground(c->column_family_data(), &job_context,
                                  *c->mutable_cf_options());
1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355
  }
  c->ReleaseCompactionFiles(s);
  c->ReleaseInputs();
  c.reset();

  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
        status.ToString().c_str());
    if (db_options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }

  // 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.
  // TODO(yhchiang): write an unit-test to make sure files are actually
  //                 deleted after CompactFiles.
  FindObsoleteFiles(&job_context, !s.ok());

  // delete unnecessary files if any, this is done outside the mutex
  if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
    mutex_.Unlock();
    // 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()) {
      PurgeObsoleteFiles(job_context);
    }
I
Igor Canadi 已提交
1356
    job_context.Clean();
1357 1358 1359 1360 1361 1362 1363
    mutex_.Lock();
  }

  bg_compaction_scheduled_--;

  return status;
}
I
Igor Canadi 已提交
1364
#endif  // ROCKSDB_LITE
1365

1366
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1367
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1368 1369 1370
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1371 1372
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1373 1374
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1375
        cfd->GetName().c_str());
1376
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1377 1378 1379
  }

  MutableCFOptions new_options;
1380
  Status s;
L
Lei Jin 已提交
1381 1382
  {
    MutexLock l(&mutex_);
1383 1384
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1385 1386 1387 1388
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

1389 1390
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1391 1392
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1393 1394
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1395
  }
1396
  if (s.ok()) {
1397 1398
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1399 1400 1401
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
  } else {
1402 1403
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1404
  }
1405
  return s;
I
Igor Canadi 已提交
1406
#endif  // ROCKSDB_LITE
1407 1408
}

1409
// return the same level if it cannot be moved
1410 1411
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
1412
  mutex_.AssertHeld();
S
sdong 已提交
1413
  const auto* vstorage = cfd->current()->storage_info();
1414
  int minimum_level = level;
1415
  for (int i = level - 1; i > 0; --i) {
1416
    // stop if level i is not empty
S
sdong 已提交
1417
    if (vstorage->NumLevelFiles(i) > 0) break;
1418
    // stop if level i is too small (cannot fit the level files)
1419
    if (mutable_cf_options.MaxBytesForLevel(i) <
S
sdong 已提交
1420
        vstorage->NumLevelBytes(level)) {
1421 1422
      break;
    }
1423 1424 1425 1426 1427 1428

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1432
  SuperVersion* superversion_to_free = nullptr;
1433
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1434 1435

  mutex_.Lock();
1436 1437 1438

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1439
    mutex_.Unlock();
1440 1441
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
I
Igor Canadi 已提交
1442
    delete new_superversion;
L
Lei Jin 已提交
1443
    return Status::NotSupported("another thread is refitting");
1444 1445 1446 1447 1448
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1449
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1450 1451
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop: %d %d",
1452
        bg_compaction_scheduled_, bg_flush_scheduled_);
1453 1454 1455
    bg_cv_.Wait();
  }

1456 1457
  const MutableCFOptions mutable_cf_options =
    *cfd->GetLatestMutableCFOptions();
1458
  // move to a smaller level
1459 1460
  int to_level = target_level;
  if (target_level < 0) {
1461
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1462
  }
1463 1464 1465

  assert(to_level <= level);

L
Lei Jin 已提交
1466
  Status status;
1467
  if (to_level < level) {
1468 1469
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Before refitting:\n%s",
1470
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1471

1472
    VersionEdit edit;
I
Igor Canadi 已提交
1473
    edit.SetColumnFamily(cfd->GetID());
S
sdong 已提交
1474
    for (const auto& f : cfd->current()->storage_info()->LevelFiles(level)) {
1475
      edit.DeleteFile(level, f->fd.GetNumber());
1476 1477 1478
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1479
    }
1480 1481
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1482
        cfd->GetName().c_str(), edit.DebugString().data());
1483

1484 1485
    status = versions_->LogAndApply(cfd,
        mutable_cf_options, &edit, &mutex_, db_directory_.get());
L
Lei Jin 已提交
1486 1487
    superversion_to_free = InstallSuperVersion(
        cfd, new_superversion, mutable_cf_options);
I
Igor Canadi 已提交
1488
    new_superversion = nullptr;
1489

1490 1491
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1492
        status.ToString().data());
1493 1494

    if (status.ok()) {
1495 1496
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1497
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1498 1499 1500 1501 1502
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1503 1504 1505 1506

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1507
  return status;
G
Gabor Cselle 已提交
1508 1509
}

1510 1511 1512
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1513 1514
}

1515 1516
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1517 1518 1519
  MutexLock l(&mutex_);
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.max_mem_compaction_level;
1520 1521
}

1522 1523
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1524 1525 1526
  MutexLock l(&mutex_);
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
1527 1528
}

L
Lei Jin 已提交
1529
Status DBImpl::Flush(const FlushOptions& flush_options,
1530 1531
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1532
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
1533 1534
}

1535
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1536 1537 1538
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1539
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1540 1541
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1542
  assert(input_level >= 0);
1543

G
Gabor Cselle 已提交
1544 1545
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1546
  ManualCompaction manual;
I
Igor Canadi 已提交
1547
  manual.cfd = cfd;
1548 1549
  manual.input_level = input_level;
  manual.output_level = output_level;
1550
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
1551
  manual.done = false;
1552
  manual.in_progress = false;
1553 1554 1555
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
1556 1557
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1558
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1559 1560 1561 1562
  } else {
    begin_storage = InternalKey(*begin, kMaxSequenceNumber, kValueTypeForSeek);
    manual.begin = &begin_storage;
  }
1563
  if (end == nullptr ||
1564 1565
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1566
    manual.end = nullptr;
G
Gabor Cselle 已提交
1567 1568 1569 1570 1571 1572
  } else {
    end_storage = InternalKey(*end, 0, static_cast<ValueType>(0));
    manual.end = &end_storage;
  }

  MutexLock l(&mutex_);
1573

1574 1575 1576 1577 1578 1579
  // 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
1580
  // RunManualCompaction(), i.e. during that time no other compaction will
1581 1582 1583
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1584
  // RunManualCompaction() from getting to the second while loop below.
1585 1586 1587 1588 1589
  // 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) {
1590
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
1591 1592 1593
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1594 1595
    bg_cv_.Wait();
  }
1596

1597 1598
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
1599
      cfd->GetName().c_str());
1600

1601 1602 1603 1604
  // 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) {
1605 1606 1607
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
1608
      bg_cv_.Wait();
1609 1610
    } else {
      manual_compaction_ = &manual;
1611 1612
      bg_compaction_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
G
Gabor Cselle 已提交
1613
    }
H
hans@chromium.org 已提交
1614
  }
1615

1616 1617 1618
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1619
  return manual.status;
J
jorlow@chromium.org 已提交
1620 1621
}

1622
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
1623
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
1624 1625 1626 1627
  Status s;
  {
    WriteContext context;
    MutexLock guard_lock(&mutex_);
1628 1629 1630 1631 1632 1633

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

I
Igor Canadi 已提交
1634 1635
    WriteThread::Writer w(&mutex_);
    s = write_thread_.EnterWriteThread(&w, 0);
S
Stanislau Hlebik 已提交
1636 1637 1638 1639 1640 1641 1642 1643
    assert(s.ok() && !w.done);  // No timeout and nobody should do our job

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

I
Igor Canadi 已提交
1644
    write_thread_.ExitWriteThread(&w, &w, s);
S
Stanislau Hlebik 已提交
1645
  }
S
Stanislau Hlebik 已提交
1646

L
Lei Jin 已提交
1647
  if (s.ok() && flush_options.wait) {
1648
    // Wait until the compaction completes
1649
    s = WaitForFlushMemTable(cfd);
1650 1651
  }
  return s;
J
jorlow@chromium.org 已提交
1652 1653
}

1654
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1655 1656 1657
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1658
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1659 1660
    bg_cv_.Wait();
  }
1661
  if (!bg_error_.ok()) {
1662 1663 1664
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1665 1666
}

1667
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1668
  mutex_.AssertHeld();
1669
  bg_schedule_needed_ = false;
1670 1671
  if (bg_work_gate_closed_) {
    // gate closed for backgrond work
I
Igor Canadi 已提交
1672
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
1673 1674
    // DB is being deleted; no more background compactions
  } else {
1675
    bool is_flush_pending = false;
1676
    // no need to refcount since we're under a mutex
1677
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
1678
      if (cfd->imm()->IsFlushPending()) {
1679 1680 1681
        is_flush_pending = true;
      }
    }
1682
    if (is_flush_pending) {
1683
      // memtable flush needed
1684
      if (bg_flush_scheduled_ < db_options_.max_background_flushes) {
1685 1686
        bg_flush_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH);
1687
      } else if (db_options_.max_background_flushes > 0) {
1688
        bg_schedule_needed_ = true;
1689
      }
1690
    }
1691
    bool is_compaction_needed = false;
1692
    // no need to refcount since we're under a mutex
1693
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1694
      if (cfd->compaction_picker()->NeedsCompaction(
Y
Yueh-Hsuan Chiang 已提交
1695
              cfd->current()->storage_info())) {
1696 1697 1698 1699
        is_compaction_needed = true;
        break;
      }
    }
1700

1701 1702
    // Schedule BGWorkCompaction if there's a compaction pending (or a memtable
    // flush, but the HIGH pool is not enabled)
1703 1704 1705 1706
    // Do it only if max_background_compactions hasn't been reached and
    // bg_manual_only_ == 0
    if (!bg_manual_only_ &&
        (is_compaction_needed ||
1707 1708
         (is_flush_pending && db_options_.max_background_flushes == 0))) {
      if (bg_compaction_scheduled_ < db_options_.max_background_compactions) {
1709 1710 1711 1712 1713
        bg_compaction_scheduled_++;
        env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
      } else {
        bg_schedule_needed_ = true;
      }
1714 1715 1716 1717
    }
  }
}

1718
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
1719
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
1720 1721 1722
  IOSTATS_RESET(bytes_written);
}

1723
void DBImpl::BGWorkFlush(void* db) {
1724
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
1725 1726 1727 1728
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
1729
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
1730 1731 1732
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
1733
Status DBImpl::BackgroundFlush(bool* madeProgress, JobContext* job_context,
H
Haobo Xu 已提交
1734
                               LogBuffer* log_buffer) {
1735
  mutex_.AssertHeld();
1736 1737 1738 1739 1740

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

1741 1742 1743 1744 1745
  // call_status is failure if at least one flush was a failure. even if
  // flushing one column family reports a failure, we will continue flushing
  // other column families. however, call_status will be a failure in that case.
  Status call_status;
  // refcounting in iteration
1746
  for (auto cfd : *versions_->GetColumnFamilySet()) {
1747 1748
    cfd->Ref();
    Status flush_status;
1749 1750
    const MutableCFOptions mutable_cf_options =
      *cfd->GetLatestMutableCFOptions();
1751
    while (flush_status.ok() && cfd->imm()->IsFlushPending()) {
1752 1753
      LogToBuffer(
          log_buffer,
1754
          "BackgroundCallFlush doing FlushMemTableToOutputFile with column "
I
Igor Canadi 已提交
1755 1756
          "family [%s], flush slots available %d",
          cfd->GetName().c_str(),
1757
          db_options_.max_background_flushes - bg_flush_scheduled_);
1758
      flush_status = FlushMemTableToOutputFile(
I
Igor Canadi 已提交
1759
          cfd, mutable_cf_options, madeProgress, job_context, log_buffer);
1760 1761 1762
    }
    if (call_status.ok() && !flush_status.ok()) {
      call_status = flush_status;
1763
    }
1764
    cfd->Unref();
J
jorlow@chromium.org 已提交
1765
  }
1766
  versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
1767
  return call_status;
J
jorlow@chromium.org 已提交
1768 1769
}

1770
void DBImpl::BackgroundCallFlush() {
1771
  bool madeProgress = false;
I
Igor Canadi 已提交
1772
  JobContext job_context(true);
1773 1774
  assert(bg_flush_scheduled_);

1775
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
1776 1777 1778
  {
    MutexLock l(&mutex_);

I
Igor Canadi 已提交
1779 1780 1781
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

H
Haobo Xu 已提交
1782
    Status s;
I
Igor Canadi 已提交
1783
    if (!shutting_down_.load(std::memory_order_acquire)) {
I
Igor Canadi 已提交
1784
      s = BackgroundFlush(&madeProgress, &job_context, &log_buffer);
H
Haobo Xu 已提交
1785 1786 1787 1788 1789
      if (!s.ok()) {
        // Wait a little bit before retrying background compaction in
        // case this is an environmental problem and we do not want to
        // chew up resources for failed compactions for the duration of
        // the problem.
1790 1791
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
H
Haobo Xu 已提交
1792 1793
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
1794
        Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
1795 1796 1797
            "Waiting after background flush error: %s"
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
H
Haobo Xu 已提交
1798
        log_buffer.FlushBufferToLog();
1799
        LogFlush(db_options_.info_log);
H
Haobo Xu 已提交
1800 1801 1802 1803 1804
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }

I
Igor Canadi 已提交
1805 1806
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

H
Haobo Xu 已提交
1807 1808
    // If !s.ok(), this means that Flush failed. In that case, we want
    // to delete all obsolete files and we force FindObsoleteFiles()
I
Igor Canadi 已提交
1809
    FindObsoleteFiles(&job_context, !s.ok());
H
Haobo Xu 已提交
1810
    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
1811
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
1812
      mutex_.Unlock();
1813 1814 1815 1816 1817
      // 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 已提交
1818
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
1819 1820
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
1821
      }
I
Igor Canadi 已提交
1822
      job_context.Clean();
1823 1824
      mutex_.Lock();
    }
I
Igor Canadi 已提交
1825

H
Haobo Xu 已提交
1826
    bg_flush_scheduled_--;
1827 1828 1829 1830 1831
    // Any time the mutex is released After finding the work to do, another
    // thread might execute MaybeScheduleFlushOrCompaction(). It is possible
    // that there is a pending job but it is not scheduled because of the
    // max thread limit.
    if (madeProgress || bg_schedule_needed_) {
H
Haobo Xu 已提交
1832 1833
      MaybeScheduleFlushOrCompaction();
    }
I
Igor Canadi 已提交
1834
    RecordFlushIOStats();
H
Haobo Xu 已提交
1835
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
1836 1837 1838 1839
    // 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.
1840
  }
J
jorlow@chromium.org 已提交
1841 1842
}

1843
void DBImpl::BackgroundCallCompaction() {
1844
  bool madeProgress = false;
I
Igor Canadi 已提交
1845
  JobContext job_context(true);
H
Haobo Xu 已提交
1846 1847

  MaybeDumpStats();
1848
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
1849 1850
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1851 1852 1853 1854

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

1855 1856
    assert(bg_compaction_scheduled_);
    Status s;
I
Igor Canadi 已提交
1857
    if (!shutting_down_.load(std::memory_order_acquire)) {
I
Igor Canadi 已提交
1858
      s = BackgroundCompaction(&madeProgress, &job_context, &log_buffer);
1859 1860 1861 1862 1863
      if (!s.ok()) {
        // Wait a little bit before retrying background compaction in
        // case this is an environmental problem and we do not want to
        // chew up resources for failed compactions for the duration of
        // the problem.
1864 1865
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
1866 1867
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
H
Haobo Xu 已提交
1868
        log_buffer.FlushBufferToLog();
1869
        Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
1870 1871 1872
            "Waiting after background compaction error: %s, "
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
1873
        LogFlush(db_options_.info_log);
1874 1875 1876 1877
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }
H
Haobo Xu 已提交
1878

I
Igor Canadi 已提交
1879 1880
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

1881 1882
    // 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
I
Igor Canadi 已提交
1883 1884 1885
    // FindObsoleteFiles(). This is because job_context does not
    // catch all created files if compaction failed.
    FindObsoleteFiles(&job_context, !s.ok());
1886 1887

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
1888
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
1889
      mutex_.Unlock();
1890 1891 1892 1893 1894
      // 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 已提交
1895
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
1896 1897
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
1898
      }
I
Igor Canadi 已提交
1899
      job_context.Clean();
1900 1901
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
1902

1903
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
1904

1905 1906
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

1907 1908 1909
    // Previous compaction may have produced too many files in a level,
    // So reschedule another compaction if we made progress in the
    // last compaction.
1910 1911 1912 1913 1914 1915
    //
    // Also, any time the mutex is released After finding the work to do,
    // another thread might execute MaybeScheduleFlushOrCompaction(). It is
    // possible  that there is a pending job but it is not scheduled because of
    // the max thread limit.
    if (madeProgress || bg_schedule_needed_) {
1916 1917
      MaybeScheduleFlushOrCompaction();
    }
1918 1919
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
I
Igor Canadi 已提交
1920
      // * madeProgress -- need to wakeup DelayWrite
1921 1922 1923 1924 1925 1926
      // * 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 已提交
1927 1928 1929 1930
    // 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.
1931
  }
J
jorlow@chromium.org 已提交
1932 1933
}

I
Igor Canadi 已提交
1934
Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
1935
                                    LogBuffer* log_buffer) {
1936
  *madeProgress = false;
J
jorlow@chromium.org 已提交
1937
  mutex_.AssertHeld();
1938

1939 1940
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
1941

1942 1943 1944 1945 1946 1947 1948 1949 1950 1951
  if (!bg_error_.ok()) {
    if (is_manual) {
      manual_compaction_->status = bg_error_;
      manual_compaction_->done = true;
      manual_compaction_->in_progress = false;
      manual_compaction_ = nullptr;
    }
    return bg_error_;
  }

1952 1953 1954
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
1955 1956 1957 1958
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
1959 1960 1961 1962 1963
  }

  // FLUSH preempts compaction
  Status flush_stat;
  for (auto cfd : *versions_->GetColumnFamilySet()) {
1964 1965
    const MutableCFOptions mutable_cf_options =
      *cfd->GetLatestMutableCFOptions();
1966 1967 1968 1969 1970
    while (cfd->imm()->IsFlushPending()) {
      LogToBuffer(
          log_buffer,
          "BackgroundCompaction doing FlushMemTableToOutputFile, "
          "compaction slots available %d",
1971
          db_options_.max_background_compactions - bg_compaction_scheduled_);
1972
      cfd->Ref();
1973
      flush_stat = FlushMemTableToOutputFile(
I
Igor Canadi 已提交
1974
          cfd, mutable_cf_options, madeProgress, job_context, log_buffer);
1975
      cfd->Unref();
1976 1977 1978 1979 1980 1981 1982
      if (!flush_stat.ok()) {
        if (is_manual) {
          manual_compaction_->status = flush_stat;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
1983
        return flush_stat;
1984 1985 1986 1987
      }
    }
  }

1988 1989 1990
  // 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
1991
  unique_ptr<Compaction> c;
1992 1993
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
1994
  if (is_manual) {
G
Gabor Cselle 已提交
1995
    ManualCompaction* m = manual_compaction_;
1996
    assert(m->in_progress);
1997 1998 1999
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
2000
    if (!c) {
2001
      m->done = true;
G
Gabor Cselle 已提交
2002
    }
I
Igor Canadi 已提交
2003 2004 2005 2006 2007 2008 2009 2010 2011
    LogToBuffer(log_buffer,
                "[%s] Manual compaction from level-%d to level-%d from %s .. "
                "%s; will stop at %s\n",
                m->cfd->GetName().c_str(), m->input_level, m->output_level,
                (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                (m->end ? m->end->DebugString().c_str() : "(end)"),
                ((m->done || manual_end == nullptr)
                     ? "(end)"
                     : manual_end->DebugString().c_str()));
I
Igor Canadi 已提交
2012
  } else {
2013
    // no need to refcount in iteration since it's always under a mutex
I
Igor Canadi 已提交
2014
    for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
2015 2016 2017 2018
      // Pick up latest mutable CF Options and use it throughout the
      // compaction job
      auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
      if (!mutable_cf_options->disable_auto_compactions) {
2019 2020 2021
        // 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
L
Lei Jin 已提交
2022
        c.reset(cfd->PickCompaction(*mutable_cf_options, log_buffer));
I
Igor Canadi 已提交
2023 2024
        if (c != nullptr) {
          // update statistics
L
Lei Jin 已提交
2025
          MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
I
Igor Canadi 已提交
2026 2027 2028
                      c->inputs(0)->size());
          break;
        }
I
Igor Canadi 已提交
2029 2030
      }
    }
J
jorlow@chromium.org 已提交
2031 2032 2033
  }

  Status status;
2034
  if (!c) {
H
hans@chromium.org 已提交
2035
    // Nothing to do
2036
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2037 2038 2039 2040 2041
  } else if (c->IsDeletionCompaction()) {
    // TODO(icanadi) Do we want to honor snapshots here? i.e. not delete old
    // file if there is alive snapshot pointing to it
    assert(c->num_input_files(1) == 0);
    assert(c->level() == 0);
2042
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2043 2044
           kCompactionStyleFIFO);
    for (const auto& f : *c->inputs(0)) {
2045
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2046
    }
2047 2048 2049
    status = versions_->LogAndApply(
        c->column_family_data(), *c->mutable_cf_options(), c->edit(),
        &mutex_, db_directory_.get());
I
Igor Canadi 已提交
2050 2051
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
I
Igor Canadi 已提交
2052 2053 2054 2055 2056
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
    c->ReleaseCompactionFiles(status);
    *madeProgress = true;
H
hans@chromium.org 已提交
2057
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
2058
    // Move file to next level
2059
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2060
    FileMetaData* f = c->input(0, 0);
2061
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
2062 2063 2064
    c->edit()->AddFile(c->level() + 1, f->fd.GetNumber(), f->fd.GetPathId(),
                       f->fd.GetFileSize(), f->smallest, f->largest,
                       f->smallest_seqno, f->largest_seqno);
2065 2066 2067
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(),
                                    c->edit(), &mutex_, db_directory_.get());
2068
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2069 2070
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
2071

S
sdong 已提交
2072
    VersionStorageInfo::LevelSummaryStorage tmp;
2073 2074
    c->column_family_data()->internal_stats()->IncBytesMoved(
        c->level() + 1, f->fd.GetFileSize());
S
sdong 已提交
2075 2076 2077 2078
    LogToBuffer(log_buffer, "[%s] Moved #%" PRIu64 " to level-%d %" PRIu64
                            " bytes %s: %s\n",
                c->column_family_data()->GetName().c_str(), f->fd.GetNumber(),
                c->level() + 1, f->fd.GetFileSize(), status.ToString().c_str(),
S
sdong 已提交
2079
                c->input_version()->storage_info()->LevelSummary(&tmp));
I
Igor Canadi 已提交
2080
    c->ReleaseCompactionFiles(status);
2081
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2082
  } else {
2083
    MaybeScheduleFlushOrCompaction(); // do more compaction work in parallel.
I
Igor Canadi 已提交
2084 2085 2086 2087 2088 2089

    auto yield_callback = [&]() {
      return CallFlushDuringCompaction(c->column_family_data(),
                                       *c->mutable_cf_options(), job_context,
                                       log_buffer);
    };
2090 2091 2092 2093 2094
    CompactionJob compaction_job(c.get(), db_options_, *c->mutable_cf_options(),
                                 env_options_, versions_.get(), &shutting_down_,
                                 log_buffer, db_directory_.get(), stats_,
                                 &snapshots_, IsSnapshotSupported(),
                                 table_cache_, std::move(yield_callback));
I
Igor Canadi 已提交
2095 2096 2097 2098
    compaction_job.Prepare();
    mutex_.Unlock();
    status = compaction_job.Run();
    mutex_.Lock();
2099
    compaction_job.Install(&status, &mutex_);
I
Igor Canadi 已提交
2100 2101 2102 2103
    if (status.ok()) {
      InstallSuperVersionBackground(c->column_family_data(), job_context,
                                    *c->mutable_cf_options());
    }
I
Igor Canadi 已提交
2104
    c->ReleaseCompactionFiles(status);
2105
    c->ReleaseInputs();
2106
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2107
  }
2108
  c.reset();
J
jorlow@chromium.org 已提交
2109 2110 2111

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2112
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2113 2114
    // Ignore compaction errors found during shutting down
  } else {
2115
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2116
        status.ToString().c_str());
2117
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2118 2119 2120
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2121 2122

  if (is_manual) {
G
Gabor Cselle 已提交
2123
    ManualCompaction* m = manual_compaction_;
2124
    if (!status.ok()) {
L
Lei Jin 已提交
2125
      m->status = status;
2126 2127
      m->done = true;
    }
2128 2129 2130 2131 2132 2133 2134 2135 2136
    // 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.
2137 2138 2139 2140 2141
    //
    // 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) {
2142 2143
      m->done = true;
    }
G
Gabor Cselle 已提交
2144 2145 2146
    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 已提交
2147
      // Universal and FIFO compactions should always compact the whole range
2148 2149
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleUniversal);
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2150
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2151 2152
      m->begin = &m->tmp_storage;
    }
2153
    m->in_progress = false; // not being processed anymore
2154
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2155
  }
2156
  return status;
J
jorlow@chromium.org 已提交
2157 2158
}

I
Igor Canadi 已提交
2159 2160 2161
uint64_t DBImpl::CallFlushDuringCompaction(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
    JobContext* job_context, LogBuffer* log_buffer) {
2162
  if (db_options_.max_background_flushes > 0) {
2163 2164 2165
    // flush thread will take care of this
    return 0;
  }
I
Igor Canadi 已提交
2166
  if (cfd->imm()->imm_flush_needed.load(std::memory_order_relaxed)) {
I
Igor Canadi 已提交
2167 2168 2169 2170
    const uint64_t imm_start = env_->NowMicros();
    mutex_.Lock();
    if (cfd->imm()->IsFlushPending()) {
      cfd->Ref();
I
Igor Canadi 已提交
2171 2172
      FlushMemTableToOutputFile(cfd, mutable_cf_options, nullptr, job_context,
                                log_buffer);
I
Igor Canadi 已提交
2173
      cfd->Unref();
I
Igor Canadi 已提交
2174
      bg_cv_.SignalAll();  // Wakeup DelayWrite() if necessary
I
Igor Canadi 已提交
2175 2176 2177 2178 2179 2180 2181 2182
    }
    mutex_.Unlock();
    log_buffer->FlushBufferToLog();
    return env_->NowMicros() - imm_start;
  }
  return 0;
}

2183 2184
namespace {
struct IterState {
I
Igor Canadi 已提交
2185 2186
  IterState(DBImpl* _db, port::Mutex* _mu, SuperVersion* _super_version)
      : db(_db), mu(_mu), super_version(_super_version) {}
2187 2188

  DBImpl* db;
2189
  port::Mutex* mu;
2190
  SuperVersion* super_version;
2191 2192 2193 2194
};

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

2196
  if (state->super_version->Unref()) {
I
Igor Canadi 已提交
2197
    JobContext job_context;
2198

2199 2200
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2201
    state->db->FindObsoleteFiles(&job_context, false, true);
2202 2203 2204
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2205 2206
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2207
    }
I
Igor Canadi 已提交
2208
    job_context.Clean();
I
Igor Canadi 已提交
2209
  }
T
Tomislav Novak 已提交
2210

2211 2212
  delete state;
}
H
Hans Wennborg 已提交
2213
}  // namespace
2214

L
Lei Jin 已提交
2215
Iterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
2216
                                      ColumnFamilyData* cfd,
2217 2218 2219
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
2220 2221 2222 2223 2224
  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 已提交
2225
      super_version->mem->NewIterator(read_options, arena));
2226
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
2227
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
2228
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
2229
  super_version->current->AddIterators(read_options, env_options_,
2230 2231
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
2232
  IterState* cleanup = new IterState(this, &mutex_, super_version);
2233
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2234 2235 2236 2237

  return internal_iter;
}

2238 2239 2240 2241
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2242
Status DBImpl::Get(const ReadOptions& read_options,
2243
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2244
                   std::string* value) {
L
Lei Jin 已提交
2245
  return GetImpl(read_options, column_family, key, value);
2246 2247
}

I
Igor Canadi 已提交
2248 2249
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
2250 2251
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
2252
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
2253
//
I
Igor Canadi 已提交
2254 2255 2256 2257
// However, if InstallSuperVersion() gets called twice with the same
// job_context, we can't reuse the SuperVersion() that got
// malloced
// because
I
Igor Canadi 已提交
2258 2259 2260
// 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 已提交
2261 2262
void DBImpl::InstallSuperVersionBackground(
    ColumnFamilyData* cfd, JobContext* job_context,
2263
    const MutableCFOptions& mutable_cf_options) {
2264
  mutex_.AssertHeld();
I
Igor Canadi 已提交
2265 2266 2267 2268
  SuperVersion* old_superversion = InstallSuperVersion(
      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 已提交
2269 2270
}

L
Lei Jin 已提交
2271 2272 2273 2274
SuperVersion* DBImpl::InstallSuperVersion(
    ColumnFamilyData* cfd, SuperVersion* new_sv,
    const MutableCFOptions& mutable_cf_options) {
  mutex_.AssertHeld();
2275 2276 2277 2278 2279 2280 2281 2282 2283

  // 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 已提交
2284 2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300
  auto* old = cfd->InstallSuperVersion(
      new_sv ? new_sv : new SuperVersion(), &mutex_, mutable_cf_options);

  // We want to schedule potential flush or compactions since new options may
  // have been picked up in this new version. New options may cause flush
  // compaction trigger condition to change.
  MaybeScheduleFlushOrCompaction();

  // 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,
2301 2302
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
L
Lei Jin 已提交
2303
  StopWatch sw(env_, stats_, DB_GET);
2304
  PERF_TIMER_GUARD(get_snapshot_time);
2305

2306 2307 2308
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2309
  SequenceNumber snapshot;
L
Lei Jin 已提交
2310 2311 2312
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2313 2314
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2315
  }
2316

2317
  // Acquire SuperVersion
2318
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
I
Igor Canadi 已提交
2319

2320
  // Prepare to store a list of merge operations if merge occurs.
2321
  MergeContext merge_context;
2322

2323
  Status s;
2324
  // First look in the memtable, then in the immutable memtable (if any).
2325
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2326
  // merge_operands will contain the sequence of merges in the latter case.
2327
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2328
  PERF_TIMER_STOP(get_snapshot_time);
2329

2330
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2331
    // Done
L
Lei Jin 已提交
2332
    RecordTick(stats_, MEMTABLE_HIT);
2333
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2334
    // Done
L
Lei Jin 已提交
2335
    RecordTick(stats_, MEMTABLE_HIT);
2336
  } else {
2337
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2338 2339
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2340
    RecordTick(stats_, MEMTABLE_MISS);
2341
  }
2342

2343 2344
  {
    PERF_TIMER_GUARD(get_post_process_time);
2345

2346
    ReturnAndCleanupSuperVersion(cfd, sv);
2347

2348 2349 2350
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2351
  return s;
J
jorlow@chromium.org 已提交
2352 2353
}

2354
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
2355
    const ReadOptions& read_options,
2356
    const std::vector<ColumnFamilyHandle*>& column_family,
2357
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
2358

L
Lei Jin 已提交
2359
  StopWatch sw(env_, stats_, DB_MULTIGET);
2360
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
2361

2362
  SequenceNumber snapshot;
2363

2364
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
2365
    ColumnFamilyData* cfd;
2366 2367 2368 2369 2370
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
2371 2372 2373 2374 2375 2376
    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});
2377 2378 2379
    }
  }

2380
  mutex_.Lock();
L
Lei Jin 已提交
2381 2382 2383
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2384 2385 2386
  } else {
    snapshot = versions_->LastSequence();
  }
2387
  for (auto mgd_iter : multiget_cf_data) {
2388 2389
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
2390
  }
2391
  mutex_.Unlock();
2392

2393 2394
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2395

2396
  // Note: this always resizes the values array
2397 2398 2399
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
2400 2401

  // Keep track of bytes that we read for statistics-recording later
2402
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
2403
  PERF_TIMER_STOP(get_snapshot_time);
2404 2405 2406 2407

  // 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.
2408
  // merge_operands will contain the sequence of merges in the latter case.
2409
  for (size_t i = 0; i < num_keys; ++i) {
2410
    merge_context.Clear();
2411
    Status& s = stat_list[i];
2412 2413 2414
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
2415 2416
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
2417 2418 2419
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
2420
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
2421
      // Done
2422
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
2423 2424
      // Done
    } else {
2425
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2426 2427
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
2428 2429 2430
    }

    if (s.ok()) {
2431
      bytes_read += value->size();
2432 2433 2434 2435
    }
  }

  // Post processing (decrement reference counts and record statistics)
2436
  PERF_TIMER_GUARD(get_post_process_time);
2437 2438
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
2439
  // TODO(icanadi) do we need lock here or just around Cleanup()?
2440 2441 2442 2443 2444 2445
  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);
2446 2447
    }
  }
2448 2449 2450 2451 2452 2453 2454
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
2455
  }
2456

L
Lei Jin 已提交
2457 2458 2459
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
2460
  PERF_TIMER_STOP(get_post_process_time);
2461

2462
  return stat_list;
2463 2464
}

L
Lei Jin 已提交
2465
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
2466
                                  const std::string& column_family_name,
2467
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
2468
  Status s;
I
Igor Canadi 已提交
2469
  *handle = nullptr;
Y
Yueh-Hsuan Chiang 已提交
2470 2471
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
2472

Y
Yueh-Hsuan Chiang 已提交
2473 2474 2475 2476 2477 2478 2479 2480 2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491 2492 2493 2494 2495 2496 2497 2498 2499 2500 2501 2502 2503 2504 2505
    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);
    s = versions_->LogAndApply(nullptr,
        MutableCFOptions(opt, ImmutableCFOptions(opt)),
        &edit, &mutex_, db_directory_.get(), false, &cf_options);
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
      delete InstallSuperVersion(
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
      *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());
    }
2506
  }  // MutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
2507 2508

  // this is outside the mutex
2509
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2510 2511
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
2512
  }
2513
  return s;
2514 2515
}

2516 2517 2518 2519
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
2520 2521
    return Status::InvalidArgument("Can't drop default column family");
  }
2522

I
Igor Canadi 已提交
2523 2524
  VersionEdit edit;
  edit.DropColumnFamily();
2525 2526
  edit.SetColumnFamily(cfd->GetID());

2527
  Status s;
2528 2529 2530 2531 2532 2533
  {
    MutexLock l(&mutex_);
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
2534
      // we drop column family from a single write thread
I
Igor Canadi 已提交
2535 2536
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
2537
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
2538 2539
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
I
Igor Canadi 已提交
2540
      write_thread_.ExitWriteThread(&w, &w, s);
2541
    }
2542
  }
2543

2544
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2545 2546 2547 2548
    // 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 已提交
2549
    assert(cfd->IsDropped());
L
Lei Jin 已提交
2550 2551 2552
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
2553 2554
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
2555
        cfd->GetID());
2556
  } else {
2557
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
2558
        "Dropping column family with id %u FAILED -- %s\n",
2559 2560 2561
        cfd->GetID(), s.ToString().c_str());
  }

2562
  return s;
2563 2564
}

L
Lei Jin 已提交
2565
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
2566 2567
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
2568
  if (value_found != nullptr) {
K
Kai Liu 已提交
2569 2570
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
2571
  }
L
Lei Jin 已提交
2572
  ReadOptions roptions = read_options;
2573
  roptions.read_tier = kBlockCacheTier; // read from block cache only
2574
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
2575

2576
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
2577 2578 2579
  // 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();
2580 2581
}

2582
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
2583 2584 2585
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
2586

2587
  if (read_options.tailing) {
I
Igor Canadi 已提交
2588 2589 2590 2591
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
2592 2593
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
2594
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2595 2596 2597
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
2598
#endif
T
Tomislav Novak 已提交
2599
  } else {
2600
    SequenceNumber latest_snapshot = versions_->LastSequence();
2601
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
2602

I
Igor Canadi 已提交
2603
    auto snapshot =
2604 2605 2606
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
2607
            : latest_snapshot;
T
Tomislav Novak 已提交
2608

2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651
    // Try to generate a DB iterator tree in continuous memory area to be
    // cache friendly. Here is an example of result:
    // +-------------------------------+
    // |                               |
    // | ArenaWrappedDBIter            |
    // |  +                            |
    // |  +---> Inner Iterator   ------------+
    // |  |                            |     |
    // |  |    +-- -- -- -- -- -- -- --+     |
    // |  +--- | Arena                 |     |
    // |       |                       |     |
    // |          Allocated Memory:    |     |
    // |       |   +-------------------+     |
    // |       |   | DBIter            | <---+
    // |           |  +                |
    // |       |   |  +-> iter_  ------------+
    // |       |   |                   |     |
    // |       |   +-------------------+     |
    // |       |   | MergingIterator   | <---+
    // |           |  +                |
    // |       |   |  +->child iter1  ------------+
    // |       |   |  |                |          |
    // |           |  +->child iter2  ----------+ |
    // |       |   |  |                |        | |
    // |       |   |  +->child iter3  --------+ | |
    // |           |                   |      | | |
    // |       |   +-------------------+      | | |
    // |       |   | Iterator1         | <--------+
    // |       |   +-------------------+      | |
    // |       |   | Iterator2         | <------+
    // |       |   +-------------------+      |
    // |       |   | Iterator3         | <----+
    // |       |   +-------------------+
    // |       |                       |
    // +-------+-----------------------+
    //
    // ArenaWrappedDBIter inlines an arena area where all the iterartor in the
    // the iterator tree is allocated in the order of being accessed when
    // querying.
    // Laying out the iterators in the order of being accessed makes it more
    // likely that any iterator pointer is close to the iterator it points to so
    // that they are likely to be in the same cache line and/or page.
    ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
2652
        env_, *cfd->ioptions(), cfd->user_comparator(),
2653
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
2654
        read_options.iterate_upper_bound);
2655

2656
    Iterator* internal_iter =
2657
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
2658 2659 2660 2661
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
2662 2663
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
2664 2665
}

2666
Status DBImpl::NewIterators(
2667
    const ReadOptions& read_options,
I
Igor Canadi 已提交
2668
    const std::vector<ColumnFamilyHandle*>& column_families,
2669
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
2670 2671 2672
  iterators->clear();
  iterators->reserve(column_families.size());

2673
  if (read_options.tailing) {
I
Igor Canadi 已提交
2674 2675 2676 2677
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
2678 2679
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
2680 2681
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
2682
      iterators->push_back(
2683
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2684 2685
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
2686
    }
I
Igor Canadi 已提交
2687
#endif
I
Igor Canadi 已提交
2688
  } else {
2689 2690
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
2691
    for (size_t i = 0; i < column_families.size(); ++i) {
2692 2693 2694
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
2695 2696

      auto snapshot =
2697 2698 2699
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
2700 2701
              : latest_snapshot;

2702
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
2703
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
2704
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
2705
      Iterator* internal_iter = NewInternalIterator(
2706
          read_options, cfd, sv, db_iter->GetArena());
2707 2708
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
2709 2710 2711 2712
    }
  }

  return Status::OK();
2713 2714
}

2715 2716 2717 2718 2719 2720 2721 2722 2723
bool DBImpl::IsSnapshotSupported() const {
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (!cfd->mem()->IsSnapshotSupported()) {
      return false;
    }
  }
  return true;
}

J
jorlow@chromium.org 已提交
2724
const Snapshot* DBImpl::GetSnapshot() {
2725 2726 2727
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error

2728
  MutexLock l(&mutex_);
2729 2730
  // returns null if the underlying memtable does not support snapshot.
  if (!IsSnapshotSupported()) return nullptr;
2731
  return snapshots_.New(versions_->LastSequence(), unix_time);
J
jorlow@chromium.org 已提交
2732 2733 2734 2735
}

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

// Convenience methods
2740 2741
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
2742
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
2743 2744
}

2745 2746 2747
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
2748
  if (!cfh->cfd()->ioptions()->merge_operator) {
2749 2750
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
2751
    return DB::Merge(o, column_family, key, val);
2752 2753 2754
  }
}

L
Lei Jin 已提交
2755
Status DBImpl::Delete(const WriteOptions& write_options,
2756
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
2757
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
2758 2759
}

L
Lei Jin 已提交
2760
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
S
Stanislau Hlebik 已提交
2761 2762 2763
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
2764
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
I
Igor Canadi 已提交
2765
  WriteThread::Writer w(&mutex_);
S
Stanislau Hlebik 已提交
2766
  w.batch = my_batch;
L
Lei Jin 已提交
2767 2768
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
2769 2770
  w.in_batch_group = false;
  w.done = false;
L
Lei Jin 已提交
2771
  w.timeout_hint_us = write_options.timeout_hint_us;
S
Stanislau Hlebik 已提交
2772 2773

  uint64_t expiration_time = 0;
I
Igor Canadi 已提交
2774
  bool has_timeout = false;
S
Stanislau Hlebik 已提交
2775
  if (w.timeout_hint_us == 0) {
I
Igor Canadi 已提交
2776
    w.timeout_hint_us = WriteThread::kNoTimeOut;
S
Stanislau Hlebik 已提交
2777 2778
  } else {
    expiration_time = env_->NowMicros() + w.timeout_hint_us;
I
Igor Canadi 已提交
2779
    has_timeout = true;
S
Stanislau Hlebik 已提交
2780 2781
  }

L
Lei Jin 已提交
2782
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
2783 2784 2785 2786 2787 2788
    RecordTick(stats_, WRITE_WITH_WAL);
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_WITH_WAL, 1);
  }

  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
2789
  Status status = write_thread_.EnterWriteThread(&w, expiration_time);
S
Stanislau Hlebik 已提交
2790 2791
  assert(status.ok() || status.IsTimedOut());
  if (status.IsTimedOut()) {
2792
    mutex_.Unlock();
L
Lei Jin 已提交
2793
    RecordTick(stats_, WRITE_TIMEDOUT);
2794
    return Status::TimedOut();
2795
  }
S
Stanislau Hlebik 已提交
2796 2797 2798 2799 2800 2801 2802 2803 2804 2805
  if (w.done) {  // write was done by someone else
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_OTHER,
                                           1);
    mutex_.Unlock();
    RecordTick(stats_, WRITE_DONE_BY_OTHER);
    return w.status;
  }

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

2807 2808 2809 2810
  // 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.
2811 2812 2813
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

2814
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
2815
                                    ? 4 * max_total_in_memory_state_
2816
                                    : db_options_.max_total_wal_size;
2817
  if (UNLIKELY(!single_column_family_mode_) &&
2818
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
2819
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
2820
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
2821
    alive_log_files_.begin()->getting_flushed = true;
2822
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
2823 2824 2825
        "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 已提交
2826 2827
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
2828
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
2829
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
S
Stanislau Hlebik 已提交
2830
        status = SetNewMemtableAndNewLogFile(cfd, &context);
I
Igor Canadi 已提交
2831 2832 2833
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
2834
        cfd->imm()->FlushRequested();
2835
      }
2836
    }
I
Igor Canadi 已提交
2837
    MaybeScheduleFlushOrCompaction();
2838 2839 2840 2841 2842 2843 2844 2845 2846 2847 2848 2849 2850 2851 2852 2853 2854
  } 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()) {
      if (!cfd->mem()->IsEmpty()) {
        status = SetNewMemtableAndNewLogFile(cfd, &context);
        if (!status.ok()) {
          break;
        }
        cfd->imm()->FlushRequested();
      }
    }
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2855 2856 2857 2858 2859 2860 2861 2862 2863 2864 2865 2866
  }

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

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

  if (UNLIKELY(status.ok()) &&
      (write_controller_.IsStopped() || write_controller_.GetDelay() > 0)) {
I
Igor Canadi 已提交
2867
    status = DelayWrite(expiration_time);
I
Igor Canadi 已提交
2868 2869 2870 2871 2872
  }

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

D
dgrogan@chromium.org 已提交
2875
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
2876
  WriteThread::Writer* last_writer = &w;
S
Stanislau Hlebik 已提交
2877
  if (status.ok()) {
2878
    autovector<WriteBatch*> write_batch_group;
I
Igor Canadi 已提交
2879
    write_thread_.BuildBatchGroup(&last_writer, &write_batch_group);
2880

2881 2882 2883
    // 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
2884
    // into memtables
2885
    {
2886
      mutex_.Unlock();
2887 2888 2889 2890 2891 2892 2893 2894 2895 2896
      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]);
        }
      }

2897 2898 2899 2900
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
2901
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
2902
      // Record statistics
L
Lei Jin 已提交
2903
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
2904
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
2905
      if (write_options.disableWAL) {
2906
        flush_on_destroy_ = true;
2907
      }
L
Lei Jin 已提交
2908
      PERF_TIMER_STOP(write_pre_and_post_process_time);
2909

2910
      uint64_t log_size = 0;
L
Lei Jin 已提交
2911
      if (!write_options.disableWAL) {
2912
        PERF_TIMER_GUARD(write_wal_time);
2913 2914
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
2915 2916
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
2917
        log_empty_ = false;
2918
        log_size = log_entry.size();
L
Lei Jin 已提交
2919
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
L
Lei Jin 已提交
2920
        if (status.ok() && write_options.sync) {
I
Igor Canadi 已提交
2921
          RecordTick(stats_, WAL_FILE_SYNCED);
2922
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
2923
          if (db_options_.use_fsync) {
2924
            status = log_->file()->Fsync();
2925
          } else {
2926
            status = log_->file()->Sync();
2927
          }
H
heyongqiang 已提交
2928
        }
2929 2930
      }
      if (status.ok()) {
2931
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
2932

2933
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
2934
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
2935
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
2936 2937 2938 2939 2940 2941 2942 2943
        // 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 已提交
2944
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
2945
      }
L
Lei Jin 已提交
2946
      PERF_TIMER_START(write_pre_and_post_process_time);
2947 2948 2949
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
2950
      mutex_.Lock();
2951 2952 2953
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
2954 2955
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
2956
      if (!write_options.disableWAL) {
2957 2958 2959 2960 2961
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
2962
      if (status.ok()) {
2963
        versions_->SetLastSequence(last_sequence);
2964
      }
J
jorlow@chromium.org 已提交
2965 2966
    }
  }
2967
  if (db_options_.paranoid_checks && !status.ok() &&
2968
      !status.IsTimedOut() && bg_error_.ok()) {
I
Igor Canadi 已提交
2969 2970
    bg_error_ = status; // stop compaction & fail any further writes
  }
2971

I
Igor Canadi 已提交
2972
  write_thread_.ExitWriteThread(&w, last_writer, status);
I
Igor Canadi 已提交
2973
  mutex_.Unlock();
2974

2975
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
2976
    RecordTick(stats_, WRITE_TIMEDOUT);
2977 2978
  }

J
jorlow@chromium.org 已提交
2979 2980 2981
  return status;
}

2982 2983
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
2984
Status DBImpl::DelayWrite(uint64_t expiration_time) {
2985 2986 2987 2988 2989
  StopWatch sw(env_, stats_, WRITE_STALL);
  bool has_timeout = (expiration_time > 0);
  auto delay = write_controller_.GetDelay();
  if (write_controller_.IsStopped() == false && delay > 0) {
    mutex_.Unlock();
2990 2991
    // hopefully we don't have to sleep more than 2 billion microseconds
    env_->SleepForMicroseconds(static_cast<int>(delay));
2992 2993 2994
    mutex_.Lock();
  }

I
Igor Canadi 已提交
2995
  while (bg_error_.ok() && write_controller_.IsStopped()) {
2996 2997 2998
    if (has_timeout) {
      bg_cv_.TimedWait(expiration_time);
      if (env_->NowMicros() > expiration_time) {
I
Igor Canadi 已提交
2999
        return Status::TimedOut();
3000 3001 3002 3003 3004
      }
    } else {
      bg_cv_.Wait();
    }
  }
I
Igor Canadi 已提交
3005 3006

  return bg_error_;
3007 3008
}

I
Igor Canadi 已提交
3009 3010 3011 3012 3013 3014 3015 3016 3017 3018 3019
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  bool schedule_bg_work = false;
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
    schedule_bg_work = true;
    auto status = SetNewMemtableAndNewLogFile(cfd, context);
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
3020 3021
    }
  }
I
Igor Canadi 已提交
3022 3023 3024 3025
  if (schedule_bg_work) {
    MaybeScheduleFlushOrCompaction();
  }
  return Status::OK();
S
Stanislau Hlebik 已提交
3026 3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037 3038
}

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

  // Attempt to switch to a new memtable and trigger flush of old.
  // Do this without holding the dbmutex lock.
3039
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
3040 3041 3042 3043
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
3044
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
3045 3046 3047 3048
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
3049 3050 3051
      s = env_->NewWritableFile(
          LogFileName(db_options_.wal_dir, new_log_number),
          &lfile, env_->OptimizeForLogWrite(env_options_));
S
Stanislau Hlebik 已提交
3052 3053 3054
      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 已提交
3055 3056
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
S
Stanislau Hlebik 已提交
3057
        new_log = new log::Writer(std::move(lfile));
I
Igor Canadi 已提交
3058
      }
S
Stanislau Hlebik 已提交
3059 3060 3061
    }

    if (s.ok()) {
3062
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options);
S
Stanislau Hlebik 已提交
3063 3064 3065
      new_superversion = new SuperVersion();
    }
  }
3066
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
3067 3068
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
3069 3070 3071 3072 3073 3074 3075 3076 3077 3078 3079 3080 3081 3082 3083
  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);
    context->logs_to_free_.push_back(log_.release());
    log_.reset(new_log);
    log_empty_ = true;
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
3084
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
3085 3086 3087 3088
      // 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 已提交
3089 3090 3091
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
          loop_cfd->imm()->size() == 0) {
        loop_cfd->SetLogNumber(logfile_number_);
3092
      }
3093 3094
    }
  }
S
Stanislau Hlebik 已提交
3095 3096 3097 3098 3099
  cfd->mem()->SetNextLogNumber(logfile_number_);
  cfd->imm()->Add(cfd->mem());
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
  context->superversions_to_free_.push_back(
L
Lei Jin 已提交
3100
      InstallSuperVersion(cfd, new_superversion, mutable_cf_options));
3101 3102 3103
  return s;
}

I
Igor Canadi 已提交
3104
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
3105 3106 3107 3108 3109
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3110 3111
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
3112
  auto version = cfd->current();
3113 3114 3115 3116 3117 3118 3119 3120 3121 3122 3123 3124
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
3125
#endif  // ROCKSDB_LITE
3126

I
Igor Canadi 已提交
3127 3128 3129 3130
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3131 3132 3133 3134
Env* DBImpl::GetEnv() const {
  return env_;
}

3135 3136
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
3137
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
3138 3139
}

3140
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
3141
                         const Slice& property, std::string* value) {
3142 3143
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3144 3145 3146
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

3147
  value->clear();
3148 3149 3150 3151 3152
  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) {
3153
      *value = ToString(int_value);
3154 3155 3156 3157 3158 3159 3160 3161 3162
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
    MutexLock l(&mutex_);
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
3163 3164
}

3165 3166
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
3167 3168
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3169 3170 3171 3172 3173 3174 3175 3176 3177 3178 3179 3180
  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) {
3181 3182
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
3183 3184 3185

  if (!need_out_of_mutex) {
    MutexLock l(&mutex_);
3186
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200
  } 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 已提交
3201
  return cfd->GetThreadLocalSuperVersion(&mutex_);
3202 3203 3204 3205
}

void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
3206
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3207 3208 3209 3210 3211 3212 3213 3214 3215 3216 3217 3218 3219

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

3222
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
3223
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
3224 3225
  // TODO(opt): better implementation
  Version* v;
3226 3227
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
3228 3229
  {
    MutexLock l(&mutex_);
3230
    v = cfd->current();
3231
    v->Ref();
J
jorlow@chromium.org 已提交
3232 3233 3234 3235 3236 3237 3238 3239 3240 3241 3242 3243 3244 3245 3246 3247 3248
  }

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
    uint64_t start = versions_->ApproximateOffsetOf(v, k1);
    uint64_t limit = versions_->ApproximateOffsetOf(v, k2);
    sizes[i] = (limit >= start ? limit - start : 0);
  }

  {
    MutexLock l(&mutex_);
    v->Unref();
  }
}

I
Igor Canadi 已提交
3249 3250 3251 3252 3253 3254 3255 3256 3257 3258 3259 3260 3261 3262 3263 3264
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 已提交
3265 3266 3267 3268 3269
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
3270
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
3271 3272 3273
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
3274
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
3275 3276
}

3277 3278 3279
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
3280 3281 3282
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
3283 3284
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
3285 3286 3287
    return Status::InvalidArgument("Invalid file name");
  }

3288 3289 3290 3291
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
3292 3293
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
3294
          name.c_str());
3295 3296
      return Status::NotSupported("Delete only supported for archived logs");
    }
3297
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
3298
    if (!status.ok()) {
3299 3300
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
3301
          name.c_str(), status.ToString().c_str());
3302 3303 3304 3305
    }
    return status;
  }

3306
  int level;
I
Igor Canadi 已提交
3307
  FileMetaData* metadata;
3308
  ColumnFamilyData* cfd;
3309
  VersionEdit edit;
I
Igor Canadi 已提交
3310
  JobContext job_context(true);
D
Dhruba Borthakur 已提交
3311 3312
  {
    MutexLock l(&mutex_);
3313
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
3314
    if (!status.ok()) {
3315 3316
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
3317
      job_context.Clean();
D
Dhruba Borthakur 已提交
3318 3319
      return Status::InvalidArgument("File not found");
    }
3320
    assert(level < cfd->NumberLevels());
3321

D
Dhruba Borthakur 已提交
3322
    // If the file is being compacted no need to delete.
3323
    if (metadata->being_compacted) {
3324
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3325
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
3326
      job_context.Clean();
D
Dhruba Borthakur 已提交
3327
      return Status::OK();
3328 3329
    }

D
Dhruba Borthakur 已提交
3330 3331 3332
    // 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 已提交
3333
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
3334
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
3335
      if (vstoreage->NumLevelFiles(i) != 0) {
3336
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
3337
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
3338
        job_context.Clean();
D
Dhruba Borthakur 已提交
3339 3340 3341
        return Status::InvalidArgument("File not in last level");
      }
    }
3342
    // if level == 0, it has to be the oldest file
S
sdong 已提交
3343 3344
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
3345 3346
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
3347
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
3348
      job_context.Clean();
3349 3350 3351
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
3352
    edit.DeleteFile(level, number);
3353 3354
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                    &edit, &mutex_, db_directory_.get());
I
Igor Canadi 已提交
3355
    if (status.ok()) {
I
Igor Canadi 已提交
3356 3357
      InstallSuperVersionBackground(cfd, &job_context,
                                    *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
3358
    }
I
Igor Canadi 已提交
3359 3360
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
3361
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
3362
  // remove files outside the db-lock
I
Igor Canadi 已提交
3363 3364
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
3365
  }
I
Igor Canadi 已提交
3366
  job_context.Clean();
3367 3368 3369 3370 3371 3372
  {
    MutexLock l(&mutex_);
    // schedule flush if file deletion means we freed the space for flushes to
    // continue
    MaybeScheduleFlushOrCompaction();
  }
3373 3374 3375
  return status;
}

3376
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
3377
  MutexLock l(&mutex_);
3378
  versions_->GetLiveFilesMetaData(metadata);
3379
}
3380 3381 3382 3383 3384 3385 3386 3387 3388 3389 3390

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 已提交
3391
#endif  // ROCKSDB_LITE
3392

I
Igor Canadi 已提交
3393 3394 3395 3396 3397 3398 3399
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
3402 3403 3404 3405 3406 3407
    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) {
3408
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
3409
                             ". Size recorded in manifest " +
3410 3411
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
3412 3413 3414 3415 3416 3417 3418 3419 3420
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

3421 3422 3423 3424 3425 3426 3427 3428 3429 3430 3431 3432 3433 3434 3435
Status DBImpl::GetDbIdentity(std::string& identity) {
  std::string idfilename = IdentityFileName(dbname_);
  unique_ptr<SequentialFile> idfile;
  const EnvOptions soptions;
  Status s = env_->NewSequentialFile(idfilename, &idfile, soptions);
  if (!s.ok()) {
    return s;
  }
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
  char buffer[file_size];
  Slice id;
3436
  s = idfile->Read(static_cast<size_t>(file_size), &id, buffer);
3437 3438 3439 3440 3441 3442 3443 3444 3445 3446 3447
  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 已提交
3448 3449
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
3450
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
3451
               const Slice& key, const Slice& value) {
3452 3453 3454 3455
  // 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);
3456
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
3457 3458 3459
  return Write(opt, &batch);
}

3460 3461
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
3462
  WriteBatch batch;
3463
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
3464 3465 3466
  return Write(opt, &batch);
}

3467 3468
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
3469
  WriteBatch batch;
3470
  batch.Merge(column_family, key, value);
3471 3472 3473
  return Write(opt, &batch);
}

3474
// Default implementation -- returns not supported status
L
Lei Jin 已提交
3475
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3476
                              const std::string& column_family_name,
3477
                              ColumnFamilyHandle** handle) {
3478
  return Status::NotSupported("");
3479
}
3480
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
3481
  return Status::NotSupported("");
3482 3483
}

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

J
Jim Paton 已提交
3486
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
3487 3488 3489 3490
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
3491
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
3492
  std::vector<ColumnFamilyHandle*> handles;
3493
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
3494 3495 3496 3497 3498 3499 3500
  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;
3501 3502
}

3503 3504
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
3505
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
3506
  Status s = SanitizeOptionsByTable(db_options, column_families);
3507 3508 3509
  if (!s.ok()) {
    return s;
  }
3510
  if (db_options.db_paths.size() > 1) {
3511 3512 3513 3514 3515 3516 3517 3518 3519 3520 3521 3522
    for (auto& cfd : column_families) {
      if (cfd.options.compaction_style != kCompactionStyleUniversal) {
        return Status::NotSupported(
            "More than one DB paths are only supported in "
            "universal compaction style. ");
      }
    }

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

3525
  *dbptr = nullptr;
3526
  handles->clear();
J
jorlow@chromium.org 已提交
3527

I
Igor Canadi 已提交
3528 3529 3530 3531
  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);
3532
  }
3533

I
Igor Canadi 已提交
3534
  DBImpl* impl = new DBImpl(db_options, dbname);
3535
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
3536
  if (s.ok()) {
3537
    for (auto db_path : impl->db_options_.db_paths) {
3538
      s = impl->env_->CreateDirIfMissing(db_path.path);
3539 3540 3541 3542 3543 3544
      if (!s.ok()) {
        break;
      }
    }
  }

3545 3546 3547 3548 3549 3550
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
3551 3552 3553 3554
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
3555
  impl->mutex_.Lock();
3556 3557
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
3558
  if (s.ok()) {
3559
    uint64_t new_log_number = impl->versions_->NewFileNumber();
3560
    unique_ptr<WritableFile> lfile;
3561
    EnvOptions soptions(db_options);
3562 3563 3564
    s = impl->db_options_.env->NewWritableFile(
        LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
        impl->db_options_.env->OptimizeForLogWrite(soptions));
J
jorlow@chromium.org 已提交
3565
    if (s.ok()) {
3566
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
3567
      impl->logfile_number_ = new_log_number;
3568
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
3569

3570 3571
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
3572 3573
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
3574 3575 3576
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
3577
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
3578 3579 3580 3581 3582 3583 3584 3585 3586 3587 3588 3589 3590 3591 3592 3593
        } 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 已提交
3594
        }
3595
      }
I
Igor Canadi 已提交
3596 3597
    }
    if (s.ok()) {
3598
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
3599 3600
        delete impl->InstallSuperVersion(
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
3601
      }
I
Igor Canadi 已提交
3602 3603
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
3604
      impl->DeleteObsoleteFiles();
3605
      impl->MaybeScheduleFlushOrCompaction();
3606
      s = impl->db_directory_->Fsync();
J
jorlow@chromium.org 已提交
3607 3608
    }
  }
3609

I
Igor Canadi 已提交
3610 3611
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
3612 3613
      if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
3614
        auto* vstorage = cfd->current()->storage_info();
3615
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
3616
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
3617
          if (num_files > 0) {
I
Igor Canadi 已提交
3618 3619 3620
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
                "open with universal or FIFO compaction style.");
I
Igor Canadi 已提交
3621 3622 3623 3624
            break;
          }
        }
      }
3625
      if (cfd->ioptions()->merge_operator != nullptr &&
3626 3627 3628 3629 3630
          !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 已提交
3631
      if (!s.ok()) {
3632 3633 3634 3635 3636
        break;
      }
    }
  }

3637 3638
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
3639
  if (s.ok()) {
3640
    impl->opened_successfully_ = true;
J
jorlow@chromium.org 已提交
3641 3642
    *dbptr = impl;
  } else {
Y
Yueh-Hsuan Chiang 已提交
3643
    for (auto* h : *handles) {
3644 3645
      delete h;
    }
3646
    handles->clear();
J
jorlow@chromium.org 已提交
3647 3648 3649 3650 3651
    delete impl;
  }
  return s;
}

3652 3653 3654
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
3655
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
3656 3657
}

3658 3659 3660
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
3661
Status DestroyDB(const std::string& dbname, const Options& options) {
3662
  const InternalKeyComparator comparator(options.comparator);
3663
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
3664
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
3665
  std::vector<std::string> filenames;
3666 3667
  std::vector<std::string> archiveFiles;

3668
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
3669 3670
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
3671 3672 3673 3674 3675 3676 3677

  if (dbname != soptions.wal_dir) {
    std::vector<std::string> logfilenames;
    env->GetChildren(soptions.wal_dir, &logfilenames);
    filenames.insert(filenames.end(), logfilenames.begin(), logfilenames.end());
    archivedir = ArchivalDirectory(soptions.wal_dir);
  }
3678

J
jorlow@chromium.org 已提交
3679 3680 3681 3682 3683
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
3684 3685
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
3686 3687 3688
  if (result.ok()) {
    uint64_t number;
    FileType type;
3689
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
3690
    for (size_t i = 0; i < filenames.size(); i++) {
3691
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
3692
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
3693 3694 3695
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
3696 3697
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
3698 3699 3700
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
3701 3702 3703 3704 3705
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3706

3707 3708
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
3709 3710 3711
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
3712
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
3713 3714 3715 3716 3717 3718 3719
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

3720
    env->GetChildren(archivedir, &archiveFiles);
3721 3722
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
3723 3724
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
3725
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
3726 3727 3728 3729 3730
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3731
    // ignore case where no archival directory is present.
3732
    env->DeleteDir(archivedir);
3733

J
jorlow@chromium.org 已提交
3734
    env->UnlockFile(lock);  // Ignore error since state is already gone
3735
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
3736
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
3737
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
3738 3739 3740 3741
  }
  return result;
}

Y
Yueh-Hsuan Chiang 已提交
3742 3743 3744
#if ROCKSDB_USING_THREAD_STATUS
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
3745 3746 3747 3748
  if (db_options_.enable_thread_tracking) {
    ThreadStatusImpl::NewColumnFamilyInfo(
        this, GetName(), cfd, cfd->GetName());
  }
Y
Yueh-Hsuan Chiang 已提交
3749 3750 3751 3752
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
3753 3754 3755
  if (db_options_.enable_thread_tracking) {
    ThreadStatusImpl::EraseColumnFamilyInfo(cfd);
  }
Y
Yueh-Hsuan Chiang 已提交
3756 3757 3758
}

void DBImpl::EraseThreadStatusDbInfo() const {
3759 3760 3761
  if (db_options_.enable_thread_tracking) {
    ThreadStatusImpl::EraseDatabaseInfo(this);
  }
Y
Yueh-Hsuan Chiang 已提交
3762 3763 3764 3765 3766 3767 3768 3769 3770 3771 3772 3773 3774 3775 3776 3777 3778 3779
}

Status GetThreadList(std::vector<ThreadStatus>* thread_list) {
  return thread_local_status.GetThreadList(thread_list);
}
#else
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
}

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

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

3780 3781
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
3782
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
3783
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
3784 3785
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
3786 3787
  Log(InfoLogLevel::INFO_LEVEL, log,
      "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
I
Igor Canadi 已提交
3788
      ROCKSDB_PATCH);
3789 3790
  Log(InfoLogLevel::INFO_LEVEL, log, "Git sha %s", rocksdb_build_git_sha);
  Log(InfoLogLevel::INFO_LEVEL, log, "Compile time %s %s",
3791
      rocksdb_build_compile_time, rocksdb_build_compile_date);
I
Igor Canadi 已提交
3792
#endif
3793 3794
}

3795
}  // namespace rocksdb