db_impl.cc 128.5 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 47
#include "db/version_set.h"
#include "db/write_batch_internal.h"
48
#include "port/port.h"
I
Igor Canadi 已提交
49
#include "rocksdb/cache.h"
50
#include "port/likely.h"
51 52 53 54
#include "rocksdb/compaction_filter.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/merge_operator.h"
I
Igor Canadi 已提交
55
#include "rocksdb/version.h"
56 57
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
S
Siying Dong 已提交
58
#include "rocksdb/table.h"
J
jorlow@chromium.org 已提交
59
#include "table/block.h"
60
#include "table/block_based_table_factory.h"
J
jorlow@chromium.org 已提交
61
#include "table/merger.h"
K
kailiu 已提交
62
#include "table/table_builder.h"
J
jorlow@chromium.org 已提交
63
#include "table/two_level_iterator.h"
64
#include "util/auto_roll_logger.h"
K
kailiu 已提交
65
#include "util/autovector.h"
66
#include "util/build_version.h"
J
jorlow@chromium.org 已提交
67
#include "util/coding.h"
68
#include "util/db_info_dumper.h"
I
Igor Canadi 已提交
69
#include "util/hash_skiplist_rep.h"
70
#include "util/hash_linklist_rep.h"
J
jorlow@chromium.org 已提交
71
#include "util/logging.h"
H
Haobo Xu 已提交
72
#include "util/log_buffer.h"
J
jorlow@chromium.org 已提交
73
#include "util/mutexlock.h"
74
#include "util/perf_context_imp.h"
75
#include "util/iostats_context_imp.h"
76
#include "util/stop_watch.h"
77
#include "util/sync_point.h"
J
jorlow@chromium.org 已提交
78

79
namespace rocksdb {
J
jorlow@chromium.org 已提交
80

81
const std::string kDefaultColumnFamilyName("default");
82

I
Igor Canadi 已提交
83
void DumpRocksDBBuildVersion(Logger * log);
84

S
Stanislau Hlebik 已提交
85 86 87 88 89 90 91 92 93 94 95 96 97 98
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 已提交
99 100 101
Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
                        const Options& src) {
102
  auto db_options = SanitizeOptions(dbname, DBOptions(src));
103
  auto cf_options = SanitizeOptions(icmp, ColumnFamilyOptions(src));
104 105 106 107 108
  return Options(db_options, cf_options);
}

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

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

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

128 129 130 131 132 133
  if (!result.rate_limiter) {
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

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

142
  if (result.db_paths.size() == 0) {
143
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
144 145
  }

J
jorlow@chromium.org 已提交
146 147 148
  return result;
}

149 150
namespace {

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

164
CompressionType GetCompressionFlush(const ImmutableCFOptions& ioptions) {
165 166 167 168 169 170
  // 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;

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

  if (can_compress) {
181
    return ioptions.compression;
182 183 184 185
  } else {
    return kNoCompression;
  }
}
186
}  // namespace
187

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

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

231 232
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
                                 table_cache_.get(), &write_controller_));
I
Igor Canadi 已提交
233 234
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
235

I
Igor Canadi 已提交
236
  DumpRocksDBBuildVersion(db_options_.info_log.get());
237 238
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
239

240
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
241 242 243
}

DBImpl::~DBImpl() {
244
  mutex_.Lock();
245

246 247
  if (flush_on_destroy_) {
    for (auto cfd : *versions_->GetColumnFamilySet()) {
248
      if (!cfd->mem()->IsEmpty()) {
249 250 251 252
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
253
        cfd->Unref();
254 255
      }
    }
256
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
257
  }
I
Igor Canadi 已提交
258 259

  // Wait for background work to finish
I
Igor Canadi 已提交
260
  shutting_down_.store(true, std::memory_order_release);
261
  while (bg_compaction_scheduled_ || bg_flush_scheduled_ || notifying_events_) {
H
hans@chromium.org 已提交
262
    bg_cv_.Wait();
J
jorlow@chromium.org 已提交
263
  }
264
  listeners_.clear();
I
Igor Canadi 已提交
265 266
  flush_scheduler_.Clear();

I
Igor Canadi 已提交
267 268 269 270 271
  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();
272 273
  }

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

294
  // versions need to be destroyed before table_cache since it can hold
295 296
  // references to table_cache.
  versions_.reset();
297
  mutex_.Unlock();
I
Igor Canadi 已提交
298 299 300
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
301

302
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
303 304 305
}

Status DBImpl::NewDB() {
306
  VersionEdit new_db;
307
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
308 309 310
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

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

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

346
const Status DBImpl::CreateArchivalDirectory() {
347 348
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
349 350 351 352 353
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

354
void DBImpl::PrintStatistics() {
355
  auto dbstats = db_options_.statistics.get();
356
  if (dbstats) {
357
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
358 359
        "STATISTCS:\n %s",
        dbstats->ToString().c_str());
360 361 362
  }
}

363
void DBImpl::MaybeDumpStats() {
364
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
365 366 367 368

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
369
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
370 371 372 373 374 375
      <= 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;
376

377 378 379 380 381 382
    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 已提交
383
    std::string stats;
384 385 386
    {
      MutexLock l(&mutex_);
      for (auto cfd : *versions_->GetColumnFamilySet()) {
387 388
        cfd->internal_stats()->GetStringProperty(cf_property_type,
                                                 "rocksdb.cfstats", &stats);
389
      }
390 391
      default_cf_internal_stats_->GetStringProperty(db_property_type,
                                                    "rocksdb.dbstats", &stats);
392
    }
393 394 395 396
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "------- DUMPING STATS -------");
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "%s", stats.c_str());
397

398
    PrintStatistics();
399 400 401
  }
}

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

412
  // if deletion is disabled, do nothing
413
  if (disable_delete_obsolete_files_ > 0) {
414 415 416
    return;
  }

417 418 419 420 421
  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;
422
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
423 424 425 426
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
    if (delete_obsolete_files_last_run_ +
427
        db_options_.delete_obsolete_files_period_micros < now_micros) {
428 429 430 431 432
      doing_the_full_scan = true;
      delete_obsolete_files_last_run_ = now_micros;
    }
  }

I
Igor Canadi 已提交
433
  // get obsolete files
I
Igor Canadi 已提交
434
  versions_->GetObsoleteFiles(&job_context->sst_delete_files);
I
Igor Canadi 已提交
435

I
Igor Canadi 已提交
436
  // store the current filenum, lognum, etc
437
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
438
  job_context->pending_manifest_file_number =
439
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
440
  job_context->log_number = versions_->MinLogNumber();
441
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
442

I
Igor Canadi 已提交
443
  if (!doing_the_full_scan && !job_context->HaveSomethingToDelete()) {
444 445 446 447 448 449 450
    // 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 已提交
451 452 453 454 455
  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();
456
  }
I
Igor Canadi 已提交
457
  versions_->AddLiveFiles(&job_context->sst_live);
I
Igor Canadi 已提交
458

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

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

493
namespace {
I
Igor Canadi 已提交
494 495
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
496 497 498 499 500
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
501
    return (first.path_id > second.path_id);
502 503 504 505
  }
}
};  // namespace

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

I
Igor Canadi 已提交
514 515 516 517
  // 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 已提交
518 519
    return;
  }
520

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

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

K
kailiu 已提交
541 542
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
543 544
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
545 546
    }
  }
547

K
kailiu 已提交
548
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
549
  // file twice
550
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
551 552
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
553

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

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

    if (keep) {
      continue;
    }

610
    std::string fname;
K
kailiu 已提交
611 612
    if (type == kTableFile) {
      // evict from cache
613
      TableCache::Evict(table_cache_.get(), number);
614
      fname = TableFileName(db_options_.db_paths, number, path_id);
615
    } else {
616 617
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
618
    }
619

I
Igor Canadi 已提交
620 621
#ifdef ROCKSDB_LITE
    Status s = env_->DeleteFile(fname);
