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

#include "db/db_impl.h"

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

J
jorlow@chromium.org 已提交
151 152 153
  return result;
}

154 155
namespace {

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

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

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

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

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

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

240
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
241 242
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
I
Igor Canadi 已提交
243 244
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
245

I
Igor Canadi 已提交
246
  DumpRocksDBBuildVersion(db_options_.info_log.get());
247 248
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
249

250
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
251 252 253
}

DBImpl::~DBImpl() {
Y
Yueh-Hsuan Chiang 已提交
254
  EraseThreadStatusDbInfo();
255
  mutex_.Lock();
256

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

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

278 279 280 281 282 283 284 285 286 287 288 289 290
  while (!flush_queue_.empty()) {
    auto cfd = PopFirstFromFlushQueue();
    if (cfd->Unref()) {
      delete cfd;
    }
  }
  while (!compaction_queue_.empty()) {
    auto cfd = PopFirstFromCompactionQueue();
    if (cfd->Unref()) {
      delete cfd;
    }
  }

I
Igor Canadi 已提交
291 292 293 294 295
  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();
296 297
  }

I
Igor Canadi 已提交
298 299 300 301 302 303 304 305 306 307
  // 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 已提交
308 309
    JobContext job_context;
    FindObsoleteFiles(&job_context, true);
I
Igor Canadi 已提交
310
    // manifest number starting from 2
I
Igor Canadi 已提交
311 312 313
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
314
    }
I
Igor Canadi 已提交
315
    job_context.Clean();
316 317
  }

318
  // versions need to be destroyed before table_cache since it can hold
319 320
  // references to table_cache.
  versions_.reset();
321
  mutex_.Unlock();
I
Igor Canadi 已提交
322 323 324
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
325

326
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
327 328 329
}

Status DBImpl::NewDB() {
330
  VersionEdit new_db;
331
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
332 333 334
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

335 336
  Log(InfoLogLevel::INFO_LEVEL,
      db_options_.info_log, "Creating manifest 1 \n");
J
jorlow@chromium.org 已提交
337
  const std::string manifest = DescriptorFileName(dbname_, 1);
338
  unique_ptr<WritableFile> file;
I
Igor Canadi 已提交
339
  Status s = env_->NewWritableFile(
L
Lei Jin 已提交
340
      manifest, &file, env_->OptimizeForManifestWrite(env_options_));
J
jorlow@chromium.org 已提交
341 342 343
  if (!s.ok()) {
    return s;
  }
344
  file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
J
jorlow@chromium.org 已提交
345
  {
346
    log::Writer log(std::move(file));
J
jorlow@chromium.org 已提交
347 348 349 350 351 352
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
353
    s = SetCurrentFile(env_, dbname_, 1, db_directory_.get());
J
jorlow@chromium.org 已提交
354 355 356 357 358 359 360
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
361
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
362 363
    // No change needed
  } else {
364 365
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
366 367 368 369
    *s = Status::OK();
  }
}

370
const Status DBImpl::CreateArchivalDirectory() {
371 372
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
373 374 375 376 377
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

378
void DBImpl::PrintStatistics() {
379
  auto dbstats = db_options_.statistics.get();
380
  if (dbstats) {
381
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
382 383
        "STATISTCS:\n %s",
        dbstats->ToString().c_str());
384 385 386
  }
}

387
void DBImpl::MaybeDumpStats() {
388
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
389 390 391 392

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
393
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
394 395 396 397 398 399
      <= 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;
400

401 402 403 404 405 406
    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 已提交
407
    std::string stats;
408 409 410
    {
      MutexLock l(&mutex_);
      for (auto cfd : *versions_->GetColumnFamilySet()) {
411 412
        cfd->internal_stats()->GetStringProperty(cf_property_type,
                                                 "rocksdb.cfstats", &stats);
413
      }
414 415
      default_cf_internal_stats_->GetStringProperty(db_property_type,
                                                    "rocksdb.dbstats", &stats);
416
    }
417 418 419 420
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "------- DUMPING STATS -------");
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "%s", stats.c_str());
421

422
    PrintStatistics();
423 424 425
  }
}

I
Igor Canadi 已提交
426 427 428 429
// If it's doing full scan:
// * Returns the list of live files in 'full_scan_sst_live' and the list
// of all files in the filesystem in 'full_scan_candidate_files'.
// Otherwise, gets obsolete files from VersionSet.
I
Igor Canadi 已提交
430 431
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
432
//  db_options_.delete_obsolete_files_period_micros
I
Igor Canadi 已提交
433
// force = true -- force the full scan
I
Igor Canadi 已提交
434
void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
I
Igor Canadi 已提交
435
                               bool no_full_scan) {
I
Igor Canadi 已提交
436
  // TODO(icanadi) clean up FindObsoleteFiles, no need to do full scans anymore
D
Dhruba Borthakur 已提交
437 438
  mutex_.AssertHeld();

439
  // if deletion is disabled, do nothing
440
  if (disable_delete_obsolete_files_ > 0) {
441 442 443
    return;
  }

444 445 446 447 448
  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;
449
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
450 451 452
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
I
Igor Canadi 已提交
453
    if (delete_obsolete_files_next_run_ < now_micros) {
454
      doing_the_full_scan = true;
I
Igor Canadi 已提交
455 456
      delete_obsolete_files_next_run_ =
          now_micros + db_options_.delete_obsolete_files_period_micros;
457 458 459
    }
  }

I
Igor Canadi 已提交
460
  // get obsolete files
I
Igor Canadi 已提交
461
  versions_->GetObsoleteFiles(&job_context->sst_delete_files);
I
Igor Canadi 已提交
462

I
Igor Canadi 已提交
463
  // store the current filenum, lognum, etc
464
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
465
  job_context->pending_manifest_file_number =
466
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
467
  job_context->log_number = versions_->MinLogNumber();
468
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
469

470
  // don't delete live files
I
Igor Canadi 已提交
471 472 473 474 475
  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();
476
  }
I
Igor Canadi 已提交
477

478
  if (doing_the_full_scan) {
I
Igor Canadi 已提交
479 480 481 482 483 484 485 486
    // Here we find all files in the DB directory and all the live files. In the
    // DeleteObsoleteFiles(), we will calculate a set difference (all_files -
    // live_files) and delete all files in that difference. If we're not doing
    // the full scan we don't need to get live files, because all files returned
    // by GetObsoleteFiles() will be dead (and need to be deleted)
    versions_->AddLiveFiles(&job_context->full_scan_sst_live);
    for (uint32_t path_id = 0; path_id < db_options_.db_paths.size();
         path_id++) {
487 488 489
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
490
      env_->GetChildren(db_options_.db_paths[path_id].path,
491
                        &files);  // Ignore errors
492
      for (std::string file : files) {
I
Igor Canadi 已提交
493
        // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
I
Igor Canadi 已提交
494 495
        job_context->full_scan_candidate_files.emplace_back("/" + file,
                                                            path_id);
496 497
      }
    }
498 499

    //Add log files in wal_dir
500
    if (db_options_.wal_dir != dbname_) {
501
      std::vector<std::string> log_files;
502
      env_->GetChildren(db_options_.wal_dir, &log_files);  // Ignore errors
503
      for (std::string log_file : log_files) {
I
Igor Canadi 已提交
504
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
505 506
      }
    }
507
    // Add info log files in db_log_dir
508
    if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) {
509
      std::vector<std::string> info_log_files;
510 511
      // Ignore errors
      env_->GetChildren(db_options_.db_log_dir, &info_log_files);
512
      for (std::string log_file : info_log_files) {
I
Igor Canadi 已提交
513
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
514 515
      }
    }
516
  }
517 518
}

519
namespace {
I
Igor Canadi 已提交
520 521
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
522 523 524 525 526
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
527
    return (first.path_id > second.path_id);
528 529 530 531
  }
}
};  // namespace

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

I
Igor Canadi 已提交
540 541 542 543
  // 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 已提交
544 545
    return;
  }
546

547
  // Now, convert live list to an unordered map, WITHOUT mutex held;
548
  // set is slow.
549
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
I
Igor Canadi 已提交
550
  for (const FileDescriptor& fd : state.full_scan_sst_live) {
551 552
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
553

I
Igor Canadi 已提交
554
  auto candidate_files = state.full_scan_candidate_files;
I
Igor Canadi 已提交
555 556 557
  candidate_files.reserve(candidate_files.size() +
                          state.sst_delete_files.size() +
                          state.log_delete_files.size());
K
kailiu 已提交
558 559
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
560
  for (auto file : state.sst_delete_files) {
561 562 563
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
564
    delete file;
I
Igor Canadi 已提交
565 566
  }

K
kailiu 已提交
567 568
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
569 570
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
571 572
    }
  }
573

K
kailiu 已提交
574
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
575
  // file twice
576
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
577 578
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
579

580
  std::vector<std::string> old_info_log_files;
581
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
582 583 584
  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 已提交
585 586 587
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
588
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
589 590
      continue;
    }
J
jorlow@chromium.org 已提交
591

K
kailiu 已提交
592 593 594 595 596 597 598 599
    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'
600
        // (can happen during manifest roll)
K
kailiu 已提交
601 602 603
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
I
Igor Canadi 已提交
604 605 606 607
        // 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 已提交
608 609 610
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
611 612 613 614
        // 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
615
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
616
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
617 618 619 620
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
621
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
622
        }
K
kailiu 已提交
623 624 625 626 627 628 629 630 631 632 633 634 635
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

636
    std::string fname;
K
kailiu 已提交
637 638
    if (type == kTableFile) {
      // evict from cache
639
      TableCache::Evict(table_cache_.get(), number);
640
      fname = TableFileName(db_options_.db_paths, number, path_id);
641
    } else {
642 643
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
644
    }
645

I
Igor Canadi 已提交
646 647
#ifdef ROCKSDB_LITE
    Status s = env_->DeleteFile(fname);
648 649
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "Delete %s type=%d #%" PRIu64 " -- %s\n",
I
Igor Canadi 已提交
650 651 652 653 654
        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 已提交
655 656
    } else {
      Status s = env_->DeleteFile(fname);
657 658
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "Delete %s type=%d #%" PRIu64 " -- %s\n",
659
          fname.c_str(), type, number, s.ToString().c_str());
J
jorlow@chromium.org 已提交
660
    }
I
Igor Canadi 已提交
661
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
662
  }
H
heyongqiang 已提交
663

664
  // Delete old info log files.
665
  size_t old_info_log_file_count = old_info_log_files.size();
