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

89
namespace rocksdb {
J
jorlow@chromium.org 已提交
90

91
const std::string kDefaultColumnFamilyName("default");
92

I
Igor Canadi 已提交
93
void DumpRocksDBBuildVersion(Logger * log);
94

S
Stanislau Hlebik 已提交
95 96
struct DBImpl::WriteContext {
  autovector<SuperVersion*> superversions_to_free_;
97
  autovector<MemTable*> memtables_to_free_;
98
  bool schedule_bg_work_ = false;
S
Stanislau Hlebik 已提交
99 100 101 102 103

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

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

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

121 122
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
123
    ClipToRange(&result.max_open_files, 20, 1000000);
124
  }
125

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

I
Igor Canadi 已提交
139
  if (result.rate_limiter.get() != nullptr) {
140 141 142 143 144
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

145 146 147 148
  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
149
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
150
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
151
  }
152

153
  if (result.db_paths.size() == 0) {
154
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
155 156
  }

J
jorlow@chromium.org 已提交
157 158 159
  return result;
}

160 161
namespace {

162 163
Status SanitizeOptionsByTable(
    const DBOptions& db_opts,
164 165 166
    const std::vector<ColumnFamilyDescriptor>& column_families) {
  Status s;
  for (auto cf : column_families) {
167
    s = cf.options.table_factory->SanitizeOptions(db_opts, cf.options);
168 169 170 171 172 173 174
    if (!s.ok()) {
      return s;
    }
  }
  return Status::OK();
}

175
CompressionType GetCompressionFlush(const ImmutableCFOptions& ioptions) {
176 177 178 179 180 181
  // 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;

182
  if (ioptions.compaction_style == kCompactionStyleUniversal) {
183
    can_compress =
184
        (ioptions.compaction_options_universal.compression_size_percent < 0);
185 186
  } else {
    // For leveled compress when min_level_to_compress == 0.
187 188
    can_compress = ioptions.compression_per_level.empty() ||
                   ioptions.compression_per_level[0] != kNoCompression;
189 190 191
  }

  if (can_compress) {
192
    return ioptions.compression;
193 194 195 196
  } else {
    return kNoCompression;
  }
}
I
Igor Canadi 已提交
197 198

void DumpCompressionInfo(Logger* logger) {
I
Igor Canadi 已提交
199 200 201 202 203 204 205 206
  Log(InfoLogLevel::INFO_LEVEL, logger, "Compression algorithms supported:");
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tSnappy supported: %d",
      Snappy_Supported());
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tZlib supported: %d",
      Zlib_Supported());
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tBzip supported: %d",
      BZip2_Supported());
  Log(InfoLogLevel::INFO_LEVEL, logger, "\tLZ4 supported: %d", LZ4_Supported());
I
Igor Canadi 已提交
207 208
}

209
}  // namespace
210

I
Igor Canadi 已提交
211
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
J
jorlow@chromium.org 已提交
212
    : env_(options.env),
H
heyongqiang 已提交
213
      dbname_(dbname),
214 215
      db_options_(SanitizeOptions(dbname, options)),
      stats_(db_options_.statistics.get()),
216
      db_lock_(nullptr),
I
Igor Canadi 已提交
217
      mutex_(stats_, env_, DB_MUTEX_WAIT_MICROS, options.use_adaptive_mutex),
I
Igor Canadi 已提交
218
      shutting_down_(false),
J
jorlow@chromium.org 已提交
219
      bg_cv_(&mutex_),
220
      logfile_number_(0),
221
      log_dir_synced_(false),
I
Igor Canadi 已提交
222
      log_empty_(true),
223
      default_cf_handle_(nullptr),
I
Igor Canadi 已提交
224 225
      total_log_size_(0),
      max_total_in_memory_state_(0),
S
sdong 已提交
226
      is_snapshot_supported_(true),
227
      write_buffer_(options.db_write_buffer_size),
228 229
      unscheduled_flushes_(0),
      unscheduled_compactions_(0),
230
      bg_compaction_scheduled_(0),
231
      bg_manual_only_(0),
232
      bg_flush_scheduled_(0),
233
      manual_compaction_(nullptr),
234
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
235 236 237
      delete_obsolete_files_next_run_(
          options.env->NowMicros() +
          db_options_.delete_obsolete_files_period_micros),
238
      last_stats_dump_time_microsec_(0),
239
      next_job_id_(1),
240
      flush_on_destroy_(false),
241
      env_options_(db_options_),
I
Igor Canadi 已提交
242 243 244
#ifndef ROCKSDB_LITE
      wal_manager_(db_options_, env_options_),
#endif  // ROCKSDB_LITE
I
Igor Canadi 已提交
245
      event_logger_(db_options_.info_log.get()),
246
      bg_work_gate_closed_(false),
247
      refitting_level_(false),
248
      opened_successfully_(false) {
H
heyongqiang 已提交
249
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
250

J
jorlow@chromium.org 已提交
251
  // Reserve ten files or so for other uses and give the rest to TableCache.
252
  // Give a large number for setting of "infinite" open files.
253 254
  const int table_cache_size = (db_options_.max_open_files == -1) ?
        4194304 : db_options_.max_open_files - 10;
255 256
  // Reserve ten files or so for other uses and give the rest to TableCache.
  table_cache_ =
257
      NewLRUCache(table_cache_size, db_options_.table_cache_numshardbits);
258

259
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
260 261
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
I
Igor Canadi 已提交
262 263
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
264

I
Igor Canadi 已提交
265
  DumpRocksDBBuildVersion(db_options_.info_log.get());
266 267
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
I
Igor Canadi 已提交
268
  DumpCompressionInfo(db_options_.info_log.get());
269

270
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
271 272
}

273 274
// Will only lock the mutex_ and wait for completion if wait is true
void DBImpl::CancelAllBackgroundWork(bool wait) {
275
  shutting_down_.store(true, std::memory_order_release);
276 277 278 279 280
  if (!wait) {
    return;
  }
  // Wait for background work to finish
  mutex_.Lock();
281
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
282 283 284
    bg_cv_.Wait();
  }
  mutex_.Unlock();
285 286
}

J
jorlow@chromium.org 已提交
287
DBImpl::~DBImpl() {
Y
Yueh-Hsuan Chiang 已提交
288
  EraseThreadStatusDbInfo();
289
  mutex_.Lock();
290

291 292
  if (flush_on_destroy_) {
    for (auto cfd : *versions_->GetColumnFamilySet()) {
293
      if (!cfd->IsDropped() && !cfd->mem()->IsEmpty()) {
294 295 296 297
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
298
        cfd->Unref();
299 300
      }
    }
301
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
302
  }
303 304 305 306 307
  // CancelAllBackgroundWork called with false means we just set the
  // shutdown marker, while holding the mutex_ here. After which we
  // do a variant of the waiting after we release the lock and unschedule work
  // (to consider: moving all the waiting into CancelAllBackgroundWork(true))
  CancelAllBackgroundWork(false);
308 309 310 311 312 313 314 315
  mutex_.Unlock();
  int compactions_unscheduled = env_->UnSchedule(this, Env::Priority::LOW);
  int flushes_unscheduled = env_->UnSchedule(this, Env::Priority::HIGH);
  mutex_.Lock();
  bg_compaction_scheduled_ -= compactions_unscheduled;
  bg_flush_scheduled_ -= flushes_unscheduled;

  // Wait for background work to finish
316
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
317 318
    bg_cv_.Wait();
  }
I
Igor Canadi 已提交
319 320
  flush_scheduler_.Clear();

321 322 323 324 325 326 327 328 329 330 331 332 333
  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 已提交
334 335 336 337 338
  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();
339 340
  }

I
Igor Canadi 已提交
341 342 343 344 345 346 347 348 349 350
  // 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_) {
351
    JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
352
    FindObsoleteFiles(&job_context, true);
I
Igor Canadi 已提交
353
    // manifest number starting from 2
I
Igor Canadi 已提交
354 355 356
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
357
    }
I
Igor Canadi 已提交
358
    job_context.Clean();
359 360
  }

361 362 363 364
  for (auto l : logs_to_free_) {
    delete l;
  }

365
  // versions need to be destroyed before table_cache since it can hold
366 367
  // references to table_cache.
  versions_.reset();
368
  mutex_.Unlock();
I
Igor Canadi 已提交
369 370 371
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
372

373
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
374 375 376
}

Status DBImpl::NewDB() {
377
  VersionEdit new_db;
378
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
379 380 381
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

382 383
  Log(InfoLogLevel::INFO_LEVEL,
      db_options_.info_log, "Creating manifest 1 \n");
J
jorlow@chromium.org 已提交
384
  const std::string manifest = DescriptorFileName(dbname_, 1);
385
  unique_ptr<WritableFile> file;
I
Igor Canadi 已提交
386
  Status s = env_->NewWritableFile(
L
Lei Jin 已提交
387
      manifest, &file, env_->OptimizeForManifestWrite(env_options_));
J
jorlow@chromium.org 已提交
388 389 390
  if (!s.ok()) {
    return s;
  }
391
  file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
J
jorlow@chromium.org 已提交
392
  {
393
    log::Writer log(std::move(file));
J
jorlow@chromium.org 已提交
394 395 396
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
S
sdong 已提交
397 398 399
    if (s.ok()) {
      s = SyncManifest(env_, &db_options_, log.file());
    }
J
jorlow@chromium.org 已提交
400 401 402
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
403
    s = SetCurrentFile(env_, dbname_, 1, directories_.GetDbDir());
J
jorlow@chromium.org 已提交
404 405 406 407 408 409 410
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
411
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
412 413
    // No change needed
  } else {
414 415
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
416 417 418 419
    *s = Status::OK();
  }
}

420
const Status DBImpl::CreateArchivalDirectory() {
421 422
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
423 424 425 426 427
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

428
void DBImpl::PrintStatistics() {
429
  auto dbstats = db_options_.statistics.get();
430
  if (dbstats) {
Y
Yueh-Hsuan Chiang 已提交
431
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
M
Mark Callaghan 已提交
432
        "STATISTICS:\n %s",
433
        dbstats->ToString().c_str());
434 435 436
  }
}

437
void DBImpl::MaybeDumpStats() {
438
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
439 440 441 442

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
443
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
444 445 446 447 448 449
      <= 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;
450

451
#ifndef ROCKSDB_LITE
452 453 454
    bool tmp1 = false;
    bool tmp2 = false;
    DBPropertyType cf_property_type =
455
        GetPropertyType(DB::Properties::kCFStats, &tmp1, &tmp2);
456
    DBPropertyType db_property_type =
457
        GetPropertyType(DB::Properties::kDBStats, &tmp1, &tmp2);
H
Haobo Xu 已提交
458
    std::string stats;
459
    {
460
      InstrumentedMutexLock l(&mutex_);
461
      for (auto cfd : *versions_->GetColumnFamilySet()) {
462
        cfd->internal_stats()->GetStringProperty(cf_property_type,
463 464
                                                 DB::Properties::kCFStats,
                                                 &stats);
465
      }
466
      default_cf_internal_stats_->GetStringProperty(db_property_type,
467 468
                                                    DB::Properties::kDBStats,
                                                    &stats);
469
    }
Y
Yueh-Hsuan Chiang 已提交
470
    Log(InfoLogLevel::WARN_LEVEL,
471
        db_options_.info_log, "------- DUMPING STATS -------");
Y
Yueh-Hsuan Chiang 已提交
472
    Log(InfoLogLevel::WARN_LEVEL,
473
        db_options_.info_log, "%s", stats.c_str());
474
#endif  // !ROCKSDB_LITE
475

476
    PrintStatistics();
477 478 479
  }
}

I
Igor Canadi 已提交
480
// * Returns the list of live files in 'sst_live'
I
Igor Canadi 已提交
481
// If it's doing full scan:
I
Igor Canadi 已提交
482 483
// * Returns the list of all files in the filesystem in
// 'full_scan_candidate_files'.
I
Igor Canadi 已提交
484
// Otherwise, gets obsolete files from VersionSet.
I
Igor Canadi 已提交
485 486
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
487
//  db_options_.delete_obsolete_files_period_micros
I
Igor Canadi 已提交
488
// force = true -- force the full scan
I
Igor Canadi 已提交
489
void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
I
Igor Canadi 已提交
490
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
491 492
  mutex_.AssertHeld();

493
  // if deletion is disabled, do nothing
494
  if (disable_delete_obsolete_files_ > 0) {
495 496 497
    return;
  }

498 499 500 501 502
  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;
503
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
504 505 506
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
I
Igor Canadi 已提交
507
    if (delete_obsolete_files_next_run_ < now_micros) {
508
      doing_the_full_scan = true;
I
Igor Canadi 已提交
509 510
      delete_obsolete_files_next_run_ =
          now_micros + db_options_.delete_obsolete_files_period_micros;
511 512 513
    }
  }

I
Igor Canadi 已提交
514 515 516 517 518 519 520 521 522
  // don't delete files that might be currently written to from compaction
  // threads
  if (!pending_outputs_.empty()) {
    job_context->min_pending_output = *pending_outputs_.begin();
  } else {
    // delete all of them
    job_context->min_pending_output = std::numeric_limits<uint64_t>::max();
  }

I
Igor Canadi 已提交
523
  // get obsolete files
I
Igor Canadi 已提交
524 525
  versions_->GetObsoleteFiles(&job_context->sst_delete_files,
                              job_context->min_pending_output);
I
Igor Canadi 已提交
526

I
Igor Canadi 已提交
527
  // store the current filenum, lognum, etc
528
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
529
  job_context->pending_manifest_file_number =
530
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
531
  job_context->log_number = versions_->MinLogNumber();
532
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
533

I
Igor Canadi 已提交
534
  versions_->AddLiveFiles(&job_context->sst_live);