622 623
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "Delete %s type=%d #%" PRIu64 " -- %s\n",
I
Igor Canadi 已提交
624 625 626 627 628
        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 已提交
629 630
    } else {
      Status s = env_->DeleteFile(fname);
631 632
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "Delete %s type=%d #%" PRIu64 " -- %s\n",
633
          fname.c_str(), type, number, s.ToString().c_str());
J
jorlow@chromium.org 已提交
634
    }
I
Igor Canadi 已提交
635
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
636
  }
H
heyongqiang 已提交
637

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

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
I
Igor Canadi 已提交
666 667 668 669
  JobContext job_context;
  FindObsoleteFiles(&job_context, true);
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
670
  }
I
Igor Canadi 已提交
671
  job_context.Clean();
672 673
}

674
Status DBImpl::Recover(
675 676
    const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
    bool error_if_log_file_exist) {
J
jorlow@chromium.org 已提交
677 678
  mutex_.AssertHeld();

679
  bool is_new_db = false;
680
  assert(db_lock_ == nullptr);
I
Igor Canadi 已提交
681
  if (!read_only) {
682 683 684 685 686 687 688 689 690 691 692 693
    // 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;
    }

694
    for (auto& db_path : db_options_.db_paths) {
695
      s = env_->CreateDirIfMissing(db_path.path);
696 697 698 699 700
      if (!s.ok()) {
        return s;
      }
    }

701 702 703 704 705
    s = env_->NewDirectory(dbname_, &db_directory_);
    if (!s.ok()) {
      return s;
    }

706
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
707 708 709
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
710

711
    if (!env_->FileExists(CurrentFileName(dbname_))) {
712
      if (db_options_.create_if_missing) {
713
        s = NewDB();
714
        is_new_db = true;
715 716 717 718 719 720
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
721 722
      }
    } else {
723
      if (db_options_.error_if_exists) {
724 725 726
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
727
    }
M
Mayank Agarwal 已提交
728 729 730 731 732 733 734
    // 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 已提交
735 736
  }

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

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

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

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

L
Lei Jin 已提交
800 801
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
802
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
803 804 805
    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 已提交
806 807
  }

J
jorlow@chromium.org 已提交
808 809 810
  return s;
}

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

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

S
Stanislau Hlebik 已提交
839 840 841 842
  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.
843
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
844 845 846 847
    // 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 已提交
848
    if (!status.ok()) {
S
Stanislau Hlebik 已提交
849 850 851 852 853 854 855 856
      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 已提交
857 858
    }

S
Stanislau Hlebik 已提交
859 860 861 862 863 864 865 866 867 868 869 870 871 872 873
    // 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*/);
874 875
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "Recovering log #%" PRIu64 "", log_number);
S
Stanislau Hlebik 已提交
876 877 878 879 880 881 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

    // 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 已提交
908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924
        // 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;
925
          }
L
Lei Jin 已提交
926
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
927
        }
J
jorlow@chromium.org 已提交
928 929 930
      }
    }

I
Igor Canadi 已提交
931
    flush_scheduler_.Clear();
S
Stanislau Hlebik 已提交
932 933 934
    if (versions_->LastSequence() < *max_sequence) {
      versions_->SetLastSequence(*max_sequence);
    }
935 936
  }

937
  if (!read_only) {
938 939
    // 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 已提交
940
    auto max_log_number = log_numbers.back();
941
    for (auto cfd : *versions_->GetColumnFamilySet()) {
942
      auto iter = version_edits.find(cfd->GetID());
943 944 945
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

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

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

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

J
jorlow@chromium.org 已提交
987 988 989
  return status;
}

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

1012 1013 1014 1015 1016
    {
      mutex_.Unlock();
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
          iter.get(), &meta, cfd->internal_comparator(), newest_snapshot,
1017 1018
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
          cfd->ioptions()->compression_opts, Env::IO_HIGH);
1019
      LogFlush(db_options_.info_log);
1020 1021
      mutex_.Lock();
    }
1022

1023 1024 1025
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
1026 1027 1028
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
        s.ToString().c_str());
  }
I
Igor Canadi 已提交
1029
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1030 1031 1032 1033

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1034
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1035 1036 1037
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1038 1039
  }

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

1051 1052
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
I
Igor Canadi 已提交
1053
    bool* madeProgress, JobContext* job_context, LogBuffer* log_buffer) {
1054
  mutex_.AssertHeld();
1055
  assert(cfd->imm()->size() != 0);
1056
  assert(cfd->imm()->IsFlushPending());
1057

I
Igor Canadi 已提交
1058 1059
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1060 1061 1062
                     snapshots_.GetNewest(), job_context, log_buffer,
                     db_directory_.get(), GetCompressionFlush(*cfd->ioptions()),
                     stats_);
1063

1064 1065
  uint64_t file_number;
  Status s = flush_job.Run(&file_number);
J
jorlow@chromium.org 已提交
1066 1067

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

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

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

1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130
void DBImpl::NotifyOnFlushCompleted(
    ColumnFamilyData* cfd, uint64_t file_number) {
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  bool triggered_flush_slowdown =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
       cfd->options()->level0_slowdown_writes_trigger);
  bool triggered_flush_stop =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
       cfd->options()->level0_stop_writes_trigger);
  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.
}

1131
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1132
                            const Slice* begin, const Slice* end,
1133 1134
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
1135
  if (target_path_id >= db_options_.db_paths.size()) {
1136 1137 1138
    return Status::InvalidArgument("Invalid target path ID");
  }

1139 1140
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1141 1142

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1143
  if (!s.ok()) {
1144
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1145 1146 1147
    return s;
  }

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

  if (reduce_level) {
I
Igor Canadi 已提交
1178
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1179
  }
1180
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1181

1182 1183 1184 1185 1186 1187 1188
  {
    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 已提交
1189
  return s;
1190 1191
}

1192 1193 1194 1195 1196
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 已提交
1197 1198 1199 1200
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219
  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 已提交
1220
#endif  // ROCKSDB_LITE
1221 1222
}

I
Igor Canadi 已提交
1223
#ifndef ROCKSDB_LITE
1224 1225 1226 1227 1228 1229 1230 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
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);
  };
1298 1299 1300 1301 1302
  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));
1303 1304 1305 1306 1307
  compaction_job.Prepare();

  mutex_.Unlock();
  Status status = compaction_job.Run();
  mutex_.Lock();
1308
  compaction_job.Install(&status, &mutex_);
1309
  if (status.ok()) {
1310 1311
    InstallSuperVersionBackground(c->column_family_data(), &job_context,
                                  *c->mutable_cf_options());
1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348
  }
  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 已提交
1349
    job_context.Clean();
1350 1351 1352 1353 1354 1355 1356
    mutex_.Lock();
  }

  bg_compaction_scheduled_--;

  return status;
}
I
Igor Canadi 已提交
1357
#endif  // ROCKSDB_LITE
1358

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

  MutableCFOptions new_options;
1373
  Status s;
L
Lei Jin 已提交
1374 1375
  {
    MutexLock l(&mutex_);
1376 1377
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1378 1379 1380 1381
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

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

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

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1425
  SuperVersion* superversion_to_free = nullptr;
1426
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1427 1428

  mutex_.Lock();
1429 1430 1431

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1432
    mutex_.Unlock();
1433 1434
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
I
Igor Canadi 已提交
1435
    delete new_superversion;
L
Lei Jin 已提交
1436
    return Status::NotSupported("another thread is refitting");
1437 1438 1439 1440 1441
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1442
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1443 1444
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop: %d %d",
1445
        bg_compaction_scheduled_, bg_flush_scheduled_);
1446 1447 1448
    bg_cv_.Wait();
  }

1449 1450
  const MutableCFOptions mutable_cf_options =
    *cfd->GetLatestMutableCFOptions();
1451
  // move to a smaller level
1452 1453
  int to_level = target_level;
  if (target_level < 0) {
1454
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1455
  }
1456 1457 1458

  assert(to_level <= level);

L
Lei Jin 已提交
1459
  Status status;