666
  if (old_info_log_file_count >= db_options_.keep_log_file_num) {
667
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
668
    size_t end = old_info_log_file_count - db_options_.keep_log_file_num;
669
    for (unsigned int i = 0; i <= end; i++) {
670
      std::string& to_delete = old_info_log_files.at(i);
671 672
      std::string full_path_to_delete = (db_options_.db_log_dir.empty() ?
           dbname_ : db_options_.db_log_dir) + "/" + to_delete;
673 674
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
          "Delete info log file %s\n",
675 676
          full_path_to_delete.c_str());
      Status s = env_->DeleteFile(full_path_to_delete);
677
      if (!s.ok()) {
678 679
        Log(InfoLogLevel::ERROR_LEVEL,
            db_options_.info_log, "Delete info log file %s FAILED -- %s\n",
680 681
            to_delete.c_str(), s.ToString().c_str());
      }
H
heyongqiang 已提交
682 683
    }
  }
I
Igor Canadi 已提交
684 685 686
#ifndef ROCKSDB_LITE
  wal_manager_.PurgeObsoleteWALFiles();
#endif  // ROCKSDB_LITE
687
  LogFlush(db_options_.info_log);
D
Dhruba Borthakur 已提交
688 689 690 691
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
I
Igor Canadi 已提交
692 693 694 695
  JobContext job_context;
  FindObsoleteFiles(&job_context, true);
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
696
  }
I
Igor Canadi 已提交
697
  job_context.Clean();
698 699
}

700
Status DBImpl::Recover(
701 702
    const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
    bool error_if_log_file_exist) {
J
jorlow@chromium.org 已提交
703 704
  mutex_.AssertHeld();

705
  bool is_new_db = false;
706
  assert(db_lock_ == nullptr);
I
Igor Canadi 已提交
707
  if (!read_only) {
708 709 710 711 712 713 714 715 716 717 718 719
    // 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;
    }

720
    for (auto& db_path : db_options_.db_paths) {
721
      s = env_->CreateDirIfMissing(db_path.path);
722 723 724 725 726
      if (!s.ok()) {
        return s;
      }
    }

727 728 729 730 731
    s = env_->NewDirectory(dbname_, &db_directory_);
    if (!s.ok()) {
      return s;
    }

732
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
733 734 735
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
736

737
    if (!env_->FileExists(CurrentFileName(dbname_))) {
738
      if (db_options_.create_if_missing) {
739
        s = NewDB();
740
        is_new_db = true;
741 742 743 744 745 746
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
747 748
      }
    } else {
749
      if (db_options_.error_if_exists) {
750 751 752
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
753
    }
M
Mayank Agarwal 已提交
754 755 756 757 758 759 760
    // 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 已提交
761 762
  }

763
  Status s = versions_->Recover(column_families, read_only);
764
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
765 766
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
767 768
  if (s.ok()) {
    SequenceNumber max_sequence(0);
769 770
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
771
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
772 773
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
774 775 776 777 778

    // 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).
    //
779
    // Note that prev_log_number() is no longer used, but we pay
780
    // attention to it in case we are recovering a database
781
    // produced by an older version of rocksdb.
782
    const uint64_t min_log = versions_->MinLogNumber();
783
    const uint64_t prev_log = versions_->prev_log_number();
784
    std::vector<std::string> filenames;
785
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
786 787
    if (!s.ok()) {
      return s;
788
    }
K
kailiu 已提交
789

790 791
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
792 793
      uint64_t number;
      FileType type;
794 795 796 797 798 799 800 801 802
      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);
        }
803
      }
J
jorlow@chromium.org 已提交
804
    }
805

H
heyongqiang 已提交
806 807 808 809 810 811
    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 已提交
812 813 814 815 816 817 818
    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 已提交
819
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
S
Stanislau Hlebik 已提交
820 821
        }
      }
822
    }
L
Lei Jin 已提交
823
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
824 825
  }

L
Lei Jin 已提交
826 827
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
828
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
829 830 831
    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 已提交
832 833
  }

J
jorlow@chromium.org 已提交
834 835 836
  return s;
}

S
Stanislau Hlebik 已提交
837 838 839
// 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 已提交
840 841
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
842
    Logger* info_log;
J
jorlow@chromium.org 已提交
843
    const char* fname;
844 845
    Status* status;  // nullptr if db_options_.paranoid_checks==false or
                     //            db_options_.skip_log_error_on_recovery==true
J
jorlow@chromium.org 已提交
846
    virtual void Corruption(size_t bytes, const Status& s) {
847 848
      Log(InfoLogLevel::WARN_LEVEL,
          info_log, "%s%s: dropping %d bytes; %s",
849
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
850
          fname, static_cast<int>(bytes), s.ToString().c_str());
851
      if (this->status != nullptr && this->status->ok()) *this->status = s;
J
jorlow@chromium.org 已提交
852 853 854 855
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
856
  Status status;
857
  std::unordered_map<int, VersionEdit> version_edits;
858
  // no need to refcount because iteration is under mutex
859 860
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
861 862
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
863
  }
I
Igor Canadi 已提交
864

S
Stanislau Hlebik 已提交
865 866 867 868
  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.
869
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
870 871 872 873
    // 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 已提交
874
    if (!status.ok()) {
S
Stanislau Hlebik 已提交
875 876 877 878 879 880 881 882
      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 已提交
883 884
    }

S
Stanislau Hlebik 已提交
885 886 887 888 889 890 891 892 893 894 895 896 897 898 899
    // 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*/);
900 901
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "Recovering log #%" PRIu64 "", log_number);
S
Stanislau Hlebik 已提交
902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933

    // 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 已提交
934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950
        // 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;
951
          }
L
Lei Jin 已提交
952
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
953
        }
J
jorlow@chromium.org 已提交
954 955 956
      }
    }

I
Igor Canadi 已提交
957
    flush_scheduler_.Clear();
S
Stanislau Hlebik 已提交
958 959 960
    if (versions_->LastSequence() < *max_sequence) {
      versions_->SetLastSequence(*max_sequence);
    }
961 962
  }

963
  if (!read_only) {
964 965
    // 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 已提交
966
    auto max_log_number = log_numbers.back();
967
    for (auto cfd : *versions_->GetColumnFamilySet()) {
968
      auto iter = version_edits.find(cfd->GetID());
969 970 971
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
972
      if (cfd->GetLogNumber() > max_log_number) {
973
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
974
        // from all logs. Memtable has to be empty because
975
        // we filter the updates based on log_number
976
        // (in WriteBatch::InsertInto)
977 978 979 980 981 982 983
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
984
        status = WriteLevel0TableForRecovery(cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
985 986 987 988
        if (!status.ok()) {
          // Recovery failed
          break;
        }
L
Lei Jin 已提交
989
        cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
990
      }
J
jorlow@chromium.org 已提交
991

992
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
993
      // writing log_number in the manifest means that any log file
994 995
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
996 997 998
      // 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);
999 1000 1001
      // 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 已提交
1002
      // log number
1003
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
1004 1005
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
1006
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1007 1008
        // Recovery failed
        break;
1009 1010
      }
    }
1011
  }
I
Igor Canadi 已提交
1012

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

1016 1017
Status DBImpl::WriteLevel0TableForRecovery(ColumnFamilyData* cfd, MemTable* mem,
                                           VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1018
  mutex_.AssertHeld();
1019
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1020
  FileMetaData meta;
1021
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1022 1023
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1024 1025
  ReadOptions ro;
  ro.total_order_seek = true;
1026
  Arena arena;
1027 1028
  Status s;
  {
1029 1030 1031 1032
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
    const SequenceNumber newest_snapshot = snapshots_.GetNewest();
    const SequenceNumber earliest_seqno_in_memtable =
        mem->GetFirstSequenceNumber();
1033 1034 1035
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1036
        cfd->GetName().c_str(), meta.fd.GetNumber());
1037

1038 1039 1040 1041 1042
    {
      mutex_.Unlock();
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
          iter.get(), &meta, cfd->internal_comparator(), newest_snapshot,
1043 1044
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
          cfd->ioptions()->compression_opts, Env::IO_HIGH);
1045
      LogFlush(db_options_.info_log);
1046 1047
      mutex_.Lock();
    }
1048

1049 1050 1051
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
1052 1053 1054
        cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
        s.ToString().c_str());
  }
I
Igor Canadi 已提交
1055
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1056 1057 1058 1059

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1060
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1061 1062 1063
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1064 1065
  }

L
Lei Jin 已提交
1066
  InternalStats::CompactionStats stats(1);
1067
  stats.micros = env_->NowMicros() - start_micros;
1068
  stats.bytes_written = meta.fd.GetFileSize();
M
Mark Callaghan 已提交
1069
  stats.files_out_levelnp1 = 1;
1070
  cfd->internal_stats()->AddCompactionStats(level, stats);
1071 1072
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1073
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1074 1075 1076
  return s;
}

1077 1078
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
I
Igor Canadi 已提交
1079
    bool* madeProgress, JobContext* job_context, LogBuffer* log_buffer) {
1080
  mutex_.AssertHeld();
1081
  assert(cfd->imm()->size() != 0);
1082
  assert(cfd->imm()->IsFlushPending());
1083

I
Igor Canadi 已提交
1084 1085
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1086 1087 1088
                     snapshots_.GetNewest(), job_context, log_buffer,
                     db_directory_.get(), GetCompressionFlush(*cfd->ioptions()),
                     stats_);
1089

1090 1091
  uint64_t file_number;
  Status s = flush_job.Run(&file_number);
J
jorlow@chromium.org 已提交
1092 1093

  if (s.ok()) {
I
Igor Canadi 已提交
1094
    InstallSuperVersionBackground(cfd, job_context, mutable_cf_options);
1095 1096 1097
    if (madeProgress) {
      *madeProgress = 1;
    }
S
sdong 已提交
1098
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1099
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1100
                cfd->current()->storage_info()->LevelSummary(&tmp));
1101

1102
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1103
      // add to deletion state
1104
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1105 1106
             alive_log_files_.begin()->number < versions_->MinLogNumber()) {
        const auto& earliest = *alive_log_files_.begin();
I
Igor Canadi 已提交
1107
        job_context->log_delete_files.push_back(earliest.number);
I
Igor Canadi 已提交
1108
        total_log_size_ -= earliest.size;
1109 1110
        alive_log_files_.pop_front();
      }
1111
    }
J
jorlow@chromium.org 已提交
1112
  }
1113

1114
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1115 1116 1117 1118 1119
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1120
  RecordFlushIOStats();
1121 1122 1123
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1124
    NotifyOnFlushCompleted(cfd, file_number, mutable_cf_options);