535
  if (doing_the_full_scan) {
I
Igor Canadi 已提交
536 537
    for (uint32_t path_id = 0; path_id < db_options_.db_paths.size();
         path_id++) {
538 539 540
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
541
      env_->GetChildren(db_options_.db_paths[path_id].path,
542
                        &files);  // Ignore errors
543
      for (std::string file : files) {
I
Igor Canadi 已提交
544
        // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
I
Igor Canadi 已提交
545 546
        job_context->full_scan_candidate_files.emplace_back("/" + file,
                                                            path_id);
547 548
      }
    }
549 550

    //Add log files in wal_dir
551
    if (db_options_.wal_dir != dbname_) {
552
      std::vector<std::string> log_files;
553
      env_->GetChildren(db_options_.wal_dir, &log_files);  // Ignore errors
554
      for (std::string log_file : log_files) {
I
Igor Canadi 已提交
555
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
556 557
      }
    }
558
    // Add info log files in db_log_dir
559
    if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) {
560
      std::vector<std::string> info_log_files;
561 562
      // Ignore errors
      env_->GetChildren(db_options_.db_log_dir, &info_log_files);
563
      for (std::string log_file : info_log_files) {
I
Igor Canadi 已提交
564
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
565 566
      }
    }
567
  }
568 569
}

570
namespace {
I
Igor Canadi 已提交
571 572
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
573 574 575 576 577
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
578
    return (first.path_id > second.path_id);
579 580 581 582
  }
}
};  // namespace

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

I
Igor Canadi 已提交
591 592 593 594
  // 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 已提交
595 596
    return;
  }
597

598
  // Now, convert live list to an unordered map, WITHOUT mutex held;
599
  // set is slow.
600
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
I
Igor Canadi 已提交
601
  for (const FileDescriptor& fd : state.sst_live) {
602 603
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
604

I
Igor Canadi 已提交
605
  auto candidate_files = state.full_scan_candidate_files;
I
Igor Canadi 已提交
606 607 608
  candidate_files.reserve(candidate_files.size() +
                          state.sst_delete_files.size() +
                          state.log_delete_files.size());
K
kailiu 已提交
609 610
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
611
  for (auto file : state.sst_delete_files) {
612 613 614
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
615
    delete file;
I
Igor Canadi 已提交
616 617
  }

K
kailiu 已提交
618 619
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
620 621
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
622 623
    }
  }
624

K
kailiu 已提交
625
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
626
  // file twice
627
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
628 629
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
630

631
  std::vector<std::string> old_info_log_files;
632
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
633 634 635
  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 已提交
636 637 638
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
639
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
640 641
      continue;
    }
J
jorlow@chromium.org 已提交
642

K
kailiu 已提交
643 644 645 646 647 648 649 650
    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'
651
        // (can happen during manifest roll)
K
kailiu 已提交
652 653 654
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
I
Igor Canadi 已提交
655 656 657 658
        // 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 已提交
659 660 661
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
662 663 664 665
        // 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
666
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
667
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
668 669 670 671
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
672
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
673
        }
K
kailiu 已提交
674 675 676 677 678 679 680 681 682 683 684 685 686
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

687
    std::string fname;
K
kailiu 已提交
688 689
    if (type == kTableFile) {
      // evict from cache
690
      TableCache::Evict(table_cache_.get(), number);
691
      fname = TableFileName(db_options_.db_paths, number, path_id);
692
    } else {
693 694
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
695
    }
696

697
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
698
    if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 ||
699
                              db_options_.WAL_size_limit_MB > 0)) {
I
Igor Canadi 已提交
700
      wal_manager_.ArchiveWALFile(fname, number);
701 702 703 704 705
      continue;
    }
#endif  // !ROCKSDB_LITE
    auto file_deletion_status = env_->DeleteFile(fname);
    if (file_deletion_status.ok()) {
706
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
707
          "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", state.job_id,
708 709 710 711 712 713 714 715 716 717 718 719 720
          fname.c_str(), type, number,
          file_deletion_status.ToString().c_str());
    } else {
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "[JOB %d] Failed to delete %s type=%d #%" PRIu64 " -- %s\n",
          state.job_id, fname.c_str(), type, number,
          file_deletion_status.ToString().c_str());
    }
    if (type == kTableFile) {
      event_logger_.Log() << "job" << state.job_id << "event"
                          << "table_file_deletion"
                          << "file_number" << number
                          << "status" << file_deletion_status.ToString();
J
jorlow@chromium.org 已提交
721 722
    }
  }
H
heyongqiang 已提交
723

724
  // Delete old info log files.
725
  size_t old_info_log_file_count = old_info_log_files.size();
726
  if (old_info_log_file_count >= db_options_.keep_log_file_num) {
727
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
728
    size_t end = old_info_log_file_count - db_options_.keep_log_file_num;
729
    for (unsigned int i = 0; i <= end; i++) {
730
      std::string& to_delete = old_info_log_files.at(i);
731 732
      std::string full_path_to_delete = (db_options_.db_log_dir.empty() ?
           dbname_ : db_options_.db_log_dir) + "/" + to_delete;
733
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
734
          "[JOB %d] Delete info log file %s\n", state.job_id,
735 736
          full_path_to_delete.c_str());
      Status s = env_->DeleteFile(full_path_to_delete);
737
      if (!s.ok()) {
738 739
        Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
            "[JOB %d] Delete info log file %s FAILED -- %s\n", state.job_id,
740 741
            to_delete.c_str(), s.ToString().c_str());
      }
H
heyongqiang 已提交
742 743
    }
  }
I
Igor Canadi 已提交
744 745 746
#ifndef ROCKSDB_LITE
  wal_manager_.PurgeObsoleteWALFiles();
#endif  // ROCKSDB_LITE
747
  LogFlush(db_options_.info_log);
D
Dhruba Borthakur 已提交
748 749 750 751
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
752
  JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
753 754 755
  FindObsoleteFiles(&job_context, true);
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
756
  }
I
Igor Canadi 已提交
757
  job_context.Clean();
758 759
}

760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775
Status DBImpl::Directories::CreateAndNewDirectory(
    Env* env, const std::string& dirname,
    std::unique_ptr<Directory>* directory) const {
  // We call CreateDirIfMissing() as the directory may already exist (if we
  // are reopening a DB), when this happens we don't want creating the
  // directory to cause an error. However, we need to check if creating the
  // directory fails or else we may get an obscure message about the lock
  // file not existing. One real-world example of this occurring is if
  // env->CreateDirIfMissing() doesn't create intermediate directories, e.g.
  // when dbname_ is "dir/db" but when "dir" doesn't exist.
  Status s = env->CreateDirIfMissing(dirname);
  if (!s.ok()) {
    return s;
  }
  return env->NewDirectory(dirname, directory);
}
J
jorlow@chromium.org 已提交
776

777 778 779 780 781 782 783 784 785
Status DBImpl::Directories::SetDirectories(
    Env* env, const std::string& dbname, const std::string& wal_dir,
    const std::vector<DbPath>& data_paths) {
  Status s = CreateAndNewDirectory(env, dbname, &db_dir_);
  if (!s.ok()) {
    return s;
  }
  if (!wal_dir.empty() && dbname != wal_dir) {
    s = CreateAndNewDirectory(env, wal_dir, &wal_dir_);
786 787 788
    if (!s.ok()) {
      return s;
    }
789
  }
790

791 792 793 794 795 796 797 798
  data_dirs_.clear();
  for (auto& p : data_paths) {
    const std::string db_path = p.path;
    if (db_path == dbname) {
      data_dirs_.emplace_back(nullptr);
    } else {
      std::unique_ptr<Directory> path_directory;
      s = CreateAndNewDirectory(env, db_path, &path_directory);
799 800 801
      if (!s.ok()) {
        return s;
      }
802
      data_dirs_.emplace_back(path_directory.release());
803
    }
804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822
  }
  assert(data_dirs_.size() == data_paths.size());
  return Status::OK();
}

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

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

824 825 826 827 828
  bool is_new_db = false;
  assert(db_lock_ == nullptr);
  if (!read_only) {
    Status s = directories_.SetDirectories(env_, dbname_, db_options_.wal_dir,
                                           db_options_.db_paths);
829 830 831 832
    if (!s.ok()) {
      return s;
    }

833
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
834 835 836
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
837

838
    if (!env_->FileExists(CurrentFileName(dbname_))) {
839
      if (db_options_.create_if_missing) {
840
        s = NewDB();
841
        is_new_db = true;
842 843 844 845 846 847
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
848 849
      }
    } else {
850
      if (db_options_.error_if_exists) {
851 852 853
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
854
    }
M
Mayank Agarwal 已提交
855 856 857 858 859 860 861
    // 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 已提交
862 863
  }

864
  Status s = versions_->Recover(column_families, read_only);
865
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
866 867
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
868
  if (s.ok()) {
A
agiardullo 已提交
869
    SequenceNumber max_sequence(kMaxSequenceNumber);
870 871
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
872
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
873 874
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
875 876 877 878 879

    // 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).
    //
880
    // Note that prev_log_number() is no longer used, but we pay
881
    // attention to it in case we are recovering a database
882
    // produced by an older version of rocksdb.
883
    const uint64_t min_log = versions_->MinLogNumber();
884
    const uint64_t prev_log = versions_->prev_log_number();
885
    std::vector<std::string> filenames;
886
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
887 888
    if (!s.ok()) {
      return s;
889
    }
K
kailiu 已提交
890

891 892
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
893 894
      uint64_t number;
      FileType type;
895 896 897 898 899 900 901 902 903
      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);
        }
904
      }
J
jorlow@chromium.org 已提交
905
    }
906

H
heyongqiang 已提交
907 908 909 910 911 912
    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 已提交
913 914 915 916 917 918 919
    if (!logs.empty()) {
      // Recover in the order in which the logs were generated
      std::sort(logs.begin(), logs.end());
      s = RecoverLogFiles(logs, &max_sequence, read_only);
      if (!s.ok()) {
        // Clear memtables if recovery failed
        for (auto cfd : *versions_->GetColumnFamilySet()) {
A
agiardullo 已提交
920 921
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 kMaxSequenceNumber);
S
Stanislau Hlebik 已提交
922 923
        }
      }
924
    }
L
Lei Jin 已提交
925
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
926 927
  }

L
Lei Jin 已提交
928 929
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
930
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
931 932 933
    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 已提交
934 935
  }

J
jorlow@chromium.org 已提交
936 937 938
  return s;
}

S
Stanislau Hlebik 已提交
939 940 941
// 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 已提交
942 943
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
944
    Logger* info_log;
J
jorlow@chromium.org 已提交
945
    const char* fname;
946
    Status* status;  // nullptr if db_options_.paranoid_checks==false
I
Igor Sugak 已提交
947
    virtual void Corruption(size_t bytes, const Status& s) override {
948 949
      Log(InfoLogLevel::WARN_LEVEL,
          info_log, "%s%s: dropping %d bytes; %s",
950
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
951
          fname, static_cast<int>(bytes), s.ToString().c_str());
952
      if (this->status != nullptr && this->status->ok()) *this->status = s;
J
jorlow@chromium.org 已提交
953 954 955 956
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
957
  Status status;
958
  std::unordered_map<int, VersionEdit> version_edits;
959
  // no need to refcount because iteration is under mutex
960 961
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
962 963
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
964
  }
965 966 967 968 969 970 971 972 973 974 975 976
  int job_id = next_job_id_.fetch_add(1);
  {
    auto stream = event_logger_.Log();
    stream << "job" << job_id << "event"
           << "recovery_started";
    stream << "log_files";
    stream.StartArray();
    for (auto log_number : log_numbers) {
      stream << log_number;
    }
    stream.EndArray();
  }
I
Igor Canadi 已提交
977

S
Stanislau Hlebik 已提交
978 979 980 981
  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.
982
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
983 984 985 986
    // 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 已提交
987
    if (!status.ok()) {
S
Stanislau Hlebik 已提交
988 989 990 991 992 993 994 995
      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 已提交
996 997
    }

S
Stanislau Hlebik 已提交
998 999 1000 1001 1002
    // Create the log reader.
    LogReporter reporter;
    reporter.env = env_;
    reporter.info_log = db_options_.info_log.get();
    reporter.fname = fname.c_str();
1003
    reporter.status = (db_options_.paranoid_checks) ? &status : nullptr;
S
Stanislau Hlebik 已提交
1004 1005 1006 1007 1008 1009
    // 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*/);
1010 1011
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "Recovering log #%" PRIu64 "", log_number);
S
Stanislau Hlebik 已提交
1012 1013 1014 1015 1016

    // Read all the records and add to a memtable
    std::string scratch;
    Slice record;
    WriteBatch batch;
I
Igor Canadi 已提交
1017
    while (reader.ReadRecord(&record, &scratch) && status.ok()) {
S
Stanislau Hlebik 已提交
1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038
      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;
A
agiardullo 已提交
1039
      if ((*max_sequence == kMaxSequenceNumber) || (last_seq > *max_sequence)) {
S
Stanislau Hlebik 已提交
1040 1041 1042 1043
        *max_sequence = last_seq;
      }

      if (!read_only) {
I
Igor Canadi 已提交
1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055
        // 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;
1056
          status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
I
Igor Canadi 已提交
1057 1058 1059 1060
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
1061
          }
A
agiardullo 已提交
1062 1063 1064

          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 *max_sequence);
1065
        }
J
jorlow@chromium.org 已提交
1066 1067 1068
      }
    }

1069 1070 1071 1072
    if (!status.ok()) {
      return status;
    }

I
Igor Canadi 已提交
1073
    flush_scheduler_.Clear();
A
agiardullo 已提交
1074 1075
    if ((*max_sequence != kMaxSequenceNumber) &&
        (versions_->LastSequence() < *max_sequence)) {
S
Stanislau Hlebik 已提交
1076 1077
      versions_->SetLastSequence(*max_sequence);
    }