1460
  if (to_level < level) {
1461 1462
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Before refitting:\n%s",
1463
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1464

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

1477 1478
    status = versions_->LogAndApply(cfd,
        mutable_cf_options, &edit, &mutex_, db_directory_.get());
L
Lei Jin 已提交
1479 1480
    superversion_to_free = InstallSuperVersion(
        cfd, new_superversion, mutable_cf_options);
I
Igor Canadi 已提交
1481
    new_superversion = nullptr;
1482

1483 1484
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1485
        status.ToString().data());
1486 1487

    if (status.ok()) {
1488 1489
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1490
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1491 1492 1493 1494 1495
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1496 1497 1498 1499

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1500
  return status;
G
Gabor Cselle 已提交
1501 1502
}

1503 1504 1505
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1506 1507
}

1508 1509
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1510 1511 1512
  MutexLock l(&mutex_);
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.max_mem_compaction_level;
1513 1514
}

1515 1516
int DBImpl::Level0StopWriteTrigger(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.level0_stop_writes_trigger;
1520 1521
}

L
Lei Jin 已提交
1522
Status DBImpl::Flush(const FlushOptions& flush_options,
1523 1524
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1525
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
1526 1527
}

1528
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1529 1530 1531
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1532
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1533 1534
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1535
  assert(input_level >= 0);
1536

G
Gabor Cselle 已提交
1537 1538
  InternalKey begin_storage, end_storage;

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

  MutexLock l(&mutex_);
1566

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

1590 1591
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
1592
      cfd->GetName().c_str());
1593

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

1609 1610 1611
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1612
  return manual.status;
J
jorlow@chromium.org 已提交
1613 1614
}

1615
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
1616
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
1617 1618 1619 1620
  Status s;
  {
    WriteContext context;
    MutexLock guard_lock(&mutex_);
1621 1622 1623 1624 1625 1626

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

I
Igor Canadi 已提交
1627 1628
    WriteThread::Writer w(&mutex_);
    s = write_thread_.EnterWriteThread(&w, 0);
S
Stanislau Hlebik 已提交
1629 1630 1631 1632 1633 1634 1635 1636
    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 已提交
1637
    write_thread_.ExitWriteThread(&w, &w, s);
S
Stanislau Hlebik 已提交
1638
  }
S
Stanislau Hlebik 已提交
1639

L
Lei Jin 已提交
1640
  if (s.ok() && flush_options.wait) {
1641
    // Wait until the compaction completes
1642
    s = WaitForFlushMemTable(cfd);
1643 1644
  }
  return s;
J
jorlow@chromium.org 已提交
1645 1646
}

1647
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1648 1649 1650
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1651
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1652 1653
    bg_cv_.Wait();
  }
1654
  if (!bg_error_.ok()) {
1655 1656 1657
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1658 1659
}

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

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

1711
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
1712
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
1713 1714 1715
  IOSTATS_RESET(bytes_written);
}

1716
void DBImpl::BGWorkFlush(void* db) {
1717
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
1718 1719 1720 1721
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
1722
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
1723 1724 1725
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
1726
Status DBImpl::BackgroundFlush(bool* madeProgress, JobContext* job_context,
H
Haobo Xu 已提交
1727
                               LogBuffer* log_buffer) {
1728
  mutex_.AssertHeld();
1729 1730 1731 1732 1733

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

1734 1735 1736 1737 1738
  // 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
1739
  for (auto cfd : *versions_->GetColumnFamilySet()) {
1740 1741
    cfd->Ref();
    Status flush_status;
1742 1743
    const MutableCFOptions mutable_cf_options =
      *cfd->GetLatestMutableCFOptions();
1744
    while (flush_status.ok() && cfd->imm()->IsFlushPending()) {
1745 1746
      LogToBuffer(
          log_buffer,
1747
          "BackgroundCallFlush doing FlushMemTableToOutputFile with column "
I
Igor Canadi 已提交
1748 1749
          "family [%s], flush slots available %d",
          cfd->GetName().c_str(),
1750
          db_options_.max_background_flushes - bg_flush_scheduled_);
1751
      flush_status = FlushMemTableToOutputFile(
I
Igor Canadi 已提交
1752
          cfd, mutable_cf_options, madeProgress, job_context, log_buffer);
1753 1754 1755
    }
    if (call_status.ok() && !flush_status.ok()) {
      call_status = flush_status;
1756
    }
1757
    cfd->Unref();
J
jorlow@chromium.org 已提交
1758
  }
1759
  versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
1760
  return call_status;
J
jorlow@chromium.org 已提交
1761 1762
}

1763
void DBImpl::BackgroundCallFlush() {
1764
  bool madeProgress = false;
I
Igor Canadi 已提交
1765
  JobContext job_context(true);
1766 1767
  assert(bg_flush_scheduled_);

1768
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
1769 1770 1771
  {
    MutexLock l(&mutex_);

I
Igor Canadi 已提交
1772 1773 1774
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

H
Haobo Xu 已提交
1775
    Status s;
I
Igor Canadi 已提交
1776
    if (!shutting_down_.load(std::memory_order_acquire)) {
I
Igor Canadi 已提交
1777
      s = BackgroundFlush(&madeProgress, &job_context, &log_buffer);
H
Haobo Xu 已提交
1778 1779 1780 1781 1782
      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.
1783 1784
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
H
Haobo Xu 已提交
1785 1786
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
1787
        Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
1788 1789 1790
            "Waiting after background flush error: %s"
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
H
Haobo Xu 已提交
1791
        log_buffer.FlushBufferToLog();
1792
        LogFlush(db_options_.info_log);
H
Haobo Xu 已提交
1793 1794 1795 1796 1797
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }

I
Igor Canadi 已提交
1798 1799
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

H
Haobo Xu 已提交
1800 1801
    // 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 已提交
1802
    FindObsoleteFiles(&job_context, !s.ok());
H
Haobo Xu 已提交
1803
    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
1804
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
1805
      mutex_.Unlock();
1806 1807 1808 1809 1810
      // 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 已提交
1811
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
1812 1813
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
1814
      }
I
Igor Canadi 已提交
1815
      job_context.Clean();
1816 1817
      mutex_.Lock();
    }
I
Igor Canadi 已提交
1818

H
Haobo Xu 已提交
1819
    bg_flush_scheduled_--;
1820 1821 1822 1823 1824
    // 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 已提交
1825 1826
      MaybeScheduleFlushOrCompaction();
    }
I
Igor Canadi 已提交
1827
    RecordFlushIOStats();
H
Haobo Xu 已提交
1828
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
1829 1830 1831 1832
    // 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.
1833
  }
J
jorlow@chromium.org 已提交
1834 1835
}

1836
void DBImpl::BackgroundCallCompaction() {
1837
  bool madeProgress = false;
I
Igor Canadi 已提交
1838
  JobContext job_context(true);
H
Haobo Xu 已提交
1839 1840

  MaybeDumpStats();
1841
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
1842 1843
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1844 1845 1846 1847

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

1848 1849
    assert(bg_compaction_scheduled_);
    Status s;
I
Igor Canadi 已提交
1850
    if (!shutting_down_.load(std::memory_order_acquire)) {
I
Igor Canadi 已提交
1851
      s = BackgroundCompaction(&madeProgress, &job_context, &log_buffer);
1852 1853 1854 1855 1856
      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.
1857 1858
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
1859 1860
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
H
Haobo Xu 已提交
1861
        log_buffer.FlushBufferToLog();
1862
        Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
1863 1864 1865
            "Waiting after background compaction error: %s, "
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
1866
        LogFlush(db_options_.info_log);
1867 1868 1869 1870
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }
H
Haobo Xu 已提交
1871

I
Igor Canadi 已提交
1872 1873
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

1874 1875
    // 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 已提交
1876 1877 1878
    // FindObsoleteFiles(). This is because job_context does not
    // catch all created files if compaction failed.
    FindObsoleteFiles(&job_context, !s.ok());
1879 1880

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
1881
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
1882
      mutex_.Unlock();
1883 1884 1885 1886 1887
      // 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 已提交
1888
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
1889 1890
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
1891
      }