1125 1126
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1127 1128 1129
  return s;
}

1130
void DBImpl::NotifyOnFlushCompleted(
1131 1132
    ColumnFamilyData* cfd, uint64_t file_number,
    const MutableCFOptions& mutable_cf_options) {
1133 1134 1135
  if (cfd->ioptions()->listeners.size() == 0U) {
    return;
  }
1136 1137 1138 1139 1140 1141
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  bool triggered_flush_slowdown =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1142
       mutable_cf_options.level0_slowdown_writes_trigger);
1143 1144
  bool triggered_flush_stop =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1145
       mutable_cf_options.level0_stop_writes_trigger);
1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160
  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.
}

1161
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1162
                            const Slice* begin, const Slice* end,
1163 1164
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
1165
  if (target_path_id >= db_options_.db_paths.size()) {
1166 1167 1168
    return Status::InvalidArgument("Invalid target path ID");
  }

1169 1170
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1171 1172

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1173
  if (!s.ok()) {
1174
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1175 1176 1177
    return s;
  }

I
Igor Canadi 已提交
1178
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1179 1180
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1181 1182
    Version* base = cfd->current();
    for (int level = 1; level < cfd->NumberLevels(); level++) {
S
sdong 已提交
1183
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1184 1185 1186 1187
        max_level_with_files = level;
      }
    }
  }
1188 1189
  for (int level = 0; level <= max_level_with_files; level++) {
    // in case the compaction is unversal or if we're compacting the
1190 1191 1192
    // 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)
1193 1194
    if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
        cfd->ioptions()->compaction_style == kCompactionStyleFIFO ||
1195
        (level == max_level_with_files && level > 0)) {
1196
      s = RunManualCompaction(cfd, level, level, target_path_id, begin, end);
1197
    } else {
1198 1199
      s = RunManualCompaction(cfd, level, level + 1, target_path_id, begin,
                              end);
L
Lei Jin 已提交
1200 1201
    }
    if (!s.ok()) {
1202
      LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1203
      return s;
1204
    }
G
Gabor Cselle 已提交
1205
  }
1206 1207

  if (reduce_level) {
I
Igor Canadi 已提交
1208
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1209
  }
1210
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1211

1212 1213 1214 1215 1216 1217 1218
  {
    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 已提交
1219
  return s;
1220 1221
}

1222 1223 1224 1225 1226
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 已提交
1227 1228 1229 1230
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249
  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 已提交
1250
#endif  // ROCKSDB_LITE
1251 1252
}

I
Igor Canadi 已提交
1253
#ifndef ROCKSDB_LITE
1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327
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);
  };
1328 1329 1330
  CompactionJob compaction_job(c.get(), db_options_, *c->mutable_cf_options(),
                               env_options_, versions_.get(), &shutting_down_,
                               &log_buffer, db_directory_.get(), stats_,
S
sdong 已提交
1331 1332
                               &snapshots_, is_snapshot_supported_,
                               table_cache_, std::move(yield_callback));
1333 1334 1335 1336 1337
  compaction_job.Prepare();

  mutex_.Unlock();
  Status status = compaction_job.Run();
  mutex_.Lock();
1338
  compaction_job.Install(&status, &mutex_);
1339
  if (status.ok()) {
1340 1341
    InstallSuperVersionBackground(c->column_family_data(), &job_context,
                                  *c->mutable_cf_options());
1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362 1363 1364 1365 1366 1367 1368 1369 1370 1371 1372 1373 1374 1375 1376 1377 1378
  }
  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 已提交
1379
    job_context.Clean();
1380 1381 1382 1383 1384 1385 1386
    mutex_.Lock();
  }

  bg_compaction_scheduled_--;

  return status;
}
I
Igor Canadi 已提交
1387
#endif  // ROCKSDB_LITE
1388

1389
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1390
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1391 1392 1393
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1394 1395
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1396 1397
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1398
        cfd->GetName().c_str());
1399
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1400 1401 1402
  }

  MutableCFOptions new_options;
1403
  Status s;
L
Lei Jin 已提交
1404 1405
  {
    MutexLock l(&mutex_);
1406 1407
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1408 1409 1410 1411
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

1412 1413
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1414 1415
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1416 1417
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1418
  }
1419
  if (s.ok()) {
1420 1421
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1422 1423 1424
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
  } else {
1425 1426
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1427
  }
1428
  return s;
I
Igor Canadi 已提交
1429
#endif  // ROCKSDB_LITE
1430 1431
}

1432
// return the same level if it cannot be moved
1433 1434
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
1435
  mutex_.AssertHeld();
S
sdong 已提交
1436
  const auto* vstorage = cfd->current()->storage_info();
1437
  int minimum_level = level;
1438
  for (int i = level - 1; i > 0; --i) {
1439
    // stop if level i is not empty
S
sdong 已提交
1440
    if (vstorage->NumLevelFiles(i) > 0) break;
1441
    // stop if level i is too small (cannot fit the level files)
1442
    if (mutable_cf_options.MaxBytesForLevel(i) <
S
sdong 已提交
1443
        vstorage->NumLevelBytes(level)) {
1444 1445
      break;
    }
1446 1447 1448 1449 1450 1451

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1455
  SuperVersion* superversion_to_free = nullptr;
1456
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1457 1458

  mutex_.Lock();
1459 1460 1461

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1462
    mutex_.Unlock();
1463 1464
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
I
Igor Canadi 已提交
1465
    delete new_superversion;
L
Lei Jin 已提交
1466
    return Status::NotSupported("another thread is refitting");
1467 1468 1469 1470 1471
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1472
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1473 1474
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop: %d %d",
1475
        bg_compaction_scheduled_, bg_flush_scheduled_);
1476 1477 1478
    bg_cv_.Wait();
  }

1479 1480
  const MutableCFOptions mutable_cf_options =
    *cfd->GetLatestMutableCFOptions();
1481
  // move to a smaller level
1482 1483
  int to_level = target_level;
  if (target_level < 0) {
1484
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1485
  }
1486 1487 1488

  assert(to_level <= level);

L
Lei Jin 已提交
1489
  Status status;
1490
  if (to_level < level) {
1491 1492
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Before refitting:\n%s",
1493
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1494

1495
    VersionEdit edit;
I
Igor Canadi 已提交
1496
    edit.SetColumnFamily(cfd->GetID());
S
sdong 已提交
1497
    for (const auto& f : cfd->current()->storage_info()->LevelFiles(level)) {
I
Igor Canadi 已提交
1498
      f->moved = true;
1499
      edit.DeleteFile(level, f->fd.GetNumber());
1500 1501 1502
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1503
    }
1504 1505
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1506
        cfd->GetName().c_str(), edit.DebugString().data());
1507

1508 1509
    status = versions_->LogAndApply(cfd,
        mutable_cf_options, &edit, &mutex_, db_directory_.get());
L
Lei Jin 已提交
1510 1511
    superversion_to_free = InstallSuperVersion(
        cfd, new_superversion, mutable_cf_options);
I
Igor Canadi 已提交
1512
    new_superversion = nullptr;
1513

1514 1515
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1516
        status.ToString().data());
1517 1518

    if (status.ok()) {
1519 1520
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1521
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1522 1523 1524 1525 1526
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1527 1528 1529 1530

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1531
  return status;
G
Gabor Cselle 已提交
1532 1533
}

1534 1535 1536
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1537 1538
}

1539 1540
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1541 1542 1543
  MutexLock l(&mutex_);
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.max_mem_compaction_level;
1544 1545
}

1546 1547
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1548 1549 1550
  MutexLock l(&mutex_);
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
1551 1552
}

L
Lei Jin 已提交
1553
Status DBImpl::Flush(const FlushOptions& flush_options,
1554 1555
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1556
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
1557 1558
}

1559
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1560 1561 1562
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1563
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1564 1565
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1566
  assert(input_level >= 0);
1567

G
Gabor Cselle 已提交
1568 1569
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1570
  ManualCompaction manual;
I
Igor Canadi 已提交
1571
  manual.cfd = cfd;
1572 1573
  manual.input_level = input_level;
  manual.output_level = output_level;
1574
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
1575
  manual.done = false;
1576
  manual.in_progress = false;
1577 1578 1579
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
1580 1581
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1582
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1583 1584 1585 1586
  } else {
    begin_storage = InternalKey(*begin, kMaxSequenceNumber, kValueTypeForSeek);
    manual.begin = &begin_storage;
  }
1587
  if (end == nullptr ||
1588 1589
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1590
    manual.end = nullptr;
G
Gabor Cselle 已提交
1591 1592 1593 1594 1595 1596
  } else {
    end_storage = InternalKey(*end, 0, static_cast<ValueType>(0));
    manual.end = &end_storage;
  }

  MutexLock l(&mutex_);
1597

1598 1599 1600 1601 1602 1603
  // 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
1604
  // RunManualCompaction(), i.e. during that time no other compaction will
1605 1606 1607
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1608
  // RunManualCompaction() from getting to the second while loop below.
1609 1610 1611 1612 1613
  // 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) {
1614
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
1615 1616 1617
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1618 1619
    bg_cv_.Wait();
  }
1620

1621 1622
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
1623
      cfd->GetName().c_str());
1624

1625 1626 1627 1628
  // 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) {
1629 1630 1631
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
1632
      bg_cv_.Wait();
1633 1634
    } else {
      manual_compaction_ = &manual;
1635 1636
      bg_compaction_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
G
Gabor Cselle 已提交
1637
    }
H
hans@chromium.org 已提交
1638
  }
1639

1640 1641 1642
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1643
  return manual.status;
J
jorlow@chromium.org 已提交
1644 1645
}

1646
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
1647
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
1648 1649 1650 1651
  Status s;
  {
    WriteContext context;
    MutexLock guard_lock(&mutex_);
1652 1653 1654 1655 1656 1657

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

I
Igor Canadi 已提交
1658 1659
    WriteThread::Writer w(&mutex_);
    s = write_thread_.EnterWriteThread(&w, 0);
S
Stanislau Hlebik 已提交
1660 1661 1662 1663 1664
    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);
1665 1666
    write_thread_.ExitWriteThread(&w, &w, s);

S
Stanislau Hlebik 已提交
1667 1668
    cfd->imm()->FlushRequested();

1669 1670 1671
    // schedule flush
    SchedulePendingFlush(cfd);
    MaybeScheduleFlushOrCompaction();
S
Stanislau Hlebik 已提交
1672
  }
S
Stanislau Hlebik 已提交
1673