1078 1079
  }

1080
  if (!read_only) {
1081 1082
    // 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 已提交
1083
    auto max_log_number = log_numbers.back();
1084
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1085
      auto iter = version_edits.find(cfd->GetID());
1086 1087 1088
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
1089
      if (cfd->GetLogNumber() > max_log_number) {
1090
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
1091
        // from all logs. Memtable has to be empty because
1092
        // we filter the updates based on log_number
1093
        // (in WriteBatch::InsertInto)
1094 1095 1096 1097 1098 1099 1100
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1101
        status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
1102 1103 1104 1105
        if (!status.ok()) {
          // Recovery failed
          break;
        }
A
agiardullo 已提交
1106 1107 1108

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

1111
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
1112
      // writing log_number in the manifest means that any log file
1113 1114
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
1115 1116 1117
      // 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);
1118 1119 1120
      // 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 已提交
1121
      // log number
1122
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
1123 1124
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
1125
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1126 1127
        // Recovery failed
        break;
1128 1129
      }
    }
1130
  }
I
Igor Canadi 已提交
1131

1132 1133 1134
  event_logger_.Log() << "job" << job_id << "event"
                      << "recovery_finished";

J
jorlow@chromium.org 已提交
1135 1136 1137
  return status;
}

1138 1139
Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
                                           MemTable* mem, VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1140
  mutex_.AssertHeld();
1141
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1142
  FileMetaData meta;
1143
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1144 1145
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1146 1147
  ReadOptions ro;
  ro.total_order_seek = true;
1148
  Arena arena;
1149
  Status s;
1150
  TableProperties table_properties;
1151
  {
1152 1153 1154 1155
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
    const SequenceNumber newest_snapshot = snapshots_.GetNewest();
    const SequenceNumber earliest_seqno_in_memtable =
        mem->GetFirstSequenceNumber();
1156 1157 1158
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1159
        cfd->GetName().c_str(), meta.fd.GetNumber());
1160

1161 1162
    bool paranoid_file_checks =
        cfd->GetLatestMutableCFOptions()->paranoid_file_checks;
1163 1164 1165 1166
    {
      mutex_.Unlock();
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
1167 1168
          iter.get(), &meta, cfd->internal_comparator(),
          cfd->int_tbl_prop_collector_factories(), newest_snapshot,
1169
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
1170 1171
          cfd->ioptions()->compression_opts, paranoid_file_checks, Env::IO_HIGH,
          &table_properties);
1172
      LogFlush(db_options_.info_log);
1173 1174 1175 1176 1177 1178 1179 1180
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] [WriteLevel0TableForRecovery]"
          " Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
          cfd->GetName().c_str(), meta.fd.GetNumber(), meta.fd.GetFileSize(),
          s.ToString().c_str());

      // output to event logger
      if (s.ok()) {
1181
        EventHelpers::LogTableFileCreation(
1182 1183 1184
            &event_logger_, job_id, meta.fd.GetNumber(), meta.fd.GetFileSize(),
            table_properties);
      }
1185 1186
      mutex_.Lock();
    }
1187
  }
I
Igor Canadi 已提交
1188
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1189 1190 1191 1192

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1193
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1194 1195 1196
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1197 1198
  }

L
Lei Jin 已提交
1199
  InternalStats::CompactionStats stats(1);
1200
  stats.micros = env_->NowMicros() - start_micros;
1201
  stats.bytes_written = meta.fd.GetFileSize();
M
Mark Callaghan 已提交
1202
  stats.files_out_levelnp1 = 1;
1203
  cfd->internal_stats()->AddCompactionStats(level, stats);
1204 1205
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1206
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1207 1208 1209
  return s;
}

1210 1211
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
I
Igor Canadi 已提交
1212
    bool* madeProgress, JobContext* job_context, LogBuffer* log_buffer) {
1213
  mutex_.AssertHeld();
1214
  assert(cfd->imm()->NumNotFlushed() != 0);
1215
  assert(cfd->imm()->IsFlushPending());
1216

I
Igor Canadi 已提交
1217 1218
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1219
                     snapshots_.GetNewest(), job_context, log_buffer,
1220
                     directories_.GetDbDir(), directories_.GetDataDir(0U),
I
Igor Canadi 已提交
1221 1222
                     GetCompressionFlush(*cfd->ioptions()), stats_,
                     &event_logger_);
1223

1224 1225
  uint64_t file_number;
  Status s = flush_job.Run(&file_number);
J
jorlow@chromium.org 已提交
1226 1227

  if (s.ok()) {
I
Igor Canadi 已提交
1228
    InstallSuperVersionBackground(cfd, job_context, mutable_cf_options);
1229 1230 1231
    if (madeProgress) {
      *madeProgress = 1;
    }
S
sdong 已提交
1232
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1233
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1234
                cfd->current()->storage_info()->LevelSummary(&tmp));
1235

1236
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1237
      // add to deletion state
1238
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1239 1240
             alive_log_files_.begin()->number < versions_->MinLogNumber()) {
        const auto& earliest = *alive_log_files_.begin();
I
Igor Canadi 已提交
1241
        job_context->log_delete_files.push_back(earliest.number);
I
Igor Canadi 已提交
1242
        total_log_size_ -= earliest.size;
1243 1244
        alive_log_files_.pop_front();
      }
1245
    }
J
jorlow@chromium.org 已提交
1246
  }
1247

1248
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1249 1250 1251 1252 1253
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1254
  RecordFlushIOStats();
1255 1256 1257
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1258
    NotifyOnFlushCompleted(cfd, file_number, mutable_cf_options);
1259 1260
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1261 1262 1263
  return s;
}

1264
void DBImpl::NotifyOnFlushCompleted(
1265 1266
    ColumnFamilyData* cfd, uint64_t file_number,
    const MutableCFOptions& mutable_cf_options) {
1267
#ifndef ROCKSDB_LITE
1268
  if (db_options_.listeners.size() == 0U) {
1269 1270
    return;
  }
1271 1272 1273 1274 1275 1276
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  bool triggered_flush_slowdown =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1277
       mutable_cf_options.level0_slowdown_writes_trigger);
1278 1279
  bool triggered_flush_stop =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1280
       mutable_cf_options.level0_stop_writes_trigger);
1281 1282
  // release lock while notifying events
  mutex_.Unlock();
1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293
  {
    // TODO(yhchiang): make db_paths dynamic.
    auto file_path = MakeTableFileName(db_options_.db_paths[0].path,
                                       file_number);
    for (auto listener : db_options_.listeners) {
      listener->OnFlushCompleted(
          this, cfd->GetName(), file_path,
          // Use path 0 as fulled memtables are first flushed into path 0.
          triggered_flush_slowdown, triggered_flush_stop);
    }
  }
1294 1295 1296
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
1297
#endif  // ROCKSDB_LITE
1298 1299
}

1300
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1301
                            const Slice* begin, const Slice* end,
1302 1303
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
1304
  if (target_path_id >= db_options_.db_paths.size()) {
1305 1306 1307
    return Status::InvalidArgument("Invalid target path ID");
  }

1308 1309
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1310 1311

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1312
  if (!s.ok()) {
1313
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1314 1315 1316
    return s;
  }

I
Igor Canadi 已提交
1317
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1318
  {
1319
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
1320
    Version* base = cfd->current();
1321 1322
    for (int level = 1; level < base->storage_info()->num_non_empty_levels();
         level++) {
S
sdong 已提交
1323
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1324 1325 1326 1327
        max_level_with_files = level;
      }
    }
  }
1328 1329 1330 1331 1332

  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
1333 1334
                            cfd->NumberLevels() - 1, target_path_id, begin,
                            end);
1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345
  } else {
    for (int level = 0; level <= max_level_with_files; level++) {
      // in case the compaction is unversal or if we're compacting the
      // bottom-most level, the output level will be the same as input one.
      // level 0 can never be the bottommost level (i.e. if all files are in
      // level 0, we will compact to level 1)
      if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO ||
          (level == max_level_with_files && level > 0)) {
        s = RunManualCompaction(cfd, level, level, target_path_id, begin, end);
      } else {
1346 1347 1348 1349 1350 1351 1352
        int output_level = level + 1;
        if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
            cfd->ioptions()->level_compaction_dynamic_level_bytes &&
            level == 0) {
          output_level = ColumnFamilyData::kCompactToBaseLevel;
        }
        s = RunManualCompaction(cfd, level, output_level, target_path_id, begin,
1353 1354 1355 1356 1357
                                end);
      }
      if (!s.ok()) {
        break;
      }
1358 1359
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
1360
    }
G
Gabor Cselle 已提交
1361
  }
1362 1363 1364 1365
  if (!s.ok()) {
    LogFlush(db_options_.info_log);
    return s;
  }
1366 1367

  if (reduce_level) {
I
Igor Canadi 已提交
1368
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1369
  }
1370
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1371

1372
  {
1373
    InstrumentedMutexLock l(&mutex_);
1374 1375 1376 1377 1378
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1379
  return s;
1380 1381
}

1382 1383 1384 1385 1386
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 已提交
1387 1388 1389 1390
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1391 1392 1393 1394 1395 1396
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);
1397 1398 1399 1400 1401 1402 1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420 1421 1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432 1433 1434 1435 1436 1437

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

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

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

  // Find and delete obsolete files
  {
    InstrumentedMutexLock l(&mutex_);
    // If !s.ok(), this means that Compaction failed. In that case, we want
    // to delete all obsolete files we might have created and we force
    // FindObsoleteFiles(). This is because job_context does not
    // catch all created files if compaction failed.
    FindObsoleteFiles(&job_context, !s.ok());
  }

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

1438
  return s;
I
Igor Canadi 已提交
1439
#endif  // ROCKSDB_LITE
1440 1441
}

I
Igor Canadi 已提交
1442
#ifndef ROCKSDB_LITE
1443 1444 1445
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
1446 1447
    const int output_level, int output_path_id, JobContext* job_context,
    LogBuffer* log_buffer) {
1448 1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479
  mutex_.AssertHeld();

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

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

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

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

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

1480
  std::vector<CompactionInputFiles> input_files;
1481 1482 1483 1484 1485 1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499 1500
  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(
1501 1502
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
1503 1504 1505 1506 1507 1508
  assert(c);
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
  assert(!c->IsDeletionCompaction());

  auto yield_callback = [&]() {
1509 1510 1511
    return CallFlushDuringCompaction(
        c->column_family_data(), *c->mutable_cf_options(),
        job_context, log_buffer);
1512
  };
I
Igor Canadi 已提交
1513
  assert(is_snapshot_supported_ || snapshots_.empty());
1514
  CompactionJob compaction_job(
I
Igor Canadi 已提交
1515 1516 1517 1518 1519
      job_context->job_id, c.get(), db_options_, env_options_, versions_.get(),
      &shutting_down_, log_buffer, directories_.GetDbDir(),
      directories_.GetDataDir(c->GetOutputPathId()), stats_,
      snapshots_.GetAll(), table_cache_, std::move(yield_callback),
      &event_logger_, c->mutable_cf_options()->paranoid_file_checks);
1520 1521 1522 1523 1524
  compaction_job.Prepare();

  mutex_.Unlock();
  Status status = compaction_job.Run();
  mutex_.Lock();
I
Igor Canadi 已提交
1525
  compaction_job.Install(&status, *c->mutable_cf_options(), &mutex_);
1526
  if (status.ok()) {
1527
    InstallSuperVersionBackground(c->column_family_data(), job_context,
1528
                                  *c->mutable_cf_options());
1529 1530 1531 1532 1533 1534 1535 1536 1537
  }
  c->ReleaseCompactionFiles(s);
  c.reset();

  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
1538 1539
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] [JOB %d] Compaction error: %s",
1540
        c->column_family_data()->GetName().c_str(), job_context->job_id,
1541 1542 1543 1544 1545 1546 1547
        status.ToString().c_str());
    if (db_options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }

  bg_compaction_scheduled_--;
1548 1549 1550
  if (bg_compaction_scheduled_ == 0) {
    bg_cv_.SignalAll();
  }
1551 1552 1553

  return status;
}
I
Igor Canadi 已提交
1554
#endif  // ROCKSDB_LITE
1555

O
Ori Bernstein 已提交
1556 1557 1558
void DBImpl::NotifyOnCompactionCompleted(
    ColumnFamilyData* cfd, Compaction *c, const Status &st) {
#ifndef ROCKSDB_LITE
1559
  if (db_options_.listeners.size() == 0U) {
O
Ori Bernstein 已提交
1560 1561 1562 1563 1564 1565 1566 1567
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  // release lock while notifying events
  mutex_.Unlock();
1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578 1579 1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590
  {
    CompactionJobInfo info;
    info.cf_name = cfd->GetName();
    info.status = st;
    info.output_level = c->output_level();
    for (size_t i = 0; i < c->num_input_levels(); ++i) {
      for (const auto fmd : *c->inputs(i)) {
        info.input_files.push_back(
            TableFileName(db_options_.db_paths,
                          fmd->fd.GetNumber(),
                          fmd->fd.GetPathId()));
      }
    }
    for (const auto newf : c->edit()->GetNewFiles()) {
      info.output_files.push_back(
          TableFileName(db_options_.db_paths,
                        newf.second.fd.GetNumber(),
                        newf.second.fd.GetPathId()));
    }
    for (auto listener : db_options_.listeners) {
      listener->OnCompactionCompleted(this, info);
    }
  }
O
Ori Bernstein 已提交
1591 1592 1593 1594 1595 1596
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
#endif  // ROCKSDB_LITE
}

1597
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1598
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1599 1600 1601
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1602 1603
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1604 1605
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1606
        cfd->GetName().c_str());