I
Igor Canadi 已提交
1892
      job_context.Clean();
1893 1894
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
1895

1896
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
1897

1898 1899
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

1900 1901 1902
    // Previous compaction may have produced too many files in a level,
    // So reschedule another compaction if we made progress in the
    // last compaction.
1903 1904 1905 1906 1907 1908
    //
    // 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_) {
1909 1910
      MaybeScheduleFlushOrCompaction();
    }
1911 1912
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
I
Igor Canadi 已提交
1913
      // * madeProgress -- need to wakeup DelayWrite
1914 1915 1916 1917 1918 1919
      // * 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 已提交
1920 1921 1922 1923
    // 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.
1924
  }
J
jorlow@chromium.org 已提交
1925 1926
}

I
Igor Canadi 已提交
1927
Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
1928
                                    LogBuffer* log_buffer) {
1929
  *madeProgress = false;
J
jorlow@chromium.org 已提交
1930
  mutex_.AssertHeld();
1931

1932 1933
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
1934

1935 1936 1937 1938 1939 1940 1941 1942 1943 1944
  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_;
  }

1945 1946 1947
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
1948 1949 1950 1951
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
1952 1953 1954 1955 1956
  }

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

1981 1982 1983
  // 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
1984
  unique_ptr<Compaction> c;
1985 1986
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
1987
  if (is_manual) {
G
Gabor Cselle 已提交
1988
    ManualCompaction* m = manual_compaction_;
1989
    assert(m->in_progress);
1990 1991 1992
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
1993
    if (!c) {
1994
      m->done = true;
G
Gabor Cselle 已提交
1995
    }
I
Igor Canadi 已提交
1996 1997 1998 1999 2000 2001 2002 2003 2004
    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 已提交
2005
  } else {
2006
    // no need to refcount in iteration since it's always under a mutex
I
Igor Canadi 已提交
2007
    for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
2008 2009 2010 2011
      // 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) {
2012 2013 2014
        // 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 已提交
2015
        c.reset(cfd->PickCompaction(*mutable_cf_options, log_buffer));
I
Igor Canadi 已提交
2016 2017
        if (c != nullptr) {
          // update statistics
L
Lei Jin 已提交
2018
          MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
I
Igor Canadi 已提交
2019 2020 2021
                      c->inputs(0)->size());
          break;
        }
I
Igor Canadi 已提交
2022 2023
      }
    }
J
jorlow@chromium.org 已提交
2024 2025 2026
  }

  Status status;
2027
  if (!c) {
H
hans@chromium.org 已提交
2028
    // Nothing to do
2029
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2030 2031 2032 2033 2034
  } 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);
2035
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2036 2037
           kCompactionStyleFIFO);
    for (const auto& f : *c->inputs(0)) {
2038
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2039
    }
2040 2041 2042
    status = versions_->LogAndApply(
        c->column_family_data(), *c->mutable_cf_options(), c->edit(),
        &mutex_, db_directory_.get());
I
Igor Canadi 已提交
2043 2044
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
I
Igor Canadi 已提交
2045 2046 2047 2048 2049
    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 已提交
2050
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
2051
    // Move file to next level
2052
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2053
    FileMetaData* f = c->input(0, 0);
2054
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
2055 2056 2057
    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);
2058 2059 2060
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(),
                                    c->edit(), &mutex_, db_directory_.get());
2061
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2062 2063
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
2064

S
sdong 已提交
2065 2066 2067 2068 2069
    VersionStorageInfo::LevelSummaryStorage tmp;
    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 已提交
2070
                c->input_version()->storage_info()->LevelSummary(&tmp));
I
Igor Canadi 已提交
2071
    c->ReleaseCompactionFiles(status);
2072
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2073
  } else {
2074
    MaybeScheduleFlushOrCompaction(); // do more compaction work in parallel.
I
Igor Canadi 已提交
2075 2076 2077 2078 2079 2080

    auto yield_callback = [&]() {
      return CallFlushDuringCompaction(c->column_family_data(),
                                       *c->mutable_cf_options(), job_context,
                                       log_buffer);
    };
2081 2082 2083 2084 2085
    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 已提交
2086 2087 2088 2089
    compaction_job.Prepare();
    mutex_.Unlock();
    status = compaction_job.Run();
    mutex_.Lock();
2090
    compaction_job.Install(&status, &mutex_);
I
Igor Canadi 已提交
2091 2092 2093 2094
    if (status.ok()) {
      InstallSuperVersionBackground(c->column_family_data(), job_context,
                                    *c->mutable_cf_options());
    }
I
Igor Canadi 已提交
2095
    c->ReleaseCompactionFiles(status);
2096
    c->ReleaseInputs();
2097
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2098
  }
2099
  c.reset();
J
jorlow@chromium.org 已提交
2100 2101 2102

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2103
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2104 2105
    // Ignore compaction errors found during shutting down
  } else {
2106
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2107
        status.ToString().c_str());
2108
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2109 2110 2111
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2112 2113

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

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

2174 2175
namespace {
struct IterState {
I
Igor Canadi 已提交
2176 2177
  IterState(DBImpl* _db, port::Mutex* _mu, SuperVersion* _super_version)
      : db(_db), mu(_mu), super_version(_super_version) {}
2178 2179

  DBImpl* db;
2180
  port::Mutex* mu;
2181
  SuperVersion* super_version;
2182 2183 2184 2185
};

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

2187
  if (state->super_version->Unref()) {
I
Igor Canadi 已提交
2188
    JobContext job_context;
2189

2190 2191
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2192
    state->db->FindObsoleteFiles(&job_context, false, true);
2193 2194 2195
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2196 2197
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2198
    }
I
Igor Canadi 已提交
2199
    job_context.Clean();
I
Igor Canadi 已提交
2200
  }
T
Tomislav Novak 已提交
2201

2202 2203
  delete state;
}
H
Hans Wennborg 已提交
2204
}  // namespace
2205

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

  return internal_iter;
}

2229 2230 2231 2232
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2233
Status DBImpl::Get(const ReadOptions& read_options,
2234
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2235
                   std::string* value) {
L
Lei Jin 已提交
2236
  return GetImpl(read_options, column_family, key, value);
2237 2238
}

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

L
Lei Jin 已提交
2262 2263 2264 2265 2266 2267 2268 2269 2270 2271 2272 2273 2274
SuperVersion* DBImpl::InstallSuperVersion(
    ColumnFamilyData* cfd, SuperVersion* new_sv,
    const MutableCFOptions& mutable_cf_options) {
  mutex_.AssertHeld();
  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_
2275
  size_t old_memtable_size = 0;
L
Lei Jin 已提交
2276 2277 2278 2279 2280 2281 2282 2283 2284 2285 2286 2287
  if (old) {
    old_memtable_size = old->mutable_cf_options.write_buffer_size *
                        old->mutable_cf_options.max_write_buffer_number;
  }
  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,
2288 2289
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
L
Lei Jin 已提交
2290
  StopWatch sw(env_, stats_, DB_GET);
2291
  PERF_TIMER_GUARD(get_snapshot_time);
2292

2293 2294 2295
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2296
  SequenceNumber snapshot;
L
Lei Jin 已提交
2297 2298 2299
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2300 2301
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2302
  }
2303

2304
  // Acquire SuperVersion
2305
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
I
Igor Canadi 已提交
2306

2307
  // Prepare to store a list of merge operations if merge occurs.
2308
  MergeContext merge_context;
2309

2310
  Status s;
2311
  // First look in the memtable, then in the immutable memtable (if any).
2312
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2313
  // merge_operands will contain the sequence of merges in the latter case.
2314
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2315
  PERF_TIMER_STOP(get_snapshot_time);
2316

2317
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2318
    // Done
L
Lei Jin 已提交
2319
    RecordTick(stats_, MEMTABLE_HIT);
2320
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2321
    // Done
L
Lei Jin 已提交
2322
    RecordTick(stats_, MEMTABLE_HIT);