L
Lei Jin 已提交
1674
  if (s.ok() && flush_options.wait) {
1675
    // Wait until the compaction completes
1676
    s = WaitForFlushMemTable(cfd);
1677 1678
  }
  return s;
J
jorlow@chromium.org 已提交
1679 1680
}

1681
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1682 1683 1684
  Status s;
  // Wait until the compaction completes
  MutexLock l(&mutex_);
1685
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1686 1687
    bg_cv_.Wait();
  }
1688
  if (!bg_error_.ok()) {
1689 1690 1691
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1692 1693
}

1694
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1695
  mutex_.AssertHeld();
1696
  if (bg_work_gate_closed_) {
1697 1698
    // gate closed for background work
    return;
I
Igor Canadi 已提交
1699
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
1700
    // DB is being deleted; no more background compactions
1701 1702 1703 1704 1705
    return;
  } else if (bg_manual_only_) {
    // manual only
    return;
  }
1706

1707 1708 1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722 1723
  while (unscheduled_flushes_ > 0 &&
         bg_flush_scheduled_ < db_options_.max_background_flushes) {
    unscheduled_flushes_--;
    bg_flush_scheduled_++;
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH);
  }

  if (db_options_.max_background_flushes == 0 &&
      bg_compaction_scheduled_ < db_options_.max_background_compactions &&
      unscheduled_flushes_ > 0) {
    // special case where flush is executed by compaction thread
    // (if max_background_flushes == 0).
    // Compaction thread will execute all the flushes
    unscheduled_flushes_ = 0;
    if (unscheduled_compactions_ > 0) {
      // bg compaction will execute one compaction
      unscheduled_compactions_--;
1724
    }
1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735 1736 1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778 1779
    bg_compaction_scheduled_++;
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
  }

  while (bg_compaction_scheduled_ < db_options_.max_background_compactions &&
         unscheduled_compactions_ > 0) {
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW);
  }
}

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

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

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

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

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

void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) {
  if (!cfd->pending_compaction() && cfd->NeedsCompaction()) {
    AddToCompactionQueue(cfd);
    ++unscheduled_compactions_;
1780 1781 1782
  }
}

1783
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
1784
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
1785 1786 1787
  IOSTATS_RESET(bytes_written);
}

1788
void DBImpl::BGWorkFlush(void* db) {
1789
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
1790 1791 1792 1793
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
1794
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
1795 1796 1797
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
1798
Status DBImpl::BackgroundFlush(bool* madeProgress, JobContext* job_context,
H
Haobo Xu 已提交
1799
                               LogBuffer* log_buffer) {
1800
  mutex_.AssertHeld();
1801 1802 1803 1804 1805

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

1806 1807 1808
  ColumnFamilyData* cfd = nullptr;
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
I
Igor Canadi 已提交
1809
    auto first_cfd = PopFirstFromFlushQueue();
1810

I
Igor Canadi 已提交
1811
    if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
1812
      // can't flush this CF, try next one
I
Igor Canadi 已提交
1813 1814
      if (first_cfd->Unref()) {
        delete first_cfd;
1815 1816
      }
      continue;
1817
    }
1818 1819

    // found a flush!
I
Igor Canadi 已提交
1820
    cfd = first_cfd;
1821 1822 1823 1824 1825 1826 1827 1828 1829 1830 1831 1832 1833 1834 1835 1836 1837 1838
    break;
  }

  Status status;
  if (cfd != nullptr) {
    const MutableCFOptions mutable_cf_options =
        *cfd->GetLatestMutableCFOptions();
    LogToBuffer(
        log_buffer,
        "Calling FlushMemTableToOutputFile with column "
        "family [%s], flush slots available %d, compaction slots available %d",
        cfd->GetName().c_str(),
        db_options_.max_background_flushes - bg_flush_scheduled_,
        db_options_.max_background_compactions - bg_compaction_scheduled_);
    status = FlushMemTableToOutputFile(cfd, mutable_cf_options, madeProgress,
                                       job_context, log_buffer);
    if (cfd->Unref()) {
      delete cfd;
1839
    }
J
jorlow@chromium.org 已提交
1840
  }
1841
  return status;
J
jorlow@chromium.org 已提交
1842 1843
}

1844
void DBImpl::BackgroundCallFlush() {
1845
  bool madeProgress = false;
I
Igor Canadi 已提交
1846
  JobContext job_context(true);
1847 1848
  assert(bg_flush_scheduled_);

1849
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
1850 1851 1852
  {
    MutexLock l(&mutex_);

I
Igor Canadi 已提交
1853 1854 1855
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

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

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

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

H
Haobo Xu 已提交
1900
    bg_flush_scheduled_--;
1901 1902
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
1903
    RecordFlushIOStats();
H
Haobo Xu 已提交
1904
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
1905 1906 1907 1908
    // 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.
1909
  }
J
jorlow@chromium.org 已提交
1910 1911
}

1912
void DBImpl::BackgroundCallCompaction() {
1913
  bool madeProgress = false;
I
Igor Canadi 已提交
1914
  JobContext job_context(true);
H
Haobo Xu 已提交
1915 1916

  MaybeDumpStats();
1917
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
1918 1919
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
1920 1921 1922 1923

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

1924 1925
    assert(bg_compaction_scheduled_);
    Status s;
I
Igor Canadi 已提交
1926
    if (!shutting_down_.load(std::memory_order_acquire)) {
I
Igor Canadi 已提交
1927
      s = BackgroundCompaction(&madeProgress, &job_context, &log_buffer);
1928 1929 1930 1931 1932
      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.
1933 1934
        uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
1935 1936
        bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
        mutex_.Unlock();
H
Haobo Xu 已提交
1937
        log_buffer.FlushBufferToLog();
1938
        Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
1939 1940 1941
            "Waiting after background compaction error: %s, "
            "Accumulated background error counts: %" PRIu64,
            s.ToString().c_str(), error_cnt);
1942
        LogFlush(db_options_.info_log);
1943 1944 1945 1946
        env_->SleepForMicroseconds(1000000);
        mutex_.Lock();
      }
    }
H
Haobo Xu 已提交
1947

I
Igor Canadi 已提交
1948 1949
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

1950 1951
    // 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 已提交
1952 1953 1954
    // FindObsoleteFiles(). This is because job_context does not
    // catch all created files if compaction failed.
    FindObsoleteFiles(&job_context, !s.ok());
1955 1956

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
1957
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
1958
      mutex_.Unlock();
1959 1960 1961 1962 1963
      // 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 已提交
1964
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
1965 1966
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
1967
      }
I
Igor Canadi 已提交
1968
      job_context.Clean();
1969 1970
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
1971

1972
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
1973

1974 1975
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

1976 1977
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
1978 1979
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
I
Igor Canadi 已提交
1980
      // * madeProgress -- need to wakeup DelayWrite
1981 1982 1983 1984 1985 1986
      // * 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 已提交
1987 1988 1989 1990
    // 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.
1991
  }
J
jorlow@chromium.org 已提交
1992 1993
}

I
Igor Canadi 已提交
1994
Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
1995
                                    LogBuffer* log_buffer) {
1996
  *madeProgress = false;
J
jorlow@chromium.org 已提交
1997
  mutex_.AssertHeld();
1998

1999 2000
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2001

2002 2003 2004 2005 2006 2007 2008 2009 2010 2011
  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_;
  }

2012 2013 2014
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2015 2016 2017 2018
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2019 2020
  }

2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031 2032 2033 2034 2035 2036 2037 2038 2039 2040 2041 2042
  // If there are no flush threads, then compaction thread needs to execute the
  // flushes
  if (db_options_.max_background_flushes == 0) {
    // BackgroundFlush() will only execute a single flush. We keep calling it as
    // long as there's more flushes to be done
    while (!flush_queue_.empty()) {
      LogToBuffer(
          log_buffer,
          "BackgroundCompaction calling BackgroundFlush. flush slots available "
          "%d, compaction slots available %d",
          db_options_.max_background_flushes - bg_flush_scheduled_,
          db_options_.max_background_compactions - bg_compaction_scheduled_);
      auto flush_status =
          BackgroundFlush(madeProgress, job_context, log_buffer);
      if (!flush_status.ok()) {
        if (is_manual) {
          manual_compaction_->status = flush_status;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
        return flush_status;
2043 2044 2045 2046 2047
      }
    }
  }

  unique_ptr<Compaction> c;
2048 2049
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2050
  if (is_manual) {
G
Gabor Cselle 已提交
2051
    ManualCompaction* m = manual_compaction_;
2052
    assert(m->in_progress);
2053 2054 2055
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
2056
    if (!c) {
2057
      m->done = true;
G
Gabor Cselle 已提交
2058
    }
I
Igor Canadi 已提交
2059 2060 2061 2062 2063 2064 2065 2066 2067
    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()));
2068 2069 2070 2071 2072 2073 2074 2075 2076 2077 2078 2079 2080 2081 2082 2083 2084 2085 2086 2087 2088 2089 2090 2091 2092 2093 2094 2095 2096 2097 2098 2099 2100 2101 2102 2103 2104 2105 2106 2107 2108 2109 2110 2111 2112 2113 2114
  } else if (!compaction_queue_.empty()) {
    // cfd is referenced here
    auto cfd = PopFirstFromCompactionQueue();
    // We unreference here because the following code will take a Ref() on
    // this cfd if it is going to use it (Compaction class holds a
    // reference).
    // This will all happen under a mutex so we don't have to be afraid of
    // somebody else deleting it.
    if (cfd->Unref()) {
      delete cfd;
      // This was the last reference of the column family, so no need to
      // compact.
      return Status::OK();
    }

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

  Status status;
2121
  if (!c) {
H
hans@chromium.org 已提交
2122
    // Nothing to do
2123
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2124 2125 2126 2127 2128
  } 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);
2129
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2130 2131
           kCompactionStyleFIFO);
    for (const auto& f : *c->inputs(0)) {
2132
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2133
    }
2134 2135 2136
    status = versions_->LogAndApply(
        c->column_family_data(), *c->mutable_cf_options(), c->edit(),
        &mutex_, db_directory_.get());
I
Igor Canadi 已提交
2137 2138
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
I
Igor Canadi 已提交
2139 2140 2141 2142 2143
    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 已提交
2144
  } else if (!is_manual && c->IsTrivialMove()) {
J
jorlow@chromium.org 已提交
2145
    // Move file to next level
2146
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2147
    FileMetaData* f = c->input(0, 0);
I
Igor Canadi 已提交
2148
    f->moved = true;
2149
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
2150 2151 2152
    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);
2153 2154 2155
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(),
                                    c->edit(), &mutex_, db_directory_.get());