1607
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1608 1609 1610
  }

  MutableCFOptions new_options;
1611
  Status s;
L
Lei Jin 已提交
1612
  {
1613
    InstrumentedMutexLock l(&mutex_);
1614 1615
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1616 1617 1618 1619
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

1620 1621
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1622 1623
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1624 1625
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1626
  }
1627
  if (s.ok()) {
1628 1629
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1630 1631 1632
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
  } else {
1633 1634
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1635
  }
1636
  return s;
I
Igor Canadi 已提交
1637
#endif  // ROCKSDB_LITE
1638 1639
}

1640
// return the same level if it cannot be moved
1641 1642
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
1643
  mutex_.AssertHeld();
S
sdong 已提交
1644
  const auto* vstorage = cfd->current()->storage_info();
1645
  int minimum_level = level;
1646
  for (int i = level - 1; i > 0; --i) {
1647
    // stop if level i is not empty
S
sdong 已提交
1648
    if (vstorage->NumLevelFiles(i) > 0) break;
1649
    // stop if level i is too small (cannot fit the level files)
1650
    if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) {
1651 1652
      break;
    }
1653 1654 1655 1656 1657 1658

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1662
  SuperVersion* superversion_to_free = nullptr;
1663
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1664 1665

  mutex_.Lock();
1666 1667 1668

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1669
    mutex_.Unlock();
1670 1671
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
I
Igor Canadi 已提交
1672
    delete new_superversion;
L
Lei Jin 已提交
1673
    return Status::NotSupported("another thread is refitting");
1674 1675 1676 1677 1678
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1679
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1680 1681
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop: %d %d",
1682
        bg_compaction_scheduled_, bg_flush_scheduled_);
1683 1684 1685
    bg_cv_.Wait();
  }

1686 1687
  const MutableCFOptions mutable_cf_options =
    *cfd->GetLatestMutableCFOptions();
1688
  // move to a smaller level
1689 1690
  int to_level = target_level;
  if (target_level < 0) {
1691
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1692
  }
1693 1694 1695

  assert(to_level <= level);

L
Lei Jin 已提交
1696
  Status status;
1697
  if (to_level < level) {
1698 1699
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Before refitting:\n%s",
1700
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1701

1702
    VersionEdit edit;
I
Igor Canadi 已提交
1703
    edit.SetColumnFamily(cfd->GetID());
S
sdong 已提交
1704
    for (const auto& f : cfd->current()->storage_info()->LevelFiles(level)) {
1705
      edit.DeleteFile(level, f->fd.GetNumber());
1706 1707 1708
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1709
    }
1710 1711
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1712
        cfd->GetName().c_str(), edit.DebugString().data());
1713

1714 1715
    status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                    directories_.GetDbDir());
L
Lei Jin 已提交
1716 1717
    superversion_to_free = InstallSuperVersion(
        cfd, new_superversion, mutable_cf_options);
I
Igor Canadi 已提交
1718
    new_superversion = nullptr;
1719

1720 1721
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1722
        status.ToString().data());
1723 1724

    if (status.ok()) {
1725 1726
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1727
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1728 1729 1730 1731 1732
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1733 1734 1735 1736

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1737
  return status;
G
Gabor Cselle 已提交
1738 1739
}

1740 1741 1742
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1743 1744
}

1745 1746
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1747
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
1748 1749
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.max_mem_compaction_level;
1750 1751
}

1752 1753
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1754
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
1755 1756
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
1757 1758
}

L
Lei Jin 已提交
1759
Status DBImpl::Flush(const FlushOptions& flush_options,
1760 1761
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1762
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
1763 1764
}

1765
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1766 1767 1768
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1769
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1770 1771
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1772 1773
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);
1774

G
Gabor Cselle 已提交
1775 1776
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1777
  ManualCompaction manual;
I
Igor Canadi 已提交
1778
  manual.cfd = cfd;
1779 1780
  manual.input_level = input_level;
  manual.output_level = output_level;
1781
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
1782
  manual.done = false;
1783
  manual.in_progress = false;
1784 1785 1786
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
1787 1788
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1789
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1790
  } else {
1791
    begin_storage.SetMaxPossibleForUserKey(*begin);
G
Gabor Cselle 已提交
1792 1793
    manual.begin = &begin_storage;
  }
1794
  if (end == nullptr ||
1795 1796
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1797
    manual.end = nullptr;
G
Gabor Cselle 已提交
1798
  } else {
1799
    end_storage.SetMinPossibleForUserKey(*end);
G
Gabor Cselle 已提交
1800 1801 1802
    manual.end = &end_storage;
  }

1803
  InstrumentedMutexLock l(&mutex_);
1804

1805 1806 1807 1808 1809 1810
  // 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
1811
  // RunManualCompaction(), i.e. during that time no other compaction will
1812 1813 1814
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1815
  // RunManualCompaction() from getting to the second while loop below.
1816 1817 1818 1819 1820
  // 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) {
1821
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
1822 1823 1824
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1825 1826
    bg_cv_.Wait();
  }
1827

1828 1829
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
1830
      cfd->GetName().c_str());
1831

1832 1833 1834 1835
  // 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) {
1836 1837 1838
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
1839
      bg_cv_.Wait();
1840 1841
    } else {
      manual_compaction_ = &manual;
1842
      bg_compaction_scheduled_++;
1843
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
G
Gabor Cselle 已提交
1844
    }
H
hans@chromium.org 已提交
1845
  }
1846

1847 1848 1849
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1850
  return manual.status;
J
jorlow@chromium.org 已提交
1851 1852
}

1853
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
1854
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
1855 1856 1857
  Status s;
  {
    WriteContext context;
1858
    InstrumentedMutexLock guard_lock(&mutex_);
1859

1860
    if (cfd->imm()->NumNotFlushed() == 0 && cfd->mem()->IsEmpty()) {
1861 1862 1863 1864
      // Nothing to flush
      return Status::OK();
    }

I
Igor Canadi 已提交
1865 1866
    WriteThread::Writer w(&mutex_);
    s = write_thread_.EnterWriteThread(&w, 0);
S
Stanislau Hlebik 已提交
1867 1868 1869 1870 1871
    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);
1872 1873
    write_thread_.ExitWriteThread(&w, &w, s);

S
Stanislau Hlebik 已提交
1874 1875
    cfd->imm()->FlushRequested();

1876 1877 1878
    // schedule flush
    SchedulePendingFlush(cfd);
    MaybeScheduleFlushOrCompaction();
S
Stanislau Hlebik 已提交
1879
  }
S
Stanislau Hlebik 已提交
1880

L
Lei Jin 已提交
1881
  if (s.ok() && flush_options.wait) {
1882
    // Wait until the compaction completes
1883
    s = WaitForFlushMemTable(cfd);
1884 1885
  }
  return s;
J
jorlow@chromium.org 已提交
1886 1887
}

1888
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1889 1890
  Status s;
  // Wait until the compaction completes
1891
  InstrumentedMutexLock l(&mutex_);
1892
  while (cfd->imm()->NumNotFlushed() > 0 && bg_error_.ok()) {
1893 1894
    bg_cv_.Wait();
  }
1895
  if (!bg_error_.ok()) {
1896 1897 1898
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1899 1900
}

1901
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1902
  mutex_.AssertHeld();
1903
  if (bg_work_gate_closed_) {
1904 1905
    // gate closed for background work
    return;
I
Igor Canadi 已提交
1906
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
1907
    // DB is being deleted; no more background compactions
1908 1909
    return;
  }
1910

1911 1912 1913 1914
  while (unscheduled_flushes_ > 0 &&
         bg_flush_scheduled_ < db_options_.max_background_flushes) {
    unscheduled_flushes_--;
    bg_flush_scheduled_++;
1915
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
1916 1917
  }

1918 1919 1920 1921 1922 1923
  if (bg_manual_only_) {
    // only manual compactions are allowed to run. don't schedule automatic
    // compactions
    return;
  }

1924 1925 1926 1927 1928 1929 1930 1931 1932 1933
  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_--;
1934
    }
1935
    bg_compaction_scheduled_++;
1936
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
1937 1938 1939 1940 1941 1942
  }

  while (bg_compaction_scheduled_ < db_options_.max_background_compactions &&
         unscheduled_compactions_ > 0) {
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
1943
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
1944 1945 1946 1947 1948 1949 1950 1951 1952 1953 1954 1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967 1968 1969 1970 1971 1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983 1984 1985 1986 1987 1988 1989
  }
}

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_;
1990 1991 1992
  }
}

1993
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
1994
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
1995 1996 1997
  IOSTATS_RESET(bytes_written);
}

1998
void DBImpl::BGWorkFlush(void* db) {
1999
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
2000 2001 2002 2003
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
2004
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
2005
  TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
2006 2007 2008
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
2009
Status DBImpl::BackgroundFlush(bool* madeProgress, JobContext* job_context,
H
Haobo Xu 已提交
2010
                               LogBuffer* log_buffer) {
2011
  mutex_.AssertHeld();
2012

2013 2014 2015 2016 2017 2018 2019
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

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

2022 2023 2024
  ColumnFamilyData* cfd = nullptr;
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
I
Igor Canadi 已提交
2025
    auto first_cfd = PopFirstFromFlushQueue();
2026

I
Igor Canadi 已提交
2027
    if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
2028
      // can't flush this CF, try next one
I
Igor Canadi 已提交
2029 2030
      if (first_cfd->Unref()) {
        delete first_cfd;
2031 2032
      }
      continue;
2033
    }
2034 2035

    // found a flush!
I
Igor Canadi 已提交
2036
    cfd = first_cfd;
2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051 2052 2053
    break;
  }

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

2059
void DBImpl::BackgroundCallFlush() {
2060
  bool madeProgress = false;
2061
  JobContext job_context(next_job_id_.fetch_add(1), true);
2062 2063
  assert(bg_flush_scheduled_);

2064
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2065
  {
2066
    InstrumentedMutexLock l(&mutex_);
H
Haobo Xu 已提交
2067

I
Igor Canadi 已提交
2068 2069 2070
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2071
    Status s = BackgroundFlush(&madeProgress, &job_context, &log_buffer);
2072 2073 2074 2075 2076 2077 2078 2079 2080 2081 2082 2083 2084 2085 2086 2087 2088
    if (!s.ok() && !s.IsShutdownInProgress()) {
      // Wait a little bit before retrying background flush in
      // case this is an environmental problem and we do not want to
      // chew up resources for failed flushes for the duration of
      // the problem.
      uint64_t error_cnt =
        default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "Waiting after background flush error: %s"
          "Accumulated background error counts: %" PRIu64,
          s.ToString().c_str(), error_cnt);
      log_buffer.FlushBufferToLog();
      LogFlush(db_options_.info_log);
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
H
Haobo Xu 已提交
2089 2090
    }

I
Igor Canadi 已提交
2091 2092
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2093 2094 2095 2096
    // We're just cleaning up for DB::Write()
    job_context.logs_to_free = logs_to_free_;
    logs_to_free_.clear();

2097 2098 2099
    // If flush failed, we want to delete all temporary files that we might have
    // created. Thus, we force full scan in FindObsoleteFiles()
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
H
Haobo Xu 已提交
2100
    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2101
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2102
      mutex_.Unlock();
2103 2104 2105 2106 2107
      // 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 已提交
2108
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2109 2110
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2111
      }
I
Igor Canadi 已提交
2112
      job_context.Clean();
2113 2114
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2115

H
Haobo Xu 已提交
2116
    bg_flush_scheduled_--;
2117 2118
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2119
    RecordFlushIOStats();
H
Haobo Xu 已提交
2120
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2121 2122 2123 2124
    // 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.
2125
  }
J
jorlow@chromium.org 已提交
2126 2127
}

2128
void DBImpl::BackgroundCallCompaction() {
2129
  bool madeProgress = false;
2130
  JobContext job_context(next_job_id_.fetch_add(1), true);
H
Haobo Xu 已提交
2131 2132

  MaybeDumpStats();
2133
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2134
  {
2135
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
2136 2137 2138 2139

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2140
    assert(bg_compaction_scheduled_);
2141
    Status s = BackgroundCompaction(&madeProgress, &job_context, &log_buffer);
2142 2143 2144 2145 2146 2147
    if (!s.ok() && !s.IsShutdownInProgress()) {
      // Wait a little bit before retrying background compaction in
      // case this is an environmental problem and we do not want to
      // chew up resources for failed compactions for the duration of
      // the problem.
      uint64_t error_cnt =
2148
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2149 2150 2151 2152 2153 2154 2155 2156 2157 2158
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
      log_buffer.FlushBufferToLog();
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "Waiting after background compaction error: %s, "
          "Accumulated background error counts: %" PRIu64,
          s.ToString().c_str(), error_cnt);
      LogFlush(db_options_.info_log);
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
2159
    }
H
Haobo Xu 已提交
2160

I
Igor Canadi 已提交
2161 2162
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2163 2164 2165 2166
    // We're just cleaning up for DB::Write()
    job_context.logs_to_free = logs_to_free_;
    logs_to_free_.clear();

2167 2168 2169 2170
    // If compaction failed, we want to delete all temporary files that we might
    // have created (they might not be all recorded in job_context in case of a
    // failure). Thus, we force full scan in FindObsoleteFiles()
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
2171 2172

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2173
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2174
      mutex_.Unlock();
2175 2176 2177 2178 2179
      // 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 已提交
2180
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2181 2182
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2183
      }
I
Igor Canadi 已提交
2184
      job_context.Clean();
2185 2186
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2187

2188
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2189

2190 2191
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2192 2193
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2194 2195
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
I
Igor Canadi 已提交
2196
      // * madeProgress -- need to wakeup DelayWrite
2197 2198 2199 2200 2201 2202
      // * 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 已提交
2203 2204 2205 2206
    // 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.
2207
  }