2323
  } else {
2324
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2325 2326
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2327
    RecordTick(stats_, MEMTABLE_MISS);
2328
  }
2329

2330 2331
  {
    PERF_TIMER_GUARD(get_post_process_time);
2332

2333
    ReturnAndCleanupSuperVersion(cfd, sv);
2334

2335 2336 2337
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2338
  return s;
J
jorlow@chromium.org 已提交
2339 2340
}

2341
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
2342
    const ReadOptions& read_options,
2343
    const std::vector<ColumnFamilyHandle*>& column_family,
2344
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
2345

L
Lei Jin 已提交
2346
  StopWatch sw(env_, stats_, DB_MULTIGET);
2347
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
2348

2349
  SequenceNumber snapshot;
2350

2351
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
2352
    ColumnFamilyData* cfd;
2353 2354 2355 2356 2357
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
2358 2359 2360 2361 2362 2363
    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});
2364 2365 2366
    }
  }

2367
  mutex_.Lock();
L
Lei Jin 已提交
2368 2369 2370
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2371 2372 2373
  } else {
    snapshot = versions_->LastSequence();
  }
2374
  for (auto mgd_iter : multiget_cf_data) {
2375 2376
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
2377
  }
2378
  mutex_.Unlock();
2379

2380 2381
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2382

2383
  // Note: this always resizes the values array
2384 2385 2386
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
2387 2388

  // Keep track of bytes that we read for statistics-recording later
2389
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
2390
  PERF_TIMER_STOP(get_snapshot_time);
2391 2392 2393 2394

  // 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.
2395
  // merge_operands will contain the sequence of merges in the latter case.
2396
  for (size_t i = 0; i < num_keys; ++i) {
2397
    merge_context.Clear();
2398
    Status& s = stat_list[i];
2399 2400 2401
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
2402 2403
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
2404 2405 2406
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
2407
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
2408
      // Done
2409
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
2410 2411
      // Done
    } else {
2412
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2413 2414
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
2415 2416 2417
    }

    if (s.ok()) {
2418
      bytes_read += value->size();
2419 2420 2421 2422
    }
  }

  // Post processing (decrement reference counts and record statistics)
2423
  PERF_TIMER_GUARD(get_post_process_time);
2424 2425
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
2426
  // TODO(icanadi) do we need lock here or just around Cleanup()?
2427 2428 2429 2430 2431 2432
  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);
2433 2434
    }
  }
2435 2436 2437 2438 2439 2440 2441
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
2442
  }
2443

L
Lei Jin 已提交
2444 2445 2446
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
2447
  PERF_TIMER_STOP(get_post_process_time);
2448

2449
  return stat_list;
2450 2451
}

L
Lei Jin 已提交
2452
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
2453
                                  const std::string& column_family_name,
2454
                                  ColumnFamilyHandle** handle) {
I
Igor Canadi 已提交
2455 2456 2457
  *handle = nullptr;
  MutexLock l(&mutex_);

I
Igor Canadi 已提交
2458 2459
  if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
      nullptr) {
I
Igor Canadi 已提交
2460 2461
    return Status::InvalidArgument("Column family already exists");
  }
2462
  VersionEdit edit;
2463
  edit.AddColumnFamily(column_family_name);
2464 2465
  uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
  edit.SetColumnFamily(new_id);
I
Igor Canadi 已提交
2466
  edit.SetLogNumber(logfile_number_);
L
Lei Jin 已提交
2467
  edit.SetComparatorName(cf_options.comparator->Name());
2468

I
Igor Canadi 已提交
2469 2470
  // LogAndApply will both write the creation in MANIFEST and create
  // ColumnFamilyData object
L
Lei Jin 已提交
2471
  Options opt(db_options_, cf_options);
2472 2473
  Status s = versions_->LogAndApply(nullptr,
      MutableCFOptions(opt, ImmutableCFOptions(opt)),
L
Lei Jin 已提交
2474
      &edit, &mutex_, db_directory_.get(), false, &cf_options);
2475
  if (s.ok()) {
2476
    single_column_family_mode_ = false;
2477 2478 2479
    auto cfd =
        versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
    assert(cfd != nullptr);
L
Lei Jin 已提交
2480
    delete InstallSuperVersion(cfd, nullptr, *cfd->GetLatestMutableCFOptions());
2481
    *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
2482 2483
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Created column family [%s] (ID %u)",
2484 2485
        column_family_name.c_str(), (unsigned)cfd->GetID());
  } else {
2486 2487
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "Creating column family [%s] FAILED -- %s",
2488 2489
        column_family_name.c_str(), s.ToString().c_str());
  }
2490
  return s;
2491 2492
}

2493 2494 2495 2496
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
2497 2498
    return Status::InvalidArgument("Can't drop default column family");
  }
2499

I
Igor Canadi 已提交
2500 2501
  VersionEdit edit;
  edit.DropColumnFamily();
2502 2503
  edit.SetColumnFamily(cfd->GetID());

2504
  Status s;
2505 2506 2507 2508 2509 2510
  {
    MutexLock l(&mutex_);
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
2511
      // we drop column family from a single write thread
I
Igor Canadi 已提交
2512 2513
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
2514
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
2515 2516
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
I
Igor Canadi 已提交
2517
      write_thread_.ExitWriteThread(&w, &w, s);
2518
    }
2519
  }
2520

2521
  if (s.ok()) {
I
Igor Canadi 已提交
2522
    assert(cfd->IsDropped());
L
Lei Jin 已提交
2523 2524 2525
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
2526 2527
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
2528
        cfd->GetID());
2529
  } else {
2530
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
2531
        "Dropping column family with id %u FAILED -- %s\n",
2532 2533 2534
        cfd->GetID(), s.ToString().c_str());
  }

2535
  return s;
2536 2537
}

L
Lei Jin 已提交
2538
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
2539 2540
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
2541
  if (value_found != nullptr) {
K
Kai Liu 已提交
2542 2543
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
2544
  }
L
Lei Jin 已提交
2545
  ReadOptions roptions = read_options;
2546
  roptions.read_tier = kBlockCacheTier; // read from block cache only
2547
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
2548

2549
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
2550 2551 2552
  // 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();
2553 2554
}

2555
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
2556 2557 2558
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
2559

2560
  if (read_options.tailing) {
I
Igor Canadi 已提交
2561 2562 2563 2564
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
2565 2566
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
2567
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2568 2569 2570
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
2571
#endif
T
Tomislav Novak 已提交
2572
  } else {
2573
    SequenceNumber latest_snapshot = versions_->LastSequence();
2574
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
2575

I
Igor Canadi 已提交
2576
    auto snapshot =
2577 2578 2579
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
2580
            : latest_snapshot;
T
Tomislav Novak 已提交
2581

2582 2583 2584 2585 2586 2587 2588 2589 2590 2591 2592 2593 2594 2595 2596 2597 2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624
    // 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(
2625
        env_, *cfd->ioptions(), cfd->user_comparator(),
2626
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
2627
        read_options.iterate_upper_bound);
2628

2629
    Iterator* internal_iter =
2630
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
2631 2632 2633 2634
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
2635 2636
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
2637 2638
}

2639
Status DBImpl::NewIterators(
2640
    const ReadOptions& read_options,
I
Igor Canadi 已提交
2641
    const std::vector<ColumnFamilyHandle*>& column_families,
2642
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
2643 2644 2645
  iterators->clear();
  iterators->reserve(column_families.size());

2646
  if (read_options.tailing) {
I
Igor Canadi 已提交
2647 2648 2649 2650
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
2651 2652
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
2653 2654
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
2655
      iterators->push_back(
2656
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2657 2658
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
2659
    }
I
Igor Canadi 已提交
2660
#endif
I
Igor Canadi 已提交
2661
  } else {
2662 2663
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
2664
    for (size_t i = 0; i < column_families.size(); ++i) {
2665 2666 2667
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
2668 2669

      auto snapshot =
2670 2671 2672
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
2673 2674
              : latest_snapshot;

2675
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
2676
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
2677
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
2678
      Iterator* internal_iter = NewInternalIterator(
2679
          read_options, cfd, sv, db_iter->GetArena());
2680 2681
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
2682 2683 2684 2685
    }
  }

  return Status::OK();
2686 2687
}