2156
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2157 2158
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
2159

S
sdong 已提交
2160
    VersionStorageInfo::LevelSummaryStorage tmp;
2161 2162
    c->column_family_data()->internal_stats()->IncBytesMoved(
        c->level() + 1, f->fd.GetFileSize());
S
sdong 已提交
2163 2164 2165 2166
    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 已提交
2167
                c->input_version()->storage_info()->LevelSummary(&tmp));
I
Igor Canadi 已提交
2168
    c->ReleaseCompactionFiles(status);
2169
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2170
  } else {
I
Igor Canadi 已提交
2171 2172 2173 2174 2175
    auto yield_callback = [&]() {
      return CallFlushDuringCompaction(c->column_family_data(),
                                       *c->mutable_cf_options(), job_context,
                                       log_buffer);
    };
2176 2177 2178
    CompactionJob compaction_job(c.get(), db_options_, *c->mutable_cf_options(),
                                 env_options_, versions_.get(), &shutting_down_,
                                 log_buffer, db_directory_.get(), stats_,
S
sdong 已提交
2179
                                 &snapshots_, is_snapshot_supported_,
2180
                                 table_cache_, std::move(yield_callback));
I
Igor Canadi 已提交
2181 2182 2183 2184
    compaction_job.Prepare();
    mutex_.Unlock();
    status = compaction_job.Run();
    mutex_.Lock();
2185
    compaction_job.Install(&status, &mutex_);
I
Igor Canadi 已提交
2186 2187 2188 2189
    if (status.ok()) {
      InstallSuperVersionBackground(c->column_family_data(), job_context,
                                    *c->mutable_cf_options());
    }
I
Igor Canadi 已提交
2190
    c->ReleaseCompactionFiles(status);
2191
    c->ReleaseInputs();
2192
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2193
  }
2194
  c.reset();
J
jorlow@chromium.org 已提交
2195 2196 2197

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2198
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2199 2200
    // Ignore compaction errors found during shutting down
  } else {
2201
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2202
        status.ToString().c_str());
2203
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2204 2205 2206
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2207 2208

  if (is_manual) {
G
Gabor Cselle 已提交
2209
    ManualCompaction* m = manual_compaction_;
2210
    if (!status.ok()) {
L
Lei Jin 已提交
2211
      m->status = status;
2212 2213
      m->done = true;
    }
2214 2215 2216 2217 2218 2219 2220 2221 2222
    // 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.
2223 2224 2225 2226 2227
    //
    // 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) {
2228 2229
      m->done = true;
    }
G
Gabor Cselle 已提交
2230 2231 2232
    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 已提交
2233
      // Universal and FIFO compactions should always compact the whole range
2234 2235
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleUniversal);
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2236
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2237 2238
      m->begin = &m->tmp_storage;
    }
2239
    m->in_progress = false; // not being processed anymore
2240
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2241
  }
2242
  return status;
J
jorlow@chromium.org 已提交
2243 2244
}

I
Igor Canadi 已提交
2245 2246 2247
uint64_t DBImpl::CallFlushDuringCompaction(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
    JobContext* job_context, LogBuffer* log_buffer) {
2248
  if (db_options_.max_background_flushes > 0) {
2249 2250 2251
    // flush thread will take care of this
    return 0;
  }
I
Igor Canadi 已提交
2252
  if (cfd->imm()->imm_flush_needed.load(std::memory_order_relaxed)) {
I
Igor Canadi 已提交
2253 2254 2255 2256
    const uint64_t imm_start = env_->NowMicros();
    mutex_.Lock();
    if (cfd->imm()->IsFlushPending()) {
      cfd->Ref();
I
Igor Canadi 已提交
2257 2258
      FlushMemTableToOutputFile(cfd, mutable_cf_options, nullptr, job_context,
                                log_buffer);
I
Igor Canadi 已提交
2259
      cfd->Unref();
I
Igor Canadi 已提交
2260
      bg_cv_.SignalAll();  // Wakeup DelayWrite() if necessary
I
Igor Canadi 已提交
2261 2262 2263 2264 2265 2266 2267 2268
    }
    mutex_.Unlock();
    log_buffer->FlushBufferToLog();
    return env_->NowMicros() - imm_start;
  }
  return 0;
}

2269 2270
namespace {
struct IterState {
I
Igor Canadi 已提交
2271 2272
  IterState(DBImpl* _db, port::Mutex* _mu, SuperVersion* _super_version)
      : db(_db), mu(_mu), super_version(_super_version) {}
2273 2274

  DBImpl* db;
2275
  port::Mutex* mu;
2276
  SuperVersion* super_version;
2277 2278 2279 2280
};

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

2282
  if (state->super_version->Unref()) {
I
Igor Canadi 已提交
2283
    JobContext job_context;
2284

2285 2286
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2287
    state->db->FindObsoleteFiles(&job_context, false, true);
2288 2289 2290
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2291 2292
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2293
    }
I
Igor Canadi 已提交
2294
    job_context.Clean();
I
Igor Canadi 已提交
2295
  }
T
Tomislav Novak 已提交
2296

2297 2298
  delete state;
}
H
Hans Wennborg 已提交
2299
}  // namespace
2300

L
Lei Jin 已提交
2301
Iterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
2302
                                      ColumnFamilyData* cfd,
2303 2304 2305
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
2306 2307 2308 2309 2310
  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 已提交
2311
      super_version->mem->NewIterator(read_options, arena));
2312
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
2313
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
2314
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
2315
  super_version->current->AddIterators(read_options, env_options_,
2316 2317
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
2318
  IterState* cleanup = new IterState(this, &mutex_, super_version);
2319
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2320 2321 2322 2323

  return internal_iter;
}

2324 2325 2326 2327
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2328
Status DBImpl::Get(const ReadOptions& read_options,
2329
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2330
                   std::string* value) {
L
Lei Jin 已提交
2331
  return GetImpl(read_options, column_family, key, value);
2332 2333
}

I
Igor Canadi 已提交
2334 2335
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
2336 2337
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
2338
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
2339
//
I
Igor Canadi 已提交
2340 2341 2342 2343
// However, if InstallSuperVersion() gets called twice with the same
// job_context, we can't reuse the SuperVersion() that got
// malloced
// because
I
Igor Canadi 已提交
2344 2345 2346
// 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 已提交
2347 2348
void DBImpl::InstallSuperVersionBackground(
    ColumnFamilyData* cfd, JobContext* job_context,
2349
    const MutableCFOptions& mutable_cf_options) {
2350
  mutex_.AssertHeld();
I
Igor Canadi 已提交
2351 2352 2353 2354
  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 已提交
2355 2356
}

L
Lei Jin 已提交
2357 2358
SuperVersion* DBImpl::InstallSuperVersion(
    ColumnFamilyData* cfd, SuperVersion* new_sv,
2359
    const MutableCFOptions& mutable_cf_options, bool dont_schedule_bg_work) {
L
Lei Jin 已提交
2360
  mutex_.AssertHeld();
2361 2362 2363 2364 2365 2366 2367 2368 2369

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

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

2373 2374 2375 2376 2377 2378 2379 2380 2381
  // Whenever we install new SuperVersion, we might need to issue new flushes or
  // compactions. dont_schedule_bg_work is true when scheduling from write
  // thread and we don't want to add additional overhead. Callers promise to
  // call SchedulePendingFlush() and MaybeScheduleFlushOrCompaction() eventually
  if (!dont_schedule_bg_work) {
    SchedulePendingFlush(cfd);
    SchedulePendingCompaction(cfd);
    MaybeScheduleFlushOrCompaction();
  }
L
Lei Jin 已提交
2382 2383 2384 2385 2386 2387 2388 2389 2390 2391

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

Status DBImpl::GetImpl(const ReadOptions& read_options,
2392 2393
                       ColumnFamilyHandle* column_family, const Slice& key,
                       std::string* value, bool* value_found) {
L
Lei Jin 已提交
2394
  StopWatch sw(env_, stats_, DB_GET);
2395
  PERF_TIMER_GUARD(get_snapshot_time);
2396

2397 2398 2399
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2400
  SequenceNumber snapshot;
L
Lei Jin 已提交
2401 2402 2403
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2404 2405
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2406
  }
2407

2408
  // Acquire SuperVersion
2409
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
I
Igor Canadi 已提交
2410

2411
  // Prepare to store a list of merge operations if merge occurs.
2412
  MergeContext merge_context;
2413

2414
  Status s;
2415
  // First look in the memtable, then in the immutable memtable (if any).
2416
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2417
  // merge_operands will contain the sequence of merges in the latter case.
2418
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2419
  PERF_TIMER_STOP(get_snapshot_time);
2420

2421
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2422
    // Done
L
Lei Jin 已提交
2423
    RecordTick(stats_, MEMTABLE_HIT);
2424
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2425
    // Done
L
Lei Jin 已提交
2426
    RecordTick(stats_, MEMTABLE_HIT);
2427
  } else {
2428
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2429 2430
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2431
    RecordTick(stats_, MEMTABLE_MISS);
2432
  }
2433

2434 2435
  {
    PERF_TIMER_GUARD(get_post_process_time);
2436

2437
    ReturnAndCleanupSuperVersion(cfd, sv);
2438

2439 2440 2441
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2442
  return s;
J
jorlow@chromium.org 已提交
2443 2444
}

2445
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
2446
    const ReadOptions& read_options,
2447
    const std::vector<ColumnFamilyHandle*>& column_family,
2448
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
2449

L
Lei Jin 已提交
2450
  StopWatch sw(env_, stats_, DB_MULTIGET);
2451
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
2452

2453
  SequenceNumber snapshot;
2454

2455
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
2456
    ColumnFamilyData* cfd;
2457 2458 2459 2460 2461
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
2462 2463 2464 2465 2466 2467
    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});
2468 2469 2470
    }
  }

2471
  mutex_.Lock();
L
Lei Jin 已提交
2472 2473 2474
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2475 2476 2477
  } else {
    snapshot = versions_->LastSequence();
  }
2478
  for (auto mgd_iter : multiget_cf_data) {
2479 2480
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
2481
  }
2482
  mutex_.Unlock();
2483

2484 2485
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2486

2487
  // Note: this always resizes the values array
2488 2489 2490
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
2491 2492

  // Keep track of bytes that we read for statistics-recording later
2493
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
2494
  PERF_TIMER_STOP(get_snapshot_time);
2495 2496 2497 2498

  // 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.
2499
  // merge_operands will contain the sequence of merges in the latter case.