J
jorlow@chromium.org 已提交
2208 2209
}

I
Igor Canadi 已提交
2210
Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
2211
                                    LogBuffer* log_buffer) {
2212
  *madeProgress = false;
J
jorlow@chromium.org 已提交
2213
  mutex_.AssertHeld();
2214

2215 2216
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2217

2218 2219 2220 2221 2222 2223
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
2224
    if (is_manual) {
2225
      manual_compaction_->status = status;
2226 2227 2228 2229
      manual_compaction_->done = true;
      manual_compaction_->in_progress = false;
      manual_compaction_ = nullptr;
    }
2230
    return status;
2231 2232
  }

2233 2234 2235
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2236 2237 2238 2239
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2240 2241
  }

2242 2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255
  // 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);
2256 2257 2258 2259
      // the second condition will be false when a column family is dropped. we
      // don't want to fail compaction because of that (because it might be a
      // different column family)
      if (!flush_status.ok() && !flush_status.IsShutdownInProgress()) {
2260 2261 2262 2263 2264 2265 2266
        if (is_manual) {
          manual_compaction_->status = flush_status;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
        return flush_status;
2267 2268 2269 2270 2271
      }
    }
  }

  unique_ptr<Compaction> c;
2272 2273
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2274
  if (is_manual) {
G
Gabor Cselle 已提交
2275
    ManualCompaction* m = manual_compaction_;
2276
    assert(m->in_progress);
2277 2278 2279
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
2280
    if (!c) {
2281
      m->done = true;
2282 2283 2284 2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297
      LogToBuffer(log_buffer,
                  "[%s] Manual compaction from level-%d from %s .. "
                  "%s; nothing to do\n",
                  m->cfd->GetName().c_str(), m->input_level,
                  (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                  (m->end ? m->end->DebugString().c_str() : "(end)"));
    } else {
      LogToBuffer(log_buffer,
                  "[%s] Manual compaction from level-%d to level-%d from %s .. "
                  "%s; will stop at %s\n",
                  m->cfd->GetName().c_str(), m->input_level, c->output_level(),
                  (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                  (m->end ? m->end->DebugString().c_str() : "(end)"),
                  ((m->done || manual_end == nullptr)
                       ? "(end)"
                       : manual_end->DebugString().c_str()));
G
Gabor Cselle 已提交
2298
    }
2299 2300 2301 2302 2303 2304 2305 2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326 2327 2328 2329 2330 2331 2332 2333 2334 2335 2336 2337 2338 2339 2340 2341 2342 2343 2344 2345
  } 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 已提交
2346
        }
I
Igor Canadi 已提交
2347 2348
      }
    }
J
jorlow@chromium.org 已提交
2349 2350
  }

2351
  if (!c) {
H
hans@chromium.org 已提交
2352
    // Nothing to do
2353
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2354 2355 2356 2357 2358
  } 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);
2359
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2360 2361
           kCompactionStyleFIFO);
    for (const auto& f : *c->inputs(0)) {
2362
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2363
    }
2364 2365 2366
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
2367 2368
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
I
Igor Canadi 已提交
2369 2370 2371 2372
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
    *madeProgress = true;
H
hans@chromium.org 已提交
2373
  } else if (!is_manual && c->IsTrivialMove()) {
2374
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2375 2376 2377 2378 2379
    // Instrument for event update
    // TODO(yhchiang): add op details for showing trivial-move.
    ThreadStatusUtil::SetColumnFamily(c->column_family_data());
    ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);

J
jorlow@chromium.org 已提交
2380
    // Move file to next level
2381
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2382
    FileMetaData* f = c->input(0, 0);
2383
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
S
sdong 已提交
2384
    c->edit()->AddFile(c->output_level(), f->fd.GetNumber(), f->fd.GetPathId(),
2385 2386
                       f->fd.GetFileSize(), f->smallest, f->largest,
                       f->smallest_seqno, f->largest_seqno);
2387
    status = versions_->LogAndApply(c->column_family_data(),
2388 2389
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2390
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2391 2392
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
2393

S
sdong 已提交
2394
    VersionStorageInfo::LevelSummaryStorage tmp;
2395 2396
    c->column_family_data()->internal_stats()->IncBytesMoved(
        c->level() + 1, f->fd.GetFileSize());
2397 2398 2399 2400 2401 2402 2403
    {
      event_logger_.LogToBuffer(log_buffer)
          << "job" << job_context->job_id << "event"
          << "trivial_move"
          << "destination_level" << c->level() + 1 << "file_number"
          << f->fd.GetNumber() << "file_size" << f->fd.GetFileSize();
    }
2404 2405 2406 2407 2408 2409
    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(),
        c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
2410
    *madeProgress = true;
2411 2412 2413

    // Clear Instrument
    ThreadStatusUtil::ResetThreadStatus();
J
jorlow@chromium.org 已提交
2414
  } else {
2415
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial");
I
Igor Canadi 已提交
2416 2417 2418 2419 2420
    auto yield_callback = [&]() {
      return CallFlushDuringCompaction(c->column_family_data(),
                                       *c->mutable_cf_options(), job_context,
                                       log_buffer);
    };
I
Igor Canadi 已提交
2421
    assert(is_snapshot_supported_ || snapshots_.empty());
2422
    CompactionJob compaction_job(
I
Igor Canadi 已提交
2423 2424 2425 2426 2427
        job_context->job_id, c.get(), db_options_, env_options_,
        versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(),
        directories_.GetDataDir(c->GetOutputPathId()), stats_,
        snapshots_.GetAll(), table_cache_, std::move(yield_callback),
        &event_logger_, c->mutable_cf_options()->paranoid_file_checks);
I
Igor Canadi 已提交
2428 2429 2430 2431
    compaction_job.Prepare();
    mutex_.Unlock();
    status = compaction_job.Run();
    mutex_.Lock();
I
Igor Canadi 已提交
2432
    compaction_job.Install(&status, *c->mutable_cf_options(), &mutex_);
I
Igor Canadi 已提交
2433 2434 2435 2436
    if (status.ok()) {
      InstallSuperVersionBackground(c->column_family_data(), job_context,
                                    *c->mutable_cf_options());
    }
O
Ori Bernstein 已提交
2437 2438 2439 2440 2441
    *madeProgress = true;
  }
  // FIXME(orib): should I check if column family data is null?
  if (c != nullptr) {
    NotifyOnCompactionCompleted(c->column_family_data(), c.get(), status);
I
Igor Canadi 已提交
2442
    c->ReleaseCompactionFiles(status);
2443
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2444
  }
2445
  // this will unref its input_version and column_family_data
2446
  c.reset();
J
jorlow@chromium.org 已提交
2447 2448 2449

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2450
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2451 2452
    // Ignore compaction errors found during shutting down
  } else {
2453
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2454
        status.ToString().c_str());
2455
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2456 2457 2458
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2459 2460

  if (is_manual) {
G
Gabor Cselle 已提交
2461
    ManualCompaction* m = manual_compaction_;
2462
    if (!status.ok()) {
L
Lei Jin 已提交
2463
      m->status = status;
2464 2465
      m->done = true;
    }
2466 2467 2468 2469 2470 2471 2472 2473 2474
    // 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.
2475 2476 2477 2478 2479
    //
    // 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) {
2480 2481
      m->done = true;
    }
G
Gabor Cselle 已提交
2482 2483 2484
    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 已提交
2485
      // Universal and FIFO compactions should always compact the whole range
S
sdong 已提交
2486 2487 2488
      assert(m->cfd->ioptions()->compaction_style !=
                 kCompactionStyleUniversal ||
             m->cfd->ioptions()->num_levels > 1);
2489
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2490
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2491 2492
      m->begin = &m->tmp_storage;
    }
2493
    m->in_progress = false; // not being processed anymore
2494
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2495
  }
2496
  return status;
J
jorlow@chromium.org 已提交
2497 2498
}

I
Igor Canadi 已提交
2499 2500 2501
uint64_t DBImpl::CallFlushDuringCompaction(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
    JobContext* job_context, LogBuffer* log_buffer) {
2502
  if (db_options_.max_background_flushes > 0) {
2503 2504 2505
    // flush thread will take care of this
    return 0;
  }
I
Igor Canadi 已提交
2506
  if (cfd->imm()->imm_flush_needed.load(std::memory_order_relaxed)) {
I
Igor Canadi 已提交
2507 2508 2509 2510
    const uint64_t imm_start = env_->NowMicros();
    mutex_.Lock();
    if (cfd->imm()->IsFlushPending()) {
      cfd->Ref();
I
Igor Canadi 已提交
2511 2512
      FlushMemTableToOutputFile(cfd, mutable_cf_options, nullptr, job_context,
                                log_buffer);
I
Igor Canadi 已提交
2513
      cfd->Unref();
I
Igor Canadi 已提交
2514
      bg_cv_.SignalAll();  // Wakeup DelayWrite() if necessary
I
Igor Canadi 已提交
2515 2516 2517 2518 2519 2520 2521 2522
    }
    mutex_.Unlock();
    log_buffer->FlushBufferToLog();
    return env_->NowMicros() - imm_start;
  }
  return 0;
}

2523 2524
namespace {
struct IterState {
2525
  IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version)
I
Igor Canadi 已提交
2526
      : db(_db), mu(_mu), super_version(_super_version) {}
2527 2528

  DBImpl* db;
2529
  InstrumentedMutex* mu;
2530
  SuperVersion* super_version;
2531 2532 2533 2534
};

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

2536
  if (state->super_version->Unref()) {
2537 2538 2539
    // Job id == 0 means that this is not our background process, but rather
    // user thread
    JobContext job_context(0);
2540

2541 2542
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2543
    state->db->FindObsoleteFiles(&job_context, false, true);
2544 2545 2546
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2547 2548
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2549
    }
I
Igor Canadi 已提交
2550
    job_context.Clean();
I
Igor Canadi 已提交
2551
  }
T
Tomislav Novak 已提交
2552

2553 2554
  delete state;
}
H
Hans Wennborg 已提交
2555
}  // namespace
2556

L
Lei Jin 已提交
2557
Iterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
2558
                                      ColumnFamilyData* cfd,
2559 2560 2561
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
2562 2563 2564 2565 2566
  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 已提交
2567
      super_version->mem->NewIterator(read_options, arena));
2568
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
2569
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
2570
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
2571
  super_version->current->AddIterators(read_options, env_options_,
2572 2573
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
2574
  IterState* cleanup = new IterState(this, &mutex_, super_version);
2575
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2576 2577 2578 2579

  return internal_iter;
}

2580 2581 2582 2583
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2584
Status DBImpl::Get(const ReadOptions& read_options,
2585
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2586
                   std::string* value) {
L
Lei Jin 已提交
2587
  return GetImpl(read_options, column_family, key, value);
2588 2589
}

I
Igor Canadi 已提交
2590 2591
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
2592 2593
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
2594
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
2595
//
I
Igor Canadi 已提交
2596 2597 2598 2599
// However, if InstallSuperVersion() gets called twice with the same
// job_context, we can't reuse the SuperVersion() that got
// malloced
// because
I
Igor Canadi 已提交
2600 2601 2602
// 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 已提交
2603 2604
void DBImpl::InstallSuperVersionBackground(
    ColumnFamilyData* cfd, JobContext* job_context,
2605
    const MutableCFOptions& mutable_cf_options) {
2606
  mutex_.AssertHeld();
I
Igor Canadi 已提交
2607 2608 2609 2610
  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 已提交
2611 2612
}

L
Lei Jin 已提交
2613 2614
SuperVersion* DBImpl::InstallSuperVersion(
    ColumnFamilyData* cfd, SuperVersion* new_sv,
2615
    const MutableCFOptions& mutable_cf_options, bool dont_schedule_bg_work) {
L
Lei Jin 已提交
2616
  mutex_.AssertHeld();
2617 2618 2619 2620 2621 2622 2623 2624 2625

  // 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 已提交
2626 2627 2628
  auto* old = cfd->InstallSuperVersion(
      new_sv ? new_sv : new SuperVersion(), &mutex_, mutable_cf_options);

2629 2630 2631 2632 2633 2634 2635 2636 2637
  // 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 已提交
2638 2639 2640 2641 2642 2643 2644 2645 2646 2647

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

2653 2654 2655
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2656
  SequenceNumber snapshot;
L
Lei Jin 已提交
2657 2658 2659
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2660 2661
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2662
  }
2663
  // Acquire SuperVersion
2664
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
2665
  // Prepare to store a list of merge operations if merge occurs.
2666
  MergeContext merge_context;
2667

2668
  Status s;
2669
  // First look in the memtable, then in the immutable memtable (if any).
2670
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2671
  // merge_operands will contain the sequence of merges in the latter case.
2672
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2673
  PERF_TIMER_STOP(get_snapshot_time);
2674

2675
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2676
    // Done
L
Lei Jin 已提交
2677
    RecordTick(stats_, MEMTABLE_HIT);
2678
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2679
    // Done
L
Lei Jin 已提交
2680
    RecordTick(stats_, MEMTABLE_HIT);
2681
  } else {
2682
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2683 2684
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2685
    RecordTick(stats_, MEMTABLE_MISS);
2686
  }
2687

2688 2689
  {
    PERF_TIMER_GUARD(get_post_process_time);
2690

2691
    ReturnAndCleanupSuperVersion(cfd, sv);
2692

2693 2694 2695
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2696
  return s;
J
jorlow@chromium.org 已提交
2697 2698
}

2699
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
2700
    const ReadOptions& read_options,
2701
    const std::vector<ColumnFamilyHandle*>& column_family,
2702
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
2703

L
Lei Jin 已提交
2704
  StopWatch sw(env_, stats_, DB_MULTIGET);
2705
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
2706

2707
  SequenceNumber snapshot;
2708

2709
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
2710
    ColumnFamilyData* cfd;