2688 2689 2690 2691 2692 2693 2694 2695 2696
bool DBImpl::IsSnapshotSupported() const {
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (!cfd->mem()->IsSnapshotSupported()) {
      return false;
    }
  }
  return true;
}

J
jorlow@chromium.org 已提交
2697
const Snapshot* DBImpl::GetSnapshot() {
2698
  MutexLock l(&mutex_);
2699 2700
  // returns null if the underlying memtable does not support snapshot.
  if (!IsSnapshotSupported()) return nullptr;
2701
  return snapshots_.New(versions_->LastSequence());
J
jorlow@chromium.org 已提交
2702 2703 2704 2705
}

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

// Convenience methods
2710 2711
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
2712
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
2713 2714
}

2715 2716 2717
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
2718
  if (!cfh->cfd()->ioptions()->merge_operator) {
2719 2720
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
2721
    return DB::Merge(o, column_family, key, val);
2722 2723 2724
  }
}

L
Lei Jin 已提交
2725
Status DBImpl::Delete(const WriteOptions& write_options,
2726
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
2727
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
2728 2729
}

L
Lei Jin 已提交
2730
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
S
Stanislau Hlebik 已提交
2731 2732 2733
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
2734
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
I
Igor Canadi 已提交
2735
  WriteThread::Writer w(&mutex_);
S
Stanislau Hlebik 已提交
2736
  w.batch = my_batch;
L
Lei Jin 已提交
2737 2738
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
2739 2740
  w.in_batch_group = false;
  w.done = false;
L
Lei Jin 已提交
2741
  w.timeout_hint_us = write_options.timeout_hint_us;
S
Stanislau Hlebik 已提交
2742 2743

  uint64_t expiration_time = 0;
I
Igor Canadi 已提交
2744
  bool has_timeout = false;
S
Stanislau Hlebik 已提交
2745
  if (w.timeout_hint_us == 0) {
I
Igor Canadi 已提交
2746
    w.timeout_hint_us = WriteThread::kNoTimeOut;
S
Stanislau Hlebik 已提交
2747 2748
  } else {
    expiration_time = env_->NowMicros() + w.timeout_hint_us;
I
Igor Canadi 已提交
2749
    has_timeout = true;
S
Stanislau Hlebik 已提交
2750 2751
  }

L
Lei Jin 已提交
2752
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
2753 2754 2755 2756 2757 2758
    RecordTick(stats_, WRITE_WITH_WAL);
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_WITH_WAL, 1);
  }

  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
2759
  Status status = write_thread_.EnterWriteThread(&w, expiration_time);
S
Stanislau Hlebik 已提交
2760 2761
  assert(status.ok() || status.IsTimedOut());
  if (status.IsTimedOut()) {
2762
    mutex_.Unlock();
L
Lei Jin 已提交
2763
    RecordTick(stats_, WRITE_TIMEDOUT);
2764
    return Status::TimedOut();
2765
  }
S
Stanislau Hlebik 已提交
2766 2767 2768 2769 2770 2771 2772 2773 2774 2775
  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);
2776

2777 2778 2779 2780
  // 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.
2781 2782 2783
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

2784
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
2785
                                    ? 4 * max_total_in_memory_state_
2786
                                    : db_options_.max_total_wal_size;
2787
  if (UNLIKELY(!single_column_family_mode_) &&
2788
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
2789
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
2790
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
2791
    alive_log_files_.begin()->getting_flushed = true;
2792
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
2793 2794 2795
        "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 已提交
2796 2797
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
2798
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
2799
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
S
Stanislau Hlebik 已提交
2800
        status = SetNewMemtableAndNewLogFile(cfd, &context);
I
Igor Canadi 已提交
2801 2802 2803
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
2804
        cfd->imm()->FlushRequested();
2805
      }
2806
    }
I
Igor Canadi 已提交
2807 2808 2809 2810 2811 2812 2813 2814 2815 2816 2817 2818 2819
    MaybeScheduleFlushOrCompaction();
  }

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

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

  if (UNLIKELY(status.ok()) &&
      (write_controller_.IsStopped() || write_controller_.GetDelay() > 0)) {
I
Igor Canadi 已提交
2820
    status = DelayWrite(expiration_time);
I
Igor Canadi 已提交
2821 2822 2823 2824 2825
  }

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

D
dgrogan@chromium.org 已提交
2828
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
2829
  WriteThread::Writer* last_writer = &w;
S
Stanislau Hlebik 已提交
2830
  if (status.ok()) {
2831
    autovector<WriteBatch*> write_batch_group;
I
Igor Canadi 已提交
2832
    write_thread_.BuildBatchGroup(&last_writer, &write_batch_group);
2833

2834 2835 2836
    // 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
2837
    // into memtables
2838
    {
2839
      mutex_.Unlock();
2840 2841 2842 2843 2844 2845 2846 2847 2848 2849
      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]);
        }
      }

2850 2851 2852 2853
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
2854
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
2855
      // Record statistics
L
Lei Jin 已提交
2856
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
2857
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
2858
      if (write_options.disableWAL) {
2859
        flush_on_destroy_ = true;
2860
      }
L
Lei Jin 已提交
2861
      PERF_TIMER_STOP(write_pre_and_post_process_time);
2862

2863
      uint64_t log_size = 0;
L
Lei Jin 已提交
2864
      if (!write_options.disableWAL) {
2865
        PERF_TIMER_GUARD(write_wal_time);
2866 2867
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
2868 2869
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
2870
        log_empty_ = false;
2871
        log_size = log_entry.size();
L
Lei Jin 已提交
2872
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
L
Lei Jin 已提交
2873
        if (status.ok() && write_options.sync) {
I
Igor Canadi 已提交
2874
          RecordTick(stats_, WAL_FILE_SYNCED);
2875
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
2876
          if (db_options_.use_fsync) {
2877
            status = log_->file()->Fsync();
2878
          } else {
2879
            status = log_->file()->Sync();
2880
          }
H
heyongqiang 已提交
2881
        }
2882 2883
      }
      if (status.ok()) {
2884
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
2885

2886
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
2887
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
2888
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
2889 2890 2891 2892 2893 2894 2895 2896
        // 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 已提交
2897
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
2898
      }
L
Lei Jin 已提交
2899
      PERF_TIMER_START(write_pre_and_post_process_time);
2900 2901 2902
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
2903
      mutex_.Lock();
2904 2905 2906
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
2907 2908
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
2909
      if (!write_options.disableWAL) {
2910 2911 2912 2913 2914
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
2915
      if (status.ok()) {
2916
        versions_->SetLastSequence(last_sequence);
2917
      }
J
jorlow@chromium.org 已提交
2918 2919
    }
  }
2920
  if (db_options_.paranoid_checks && !status.ok() &&
2921
      !status.IsTimedOut() && bg_error_.ok()) {
I
Igor Canadi 已提交
2922 2923
    bg_error_ = status; // stop compaction & fail any further writes
  }
2924

I
Igor Canadi 已提交
2925
  write_thread_.ExitWriteThread(&w, last_writer, status);
I
Igor Canadi 已提交
2926
  mutex_.Unlock();
2927

2928
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
2929
    RecordTick(stats_, WRITE_TIMEDOUT);
2930 2931
  }

J
jorlow@chromium.org 已提交
2932 2933 2934
  return status;
}

2935 2936
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
2937
Status DBImpl::DelayWrite(uint64_t expiration_time) {
2938 2939 2940 2941 2942
  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();
2943 2944
    // hopefully we don't have to sleep more than 2 billion microseconds
    env_->SleepForMicroseconds(static_cast<int>(delay));
2945 2946 2947
    mutex_.Lock();
  }