2500
  for (size_t i = 0; i < num_keys; ++i) {
2501
    merge_context.Clear();
2502
    Status& s = stat_list[i];
2503 2504 2505
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
2506 2507
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
2508 2509 2510
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
2511
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
2512
      // Done
2513
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
2514 2515
      // Done
    } else {
2516
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2517 2518
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
2519 2520 2521
    }

    if (s.ok()) {
2522
      bytes_read += value->size();
2523 2524 2525 2526
    }
  }

  // Post processing (decrement reference counts and record statistics)
2527
  PERF_TIMER_GUARD(get_post_process_time);
2528 2529
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
2530
  // TODO(icanadi) do we need lock here or just around Cleanup()?
2531 2532 2533 2534 2535 2536
  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);
2537 2538
    }
  }
2539 2540 2541 2542 2543 2544 2545
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
2546
  }
2547

L
Lei Jin 已提交
2548 2549 2550
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
2551
  PERF_TIMER_STOP(get_post_process_time);
2552

2553
  return stat_list;
2554 2555
}

L
Lei Jin 已提交
2556
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
2557
                                  const std::string& column_family_name,
2558
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
2559
  Status s;
I
Igor Canadi 已提交
2560
  *handle = nullptr;
Y
Yueh-Hsuan Chiang 已提交
2561 2562
  {
    MutexLock l(&mutex_);
I
Igor Canadi 已提交
2563

Y
Yueh-Hsuan Chiang 已提交
2564 2565 2566 2567 2568 2569 2570 2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587
    if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
        nullptr) {
      return Status::InvalidArgument("Column family already exists");
    }
    VersionEdit edit;
    edit.AddColumnFamily(column_family_name);
    uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
    edit.SetColumnFamily(new_id);
    edit.SetLogNumber(logfile_number_);
    edit.SetComparatorName(cf_options.comparator->Name());

    // LogAndApply will both write the creation in MANIFEST and create
    // ColumnFamilyData object
    Options opt(db_options_, cf_options);
    s = versions_->LogAndApply(nullptr,
        MutableCFOptions(opt, ImmutableCFOptions(opt)),
        &edit, &mutex_, db_directory_.get(), false, &cf_options);
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
      delete InstallSuperVersion(
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
S
sdong 已提交
2588 2589 2590 2591 2592

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

Y
Yueh-Hsuan Chiang 已提交
2593 2594 2595 2596 2597 2598 2599 2600 2601
      *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
          "Created column family [%s] (ID %u)",
          column_family_name.c_str(), (unsigned)cfd->GetID());
    } else {
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "Creating column family [%s] FAILED -- %s",
          column_family_name.c_str(), s.ToString().c_str());
    }
2602
  }  // MutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
2603 2604

  // this is outside the mutex
2605
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2606 2607
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
2608
  }
2609
  return s;
2610 2611
}

2612 2613 2614 2615
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
2616 2617
    return Status::InvalidArgument("Can't drop default column family");
  }
2618

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

I
Igor Canadi 已提交
2621 2622
  VersionEdit edit;
  edit.DropColumnFamily();
2623 2624
  edit.SetColumnFamily(cfd->GetID());

2625
  Status s;
2626 2627 2628 2629 2630 2631
  {
    MutexLock l(&mutex_);
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
2632
      // we drop column family from a single write thread
I
Igor Canadi 已提交
2633 2634
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
2635
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
2636 2637
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
I
Igor Canadi 已提交
2638
      write_thread_.ExitWriteThread(&w, &w, s);
2639
    }
S
sdong 已提交
2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652

    if (!cf_support_snapshot) {
      // Dropped Column Family doesn't support snapshot. Need to recalculate
      // is_snapshot_supported_.
      bool new_is_snapshot_supported = true;
      for (auto c : *versions_->GetColumnFamilySet()) {
        if (!c->mem()->IsSnapshotSupported()) {
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
2653
  }
2654

2655
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2656 2657 2658 2659
    // Note that here we erase the associated cf_info of the to-be-dropped
    // cfd before its ref-count goes to zero to avoid having to erase cf_info
    // later inside db_mutex.
    EraseThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
2660
    assert(cfd->IsDropped());
L
Lei Jin 已提交
2661 2662 2663
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
2664 2665
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
2666
        cfd->GetID());
2667
  } else {
2668
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
2669
        "Dropping column family with id %u FAILED -- %s\n",
2670 2671 2672
        cfd->GetID(), s.ToString().c_str());
  }

2673
  return s;
2674 2675
}

L
Lei Jin 已提交
2676
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
2677 2678
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
2679
  if (value_found != nullptr) {
K
Kai Liu 已提交
2680 2681
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
2682
  }
L
Lei Jin 已提交
2683
  ReadOptions roptions = read_options;
2684
  roptions.read_tier = kBlockCacheTier; // read from block cache only
2685
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
2686

2687
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
2688 2689 2690
  // 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();
2691 2692
}

2693
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
2694 2695 2696
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
2697

2698
  if (read_options.tailing) {
I
Igor Canadi 已提交
2699 2700 2701 2702
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
2703 2704
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
2705
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2706 2707 2708
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
2709
#endif
T
Tomislav Novak 已提交
2710
  } else {
2711
    SequenceNumber latest_snapshot = versions_->LastSequence();
2712
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
2713

I
Igor Canadi 已提交
2714
    auto snapshot =
2715 2716 2717
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
2718
            : latest_snapshot;
T
Tomislav Novak 已提交
2719

2720 2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731 2732 2733 2734 2735 2736 2737 2738 2739 2740 2741 2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752 2753 2754 2755 2756 2757 2758 2759 2760 2761 2762
    // 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(
2763
        env_, *cfd->ioptions(), cfd->user_comparator(),
2764
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
2765
        read_options.iterate_upper_bound);
2766

2767
    Iterator* internal_iter =
2768
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
2769 2770 2771 2772
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
2773 2774
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
2775 2776
}

2777
Status DBImpl::NewIterators(
2778
    const ReadOptions& read_options,
I
Igor Canadi 已提交
2779
    const std::vector<ColumnFamilyHandle*>& column_families,
2780
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
2781 2782 2783
  iterators->clear();
  iterators->reserve(column_families.size());

2784
  if (read_options.tailing) {
I
Igor Canadi 已提交
2785 2786 2787 2788
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
2789 2790
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
2791 2792
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
2793
      iterators->push_back(
2794
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2795 2796
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
2797
    }
I
Igor Canadi 已提交
2798
#endif
I
Igor Canadi 已提交
2799
  } else {
2800 2801
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
2802
    for (size_t i = 0; i < column_families.size(); ++i) {
2803 2804 2805
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
2806 2807

      auto snapshot =
2808 2809 2810
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
2811 2812
              : latest_snapshot;

2813
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
2814
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
2815
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
2816
      Iterator* internal_iter = NewInternalIterator(
2817
          read_options, cfd, sv, db_iter->GetArena());
2818 2819
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
2820 2821 2822 2823
    }
  }

  return Status::OK();
2824 2825
}

J
jorlow@chromium.org 已提交
2826
const Snapshot* DBImpl::GetSnapshot() {
2827 2828 2829
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error

2830
  MutexLock l(&mutex_);
2831
  // returns null if the underlying memtable does not support snapshot.
S
sdong 已提交
2832
  if (!is_snapshot_supported_) return nullptr;
2833
  return snapshots_.New(versions_->LastSequence(), unix_time);
J
jorlow@chromium.org 已提交
2834 2835 2836 2837
}

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

// Convenience methods
2842 2843
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
2844
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
2845 2846
}

2847 2848 2849
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
2850
  if (!cfh->cfd()->ioptions()->merge_operator) {
2851 2852
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
2853
    return DB::Merge(o, column_family, key, val);
2854 2855 2856
  }
}

L
Lei Jin 已提交
2857
Status DBImpl::Delete(const WriteOptions& write_options,
2858
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
2859
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
2860 2861
}

L
Lei Jin 已提交
2862
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
S
Stanislau Hlebik 已提交
2863 2864 2865
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
2866
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
I
Igor Canadi 已提交
2867
  WriteThread::Writer w(&mutex_);
S
Stanislau Hlebik 已提交
2868
  w.batch = my_batch;
L
Lei Jin 已提交
2869 2870
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
2871 2872
  w.in_batch_group = false;
  w.done = false;
L
Lei Jin 已提交
2873
  w.timeout_hint_us = write_options.timeout_hint_us;
S
Stanislau Hlebik 已提交
2874 2875

  uint64_t expiration_time = 0;
I
Igor Canadi 已提交
2876
  bool has_timeout = false;
S
Stanislau Hlebik 已提交
2877
  if (w.timeout_hint_us == 0) {
I
Igor Canadi 已提交
2878
    w.timeout_hint_us = WriteThread::kNoTimeOut;
S
Stanislau Hlebik 已提交
2879 2880
  } else {
    expiration_time = env_->NowMicros() + w.timeout_hint_us;
I
Igor Canadi 已提交
2881
    has_timeout = true;
S
Stanislau Hlebik 已提交
2882 2883
  }

L
Lei Jin 已提交
2884
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
2885 2886 2887 2888 2889 2890
    RecordTick(stats_, WRITE_WITH_WAL);
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_WITH_WAL, 1);
  }

  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
2891
  Status status = write_thread_.EnterWriteThread(&w, expiration_time);
S
Stanislau Hlebik 已提交
2892 2893
  assert(status.ok() || status.IsTimedOut());
  if (status.IsTimedOut()) {
2894
    mutex_.Unlock();
L
Lei Jin 已提交
2895
    RecordTick(stats_, WRITE_TIMEDOUT);
2896
    return Status::TimedOut();
2897
  }
S
Stanislau Hlebik 已提交
2898 2899 2900 2901 2902 2903 2904 2905 2906 2907
  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);
2908

2909 2910 2911 2912
  // 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.
2913 2914 2915
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

2916
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
2917
                                    ? 4 * max_total_in_memory_state_
2918
                                    : db_options_.max_total_wal_size;
2919
  if (UNLIKELY(!single_column_family_mode_) &&
2920
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
2921
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
2922
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
2923
    alive_log_files_.begin()->getting_flushed = true;
2924
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
2925 2926 2927
        "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 已提交
2928 2929
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
2930
    for (auto cfd : *versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
2931
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
S
Stanislau Hlebik 已提交
2932
        status = SetNewMemtableAndNewLogFile(cfd, &context);
I
Igor Canadi 已提交
2933 2934 2935
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
2936
        cfd->imm()->FlushRequested();
2937 2938
        SchedulePendingFlush(cfd);
        context.schedule_bg_work_ = true;
2939
      }
2940
    }