2711 2712 2713 2714 2715
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
2716 2717 2718 2719 2720 2721
    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});
2722 2723 2724
    }
  }

2725
  mutex_.Lock();
L
Lei Jin 已提交
2726 2727 2728
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2729 2730 2731
  } else {
    snapshot = versions_->LastSequence();
  }
2732
  for (auto mgd_iter : multiget_cf_data) {
2733 2734
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
2735
  }
2736
  mutex_.Unlock();
2737

2738 2739
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2740

2741
  // Note: this always resizes the values array
2742 2743 2744
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
2745 2746

  // Keep track of bytes that we read for statistics-recording later
2747
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
2748
  PERF_TIMER_STOP(get_snapshot_time);
2749 2750 2751 2752

  // 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.
2753
  // merge_operands will contain the sequence of merges in the latter case.
2754
  for (size_t i = 0; i < num_keys; ++i) {
2755
    merge_context.Clear();
2756
    Status& s = stat_list[i];
2757 2758 2759
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
2760 2761
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
2762 2763 2764
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
2765
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
2766
      // Done
2767
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
2768 2769
      // Done
    } else {
2770
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2771 2772
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
2773 2774 2775
    }

    if (s.ok()) {
2776
      bytes_read += value->size();
2777 2778 2779 2780
    }
  }

  // Post processing (decrement reference counts and record statistics)
2781
  PERF_TIMER_GUARD(get_post_process_time);
2782 2783
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
2784
  // TODO(icanadi) do we need lock here or just around Cleanup()?
2785 2786 2787 2788 2789 2790
  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);
2791 2792
    }
  }
2793 2794 2795 2796 2797 2798 2799
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
2800
  }
2801

L
Lei Jin 已提交
2802 2803 2804
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
2805
  PERF_TIMER_STOP(get_post_process_time);
2806

2807
  return stat_list;
2808 2809
}

L
Lei Jin 已提交
2810
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
2811
                                  const std::string& column_family_name,
2812
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
2813
  Status s;
I
Igor Canadi 已提交
2814
  *handle = nullptr;
Y
Yueh-Hsuan Chiang 已提交
2815
  {
2816
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
2817

Y
Yueh-Hsuan Chiang 已提交
2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828 2829 2830 2831
    if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
        nullptr) {
      return Status::InvalidArgument("Column family already exists");
    }
    VersionEdit edit;
    edit.AddColumnFamily(column_family_name);
    uint32_t new_id = versions_->GetColumnFamilySet()->GetNextColumnFamilyID();
    edit.SetColumnFamily(new_id);
    edit.SetLogNumber(logfile_number_);
    edit.SetComparatorName(cf_options.comparator->Name());

    // LogAndApply will both write the creation in MANIFEST and create
    // ColumnFamilyData object
    Options opt(db_options_, cf_options);
I
Igor Canadi 已提交
2832 2833 2834 2835 2836 2837 2838 2839
    {  // write thread
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
      // LogAndApply will both write the creation in MANIFEST and create
      // ColumnFamilyData object
      s = versions_->LogAndApply(
          nullptr, MutableCFOptions(opt, ImmutableCFOptions(opt)), &edit,
2840
          &mutex_, directories_.GetDbDir(), false, &cf_options);
I
Igor Canadi 已提交
2841 2842
      write_thread_.ExitWriteThread(&w, &w, s);
    }
Y
Yueh-Hsuan Chiang 已提交
2843 2844 2845 2846 2847 2848 2849
    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 已提交
2850 2851 2852 2853 2854

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

Y
Yueh-Hsuan Chiang 已提交
2855 2856 2857 2858 2859 2860 2861 2862 2863
      *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());
    }
2864
  }  // InstrumentedMutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
2865 2866

  // this is outside the mutex
2867
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2868 2869
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
2870
  }
2871
  return s;
2872 2873
}

2874 2875 2876 2877
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
2878 2879
    return Status::InvalidArgument("Can't drop default column family");
  }
2880

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

I
Igor Canadi 已提交
2883 2884
  VersionEdit edit;
  edit.DropColumnFamily();
2885 2886
  edit.SetColumnFamily(cfd->GetID());

2887
  Status s;
2888
  {
2889
    InstrumentedMutexLock l(&mutex_);
2890 2891 2892 2893
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
2894
      // we drop column family from a single write thread
I
Igor Canadi 已提交
2895 2896
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
2897
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
2898 2899
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
I
Igor Canadi 已提交
2900
      write_thread_.ExitWriteThread(&w, &w, s);
2901
    }
S
sdong 已提交
2902 2903 2904 2905 2906 2907

    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()) {
2908
        if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) {
S
sdong 已提交
2909 2910 2911 2912 2913 2914
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
2915
  }
2916

2917
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2918 2919 2920 2921
    // 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 已提交
2922
    assert(cfd->IsDropped());
L
Lei Jin 已提交
2923 2924 2925
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
2926 2927
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
2928
        cfd->GetID());
2929
  } else {
2930
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
2931
        "Dropping column family with id %u FAILED -- %s\n",
2932 2933 2934
        cfd->GetID(), s.ToString().c_str());
  }

2935
  return s;
2936 2937
}

L
Lei Jin 已提交
2938
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
2939 2940
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
2941
  if (value_found != nullptr) {
K
Kai Liu 已提交
2942 2943
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
2944
  }
L
Lei Jin 已提交
2945
  ReadOptions roptions = read_options;
2946
  roptions.read_tier = kBlockCacheTier; // read from block cache only
2947
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
2948

2949
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
2950 2951 2952
  // 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();
2953 2954
}

2955
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
2956 2957 2958
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
2959

V
Venkatesh Radhakrishnan 已提交
2960 2961 2962 2963 2964 2965 2966 2967 2968 2969 2970 2971 2972 2973 2974 2975 2976 2977
  XFUNC_TEST("", "managed_new", managed_new1, xf_manage_new,
             reinterpret_cast<DBImpl*>(this),
             const_cast<ReadOptions*>(&read_options), is_snapshot_supported_);
  if (read_options.managed) {
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return NewErrorIterator(Status::InvalidArgument(
        "Managed Iterators not supported in RocksDBLite."));
#else
    if ((read_options.tailing) || (read_options.snapshot != nullptr) ||
        (is_snapshot_supported_)) {
      return new ManagedIterator(this, read_options, cfd);
    }
    // Managed iter not supported
    return NewErrorIterator(Status::InvalidArgument(
        "Managed Iterators not supported without snapshots."));
#endif
  } else if (read_options.tailing) {
I
Igor Canadi 已提交
2978 2979 2980 2981
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
2982 2983
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
2984
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2985 2986 2987
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
2988
#endif
T
Tomislav Novak 已提交
2989
  } else {
2990
    SequenceNumber latest_snapshot = versions_->LastSequence();
2991
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
2992

I
Igor Canadi 已提交
2993
    auto snapshot =
2994 2995 2996
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
2997
            : latest_snapshot;
T
Tomislav Novak 已提交
2998

2999 3000 3001 3002 3003 3004 3005 3006 3007 3008 3009 3010 3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022 3023 3024 3025 3026 3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037 3038 3039 3040 3041
    // 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(
3042
        env_, *cfd->ioptions(), cfd->user_comparator(),
3043
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
3044
        read_options.iterate_upper_bound);
3045

3046
    Iterator* internal_iter =
3047
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3048 3049 3050 3051
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
3052 3053
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
3054 3055
}

3056
Status DBImpl::NewIterators(
3057
    const ReadOptions& read_options,
I
Igor Canadi 已提交
3058
    const std::vector<ColumnFamilyHandle*>& column_families,
3059
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3060 3061
  iterators->clear();
  iterators->reserve(column_families.size());
V
Venkatesh Radhakrishnan 已提交
3062 3063 3064 3065 3066 3067 3068 3069 3070 3071 3072 3073 3074 3075 3076 3077 3078 3079 3080 3081
  XFUNC_TEST("", "managed_new", managed_new1, xf_manage_new,
             reinterpret_cast<DBImpl*>(this),
             const_cast<ReadOptions*>(&read_options), is_snapshot_supported_);
  if (read_options.managed) {
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Managed interator not supported in RocksDB lite");
#else
    if ((!read_options.tailing) && (read_options.snapshot == nullptr) &&
        (!is_snapshot_supported_)) {
      return Status::InvalidArgument(
          "Managed interator not supported without snapshots");
    }
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
      auto iter = new ManagedIterator(this, read_options, cfd);
      iterators->push_back(iter);
    }
#endif
  } else if (read_options.tailing) {
I
Igor Canadi 已提交
3082 3083 3084 3085
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3086 3087
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
3088 3089
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
3090
      iterators->push_back(
3091
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3092 3093
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
3094
    }
I
Igor Canadi 已提交
3095
#endif
I
Igor Canadi 已提交
3096
  } else {
3097 3098
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
3099
    for (size_t i = 0; i < column_families.size(); ++i) {
3100 3101 3102
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
3103 3104

      auto snapshot =
3105 3106 3107
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
3108 3109
              : latest_snapshot;

3110
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3111
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
3112
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
3113
      Iterator* internal_iter = NewInternalIterator(
3114
          read_options, cfd, sv, db_iter->GetArena());
3115 3116
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3117 3118 3119 3120
    }
  }

  return Status::OK();
3121 3122
}

J
jorlow@chromium.org 已提交
3123
const Snapshot* DBImpl::GetSnapshot() {
3124 3125 3126
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error

3127
  InstrumentedMutexLock l(&mutex_);
3128
  // returns null if the underlying memtable does not support snapshot.
S
sdong 已提交
3129
  if (!is_snapshot_supported_) return nullptr;
3130
  return snapshots_.New(versions_->LastSequence(), unix_time);
J
jorlow@chromium.org 已提交
3131 3132 3133
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
3134
  InstrumentedMutexLock l(&mutex_);
3135
  snapshots_.Delete(reinterpret_cast<const SnapshotImpl*>(s));
J
jorlow@chromium.org 已提交
3136 3137 3138
}

// Convenience methods
3139 3140
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3141
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3142 3143
}

3144 3145 3146
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
3147
  if (!cfh->cfd()->ioptions()->merge_operator) {
3148 3149
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3150
    return DB::Merge(o, column_family, key, val);
3151 3152 3153
  }
}

L
Lei Jin 已提交
3154
Status DBImpl::Delete(const WriteOptions& write_options,
3155
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
3156
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
3157 3158
}

L
Lei Jin 已提交
3159
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
A
agiardullo 已提交
3160 3161 3162
  return WriteImpl(write_options, my_batch, nullptr);
}

A
agiardullo 已提交
3163
#ifndef ROCKSDB_LITE
A
agiardullo 已提交
3164 3165 3166 3167 3168
Status DBImpl::WriteWithCallback(const WriteOptions& write_options,
                                 WriteBatch* my_batch,
                                 WriteCallback* callback) {
  return WriteImpl(write_options, my_batch, callback);
}
A
agiardullo 已提交
3169
#endif  // ROCKSDB_LITE
A
agiardullo 已提交
3170 3171 3172

Status DBImpl::WriteImpl(const WriteOptions& write_options,
                         WriteBatch* my_batch, WriteCallback* callback) {
S
Stanislau Hlebik 已提交
3173 3174 3175
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
A
agiardullo 已提交
3176 3177 3178 3179 3180 3181 3182 3183 3184 3185 3186 3187

  Status status;
  bool xfunc_attempted_write = false;
  XFUNC_TEST("transaction", "transaction_xftest_write_impl",
             xf_transaction_write1, xf_transaction_write, write_options,
             db_options_, my_batch, callback, this, &status,
             &xfunc_attempted_write);
  if (xfunc_attempted_write) {
    // Test already did the write
    return status;
  }

3188
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
I
Igor Canadi 已提交
3189
  WriteThread::Writer w(&mutex_);
S
Stanislau Hlebik 已提交
3190
  w.batch = my_batch;
L
Lei Jin 已提交
3191 3192
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
3193 3194
  w.in_batch_group = false;
  w.done = false;
A
agiardullo 已提交
3195
  w.has_callback = (callback != nullptr) ? true : false;
L
Lei Jin 已提交
3196
  w.timeout_hint_us = write_options.timeout_hint_us;
S
Stanislau Hlebik 已提交
3197 3198

  uint64_t expiration_time = 0;
I
Igor Canadi 已提交
3199
  bool has_timeout = false;
S
Stanislau Hlebik 已提交
3200
  if (w.timeout_hint_us == 0) {
I
Igor Canadi 已提交
3201
    w.timeout_hint_us = WriteThread::kNoTimeOut;
S
Stanislau Hlebik 已提交
3202 3203
  } else {
    expiration_time = env_->NowMicros() + w.timeout_hint_us;
I
Igor Canadi 已提交
3204
    has_timeout = true;
S
Stanislau Hlebik 已提交
3205 3206
  }

L
Lei Jin 已提交
3207
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
3208 3209 3210 3211 3212
    RecordTick(stats_, WRITE_WITH_WAL);
  }

  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
3213 3214 3215 3216 3217

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

A
agiardullo 已提交
3218
  status = write_thread_.EnterWriteThread(&w, expiration_time);
S
Stanislau Hlebik 已提交
3219 3220
  assert(status.ok() || status.IsTimedOut());
  if (status.IsTimedOut()) {
3221
    mutex_.Unlock();
L
Lei Jin 已提交
3222
    RecordTick(stats_, WRITE_TIMEDOUT);
3223
    return Status::TimedOut();
3224
  }
S
Stanislau Hlebik 已提交
3225 3226 3227 3228 3229 3230 3231 3232 3233 3234
  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);
3235

3236 3237 3238 3239
  // 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.
3240 3241 3242
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

3243
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
3244
                                    ? 4 * max_total_in_memory_state_