I
Igor Canadi 已提交
2948
  while (bg_error_.ok() && write_controller_.IsStopped()) {
2949 2950 2951
    if (has_timeout) {
      bg_cv_.TimedWait(expiration_time);
      if (env_->NowMicros() > expiration_time) {
I
Igor Canadi 已提交
2952
        return Status::TimedOut();
2953 2954 2955 2956 2957
      }
    } else {
      bg_cv_.Wait();
    }
  }
I
Igor Canadi 已提交
2958 2959

  return bg_error_;
2960 2961
}

I
Igor Canadi 已提交
2962 2963 2964 2965 2966 2967 2968 2969 2970 2971 2972
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 已提交
2973 2974
    }
  }
I
Igor Canadi 已提交
2975 2976 2977 2978
  if (schedule_bg_work) {
    MaybeScheduleFlushOrCompaction();
  }
  return Status::OK();
S
Stanislau Hlebik 已提交
2979 2980 2981 2982 2983 2984 2985 2986 2987 2988 2989 2990 2991
}

// 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.
2992
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
2993 2994 2995 2996
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
2997
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
2998 2999 3000 3001
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
3002 3003 3004
      s = env_->NewWritableFile(
          LogFileName(db_options_.wal_dir, new_log_number),
          &lfile, env_->OptimizeForLogWrite(env_options_));
S
Stanislau Hlebik 已提交
3005 3006 3007
      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 已提交
3008 3009
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
S
Stanislau Hlebik 已提交
3010
        new_log = new log::Writer(std::move(lfile));
I
Igor Canadi 已提交
3011
      }
S
Stanislau Hlebik 已提交
3012 3013 3014
    }

    if (s.ok()) {
L
Lei Jin 已提交
3015 3016
      new_mem = new MemTable(cfd->internal_comparator(), *cfd->ioptions(),
                             mutable_cf_options);
S
Stanislau Hlebik 已提交
3017 3018 3019
      new_superversion = new SuperVersion();
    }
  }
3020
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
3021 3022
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
3023 3024 3025 3026 3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037
  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 已提交
3038
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
3039 3040 3041 3042
      // 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 已提交
3043 3044 3045
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
          loop_cfd->imm()->size() == 0) {
        loop_cfd->SetLogNumber(logfile_number_);
3046
      }
3047 3048
    }
  }
S
Stanislau Hlebik 已提交
3049 3050 3051 3052 3053
  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 已提交
3054
      InstallSuperVersion(cfd, new_superversion, mutable_cf_options));
3055 3056 3057
  return s;
}

I
Igor Canadi 已提交
3058
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
3059 3060 3061 3062 3063
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3064 3065
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
3066
  auto version = cfd->current();
3067 3068 3069 3070 3071 3072 3073 3074 3075 3076 3077 3078
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
3079
#endif  // ROCKSDB_LITE
3080

I
Igor Canadi 已提交
3081 3082 3083 3084
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3085 3086 3087 3088
Env* DBImpl::GetEnv() const {
  return env_;
}

3089 3090
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
3091
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
3092 3093
}

3094
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
3095
                         const Slice& property, std::string* value) {
3096 3097
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3098 3099 3100
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

3101
  value->clear();
3102 3103 3104 3105 3106 3107 3108 3109 3110 3111 3112 3113 3114 3115 3116
  if (is_int_property) {
    uint64_t int_value;
    bool ret_value = GetIntPropertyInternal(column_family, property_type,
                                            need_out_of_mutex, &int_value);
    if (ret_value) {
      *value = std::to_string(int_value);
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
    MutexLock l(&mutex_);
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
3117 3118
}

3119 3120
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
3121 3122
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3123 3124 3125 3126 3127 3128 3129 3130 3131 3132 3133 3134
  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) {
3135 3136
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
3137 3138 3139

  if (!need_out_of_mutex) {
    MutexLock l(&mutex_);
3140
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
3141 3142 3143 3144 3145 3146 3147 3148 3149 3150 3151 3152 3153 3154
  } 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 已提交
3155
  return cfd->GetThreadLocalSuperVersion(&mutex_);
3156 3157 3158 3159
}

void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
3160
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3161 3162 3163 3164 3165 3166 3167 3168 3169 3170 3171 3172 3173

  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);
  }
3174 3175
}

3176
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
3177
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
3178 3179
  // TODO(opt): better implementation
  Version* v;
3180 3181
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
3182 3183
  {
    MutexLock l(&mutex_);
3184
    v = cfd->current();
3185
    v->Ref();
J
jorlow@chromium.org 已提交
3186 3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202
  }

  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 已提交
3203 3204 3205 3206 3207 3208 3209 3210 3211 3212 3213 3214 3215 3216 3217 3218
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 已提交
3219 3220 3221 3222 3223
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
3224
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
3225 3226 3227
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
3228
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
3229 3230
}

3231 3232 3233
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
3234 3235 3236
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
3237 3238
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
3239 3240 3241
    return Status::InvalidArgument("Invalid file name");
  }

3242 3243 3244 3245
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
3246 3247
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
3248
          name.c_str());
3249 3250
      return Status::NotSupported("Delete only supported for archived logs");
    }
3251
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
3252
    if (!status.ok()) {
3253 3254
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
3255
          name.c_str(), status.ToString().c_str());
3256 3257 3258 3259
    }
    return status;
  }

3260
  int level;
I
Igor Canadi 已提交
3261
  FileMetaData* metadata;
3262
  ColumnFamilyData* cfd;
3263
  VersionEdit edit;
I
Igor Canadi 已提交
3264
  JobContext job_context(true);
D
Dhruba Borthakur 已提交
3265 3266
  {
    MutexLock l(&mutex_);
3267
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
3268
    if (!status.ok()) {
3269 3270
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
D
Dhruba Borthakur 已提交
3271 3272
      return Status::InvalidArgument("File not found");
    }
3273
    assert(level < cfd->NumberLevels());
3274

D
Dhruba Borthakur 已提交
3275
    // If the file is being compacted no need to delete.
3276
    if (metadata->being_compacted) {
3277
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3278
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
D
Dhruba Borthakur 已提交
3279
      return Status::OK();
3280 3281
    }

D
Dhruba Borthakur 已提交
3282 3283 3284
    // 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 已提交
3285
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
3286
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
3287
      if (vstoreage->NumLevelFiles(i) != 0) {
3288
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
3289
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
D
Dhruba Borthakur 已提交
3290 3291 3292
        return Status::InvalidArgument("File not in last level");
      }
    }
3293
    // if level == 0, it has to be the oldest file
S
sdong 已提交
3294 3295
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
3296 3297
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
3298
          " target file in level 0 must be the oldest.", name.c_str());
3299 3300 3301
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
3302
    edit.DeleteFile(level, number);
3303 3304
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                    &edit, &mutex_, db_directory_.get());
I
Igor Canadi 已提交
3305
    if (status.ok()) {
I
Igor Canadi 已提交
3306 3307
      InstallSuperVersionBackground(cfd, &job_context,
                                    *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
3308
    }
I
Igor Canadi 已提交
3309 3310
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
3311
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
3312
  // remove files outside the db-lock
I
Igor Canadi 已提交
3313 3314
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
3315
  }
I
Igor Canadi 已提交
3316
  job_context.Clean();
3317 3318 3319 3320 3321 3322
  {
    MutexLock l(&mutex_);
    // schedule flush if file deletion means we freed the space for flushes to
    // continue
    MaybeScheduleFlushOrCompaction();
  }
3323 3324 3325
  return status;
}

3326
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
3327
  MutexLock l(&mutex_);
3328
  versions_->GetLiveFilesMetaData(metadata);
3329
}
3330 3331 3332 3333 3334 3335 3336 3337 3338 3339 3340

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 已提交
3341
#endif  // ROCKSDB_LITE
3342