2941 2942 2943 2944 2945 2946 2947 2948 2949 2950 2951 2952 2953 2954
  } else if (UNLIKELY(write_buffer_.ShouldFlush())) {
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Flushing all column families. Write buffer is using %" PRIu64
        " bytes out of a total of %" PRIu64 ".",
        write_buffer_.memory_usage(), write_buffer_.buffer_size());
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
    for (auto cfd : *versions_->GetColumnFamilySet()) {
      if (!cfd->mem()->IsEmpty()) {
        status = SetNewMemtableAndNewLogFile(cfd, &context);
        if (!status.ok()) {
          break;
        }
        cfd->imm()->FlushRequested();
2955 2956
        SchedulePendingFlush(cfd);
        context.schedule_bg_work_ = true;
2957 2958 2959
      }
    }
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2960 2961 2962 2963 2964 2965 2966 2967 2968 2969 2970 2971
  }

  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 已提交
2972
    status = DelayWrite(expiration_time);
I
Igor Canadi 已提交
2973 2974 2975 2976 2977
  }

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

D
dgrogan@chromium.org 已提交
2980
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
2981
  WriteThread::Writer* last_writer = &w;
S
Stanislau Hlebik 已提交
2982
  if (status.ok()) {
2983
    autovector<WriteBatch*> write_batch_group;
I
Igor Canadi 已提交
2984
    write_thread_.BuildBatchGroup(&last_writer, &write_batch_group);
2985

2986 2987 2988
    // 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
2989
    // into memtables
2990
    {
2991
      mutex_.Unlock();
2992 2993 2994 2995 2996 2997 2998 2999 3000 3001
      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]);
        }
      }

3002 3003 3004 3005
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
3006
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
3007
      // Record statistics
L
Lei Jin 已提交
3008
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
3009
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
3010
      if (write_options.disableWAL) {
3011
        flush_on_destroy_ = true;
3012
      }
L
Lei Jin 已提交
3013
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3014

3015
      uint64_t log_size = 0;
L
Lei Jin 已提交
3016
      if (!write_options.disableWAL) {
3017
        PERF_TIMER_GUARD(write_wal_time);
3018 3019
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
3020 3021
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
3022
        log_empty_ = false;
3023
        log_size = log_entry.size();
L
Lei Jin 已提交
3024
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
L
Lei Jin 已提交
3025
        if (status.ok() && write_options.sync) {
I
Igor Canadi 已提交
3026
          RecordTick(stats_, WAL_FILE_SYNCED);
3027
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
3028
          if (db_options_.use_fsync) {
3029
            status = log_->file()->Fsync();
3030
          } else {
3031
            status = log_->file()->Sync();
3032
          }
H
heyongqiang 已提交
3033
        }
3034 3035
      }
      if (status.ok()) {
3036
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
3037

3038
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3039
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
3040
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
3041 3042 3043 3044 3045 3046 3047 3048
        // 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 已提交
3049
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
3050
      }
L
Lei Jin 已提交
3051
      PERF_TIMER_START(write_pre_and_post_process_time);
3052 3053 3054
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
3055
      mutex_.Lock();
3056 3057 3058
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
3059 3060
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
3061
      if (!write_options.disableWAL) {
3062 3063 3064 3065 3066
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
3067
      if (status.ok()) {
3068
        versions_->SetLastSequence(last_sequence);
3069
      }
J
jorlow@chromium.org 已提交
3070 3071
    }
  }
3072
  if (db_options_.paranoid_checks && !status.ok() &&
3073
      !status.IsTimedOut() && bg_error_.ok()) {
I
Igor Canadi 已提交
3074 3075
    bg_error_ = status; // stop compaction & fail any further writes
  }
3076

I
Igor Canadi 已提交
3077
  write_thread_.ExitWriteThread(&w, last_writer, status);
3078 3079 3080 3081

  if (context.schedule_bg_work_) {
    MaybeScheduleFlushOrCompaction();
  }
I
Igor Canadi 已提交
3082
  mutex_.Unlock();
3083

3084
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
3085
    RecordTick(stats_, WRITE_TIMEDOUT);
3086 3087
  }

J
jorlow@chromium.org 已提交
3088 3089 3090
  return status;
}

3091 3092
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
3093
Status DBImpl::DelayWrite(uint64_t expiration_time) {
3094 3095 3096 3097 3098
  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();
3099 3100
    // hopefully we don't have to sleep more than 2 billion microseconds
    env_->SleepForMicroseconds(static_cast<int>(delay));
3101 3102 3103
    mutex_.Lock();
  }

I
Igor Canadi 已提交
3104
  while (bg_error_.ok() && write_controller_.IsStopped()) {
3105 3106 3107
    if (has_timeout) {
      bg_cv_.TimedWait(expiration_time);
      if (env_->NowMicros() > expiration_time) {
I
Igor Canadi 已提交
3108
        return Status::TimedOut();
3109 3110 3111 3112 3113
      }
    } else {
      bg_cv_.Wait();
    }
  }
I
Igor Canadi 已提交
3114 3115

  return bg_error_;
3116 3117
}

I
Igor Canadi 已提交
3118 3119 3120 3121
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
    auto status = SetNewMemtableAndNewLogFile(cfd, context);
3122 3123
    SchedulePendingFlush(cfd);
    context->schedule_bg_work_ = true;
I
Igor Canadi 已提交
3124 3125 3126 3127 3128
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
3129 3130
    }
  }
I
Igor Canadi 已提交
3131
  return Status::OK();
S
Stanislau Hlebik 已提交
3132 3133 3134 3135 3136 3137 3138 3139 3140 3141 3142 3143 3144
}

// 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.
3145
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
3146 3147 3148 3149
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
3150
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
3151 3152 3153 3154
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
3155 3156 3157
      s = env_->NewWritableFile(
          LogFileName(db_options_.wal_dir, new_log_number),
          &lfile, env_->OptimizeForLogWrite(env_options_));
S
Stanislau Hlebik 已提交
3158 3159 3160
      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 已提交
3161 3162
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
S
Stanislau Hlebik 已提交
3163
        new_log = new log::Writer(std::move(lfile));
I
Igor Canadi 已提交
3164
      }
S
Stanislau Hlebik 已提交
3165 3166 3167
    }

    if (s.ok()) {
3168
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options);
S
Stanislau Hlebik 已提交
3169 3170 3171
      new_superversion = new SuperVersion();
    }
  }
3172
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
3173 3174
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
3175 3176 3177 3178 3179 3180 3181 3182 3183 3184 3185 3186 3187 3188 3189
  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 已提交
3190
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
3191 3192 3193 3194
      // 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 已提交
3195 3196 3197
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
          loop_cfd->imm()->size() == 0) {
        loop_cfd->SetLogNumber(logfile_number_);
3198
      }
3199 3200
    }
  }
S
Stanislau Hlebik 已提交
3201 3202 3203 3204 3205
  cfd->mem()->SetNextLogNumber(logfile_number_);
  cfd->imm()->Add(cfd->mem());
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
  context->superversions_to_free_.push_back(
3206
      InstallSuperVersion(cfd, new_superversion, mutable_cf_options, true));
3207 3208 3209
  return s;
}

I
Igor Canadi 已提交
3210
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
3211 3212 3213 3214 3215
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3216 3217
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
3218
  auto version = cfd->current();
3219 3220 3221 3222 3223 3224 3225 3226 3227 3228 3229 3230
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
3231
#endif  // ROCKSDB_LITE
3232

I
Igor Canadi 已提交
3233 3234 3235 3236
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3237 3238 3239 3240
Env* DBImpl::GetEnv() const {
  return env_;
}

3241 3242
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
3243
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
3244 3245
}

3246
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
3247
                         const Slice& property, std::string* value) {
3248 3249
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3250 3251 3252
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

3253
  value->clear();
3254 3255 3256 3257 3258
  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) {
3259
      *value = ToString(int_value);
3260 3261 3262 3263 3264 3265 3266 3267 3268
    }
    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 已提交
3269 3270
}

3271 3272
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
3273 3274
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3275 3276 3277 3278 3279 3280 3281 3282 3283 3284 3285 3286
  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) {
3287 3288
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
3289 3290 3291

  if (!need_out_of_mutex) {
    MutexLock l(&mutex_);
3292
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
3293 3294 3295 3296 3297 3298 3299 3300 3301 3302 3303 3304 3305 3306
  } 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 已提交
3307
  return cfd->GetThreadLocalSuperVersion(&mutex_);
3308 3309 3310 3311
}

void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
3312
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3313 3314 3315 3316 3317 3318 3319 3320 3321 3322 3323 3324 3325

  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);
  }
3326 3327
}

3328
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
3329
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
3330 3331
  // TODO(opt): better implementation
  Version* v;
3332 3333
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
3334 3335
  {
    MutexLock l(&mutex_);
3336
    v = cfd->current();
3337
    v->Ref();
J
jorlow@chromium.org 已提交
3338 3339 3340 3341 3342 3343 3344 3345 3346 3347 3348 3349 3350 3351 3352 3353 3354
  }

  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 已提交
3355 3356 3357 3358 3359 3360 3361 3362 3363 3364 3365 3366 3367 3368 3369 3370
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 已提交
3371 3372 3373 3374 3375
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
3376
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
3377 3378 3379
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
3380
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
3381 3382
}

3383 3384 3385
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
3386 3387 3388
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
3389 3390
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
3391 3392 3393
    return Status::InvalidArgument("Invalid file name");
  }

3394 3395 3396 3397
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
3398 3399
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
3400
          name.c_str());
3401 3402
      return Status::NotSupported("Delete only supported for archived logs");
    }
3403
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
3404
    if (!status.ok()) {
3405 3406
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
3407
          name.c_str(), status.ToString().c_str());
3408 3409 3410 3411
    }
    return status;
  }

3412
  int level;
I
Igor Canadi 已提交
3413
  FileMetaData* metadata;
3414
  ColumnFamilyData* cfd;
3415
  VersionEdit edit;
I
Igor Canadi 已提交
3416
  JobContext job_context(true);