3245
                                    : db_options_.max_total_wal_size;
3246
  if (UNLIKELY(!single_column_family_mode_) &&
3247
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3248
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
3249
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
3250
    alive_log_files_.begin()->getting_flushed = true;
3251
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3252 3253 3254
        "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 已提交
3255 3256
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
3257
    for (auto cfd : *versions_->GetColumnFamilySet()) {
3258 3259 3260
      if (cfd->IsDropped()) {
        continue;
      }
I
Igor Canadi 已提交
3261
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
S
Stanislau Hlebik 已提交
3262
        status = SetNewMemtableAndNewLogFile(cfd, &context);
I
Igor Canadi 已提交
3263 3264 3265
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
3266
        cfd->imm()->FlushRequested();
3267 3268
        SchedulePendingFlush(cfd);
        context.schedule_bg_work_ = true;
3269
      }
3270
    }
3271 3272 3273 3274 3275 3276 3277 3278
  } 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()) {
3279 3280 3281
      if (cfd->IsDropped()) {
        continue;
      }
3282 3283 3284 3285 3286 3287
      if (!cfd->mem()->IsEmpty()) {
        status = SetNewMemtableAndNewLogFile(cfd, &context);
        if (!status.ok()) {
          break;
        }
        cfd->imm()->FlushRequested();
3288 3289
        SchedulePendingFlush(cfd);
        context.schedule_bg_work_ = true;
3290 3291 3292
      }
    }
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
3293 3294 3295 3296 3297 3298 3299 3300 3301 3302
  }

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

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

I
Igor Canadi 已提交
3303 3304
  if (UNLIKELY(status.ok() && (write_controller_.IsStopped() ||
                               write_controller_.GetDelay() > 0))) {
3305 3306 3307 3308 3309
    // If writer is stopped, we need to get it going,
    // so schedule flushes/compactions
    if (context.schedule_bg_work_) {
      MaybeScheduleFlushOrCompaction();
    }
I
Igor Canadi 已提交
3310
    status = DelayWrite(expiration_time);
I
Igor Canadi 已提交
3311 3312 3313 3314 3315
  }

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

D
dgrogan@chromium.org 已提交
3318
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
3319
  WriteThread::Writer* last_writer = &w;
A
agiardullo 已提交
3320 3321
  autovector<WriteBatch*> write_batch_group;

S
Stanislau Hlebik 已提交
3322
  if (status.ok()) {
I
Igor Canadi 已提交
3323
    write_thread_.BuildBatchGroup(&last_writer, &write_batch_group);
3324

3325 3326 3327
    // 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
3328
    // into memtables
A
agiardullo 已提交
3329 3330 3331 3332 3333 3334 3335 3336 3337 3338 3339 3340 3341 3342 3343

    mutex_.Unlock();

    if (callback != nullptr) {
      // If this write has a validation callback, check to see if this write
      // is able to be written.  Must be called on the write thread.
      status = callback->Callback(this);
    }
  } else {
    mutex_.Unlock();
  }

  // At this point the mutex is unlocked

  if (status.ok()) {
3344 3345 3346 3347 3348 3349 3350 3351 3352 3353
      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]);
        }
      }

3354 3355 3356 3357
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
3358
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
3359
      // Record statistics
L
Lei Jin 已提交
3360
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
3361
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
3362
      if (write_options.disableWAL) {
3363
        flush_on_destroy_ = true;
3364
      }
L
Lei Jin 已提交
3365
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3366

3367
      uint64_t log_size = 0;
L
Lei Jin 已提交
3368
      if (!write_options.disableWAL) {
3369
        PERF_TIMER_GUARD(write_wal_time);
3370 3371
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
3372 3373
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
3374
        log_empty_ = false;
3375
        log_size = log_entry.size();
L
Lei Jin 已提交
3376
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
L
Lei Jin 已提交
3377
        if (status.ok() && write_options.sync) {
I
Igor Canadi 已提交
3378
          RecordTick(stats_, WAL_FILE_SYNCED);
3379
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
3380
          if (db_options_.use_fsync) {
3381
            status = log_->file()->Fsync();
3382
          } else {
3383
            status = log_->file()->Sync();
3384
          }
3385
          if (status.ok() && !log_dir_synced_) {
3386 3387 3388 3389 3390
            // We only sync WAL directory the first time WAL syncing is
            // requested, so that in case users never turn on WAL sync,
            // we can avoid the disk I/O in the write code path.
            status = directories_.GetWalDir()->Fsync();
          }
3391
          log_dir_synced_ = true;
H
heyongqiang 已提交
3392
        }
3393 3394
      }
      if (status.ok()) {
3395
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
3396

3397
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3398
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
3399
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
3400 3401 3402 3403 3404 3405 3406 3407
        // 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 已提交
3408
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
3409
      }
L
Lei Jin 已提交
3410
      PERF_TIMER_START(write_pre_and_post_process_time);
3411 3412 3413
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
3414
      mutex_.Lock();
A
agiardullo 已提交
3415

3416 3417 3418
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
3419 3420
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
3421
      if (!write_options.disableWAL) {
3422 3423 3424 3425 3426
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
3427
      if (status.ok()) {
3428
        versions_->SetLastSequence(last_sequence);
3429
      }
A
agiardullo 已提交
3430 3431 3432
  } else {
    // Operation failed.  Make sure sure mutex is held for cleanup code below.
    mutex_.Lock();
J
jorlow@chromium.org 已提交
3433
    }
A
agiardullo 已提交
3434 3435 3436

    if (db_options_.paranoid_checks && !status.ok() && !status.IsTimedOut() &&
        !status.IsBusy() && bg_error_.ok()) {
I
Igor Canadi 已提交
3437 3438
    bg_error_ = status; // stop compaction & fail any further writes
  }
3439

A
agiardullo 已提交
3440
  mutex_.AssertHeld();
I
Igor Canadi 已提交
3441
  write_thread_.ExitWriteThread(&w, last_writer, status);
3442 3443 3444 3445

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

3448
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
3449
    RecordTick(stats_, WRITE_TIMEDOUT);
3450 3451
  }

J
jorlow@chromium.org 已提交
3452 3453 3454
  return status;
}

3455 3456
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
3457
Status DBImpl::DelayWrite(uint64_t expiration_time) {
S
sdong 已提交
3458 3459
  uint64_t time_delayed = 0;
  bool delayed = false;
3460
  bool timed_out = false;
S
sdong 已提交
3461 3462 3463 3464 3465 3466 3467 3468
  {
    StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed);
    bool has_timeout = (expiration_time > 0);
    auto delay = write_controller_.GetDelay();
    if (write_controller_.IsStopped() == false && delay > 0) {
      mutex_.Unlock();
      delayed = true;
      // hopefully we don't have to sleep more than 2 billion microseconds
S
sdong 已提交
3469
      TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep");
S
sdong 已提交
3470 3471 3472
      env_->SleepForMicroseconds(static_cast<int>(delay));
      mutex_.Lock();
    }
3473

S
sdong 已提交
3474 3475 3476
    while (bg_error_.ok() && write_controller_.IsStopped()) {
      delayed = true;
      if (has_timeout) {
S
sdong 已提交
3477
        TEST_SYNC_POINT("DBImpl::DelayWrite:TimedWait");
S
sdong 已提交
3478 3479
        bg_cv_.TimedWait(expiration_time);
        if (env_->NowMicros() > expiration_time) {
3480 3481
          timed_out = true;
          break;
S
sdong 已提交
3482 3483 3484
        }
      } else {
        bg_cv_.Wait();
3485 3486 3487
      }
    }
  }
S
sdong 已提交
3488 3489 3490 3491 3492
  if (delayed) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_STALL_MICROS,
                                           time_delayed);
    RecordTick(stats_, STALL_MICROS, time_delayed);
  }
I
Igor Canadi 已提交
3493

3494 3495 3496 3497
  if (timed_out) {
    return Status::TimedOut();
  }

I
Igor Canadi 已提交
3498
  return bg_error_;
3499 3500
}

I
Igor Canadi 已提交
3501 3502 3503 3504
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
    auto status = SetNewMemtableAndNewLogFile(cfd, context);
3505 3506
    SchedulePendingFlush(cfd);
    context->schedule_bg_work_ = true;
I
Igor Canadi 已提交
3507 3508 3509 3510 3511
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
3512 3513
    }
  }
I
Igor Canadi 已提交
3514
  return Status::OK();
S
Stanislau Hlebik 已提交
3515 3516 3517 3518 3519 3520 3521 3522 3523 3524 3525 3526 3527
}

// 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.
3528
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
3529 3530 3531 3532
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
3533
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
3534 3535 3536 3537
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
3538
      s = env_->NewWritableFile(
3539
          LogFileName(db_options_.wal_dir, new_log_number), &lfile,
3540
          env_->OptimizeForLogWrite(env_options_, db_options_));
S
Stanislau Hlebik 已提交
3541 3542 3543
      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 已提交
3544 3545
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
S
Stanislau Hlebik 已提交
3546
        new_log = new log::Writer(std::move(lfile));
3547
        log_dir_synced_ = false;
I
Igor Canadi 已提交
3548
      }
S
Stanislau Hlebik 已提交
3549 3550 3551
    }

    if (s.ok()) {
A
agiardullo 已提交
3552 3553
      SequenceNumber seq = versions_->LastSequence();
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options, seq);
S
Stanislau Hlebik 已提交
3554 3555 3556
      new_superversion = new SuperVersion();
    }
  }
3557
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
3558 3559
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
3560 3561 3562 3563 3564 3565 3566 3567 3568 3569 3570
  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);
3571
    logs_to_free_.push_back(log_.release());
S
Stanislau Hlebik 已提交
3572 3573 3574
    log_.reset(new_log);
    log_empty_ = true;
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
3575
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
3576 3577 3578 3579
      // 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 已提交
3580
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
3581
          loop_cfd->imm()->NumNotFlushed() == 0) {
I
Igor Canadi 已提交
3582
        loop_cfd->SetLogNumber(logfile_number_);
3583
      }
3584 3585
    }
  }
S
Stanislau Hlebik 已提交
3586
  cfd->mem()->SetNextLogNumber(logfile_number_);
3587
  cfd->imm()->Add(cfd->mem(), &context->memtables_to_free_);
S
Stanislau Hlebik 已提交
3588 3589 3590
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
  context->superversions_to_free_.push_back(
3591
      InstallSuperVersion(cfd, new_superversion, mutable_cf_options, true));
3592 3593 3594
  return s;
}

I
Igor Canadi 已提交
3595
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
3596 3597 3598 3599 3600
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3601 3602
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
3603
  auto version = cfd->current();
3604 3605 3606 3607 3608 3609 3610 3611 3612 3613 3614 3615
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
3616
#endif  // ROCKSDB_LITE
3617

I
Igor Canadi 已提交
3618 3619 3620 3621
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3622 3623 3624 3625
Env* DBImpl::GetEnv() const {
  return env_;
}

3626 3627
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
3628
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
3629 3630
}

3631 3632
const DBOptions& DBImpl::GetDBOptions() const { return db_options_; }

3633
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
3634
                         const Slice& property, std::string* value) {
3635 3636
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3637 3638 3639
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

3640
  value->clear();
3641 3642 3643 3644 3645
  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) {
3646
      *value = ToString(int_value);
3647 3648 3649 3650 3651
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
3652
    InstrumentedMutexLock l(&mutex_);
3653 3654 3655
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
3656 3657
}

3658 3659
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
3660 3661
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3662 3663 3664 3665 3666 3667 3668 3669 3670 3671 3672 3673
  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) {
3674 3675
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
3676 3677

  if (!need_out_of_mutex) {
3678
    InstrumentedMutexLock l(&mutex_);
3679
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
3680 3681 3682 3683 3684 3685 3686 3687 3688 3689 3690 3691 3692 3693
  } 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 已提交
3694
  return cfd->GetThreadLocalSuperVersion(&mutex_);
3695 3696
}

A
agiardullo 已提交
3697 3698 3699 3700 3701 3702 3703 3704 3705 3706 3707 3708
// REQUIRED: this function should only be called on the write thread or if the
// mutex is held.
SuperVersion* DBImpl::GetAndRefSuperVersion(uint32_t column_family_id) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);
  if (!cfd) {
    return nullptr;
  }

  return GetAndRefSuperVersion(cfd);
}

3709 3710
void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
3711
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3712 3713 3714 3715 3716

  if (unref_sv) {
    // Release SuperVersion
    if (sv->Unref()) {
      {
3717
        InstrumentedMutexLock l(&mutex_);
3718 3719 3720 3721 3722 3723 3724
        sv->Cleanup();
      }
      delete sv;
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
    }
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
  }
3725 3726
}

A
agiardullo 已提交
3727 3728 3729 3730 3731 3732 3733 3734 3735 3736 3737 3738 3739 3740 3741 3742 3743 3744 3745 3746 3747 3748 3749 3750
// REQUIRED: this function should only be called on the write thread.
void DBImpl::ReturnAndCleanupSuperVersion(uint32_t column_family_id,
                                          SuperVersion* sv) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);

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

// REQUIRED: this function should only be called on the write thread or if the
// mutex is held.
ColumnFamilyHandle* DBImpl::GetColumnFamilyHandle(uint32_t column_family_id) {
  ColumnFamilyMemTables* cf_memtables = column_family_memtables_.get();

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

  return cf_memtables->GetColumnFamilyHandle();
}

3751
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
3752
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
3753
  Version* v;
3754 3755
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
3756
  {
3757
    InstrumentedMutexLock l(&mutex_);
3758
    v = cfd->current();
3759
    v->Ref();
J
jorlow@chromium.org 已提交
3760 3761 3762 3763
  }

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
3764 3765 3766
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
    sizes[i] = versions_->ApproximateSize(v, k1.Encode(), k2.Encode());