I
Igor Canadi 已提交
3343 3344 3345 3346 3347 3348 3349
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
3352 3353 3354 3355 3356 3357
    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) {
3358
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
3359 3360 3361 3362 3363 3364 3365 3366 3367 3368 3369 3370
                             ". Size recorded in manifest " +
                             std::to_string(md.size) + ", actual size " +
                             std::to_string(fsize) + "\n";
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

3371 3372 3373 3374 3375 3376 3377 3378 3379 3380 3381 3382 3383 3384 3385
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;
3386
  s = idfile->Read(static_cast<size_t>(file_size), &id, buffer);
3387 3388 3389 3390 3391 3392 3393 3394 3395 3396 3397
  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 已提交
3398 3399
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
3400
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
3401
               const Slice& key, const Slice& value) {
3402 3403 3404 3405
  // 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);
3406
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
3407 3408 3409
  return Write(opt, &batch);
}

3410 3411
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
3412
  WriteBatch batch;
3413
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
3414 3415 3416
  return Write(opt, &batch);
}

3417 3418
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
3419
  WriteBatch batch;
3420
  batch.Merge(column_family, key, value);
3421 3422 3423
  return Write(opt, &batch);
}

3424
// Default implementation -- returns not supported status
L
Lei Jin 已提交
3425
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3426
                              const std::string& column_family_name,
3427
                              ColumnFamilyHandle** handle) {
3428
  return Status::NotSupported("");
3429
}
3430
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
3431
  return Status::NotSupported("");
3432 3433
}

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

J
Jim Paton 已提交
3436
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
3437 3438 3439 3440
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
3441
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
3442
  std::vector<ColumnFamilyHandle*> handles;
3443
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
3444 3445 3446 3447 3448 3449 3450
  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;
3451 3452
}

3453 3454
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
3455
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
3456
  Status s = SanitizeOptionsByTable(db_options, column_families);
3457 3458 3459
  if (!s.ok()) {
    return s;
  }
3460
  if (db_options.db_paths.size() > 1) {
3461 3462 3463 3464 3465 3466 3467 3468 3469 3470 3471 3472
    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. ");
    }
3473 3474
  }

3475
  *dbptr = nullptr;
3476
  handles->clear();
J
jorlow@chromium.org 已提交
3477

I
Igor Canadi 已提交
3478 3479 3480 3481
  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);
3482
  }
3483

I
Igor Canadi 已提交
3484
  DBImpl* impl = new DBImpl(db_options, dbname);
3485
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
3486
  if (s.ok()) {
3487
    for (auto db_path : impl->db_options_.db_paths) {
3488
      s = impl->env_->CreateDirIfMissing(db_path.path);
3489 3490 3491 3492 3493 3494
      if (!s.ok()) {
        break;
      }
    }
  }

3495 3496 3497 3498 3499 3500
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
3501 3502 3503 3504
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
3505
  impl->mutex_.Lock();
3506 3507
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
3508
  if (s.ok()) {
3509
    uint64_t new_log_number = impl->versions_->NewFileNumber();
3510
    unique_ptr<WritableFile> lfile;
3511
    EnvOptions soptions(db_options);
3512 3513 3514
    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 已提交
3515
    if (s.ok()) {
3516
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
3517
      impl->logfile_number_ = new_log_number;
3518
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
3519

3520 3521
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
3522 3523
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
3524 3525 3526 3527 3528 3529 3530 3531 3532 3533 3534 3535 3536 3537 3538 3539 3540 3541 3542
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
        } else {
          if (db_options.create_missing_column_families) {
            // missing column family, create it
            ColumnFamilyHandle* handle;
            impl->mutex_.Unlock();
            s = impl->CreateColumnFamily(cf.options, cf.name, &handle);
            impl->mutex_.Lock();
            if (s.ok()) {
              handles->push_back(handle);
            } else {
              break;
            }
          } else {
            s = Status::InvalidArgument("Column family not found: ", cf.name);
            break;
          }
I
Igor Canadi 已提交
3543
        }
3544
      }
I
Igor Canadi 已提交
3545 3546
    }
    if (s.ok()) {
3547
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
3548 3549
        delete impl->InstallSuperVersion(
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
3550
      }
I
Igor Canadi 已提交
3551 3552
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
3553
      impl->DeleteObsoleteFiles();
3554
      impl->MaybeScheduleFlushOrCompaction();
3555
      s = impl->db_directory_->Fsync();
J
jorlow@chromium.org 已提交
3556 3557
    }
  }
3558

I
Igor Canadi 已提交
3559 3560
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
3561 3562
      if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
3563
        auto* vstorage = cfd->current()->storage_info();
3564
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
3565
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
3566
          if (num_files > 0) {
I
Igor Canadi 已提交
3567 3568 3569
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
                "open with universal or FIFO compaction style.");
I
Igor Canadi 已提交
3570 3571 3572 3573
            break;
          }
        }
      }
3574
      if (cfd->ioptions()->merge_operator != nullptr &&
3575 3576 3577 3578 3579
          !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 已提交
3580
      if (!s.ok()) {
3581 3582 3583 3584 3585
        break;
      }
    }
  }

3586 3587
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
3588
  if (s.ok()) {
3589
    impl->opened_successfully_ = true;
J
jorlow@chromium.org 已提交
3590
    *dbptr = impl;
3591 3592 3593 3594 3595 3596 3597 3598 3599
    // TODO(yhchiang): Add NotifyOnDatabaseOpen() here.
    // Since the column-family handles are only available after DB::Open(),
    // typically developers will need to pass the returned ColumnFamilyHandles
    // to their EventListeners in order to maintain the mapping between
    // column-family-name to ColumnFamilyHandle.  However, some database
    // events might happen before the user passing those ColumnFamilyHandle to
    // their Listeners.  To address this, we should have NotifyOnDatabaseOpen()
    // here which passes the created ColumnFamilyHandle to the Listeners
    // as the first event after DB::Open().
J
jorlow@chromium.org 已提交
3600
  } else {
3601 3602 3603
    for (auto h : *handles) {
      delete h;
    }
3604
    handles->clear();
J
jorlow@chromium.org 已提交
3605 3606 3607 3608 3609
    delete impl;
  }
  return s;
}

3610 3611 3612
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
3613
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
3614 3615
}

3616 3617 3618
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
3619
Status DestroyDB(const std::string& dbname, const Options& options) {
3620
  const InternalKeyComparator comparator(options.comparator);
3621
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
3622
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
3623
  std::vector<std::string> filenames;
3624 3625
  std::vector<std::string> archiveFiles;

3626
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
3627 3628
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
3629 3630 3631 3632 3633 3634 3635

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

J
jorlow@chromium.org 已提交
3637 3638 3639 3640 3641
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
3642 3643
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
3644 3645 3646
  if (result.ok()) {
    uint64_t number;
    FileType type;
3647
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
3648
    for (size_t i = 0; i < filenames.size(); i++) {
3649
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
3650
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
3651 3652 3653
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
3654 3655
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
3656 3657 3658
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
3659 3660 3661 3662 3663
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3664

3665 3666
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
3667 3668 3669
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
3670
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
3671 3672 3673 3674 3675 3676 3677
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

3678
    env->GetChildren(archivedir, &archiveFiles);
3679 3680
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
3681 3682
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
3683
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
3684 3685 3686 3687 3688
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3689
    // ignore case where no archival directory is present.
3690
    env->DeleteDir(archivedir);
3691

J
jorlow@chromium.org 已提交
3692
    env->UnlockFile(lock);  // Ignore error since state is already gone
3693
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
3694
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
3695
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
3696 3697 3698 3699
  }
  return result;
}

3700 3701
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
3702
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
3703
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
3704 3705
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
3706 3707
  Log(InfoLogLevel::INFO_LEVEL, log,
      "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
I
Igor Canadi 已提交
3708
      ROCKSDB_PATCH);
3709 3710
  Log(InfoLogLevel::INFO_LEVEL, log, "Git sha %s", rocksdb_build_git_sha);
  Log(InfoLogLevel::INFO_LEVEL, log, "Compile time %s %s",
3711
      rocksdb_build_compile_time, rocksdb_build_compile_date);
I
Igor Canadi 已提交
3712
#endif
3713 3714
}

3715
}  // namespace rocksdb