D
Dhruba Borthakur 已提交
3417 3418
  {
    MutexLock l(&mutex_);
3419
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
3420
    if (!status.ok()) {
3421 3422
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
3423
      job_context.Clean();
D
Dhruba Borthakur 已提交
3424 3425
      return Status::InvalidArgument("File not found");
    }
3426
    assert(level < cfd->NumberLevels());
3427

D
Dhruba Borthakur 已提交
3428
    // If the file is being compacted no need to delete.
3429
    if (metadata->being_compacted) {
3430
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3431
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
3432
      job_context.Clean();
D
Dhruba Borthakur 已提交
3433
      return Status::OK();
3434 3435
    }

D
Dhruba Borthakur 已提交
3436 3437 3438
    // 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 已提交
3439
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
3440
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
3441
      if (vstoreage->NumLevelFiles(i) != 0) {
3442
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
3443
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
3444
        job_context.Clean();
D
Dhruba Borthakur 已提交
3445 3446 3447
        return Status::InvalidArgument("File not in last level");
      }
    }
3448
    // if level == 0, it has to be the oldest file
S
sdong 已提交
3449 3450
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
3451 3452
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
3453
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
3454
      job_context.Clean();
3455 3456 3457
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
3458
    edit.DeleteFile(level, number);
3459 3460
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                    &edit, &mutex_, db_directory_.get());
I
Igor Canadi 已提交
3461
    if (status.ok()) {
I
Igor Canadi 已提交
3462 3463
      InstallSuperVersionBackground(cfd, &job_context,
                                    *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
3464
    }
I
Igor Canadi 已提交
3465 3466
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
3467
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
3468
  // remove files outside the db-lock
I
Igor Canadi 已提交
3469 3470
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
3471
  }
I
Igor Canadi 已提交
3472
  job_context.Clean();
3473 3474 3475
  return status;
}

3476
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
3477
  MutexLock l(&mutex_);
3478
  versions_->GetLiveFilesMetaData(metadata);
3479
}
3480 3481 3482 3483 3484 3485 3486 3487 3488 3489 3490

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 已提交
3491
#endif  // ROCKSDB_LITE
3492

I
Igor Canadi 已提交
3493 3494 3495 3496 3497 3498 3499
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
3502 3503 3504 3505 3506 3507
    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) {
3508
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
3509
                             ". Size recorded in manifest " +
3510 3511
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
3512 3513 3514 3515 3516 3517 3518 3519 3520
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

3521 3522 3523 3524 3525 3526 3527 3528 3529 3530 3531 3532 3533 3534 3535
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;
3536
  s = idfile->Read(static_cast<size_t>(file_size), &id, buffer);
3537 3538 3539 3540 3541 3542 3543 3544 3545 3546 3547
  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 已提交
3548 3549
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
3550
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
3551
               const Slice& key, const Slice& value) {
3552 3553 3554 3555
  // 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);
3556
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
3557 3558 3559
  return Write(opt, &batch);
}

3560 3561
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
3562
  WriteBatch batch;
3563
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
3564 3565 3566
  return Write(opt, &batch);
}

3567 3568
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
3569
  WriteBatch batch;
3570
  batch.Merge(column_family, key, value);
3571 3572 3573
  return Write(opt, &batch);
}

3574
// Default implementation -- returns not supported status
L
Lei Jin 已提交
3575
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3576
                              const std::string& column_family_name,
3577
                              ColumnFamilyHandle** handle) {
3578
  return Status::NotSupported("");
3579
}
3580
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
3581
  return Status::NotSupported("");
3582 3583
}

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

J
Jim Paton 已提交
3586
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
3587 3588 3589 3590
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
3591
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
3592
  std::vector<ColumnFamilyHandle*> handles;
3593
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
3594 3595 3596 3597 3598 3599 3600
  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;
3601 3602
}

3603 3604
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
3605
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
3606
  Status s = SanitizeOptionsByTable(db_options, column_families);
3607 3608 3609
  if (!s.ok()) {
    return s;
  }
3610

3611
  if (db_options.db_paths.size() > 1) {
3612
    for (auto& cfd : column_families) {
3613 3614
      if ((cfd.options.compaction_style != kCompactionStyleUniversal) &&
          (cfd.options.compaction_style != kCompactionStyleLevel)) {
3615 3616
        return Status::NotSupported(
            "More than one DB paths are only supported in "
3617
            "universal and level compaction styles. ");
3618 3619 3620 3621 3622
      }
    }

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

3627
  *dbptr = nullptr;
3628
  handles->clear();
J
jorlow@chromium.org 已提交
3629

I
Igor Canadi 已提交
3630 3631 3632 3633
  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);
3634
  }
3635

I
Igor Canadi 已提交
3636
  DBImpl* impl = new DBImpl(db_options, dbname);
3637
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
3638
  if (s.ok()) {
3639
    for (auto db_path : impl->db_options_.db_paths) {
3640
      s = impl->env_->CreateDirIfMissing(db_path.path);
3641 3642 3643 3644 3645 3646
      if (!s.ok()) {
        break;
      }
    }
  }

3647 3648 3649 3650 3651 3652
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
3653 3654 3655 3656
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
3657
  impl->mutex_.Lock();
3658 3659
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
3660
  if (s.ok()) {
3661
    uint64_t new_log_number = impl->versions_->NewFileNumber();
3662
    unique_ptr<WritableFile> lfile;
3663
    EnvOptions soptions(db_options);
3664 3665 3666
    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 已提交
3667
    if (s.ok()) {
3668
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
3669
      impl->logfile_number_ = new_log_number;
3670
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
3671

3672 3673
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
3674 3675
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
3676 3677 3678
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
3679
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
3680 3681 3682 3683 3684 3685 3686 3687 3688 3689 3690 3691 3692 3693 3694 3695
        } 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 已提交
3696
        }
3697
      }
I
Igor Canadi 已提交
3698 3699
    }
    if (s.ok()) {
3700
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
3701 3702
        delete impl->InstallSuperVersion(
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
3703
      }
I
Igor Canadi 已提交
3704 3705
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
3706
      impl->DeleteObsoleteFiles();
3707
      s = impl->db_directory_->Fsync();
J
jorlow@chromium.org 已提交
3708 3709
    }
  }
3710

I
Igor Canadi 已提交
3711 3712
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
3713 3714
      if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
3715
        auto* vstorage = cfd->current()->storage_info();
3716
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
3717
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
3718
          if (num_files > 0) {
I
Igor Canadi 已提交
3719 3720 3721
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
                "open with universal or FIFO compaction style.");
I
Igor Canadi 已提交
3722 3723 3724 3725
            break;
          }
        }
      }
S
sdong 已提交
3726 3727 3728
      if (!cfd->mem()->IsSnapshotSupported()) {
        impl->is_snapshot_supported_ = false;
      }
3729
      if (cfd->ioptions()->merge_operator != nullptr &&
3730 3731 3732 3733 3734
          !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 已提交
3735
      if (!s.ok()) {
3736 3737 3738 3739 3740
        break;
      }
    }
  }

3741 3742
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
3743
  if (s.ok()) {
3744
    impl->opened_successfully_ = true;
J
jorlow@chromium.org 已提交
3745 3746
    *dbptr = impl;
  } else {
Y
Yueh-Hsuan Chiang 已提交
3747
    for (auto* h : *handles) {
3748 3749
      delete h;
    }
3750
    handles->clear();
J
jorlow@chromium.org 已提交
3751 3752 3753 3754 3755
    delete impl;
  }
  return s;
}

3756 3757 3758
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
3759
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
3760 3761
}

3762 3763 3764
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
3765
Status DestroyDB(const std::string& dbname, const Options& options) {
3766
  const InternalKeyComparator comparator(options.comparator);
3767
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
3768
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
3769
  std::vector<std::string> filenames;
3770 3771
  std::vector<std::string> archiveFiles;

3772
  std::string archivedir = ArchivalDirectory(dbname);
J
jorlow@chromium.org 已提交
3773 3774
  // Ignore error in case directory does not exist
  env->GetChildren(dbname, &filenames);
3775 3776 3777 3778 3779 3780 3781

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

J
jorlow@chromium.org 已提交
3783 3784 3785 3786 3787
  if (filenames.empty()) {
    return Status::OK();
  }

  FileLock* lock;
3788 3789
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
3790 3791 3792
  if (result.ok()) {
    uint64_t number;
    FileType type;
3793
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
3794
    for (size_t i = 0; i < filenames.size(); i++) {
3795
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
3796
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
3797 3798 3799
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
3800 3801
        } else if (type == kLogFile) {
          del = env->DeleteFile(soptions.wal_dir + "/" + filenames[i]);
K
Kosie van der Merwe 已提交
3802 3803 3804
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
3805 3806 3807 3808 3809
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3810

3811 3812
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
3813 3814 3815
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
3816
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
3817 3818 3819 3820 3821 3822 3823
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

3824
    env->GetChildren(archivedir, &archiveFiles);
3825 3826
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
3827 3828
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
3829
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
3830 3831 3832 3833 3834
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
3835
    // ignore case where no archival directory is present.
3836
    env->DeleteDir(archivedir);
3837

J
jorlow@chromium.org 已提交
3838
    env->UnlockFile(lock);  // Ignore error since state is already gone
3839
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
3840
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
3841
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
3842 3843 3844 3845
  }
  return result;
}

Y
Yueh-Hsuan Chiang 已提交
3846 3847 3848
#if ROCKSDB_USING_THREAD_STATUS
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
3849 3850 3851 3852
  if (db_options_.enable_thread_tracking) {
    ThreadStatusImpl::NewColumnFamilyInfo(
        this, GetName(), cfd, cfd->GetName());
  }
Y
Yueh-Hsuan Chiang 已提交
3853 3854 3855 3856
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
3857 3858 3859
  if (db_options_.enable_thread_tracking) {
    ThreadStatusImpl::EraseColumnFamilyInfo(cfd);
  }
Y
Yueh-Hsuan Chiang 已提交
3860 3861 3862
}

void DBImpl::EraseThreadStatusDbInfo() const {
3863 3864 3865
  if (db_options_.enable_thread_tracking) {
    ThreadStatusImpl::EraseDatabaseInfo(this);
  }
Y
Yueh-Hsuan Chiang 已提交
3866 3867 3868 3869 3870 3871 3872 3873 3874 3875 3876 3877 3878 3879 3880 3881 3882 3883
}

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

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

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

3884 3885
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
3886
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
3887
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
3888 3889
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
3890 3891
  Log(InfoLogLevel::INFO_LEVEL, log,
      "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
I
Igor Canadi 已提交
3892
      ROCKSDB_PATCH);
3893 3894
  Log(InfoLogLevel::INFO_LEVEL, log, "Git sha %s", rocksdb_build_git_sha);
  Log(InfoLogLevel::INFO_LEVEL, log, "Compile time %s %s",
3895
      rocksdb_build_compile_time, rocksdb_build_compile_date);
I
Igor Canadi 已提交
3896
#endif
3897 3898
}

3899
}  // namespace rocksdb