J
jorlow@chromium.org 已提交
3767 3768 3769
  }

  {
3770
    InstrumentedMutexLock l(&mutex_);
J
jorlow@chromium.org 已提交
3771 3772 3773 3774
    v->Unref();
  }
}

I
Igor Canadi 已提交
3775 3776 3777 3778 3779 3780 3781 3782 3783 3784 3785 3786 3787 3788 3789 3790
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 已提交
3791 3792 3793 3794 3795
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
3796
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
3797 3798 3799
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
3800
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
3801 3802
}

3803 3804 3805
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
3806 3807 3808
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
3809 3810
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
3811 3812 3813
    return Status::InvalidArgument("Invalid file name");
  }

3814 3815 3816 3817
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
3818 3819
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
3820
          name.c_str());
3821 3822
      return Status::NotSupported("Delete only supported for archived logs");
    }
3823
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
3824
    if (!status.ok()) {
3825 3826
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
3827
          name.c_str(), status.ToString().c_str());
3828 3829 3830 3831
    }
    return status;
  }

3832
  int level;
I
Igor Canadi 已提交
3833
  FileMetaData* metadata;
3834
  ColumnFamilyData* cfd;
3835
  VersionEdit edit;
3836
  JobContext job_context(next_job_id_.fetch_add(1), true);
D
Dhruba Borthakur 已提交
3837
  {
3838
    InstrumentedMutexLock l(&mutex_);
3839
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
3840
    if (!status.ok()) {
3841 3842
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
3843
      job_context.Clean();
D
Dhruba Borthakur 已提交
3844 3845
      return Status::InvalidArgument("File not found");
    }
3846
    assert(level < cfd->NumberLevels());
3847

D
Dhruba Borthakur 已提交
3848
    // If the file is being compacted no need to delete.
3849
    if (metadata->being_compacted) {
3850
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3851
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
3852
      job_context.Clean();
D
Dhruba Borthakur 已提交
3853
      return Status::OK();
3854 3855
    }

D
Dhruba Borthakur 已提交
3856 3857 3858
    // 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 已提交
3859
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
3860
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
3861
      if (vstoreage->NumLevelFiles(i) != 0) {
3862
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
3863
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
3864
        job_context.Clean();
D
Dhruba Borthakur 已提交
3865 3866 3867
        return Status::InvalidArgument("File not in last level");
      }
    }
3868
    // if level == 0, it has to be the oldest file
S
sdong 已提交
3869 3870
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
3871 3872
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
3873
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
3874
      job_context.Clean();
3875 3876 3877
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
3878
    edit.DeleteFile(level, number);
3879
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
3880
                                    &edit, &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
3881
    if (status.ok()) {
I
Igor Canadi 已提交
3882 3883
      InstallSuperVersionBackground(cfd, &job_context,
                                    *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
3884
    }
I
Igor Canadi 已提交
3885 3886
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
3887
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
3888
  // remove files outside the db-lock
I
Igor Canadi 已提交
3889 3890
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
3891
  }
I
Igor Canadi 已提交
3892
  job_context.Clean();
3893 3894 3895
  return status;
}

3896
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
3897
  InstrumentedMutexLock l(&mutex_);
3898
  versions_->GetLiveFilesMetaData(metadata);
3899
}
3900 3901 3902 3903 3904 3905 3906 3907 3908 3909 3910

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 已提交
3911
#endif  // ROCKSDB_LITE
3912

I
Igor Canadi 已提交
3913 3914 3915 3916 3917 3918 3919
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
3923 3924 3925 3926 3927 3928
    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) {
3929
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
3930
                             ". Size recorded in manifest " +
3931 3932
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
3933 3934 3935 3936 3937 3938 3939 3940 3941
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

3942
Status DBImpl::GetDbIdentity(std::string& identity) const {
3943 3944 3945 3946 3947 3948 3949 3950 3951 3952 3953 3954 3955 3956
  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;
3957
  s = idfile->Read(static_cast<size_t>(file_size), &id, buffer);
3958 3959 3960 3961 3962 3963 3964 3965 3966 3967 3968
  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 已提交
3969 3970
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
3971
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
3972
               const Slice& key, const Slice& value) {
3973 3974 3975 3976
  // 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);
3977
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
3978 3979 3980
  return Write(opt, &batch);
}

3981 3982
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
3983
  WriteBatch batch;
3984
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
3985 3986 3987
  return Write(opt, &batch);
}

3988 3989
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
3990
  WriteBatch batch;
3991
  batch.Merge(column_family, key, value);
3992 3993 3994
  return Write(opt, &batch);
}

3995
// Default implementation -- returns not supported status
L
Lei Jin 已提交
3996
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3997
                              const std::string& column_family_name,
3998
                              ColumnFamilyHandle** handle) {
3999
  return Status::NotSupported("");
4000
}
4001
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4002
  return Status::NotSupported("");
4003 4004
}

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

J
Jim Paton 已提交
4007
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4008 4009 4010 4011
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4012
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4013
  std::vector<ColumnFamilyHandle*> handles;
4014
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
4015 4016 4017 4018 4019 4020 4021
  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;
4022 4023
}

4024 4025
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4026
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4027
  Status s = SanitizeOptionsByTable(db_options, column_families);
4028 4029 4030
  if (!s.ok()) {
    return s;
  }
4031

4032
  if (db_options.db_paths.size() > 1) {
4033
    for (auto& cfd : column_families) {
4034 4035
      if ((cfd.options.compaction_style != kCompactionStyleUniversal) &&
          (cfd.options.compaction_style != kCompactionStyleLevel)) {
4036 4037
        return Status::NotSupported(
            "More than one DB paths are only supported in "
4038
            "universal and level compaction styles. ");
4039 4040 4041 4042 4043
      }
    }

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

4048
  *dbptr = nullptr;
4049
  handles->clear();
J
jorlow@chromium.org 已提交
4050

I
Igor Canadi 已提交
4051 4052 4053 4054
  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);
4055
  }
4056

I
Igor Canadi 已提交
4057
  DBImpl* impl = new DBImpl(db_options, dbname);
4058
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
4059
  if (s.ok()) {
4060
    for (auto db_path : impl->db_options_.db_paths) {
4061
      s = impl->env_->CreateDirIfMissing(db_path.path);
4062 4063 4064 4065 4066 4067
      if (!s.ok()) {
        break;
      }
    }
  }

4068 4069 4070 4071 4072 4073
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4074 4075 4076 4077
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4078
  impl->mutex_.Lock();
4079 4080
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4081
  if (s.ok()) {
4082
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4083
    unique_ptr<WritableFile> lfile;
4084
    EnvOptions soptions(db_options);
4085 4086
    s = impl->db_options_.env->NewWritableFile(
        LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
4087 4088
        impl->db_options_.env->OptimizeForLogWrite(soptions,
                                                   impl->db_options_));
J
jorlow@chromium.org 已提交
4089
    if (s.ok()) {
4090
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4091
      impl->logfile_number_ = new_log_number;
4092
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
4093

4094 4095
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4096 4097
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
4098 4099 4100
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
4101
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
4102 4103 4104 4105 4106 4107 4108 4109 4110 4111 4112 4113 4114 4115 4116 4117
        } 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 已提交
4118
        }
4119
      }
I
Igor Canadi 已提交
4120 4121
    }
    if (s.ok()) {
4122
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
4123 4124
        delete impl->InstallSuperVersion(
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
4125
      }
I
Igor Canadi 已提交
4126 4127
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4128
      impl->DeleteObsoleteFiles();
4129
      s = impl->directories_.GetDbDir()->Fsync();
J
jorlow@chromium.org 已提交
4130 4131
    }
  }
4132

I
Igor Canadi 已提交
4133 4134
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
S
sdong 已提交
4135
      if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
4136
        auto* vstorage = cfd->current()->storage_info();
4137
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
4138
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
4139
          if (num_files > 0) {
I
Igor Canadi 已提交
4140 4141
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
S
sdong 已提交
4142
                "open with FIFO compaction style.");
I
Igor Canadi 已提交
4143 4144 4145 4146
            break;
          }
        }
      }
S
sdong 已提交
4147 4148 4149
      if (!cfd->mem()->IsSnapshotSupported()) {
        impl->is_snapshot_supported_ = false;
      }
4150
      if (cfd->ioptions()->merge_operator != nullptr &&
4151 4152 4153 4154 4155
          !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 已提交
4156
      if (!s.ok()) {
4157 4158 4159 4160 4161
        break;
      }
    }
  }

4162 4163
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4164
  if (s.ok()) {
4165
    impl->opened_successfully_ = true;
S
sdong 已提交
4166 4167
    Log(InfoLogLevel::INFO_LEVEL, impl->db_options_.info_log, "DB pointer %p",
        impl);
J
jorlow@chromium.org 已提交
4168 4169
    *dbptr = impl;
  } else {
Y
Yueh-Hsuan Chiang 已提交
4170
    for (auto* h : *handles) {
4171 4172
      delete h;
    }
4173
    handles->clear();
J
jorlow@chromium.org 已提交
4174 4175 4176 4177 4178
    delete impl;
  }
  return s;
}

4179 4180 4181
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4182
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4183 4184
}

4185 4186 4187
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4188
Status DestroyDB(const std::string& dbname, const Options& options) {
4189
  const InternalKeyComparator comparator(options.comparator);
4190
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
4191
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4192
  std::vector<std::string> filenames;
4193

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

J
jorlow@chromium.org 已提交
4197
  FileLock* lock;
4198 4199
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4200 4201 4202
  if (result.ok()) {
    uint64_t number;
    FileType type;
4203
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
4204
    for (size_t i = 0; i < filenames.size(); i++) {
4205
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
4206
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4207 4208 4209 4210 4211 4212
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
4213 4214 4215 4216 4217
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4218

4219 4220
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
4221 4222 4223
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
4224
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
4225 4226 4227 4228 4229 4230 4231
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

I
Igor Canadi 已提交
4232 4233 4234 4235 4236 4237 4238 4239 4240 4241 4242 4243 4244 4245 4246 4247 4248 4249
    std::vector<std::string> walDirFiles;
    std::string archivedir = ArchivalDirectory(dbname);
    if (dbname != soptions.wal_dir) {
      env->GetChildren(soptions.wal_dir, &walDirFiles);
      archivedir = ArchivalDirectory(soptions.wal_dir);
    }

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

    std::vector<std::string> archiveFiles;
4250
    env->GetChildren(archivedir, &archiveFiles);
4251 4252
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4253 4254
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4255
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4256 4257 4258 4259 4260
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4261
    // ignore case where no archival directory is present.
4262
    env->DeleteDir(archivedir);
4263

J
jorlow@chromium.org 已提交
4264
    env->UnlockFile(lock);  // Ignore error since state is already gone
4265
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4266
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4267
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4268 4269 4270 4271
  }
  return result;
}

Y
Yueh-Hsuan Chiang 已提交
4272
#if ROCKSDB_USING_THREAD_STATUS
4273

Y
Yueh-Hsuan Chiang 已提交
4274 4275
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4276
  if (db_options_.enable_thread_tracking) {
4277
    ThreadStatusUtil::NewColumnFamilyInfo(this, cfd);
4278
  }
Y
Yueh-Hsuan Chiang 已提交
4279 4280 4281 4282
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4283
  if (db_options_.enable_thread_tracking) {
4284
    ThreadStatusUtil::EraseColumnFamilyInfo(cfd);
4285
  }
Y
Yueh-Hsuan Chiang 已提交
4286 4287 4288
}

void DBImpl::EraseThreadStatusDbInfo() const {
4289
  if (db_options_.enable_thread_tracking) {
4290
    ThreadStatusUtil::EraseDatabaseInfo(this);
4291
  }
Y
Yueh-Hsuan Chiang 已提交
4292 4293 4294 4295 4296 4297 4298 4299 4300 4301 4302 4303 4304 4305 4306
}

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

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

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

4307 4308
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
4309
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4310
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
4311 4312
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
4313
  Warn(log,
4314
      "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
I
Igor Canadi 已提交
4315
      ROCKSDB_PATCH);
4316 4317
  Warn(log, "Git sha %s", rocksdb_build_git_sha);
  Warn(log, "Compile date %s", rocksdb_build_compile_date);
I
Igor Canadi 已提交
4318
#endif
4319 4320
}

A
agiardullo 已提交
4321 4322 4323 4324 4325 4326 4327 4328 4329 4330 4331 4332 4333 4334 4335 4336 4337 4338 4339 4340 4341 4342 4343 4344 4345 4346 4347 4348 4349 4350 4351 4352 4353 4354 4355 4356 4357 4358 4359 4360 4361 4362 4363 4364 4365 4366 4367 4368 4369 4370 4371 4372 4373 4374 4375 4376 4377 4378 4379 4380 4381 4382 4383 4384 4385 4386 4387 4388 4389 4390 4391 4392 4393 4394 4395 4396 4397 4398 4399
#ifndef ROCKSDB_LITE
SequenceNumber DBImpl::GetEarliestMemTableSequenceNumber(SuperVersion* sv,
                                                         bool include_history) {
  // Find the earliest sequence number that we know we can rely on reading
  // from the memtable without needing to check sst files.
  SequenceNumber earliest_seq =
      sv->imm->GetEarliestSequenceNumber(include_history);
  if (earliest_seq == kMaxSequenceNumber) {
    earliest_seq = sv->mem->GetEarliestSequenceNumber();
  }
  assert(sv->mem->GetEarliestSequenceNumber() >= earliest_seq);

  return earliest_seq;
}
#endif  // ROCKSDB_LITE

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

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

  *seq = kMaxSequenceNumber;

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

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

    return s;
  }

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

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

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

    return s;
  }

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

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

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

    return s;
  }

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

4400
}  // namespace rocksdb